You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by ka...@apache.org on 2018/02/07 21:05:23 UTC

[1/7] storm git commit: STORM-2936 Overwrite latest storm-kafka-client 1.x-branch into 1.1.x-branch

Repository: storm
Updated Branches:
  refs/heads/1.1.x-branch 8ed22cf55 -> c270ff4bd


http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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 dc85123..40a2d3c 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
@@ -24,7 +24,7 @@ import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.util.List;
-
+import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.storm.Config;
 import org.apache.storm.LocalCluster;
@@ -110,7 +110,7 @@ public class KafkaSpoutTopologyMainNamedTopics {
                 TOPIC_PART_OFF_KEY_VALUE_FUNC,
                 new Fields("topic", "partition", "offset", "key", "value"), TOPIC_2_STREAM);
         return KafkaSpoutConfig.builder("127.0.0.1:9092", TOPICS)
-                .setGroupId("kafkaSpoutTestGroup")
+                .setProp(ConsumerConfig.GROUP_ID_CONFIG, "kafkaSpoutTestGroup")
                 .setRetry(getRetryService())
                 .setRecordTranslator(trans)
                 .setOffsetCommitPeriodMs(10_000)

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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 cfc3446..f0004ea 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
@@ -22,7 +22,7 @@ import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrat
 
 import java.util.List;
 import java.util.regex.Pattern;
-
+import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.kafka.spout.Func;
@@ -56,7 +56,7 @@ public class KafkaSpoutTopologyMainWildcardTopics extends KafkaSpoutTopologyMain
     
     protected KafkaSpoutConfig<String,String> getKafkaSpoutConfig() {
         return KafkaSpoutConfig.builder("127.0.0.1:9092", TOPIC_WILDCARD_PATTERN)
-                .setGroupId("kafkaSpoutTestGroup")
+                .setProp(ConsumerConfig.GROUP_ID_CONFIG, "kafkaSpoutTestGroup")
                 .setRetry(getRetryService())
                 .setRecordTranslator(TOPIC_PART_OFF_KEY_VALUE_FUNC,
                         new Fields("topic", "partition", "offset", "key", "value"), STREAM)

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutBatchMetadataTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutBatchMetadataTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutBatchMetadataTest.java
new file mode 100644
index 0000000..da87a03
--- /dev/null
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutBatchMetadataTest.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2017 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.trident;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+import java.util.Collections;
+import java.util.Map;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.storm.kafka.spout.SpoutWithMockedConsumerSetupHelper;
+import org.apache.storm.shade.org.json.simple.JSONValue;
+import org.junit.Test;
+
+public class KafkaTridentSpoutBatchMetadataTest {
+
+    @SuppressWarnings("rawtypes")
+    @Test
+    public void testMetadataIsRoundTripSerializableWithJsonSimple() throws Exception {
+        /**
+         * Tests that the metadata object can be converted to and from a Map. This is needed because Trident metadata is written to
+         * Zookeeper as JSON with the json-simple library, so the spout converts the metadata to Map before returning it to Trident.
+         * It is important that all map entries are types json-simple knows about,
+         * since otherwise the library just calls toString on them which will likely produce invalid JSON.
+         */
+        TopicPartition tp = new TopicPartition("topic", 0);
+        long startOffset = 10;
+        long endOffset = 20;
+
+        KafkaTridentSpoutBatchMetadata metadata = new KafkaTridentSpoutBatchMetadata(tp, startOffset, endOffset);
+        Map<String, Object> map = metadata.toMap();
+        Map deserializedMap = (Map)JSONValue.parseWithException(JSONValue.toJSONString(map));
+        KafkaTridentSpoutBatchMetadata deserializedMetadata = KafkaTridentSpoutBatchMetadata.fromMap(deserializedMap);
+        assertThat(deserializedMetadata.getTopicPartition(), is(metadata.getTopicPartition()));
+        assertThat(deserializedMetadata.getFirstOffset(), is(metadata.getFirstOffset()));
+        assertThat(deserializedMetadata.getLastOffset(), is(metadata.getLastOffset()));
+    }
+
+    @Test
+    public void testCreateMetadataFromRecords() {
+        TopicPartition tp = new TopicPartition("topic", 0);
+        long firstOffset = 15;
+        long lastOffset = 55;
+        ConsumerRecords<?, ?> records = new ConsumerRecords<>(Collections.singletonMap(tp, SpoutWithMockedConsumerSetupHelper.createRecords(tp, firstOffset, (int) (lastOffset - firstOffset + 1))));
+
+        KafkaTridentSpoutBatchMetadata metadata = new KafkaTridentSpoutBatchMetadata(tp, records);
+        assertThat("The first offset should be the first offset in the record set", metadata.getFirstOffset(), is(firstOffset));
+        assertThat("The last offset should be the last offset in the record set", metadata.getLastOffset(), is(lastOffset));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index cefb6f7..af1426e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -278,7 +278,7 @@
         <storm.kafka.artifact.id>kafka_2.10</storm.kafka.artifact.id>
 
         <!-- kafka version used by new storm-kafka-client spout code -->
-        <storm.kafka.client.version>0.10.0.0</storm.kafka.client.version>
+        <storm.kafka.client.version>0.10.1.0</storm.kafka.client.version>
 
         <!-- Java and clojure build lifecycle test properties are defined here to avoid having to create a default profile -->
         <java.unit.test.exclude>org.apache.storm.testing.IntegrationTest</java.unit.test.exclude>

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/storm-core/src/jvm/org/apache/storm/annotation/InterfaceStability.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/annotation/InterfaceStability.java b/storm-core/src/jvm/org/apache/storm/annotation/InterfaceStability.java
new file mode 100644
index 0000000..6fcbac2
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/annotation/InterfaceStability.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.annotation;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * Annotation to inform users of how much to rely on a particular package,
+ * class or method not changing over time.
+ * </ul>
+ */
+@InterfaceStability.Evolving
+public class InterfaceStability {
+    /**
+     * Can evolve while retaining compatibility for minor release boundaries.;
+     * can break compatibility only at major release (ie. at m.0).
+     */
+    @Documented
+    @Retention(RetentionPolicy.RUNTIME)
+    public @interface Stable {};
+
+    /**
+     * Evolving, but can break compatibility at minor release (i.e. m.x)
+     */
+    @Documented
+    @Retention(RetentionPolicy.RUNTIME)
+    public @interface Evolving {};
+
+    /**
+     * No guarantee is provided as to reliability or stability across any
+     * level of release granularity.
+     */
+    @Documented
+    @Retention(RetentionPolicy.RUNTIME)
+    public @interface Unstable {};
+}


[4/7] storm git commit: STORM-2936 Overwrite latest storm-kafka-client 1.x-branch into 1.1.x-branch

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/OffsetManager.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/OffsetManager.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/OffsetManager.java
index 7dfe7f6..c9f9541 100755
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/OffsetManager.java
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/OffsetManager.java
@@ -13,13 +13,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.storm.kafka.spout.internal;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.NavigableSet;
+import java.util.NoSuchElementException;
 import java.util.TreeSet;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
+
 import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.storm.kafka.spout.KafkaSpoutMessageId;
@@ -30,23 +34,27 @@ import org.slf4j.LoggerFactory;
  * Manages acked and committed offsets for a TopicPartition. This class is not thread safe
  */
 public class OffsetManager {
-
     private static final Comparator<KafkaSpoutMessageId> OFFSET_COMPARATOR = new OffsetComparator();
     private static final Logger LOG = LoggerFactory.getLogger(OffsetManager.class);
+
     private final TopicPartition tp;
-    /* First offset to be fetched. It is either set to the beginning, end, or to the first uncommitted offset.
-    * Initial value depends on offset strategy. See KafkaSpoutConsumerRebalanceListener */
-    private final long initialFetchOffset;
-    // Committed offset, i.e. the offset where processing will resume if the spout restarts. Initially it is set to fetchOffset.
-    private long committedOffset;
     // Emitted Offsets List
     private final NavigableSet<Long> emittedOffsets = new TreeSet<>();
     // Acked messages sorted by ascending order of offset
     private final NavigableSet<KafkaSpoutMessageId> ackedMsgs = new TreeSet<>(OFFSET_COMPARATOR);
+    // Committed offset, i.e. the offset where processing will resume upon spout restart. Initially it is set to fetchOffset.
+    private long committedOffset;
+    // True if this OffsetManager has made at least one commit to Kafka
+    private boolean committed;
+    private long latestEmittedOffset;
 
+    /**
+     * Creates a new OffsetManager.
+     * @param tp The TopicPartition
+     * @param initialFetchOffset The initial fetch offset for the given TopicPartition
+     */
     public OffsetManager(TopicPartition tp, long initialFetchOffset) {
         this.tp = tp;
-        this.initialFetchOffset = initialFetchOffset;
         this.committedOffset = initialFetchOffset;
         LOG.debug("Instantiated {}", this.toString());
     }
@@ -56,7 +64,29 @@ public class OffsetManager {
     }
 
     public void addToEmitMsgs(long offset) {
-        this.emittedOffsets.add(offset);                  // O(Log N)
+        this.emittedOffsets.add(offset);  // O(Log N)
+        this.latestEmittedOffset = Math.max(latestEmittedOffset, offset);
+    }
+    
+    public int getNumUncommittedOffsets() {
+        return this.emittedOffsets.size();
+    }
+    
+    /**
+     * Gets the offset of the nth emitted message after the committed offset. 
+     * Example: If the committed offset is 0 and offsets 1, 2, 8, 10 have been emitted,
+     * getNthUncommittedOffsetAfterCommittedOffset(3) returns 8.
+     * 
+     * @param index The index of the message to get the offset for
+     * @return The offset
+     * @throws NoSuchElementException if the index is out of range
+     */
+    public long getNthUncommittedOffsetAfterCommittedOffset(int index) {
+        Iterator<Long> offsetIter = emittedOffsets.iterator();
+        for (int i = 0; i < index - 1; i++) {
+            offsetIter.next();
+        }
+        return offsetIter.next();
     }
 
     /**
@@ -66,20 +96,20 @@ public class OffsetManager {
      * <p/>
      * The returned offset points to the earliest uncommitted offset, and matches the semantics of the KafkaConsumer.commitSync API.
      *
+     * @param commitMetadata Metadata information to commit to Kafka. It is constant per KafkaSpout instance per topology
      * @return the next OffsetAndMetadata to commit, or null if no offset is
-     * ready to commit.
+     *     ready to commit.
      */
-    public OffsetAndMetadata findNextCommitOffset() {
+    public OffsetAndMetadata findNextCommitOffset(final String commitMetadata) {
         boolean found = false;
         long currOffset;
         long nextCommitOffset = committedOffset;
-        KafkaSpoutMessageId nextCommitMsg = null;     // this is a convenience variable to make it faster to create OffsetAndMetadata
 
         for (KafkaSpoutMessageId currAckedMsg : ackedMsgs) {  // complexity is that of a linear scan on a TreeMap
             currOffset = currAckedMsg.offset();
-            if (currOffset == nextCommitOffset) {            // found the next offset to commit
+            if (currOffset == nextCommitOffset) {
+                // found the next offset to commit
                 found = true;
-                nextCommitMsg = currAckedMsg;
                 nextCommitOffset = currOffset + 1;
             } else if (currOffset > nextCommitOffset) {
                 if (emittedOffsets.contains(nextCommitOffset)) {
@@ -101,7 +131,6 @@ public class OffsetManager {
                     if (nextEmittedOffset != null && currOffset == nextEmittedOffset) {
                         LOG.debug("Found committable offset: [{}] after missing offset: [{}], skipping to the committable offset",
                             currOffset, nextCommitOffset);
-                        nextCommitMsg = currAckedMsg;
                         nextCommitOffset = currOffset + 1;
                     } else {
                         LOG.debug("Topic-partition [{}] has non-sequential offset [{}]."
@@ -118,10 +147,10 @@ public class OffsetManager {
 
         OffsetAndMetadata nextCommitOffsetAndMetadata = null;
         if (found) {
-            nextCommitOffsetAndMetadata = new OffsetAndMetadata(nextCommitOffset,
-                nextCommitMsg.getMetadata(Thread.currentThread()));
+            nextCommitOffsetAndMetadata = new OffsetAndMetadata(nextCommitOffset, commitMetadata);
+
             LOG.debug("Topic-partition [{}] has offsets [{}-{}] ready to be committed."
-                + " Processing will resume at offset [{}] if the spout restarts",
+                + " Processing will resume at offset [{}] upon spout restart",
                 tp, committedOffset, nextCommitOffsetAndMetadata.offset() - 1, nextCommitOffsetAndMetadata.offset());
         } else {
             LOG.debug("Topic-partition [{}] has no offsets ready to be committed", tp);
@@ -133,18 +162,19 @@ public class OffsetManager {
     /**
      * Marks an offset as committed. This method has side effects - it sets the
      * internal state in such a way that future calls to
-     * {@link #findNextCommitOffset()} will return offsets greater than or equal to the
+     * {@link #findNextCommitOffset(String)} will return offsets greater than or equal to the
      * offset specified, if any.
      *
-     * @param committedOffset The committed offset. All lower offsets are expected to have been committed.
+     * @param committedOffsetAndMeta The committed offset. All lower offsets are expected to have been committed.
      * @return Number of offsets committed in this commit
      */
-    public long commit(OffsetAndMetadata committedOffset) {
+    public long commit(OffsetAndMetadata committedOffsetAndMeta) {
+        committed = true;
         final long preCommitCommittedOffset = this.committedOffset;
         long numCommittedOffsets = 0;
-        this.committedOffset = committedOffset.offset();
+        this.committedOffset = committedOffsetAndMeta.offset();
         for (Iterator<KafkaSpoutMessageId> iterator = ackedMsgs.iterator(); iterator.hasNext();) {
-            if (iterator.next().offset() < committedOffset.offset()) {
+            if (iterator.next().offset() < committedOffsetAndMeta.offset()) {
                 iterator.remove();
                 numCommittedOffsets++;
             } else {
@@ -153,7 +183,7 @@ public class OffsetManager {
         }
 
         for (Iterator<Long> iterator = emittedOffsets.iterator(); iterator.hasNext();) {
-            if (iterator.next() < committedOffset.offset()) {
+            if (iterator.next() < committedOffsetAndMeta.offset()) {
                 iterator.remove();
             } else {
                 break;
@@ -163,41 +193,46 @@ public class OffsetManager {
         LOG.trace("{}", this);
         
         LOG.debug("Committed [{}] offsets in the range [{}-{}] for topic-partition [{}]."
-            + " Processing will resume at [{}] if the spout restarts.",
+            + " Processing will resume at [{}] upon spout restart",
                 numCommittedOffsets, preCommitCommittedOffset, this.committedOffset - 1, tp, this.committedOffset);
         
         return numCommittedOffsets;
     }
 
-    public long getCommittedOffset() {
-        return committedOffset;
-    }
-
-    public boolean isEmpty() {
-        return ackedMsgs.isEmpty();
-    }
-
-    public boolean contains(ConsumerRecord record) {
-        return contains(new KafkaSpoutMessageId(record));
+    /**
+     * Checks if this OffsetManager has committed to Kafka.
+     *
+     * @return true if this OffsetManager has made at least one commit to Kafka, false otherwise
+     */
+    public boolean hasCommitted() {
+        return committed;
     }
 
     public boolean contains(KafkaSpoutMessageId msgId) {
         return ackedMsgs.contains(msgId);
     }
-    
-    //VisibleForTesting
+
+    @VisibleForTesting
     boolean containsEmitted(long offset) {
         return emittedOffsets.contains(offset);
     }
 
+    public long getLatestEmittedOffset() {
+        return latestEmittedOffset;
+    }
+
+    public long getCommittedOffset() {
+        return committedOffset;
+    }
+
     @Override
     public final String toString() {
         return "OffsetManager{"
             + "topic-partition=" + tp
-            + ", fetchOffset=" + initialFetchOffset
             + ", committedOffset=" + committedOffset
             + ", emittedOffsets=" + emittedOffsets
             + ", ackedMsgs=" + ackedMsgs
+            + ", latestEmittedOffset=" + latestEmittedOffset
             + '}';
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/metrics/KafkaOffsetMetric.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/metrics/KafkaOffsetMetric.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/metrics/KafkaOffsetMetric.java
new file mode 100644
index 0000000..afe8f74
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/metrics/KafkaOffsetMetric.java
@@ -0,0 +1,141 @@
+/**
+ * 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.metrics;
+
+import com.google.common.base.Supplier;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.storm.kafka.spout.internal.OffsetManager;
+import org.apache.storm.metric.api.IMetric;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This class is used compute the partition and topic level offset metrics
+ * <p>
+ * Partition level metrics are:
+ * topicName/partition_{number}/earliestTimeOffset //gives beginning offset of the partition
+ * topicName/partition_{number}/latestTimeOffset //gives end offset of the partition
+ * topicName/partition_{number}/latestEmittedOffset //gives latest emitted offset of the partition from the spout
+ * topicName/partition_{number}/latestCompletedOffset //gives latest committed offset of the partition from the spout
+ * topicName/partition_{number}/spoutLag // the delta between the latest Offset and latestCompletedOffset
+ * topicName/partition_{number}/recordsInPartition // total number of records in the partition
+ * </p>
+ * <p>
+ * Topic level metrics are:
+ * topicName/totalEarliestTimeOffset //gives the total beginning offset of all the associated partitions of this spout
+ * topicName/totalLatestTimeOffset //gives the total end offset of all the associated partitions of this spout
+ * topicName/totalLatestEmittedOffset //gives the total latest emitted offset of all the associated partitions of this spout
+ * topicName/totalLatestCompletedOffset //gives the total latest committed offset of all the associated partitions of this spout
+ * topicName/spoutLag // total spout lag of all the associated partitions of this spout
+ * topicName/totalRecordsInPartitions //total number of records in all the associated partitions of this spout
+ * </p>
+ */
+public class KafkaOffsetMetric implements IMetric {
+
+    private static final Logger LOG = LoggerFactory.getLogger(KafkaOffsetMetric.class);
+    private final Supplier<Map<TopicPartition, OffsetManager>> offsetManagerSupplier;
+    private final Supplier<KafkaConsumer> consumerSupplier;
+
+    public KafkaOffsetMetric(Supplier offsetManagerSupplier, Supplier consumerSupplier) {
+        this.offsetManagerSupplier = offsetManagerSupplier;
+        this.consumerSupplier = consumerSupplier;
+    }
+
+    @Override
+    public Object getValueAndReset() {
+
+        Map<TopicPartition, OffsetManager> offsetManagers = offsetManagerSupplier.get();
+        KafkaConsumer kafkaConsumer = consumerSupplier.get();
+
+        if (offsetManagers == null || offsetManagers.isEmpty() || kafkaConsumer == null) {
+            LOG.debug("Metrics Tick: offsetManagers or kafkaConsumer is null.");
+            return null;
+        }
+
+        Map<String,TopicMetrics> topicMetricsMap = new HashMap<>();
+        Set<TopicPartition> topicPartitions = offsetManagers.keySet();
+
+        Map<TopicPartition, Long> beginningOffsets = kafkaConsumer.beginningOffsets(topicPartitions);
+        Map<TopicPartition, Long> endOffsets = kafkaConsumer.endOffsets(topicPartitions);
+        //map to hold partition level and topic level metrics
+        Map<String, Long> result = new HashMap<>();
+
+        for (Map.Entry<TopicPartition, OffsetManager> entry : offsetManagers.entrySet()) {
+            TopicPartition topicPartition = entry.getKey();
+            OffsetManager offsetManager = entry.getValue();
+
+            long latestTimeOffset = endOffsets.get(topicPartition);
+            long earliestTimeOffset = beginningOffsets.get(topicPartition);
+
+            long latestEmittedOffset = offsetManager.getLatestEmittedOffset();
+            long latestCompletedOffset = offsetManager.getCommittedOffset();
+            long spoutLag = latestTimeOffset - latestCompletedOffset;
+            long recordsInPartition =  latestTimeOffset - earliestTimeOffset;
+
+            String metricPath = topicPartition.topic()  + "/partition_" + topicPartition.partition();
+            result.put(metricPath + "/" + "spoutLag", spoutLag);
+            result.put(metricPath + "/" + "earliestTimeOffset", earliestTimeOffset);
+            result.put(metricPath + "/" + "latestTimeOffset", latestTimeOffset);
+            result.put(metricPath + "/" + "latestEmittedOffset", latestEmittedOffset);
+            result.put(metricPath + "/" + "latestCompletedOffset", latestCompletedOffset);
+            result.put(metricPath + "/" + "recordsInPartition", recordsInPartition);
+
+            TopicMetrics topicMetrics = topicMetricsMap.get(topicPartition.topic());
+            if (topicMetrics == null) {
+                topicMetrics = new TopicMetrics();
+                topicMetricsMap.put(topicPartition.topic(), topicMetrics);
+            }
+
+            topicMetrics.totalSpoutLag += spoutLag;
+            topicMetrics.totalEarliestTimeOffset += earliestTimeOffset;
+            topicMetrics.totalLatestTimeOffset += latestTimeOffset;
+            topicMetrics.totalLatestEmittedOffset += latestEmittedOffset;
+            topicMetrics.totalLatestCompletedOffset += latestCompletedOffset;
+            topicMetrics.totalRecordsInPartitions += recordsInPartition;
+        }
+
+        for (Map.Entry<String, TopicMetrics> e : topicMetricsMap.entrySet()) {
+            String topic = e.getKey();
+            TopicMetrics topicMetrics = e.getValue();
+            result.put(topic + "/" + "totalSpoutLag", topicMetrics.totalSpoutLag);
+            result.put(topic + "/" + "totalEarliestTimeOffset", topicMetrics.totalEarliestTimeOffset);
+            result.put(topic + "/" + "totalLatestTimeOffset", topicMetrics.totalLatestTimeOffset);
+            result.put(topic + "/" + "totalLatestEmittedOffset", topicMetrics.totalLatestEmittedOffset);
+            result.put(topic + "/" + "totalLatestCompletedOffset", topicMetrics.totalLatestCompletedOffset);
+            result.put(topic + "/" + "totalRecordsInPartitions", topicMetrics.totalRecordsInPartitions);
+        }
+
+        LOG.debug("Metrics Tick: value : {}", result);
+        return result;
+    }
+
+    private class TopicMetrics {
+        long totalSpoutLag = 0;
+        long totalEarliestTimeOffset = 0;
+        long totalLatestTimeOffset = 0;
+        long totalLatestEmittedOffset = 0;
+        long totalLatestCompletedOffset = 0;
+        long totalRecordsInPartitions = 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutBatchMetadata.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutBatchMetadata.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutBatchMetadata.java
index 7079cb2..6fa81aa 100644
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutBatchMetadata.java
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutBatchMetadata.java
@@ -18,6 +18,11 @@
 
 package org.apache.storm.kafka.spout.trident;
 
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.lang.Validate;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.ConsumerRecords;
 import org.apache.kafka.common.TopicPartition;
@@ -30,37 +35,47 @@ import java.util.List;
 /**
  * Wraps transaction batch information
  */
-public class KafkaTridentSpoutBatchMetadata<K,V> implements Serializable {
+public class KafkaTridentSpoutBatchMetadata implements Serializable {
     private static final Logger LOG = LoggerFactory.getLogger(KafkaTridentSpoutBatchMetadata.class);
-
+    private static final TopicPartitionSerializer TP_SERIALIZER = new TopicPartitionSerializer();
+    
+    public static final String TOPIC_PARTITION_KEY = "tp";
+    public static final String FIRST_OFFSET_KEY = "firstOffset";
+    public static final String LAST_OFFSET_KEY = "lastOffset";
+    
     // topic partition of this batch
-    private TopicPartition topicPartition;  
+    private final TopicPartition topicPartition;  
     // first offset of this batch
-    private long firstOffset;               
+    private final long firstOffset;               
     // last offset of this batch
-    private long lastOffset;
+    private final long lastOffset;
 
+    /**
+     * Builds a metadata object.
+     * @param topicPartition The topic partition
+     * @param firstOffset The first offset for the batch
+     * @param lastOffset The last offset for the batch
+     */
     public KafkaTridentSpoutBatchMetadata(TopicPartition topicPartition, long firstOffset, long lastOffset) {
         this.topicPartition = topicPartition;
         this.firstOffset = firstOffset;
         this.lastOffset = lastOffset;
     }
 
-    public KafkaTridentSpoutBatchMetadata(TopicPartition topicPartition, ConsumerRecords<K, V> consumerRecords, KafkaTridentSpoutBatchMetadata<K, V> lastBatch) {
-        this.topicPartition = topicPartition;
-
+    /**
+     * Builds a metadata object from a non-empty set of records.
+     * @param topicPartition The topic partition the records belong to.
+     * @param consumerRecords The non-empty set of records.
+     */
+    public <K, V> KafkaTridentSpoutBatchMetadata(TopicPartition topicPartition, ConsumerRecords<K, V> consumerRecords) {
+        Validate.notNull(consumerRecords.records(topicPartition));
         List<ConsumerRecord<K, V>> records = consumerRecords.records(topicPartition);
-
-        if (records != null && !records.isEmpty()) {
-            firstOffset = records.get(0).offset();
-            lastOffset = records.get(records.size() - 1).offset();
-        } else {
-            if (lastBatch != null) {
-                firstOffset = lastBatch.firstOffset;
-                lastOffset = lastBatch.lastOffset;
-            }
-        }
-        LOG.debug("Created {}", this);
+        Validate.isTrue(!records.isEmpty(), "There must be at least one record in order to build metadata");
+        
+        this.topicPartition = topicPartition;
+        firstOffset = records.get(0).offset();
+        lastOffset = records.get(records.size() - 1).offset();
+        LOG.debug("Created {}", this.toString());
     }
 
     public long getFirstOffset() {
@@ -74,9 +89,32 @@ public class KafkaTridentSpoutBatchMetadata<K,V> implements Serializable {
     public TopicPartition getTopicPartition() {
         return topicPartition;
     }
+    
+    /**
+     * Constructs a metadata object from a Map in the format produced by {@link #toMap() }.
+     * @param map The source map
+     * @return A new metadata object
+     */
+    public static KafkaTridentSpoutBatchMetadata fromMap(Map<String, Object> map) {
+        Map<String, Object> topicPartitionMap = (Map<String, Object>)map.get(TOPIC_PARTITION_KEY);
+        TopicPartition tp = TP_SERIALIZER.fromMap(topicPartitionMap);
+        return new KafkaTridentSpoutBatchMetadata(tp, ((Number)map.get(FIRST_OFFSET_KEY)).longValue(),
+            ((Number)map.get(LAST_OFFSET_KEY)).longValue());
+    }
+    
+    /**
+     * Writes this metadata object to a Map so Trident can read/write it to Zookeeper.
+     */
+    public Map<String, Object> toMap() {
+        Map<String, Object> map = new HashMap<>();
+        map.put(TOPIC_PARTITION_KEY, TP_SERIALIZER.toMap(topicPartition));
+        map.put(FIRST_OFFSET_KEY, firstOffset);
+        map.put(LAST_OFFSET_KEY, lastOffset);
+        return map;
+    }
 
     @Override
-    public String toString() {
+    public final String toString() {
         return super.toString() +
                 "{topicPartition=" + topicPartition +
                 ", firstOffset=" + firstOffset +

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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 87af0cf..3b4aa4b 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,10 +18,11 @@
 
 package org.apache.storm.kafka.spout.trident;
 
+import java.util.HashMap;
+import java.util.Map;
 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.RecordTranslator;
@@ -48,9 +49,9 @@ import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrat
 import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_LATEST;
 
 public class KafkaTridentSpoutEmitter<K, V> implements IOpaquePartitionedTridentSpout.Emitter<
-        List<TopicPartition>,
+        List<Map<String, Object>>,
         KafkaTridentSpoutTopicPartition,
-        KafkaTridentSpoutBatchMetadata<K, V>>,
+        Map<String, Object>>,
         Serializable {
 
     private static final long serialVersionUID = -7343927794834130435L;
@@ -61,16 +62,24 @@ public class KafkaTridentSpoutEmitter<K, V> implements IOpaquePartitionedTrident
 
     // Bookkeeping
     private final KafkaTridentSpoutManager<K, V> kafkaManager;
-    private Set<TopicPartition> firstPoll = new HashSet<>();        // set of topic-partitions for which first poll has already occurred
+    // set of topic-partitions for which first poll has already occurred, and the first polled txid
+    private final Map<TopicPartition, Long> firstPollTransaction = new HashMap<>(); 
 
     // Declare some KafkaTridentSpoutManager references for convenience
     private final long pollTimeoutMs;
     private final KafkaSpoutConfig.FirstPollOffsetStrategy firstPollOffsetStrategy;
     private final RecordTranslator<K, V> translator;
     private final Timer refreshSubscriptionTimer;
+    private final TopicPartitionSerializer tpSerializer = new TopicPartitionSerializer();
 
     private TopologyContext topologyContext;
 
+    /**
+     * Create a new Kafka spout emitter.
+     * @param kafkaManager The Kafka consumer manager to use
+     * @param topologyContext The topology context
+     * @param refreshSubscriptionTimer The timer for deciding when to recheck the subscription
+     */
     public KafkaTridentSpoutEmitter(KafkaTridentSpoutManager<K, V> kafkaManager, TopologyContext topologyContext, Timer refreshSubscriptionTimer) {
         this.kafkaConsumer = kafkaManager.createAndSubscribeKafkaConsumer(topologyContext);
         this.kafkaManager = kafkaManager;
@@ -81,7 +90,7 @@ public class KafkaTridentSpoutEmitter<K, V> implements IOpaquePartitionedTrident
         final KafkaSpoutConfig<K, V> kafkaSpoutConfig = kafkaManager.getKafkaSpoutConfig();
         this.pollTimeoutMs = kafkaSpoutConfig.getPollTimeoutMs();
         this.firstPollOffsetStrategy = kafkaSpoutConfig.getFirstPollOffsetStrategy();
-        LOG.debug("Created {}", this);
+        LOG.debug("Created {}", this.toString());
     }
 
     /**
@@ -93,15 +102,16 @@ public class KafkaTridentSpoutEmitter<K, V> implements IOpaquePartitionedTrident
     }
 
     @Override
-    public KafkaTridentSpoutBatchMetadata<K, V> emitPartitionBatch(TransactionAttempt tx, TridentCollector collector,
-                                                                   KafkaTridentSpoutTopicPartition currBatchPartition, KafkaTridentSpoutBatchMetadata<K, V> lastBatch) {
+    public Map<String, Object> emitPartitionBatch(TransactionAttempt tx, TridentCollector collector,
+            KafkaTridentSpoutTopicPartition currBatchPartition, Map<String, Object> lastBatch) {
 
         LOG.debug("Processing batch: [transaction = {}], [currBatchPartition = {}], [lastBatchMetadata = {}], [collector = {}]",
                 tx, currBatchPartition, lastBatch, collector);
 
         final TopicPartition currBatchTp = currBatchPartition.getTopicPartition();
         final Set<TopicPartition> assignments = kafkaConsumer.assignment();
-        KafkaTridentSpoutBatchMetadata<K, V> currentBatch = lastBatch;
+        KafkaTridentSpoutBatchMetadata lastBatchMeta = lastBatch == null ? null : KafkaTridentSpoutBatchMetadata.fromMap(lastBatch);
+        KafkaTridentSpoutBatchMetadata currentBatch = lastBatchMeta;
         Collection<TopicPartition> pausedTopicPartitions = Collections.emptySet();
 
         if (assignments == null || !assignments.contains(currBatchPartition.getTopicPartition())) {
@@ -114,7 +124,7 @@ public class KafkaTridentSpoutEmitter<K, V> implements IOpaquePartitionedTrident
                 // pause other topic-partitions to only poll from current topic-partition
                 pausedTopicPartitions = pauseTopicPartitions(currBatchTp);
 
-                seek(currBatchTp, lastBatch);
+                seek(currBatchTp, lastBatchMeta, tx.getTransactionId());
 
                 // poll
                 if (refreshSubscriptionTimer.isExpiredResetOnTrue()) {
@@ -127,7 +137,7 @@ public class KafkaTridentSpoutEmitter<K, V> implements IOpaquePartitionedTrident
                 if (!records.isEmpty()) {
                     emitTuples(collector, records);
                     // build new metadata
-                    currentBatch = new KafkaTridentSpoutBatchMetadata<>(currBatchTp, records, lastBatch);
+                    currentBatch = new KafkaTridentSpoutBatchMetadata(currBatchTp, records);
                 }
             } finally {
                 kafkaConsumer.resume(pausedTopicPartitions);
@@ -137,7 +147,7 @@ public class KafkaTridentSpoutEmitter<K, V> implements IOpaquePartitionedTrident
                     "[currBatchMetadata = {}], [collector = {}]", tx, currBatchPartition, lastBatch, currentBatch, collector);
         }
 
-        return currentBatch;
+        return currentBatch == null ? null : currentBatch.toMap();
     }
 
     private void emitTuples(TridentCollector collector, ConsumerRecords<K, V> records) {
@@ -149,46 +159,49 @@ public class KafkaTridentSpoutEmitter<K, V> implements IOpaquePartitionedTrident
     }
 
     /**
-     * Determines the offset of the next fetch. For failed batches lastBatchMeta is not null and contains the fetch
-     * offset of the failed batch. In this scenario the next fetch will take place at offset of the failed batch + 1.
-     * When the previous batch is successful, lastBatchMeta is null, and the offset of the next fetch is either the
-     * offset of the last commit to kafka, or if no commit was yet made, the offset dictated by
-     * {@link KafkaSpoutConfig.FirstPollOffsetStrategy}
+     * Determines the offset of the next fetch. Will use the firstPollOffsetStrategy if this is the first poll for the topic partition.
+     * Otherwise the next offset will be one past the last batch, based on lastBatchMeta.
+     * 
+     * <p>lastBatchMeta should only be null when the previous txid was not emitted (e.g. new topic),
+     * it is the first poll for the spout instance, or it is a replay of the first txid this spout emitted on this partition.
+     * In the second case, there are either no previous transactions, or the MBC is still committing them
+     * and they will fail because this spout did not emit the corresponding batches. If it had emitted them, the meta could not be null. 
+     * In any case, the lastBatchMeta should never be null if this is not the first poll for this spout instance.
      *
      * @return the offset of the next fetch
      */
-    private long seek(TopicPartition tp, KafkaTridentSpoutBatchMetadata<K, V> lastBatchMeta) {
-        if (lastBatchMeta != null) {
-            kafkaConsumer.seek(tp, lastBatchMeta.getLastOffset() + 1);  // seek next offset after last offset from previous batch
-            LOG.debug("Seeking fetch offset to next offset after last offset from previous batch for topic-partition [{}]", tp);
-        } else if (isFirstPoll(tp)) {
-            LOG.debug("Seeking fetch offset from firstPollOffsetStrategy and last commit to Kafka for topic-partition [{}]", tp);
-            firstPoll.add(tp);
-            final OffsetAndMetadata committedOffset = kafkaConsumer.committed(tp);
-            if (committedOffset != null) {             // offset was committed for this TopicPartition
-                if (firstPollOffsetStrategy.equals(EARLIEST)) {
-                    kafkaConsumer.seekToBeginning(Collections.singleton(tp));
-                } else if (firstPollOffsetStrategy.equals(LATEST)) {
-                    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(Collections.singleton(tp));
-                } else if (firstPollOffsetStrategy.equals(LATEST) || firstPollOffsetStrategy.equals(UNCOMMITTED_LATEST)) {
-                    kafkaConsumer.seekToEnd(Collections.singleton(tp));
-                }
+    private long seek(TopicPartition tp, KafkaTridentSpoutBatchMetadata lastBatchMeta, long transactionId) {
+        if (isFirstPoll(tp, transactionId)) {
+            if (firstPollOffsetStrategy == EARLIEST) {
+                LOG.debug("First poll for topic partition [{}], seeking to partition beginning", tp);
+                kafkaConsumer.seekToBeginning(Collections.singleton(tp));
+            } else if (firstPollOffsetStrategy == LATEST) {
+                LOG.debug("First poll for topic partition [{}], seeking to partition end", tp);
+                kafkaConsumer.seekToEnd(Collections.singleton(tp));
+            } else if (lastBatchMeta != null) {
+                LOG.debug("First poll for topic partition [{}], using last batch metadata", tp);
+                kafkaConsumer.seek(tp, lastBatchMeta.getLastOffset() + 1);  // seek next offset after last offset from previous batch
+            } else if (firstPollOffsetStrategy == UNCOMMITTED_EARLIEST) {
+                LOG.debug("First poll for topic partition [{}] with no last batch metadata, seeking to partition beginning", tp);
+                kafkaConsumer.seekToBeginning(Collections.singleton(tp));
+            } else if (firstPollOffsetStrategy == UNCOMMITTED_LATEST) {
+                LOG.debug("First poll for topic partition [{}] with no last batch metadata, seeking to partition end", tp);
+                kafkaConsumer.seekToEnd(Collections.singleton(tp));
             }
+            firstPollTransaction.put(tp, transactionId);
+        } else {
+            kafkaConsumer.seek(tp, lastBatchMeta.getLastOffset() + 1);  // seek next offset after last offset from previous batch
+            LOG.debug("First poll for topic partition [{}], using last batch metadata", tp);
         }
+
         final long fetchOffset = kafkaConsumer.position(tp);
         LOG.debug("Set [fetchOffset = {}]", fetchOffset);
         return fetchOffset;
     }
 
-    private boolean isFirstPoll(TopicPartition tp) {
-         return !firstPoll.contains(tp);
+    private boolean isFirstPoll(TopicPartition tp, long txid) {
+        // The first poll is either the "real" first transaction, or a replay of the first transaction
+        return !firstPollTransaction.containsKey(tp) || firstPollTransaction.get(tp) == txid;
     }
 
     // returns paused topic-partitions.
@@ -215,15 +228,20 @@ public class KafkaTridentSpoutEmitter<K, V> implements IOpaquePartitionedTrident
      * @return ordered list of topic partitions for this task
      */
     @Override
-    public List<KafkaTridentSpoutTopicPartition> getOrderedPartitions(final List<TopicPartition> allPartitionInfo) {
-        final List<KafkaTridentSpoutTopicPartition> allPartitions = newKafkaTridentSpoutTopicPartitions(allPartitionInfo);
+    public List<KafkaTridentSpoutTopicPartition> getOrderedPartitions(final List<Map<String, Object>> allPartitionInfo) {
+        List<TopicPartition> allTopicPartitions = new ArrayList<>();
+        for(Map<String, Object> map : allPartitionInfo) {
+            allTopicPartitions.add(tpSerializer.fromMap(map));
+        }
+        final List<KafkaTridentSpoutTopicPartition> allPartitions = newKafkaTridentSpoutTopicPartitions(allTopicPartitions);
         LOG.debug("Returning all topic-partitions {} across all tasks. Current task index [{}]. Total tasks [{}] ",
                 allPartitions, topologyContext.getThisTaskIndex(), getNumTasks());
         return allPartitions;
     }
 
     @Override
-    public List<KafkaTridentSpoutTopicPartition> getPartitionsForTask(int taskId, int numTasks, List<TopicPartition> allPartitionInfo) {
+    public List<KafkaTridentSpoutTopicPartition> getPartitionsForTask(int taskId, int numTasks,
+        List<Map<String, Object>> allPartitionInfo) {
         final Set<TopicPartition> assignedTps = kafkaConsumer.assignment();
         LOG.debug("Consumer [{}], running on task with index [{}], has assigned topic-partitions {}", kafkaConsumer, taskId, assignedTps);
         final List<KafkaTridentSpoutTopicPartition> taskTps = newKafkaTridentSpoutTopicPartitions(assignedTps);
@@ -253,7 +271,7 @@ public class KafkaTridentSpoutEmitter<K, V> implements IOpaquePartitionedTrident
     }
 
     @Override
-    public String toString() {
+    public final String toString() {
         return super.toString() +
                 "{kafkaManager=" + kafkaManager +
                 '}';

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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 4054b49..26db5c9 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
@@ -43,15 +43,18 @@ public class KafkaTridentSpoutManager<K, V> implements Serializable {
     // Declare some KafkaSpoutConfig references for convenience
     private Fields fields;
 
+    /**
+     * Create a KafkaConsumer manager for the trident spout.
+     * @param kafkaSpoutConfig The consumer config
+     */
     public KafkaTridentSpoutManager(KafkaSpoutConfig<K, V> kafkaSpoutConfig) {
         this.kafkaSpoutConfig = kafkaSpoutConfig;
         this.fields = getFields();
-        LOG.debug("Created {}", this);
+        LOG.debug("Created {}", this.toString());
     }
 
     KafkaConsumer<K,V> createAndSubscribeKafkaConsumer(TopologyContext context) {
-        kafkaConsumer = new KafkaConsumer<>(kafkaSpoutConfig.getKafkaProps(),
-                kafkaSpoutConfig.getKeyDeserializer(), kafkaSpoutConfig.getValueDeserializer());
+        kafkaConsumer = new KafkaConsumer<>(kafkaSpoutConfig.getKafkaProps());
 
         kafkaSpoutConfig.getSubscription().subscribe(kafkaConsumer, new KafkaSpoutConsumerRebalanceListener(), context);
         return kafkaConsumer;
@@ -65,7 +68,7 @@ public class KafkaTridentSpoutManager<K, V> implements Serializable {
         return KafkaTridentSpoutTopicPartitionRegistry.INSTANCE.getTopicPartitions();
     }
 
-    Fields getFields() {
+    final Fields getFields() {
         if (fields == null) {
             RecordTranslator<K, V> translator = kafkaSpoutConfig.getTranslator();
             Fields fs = null;
@@ -89,7 +92,7 @@ public class KafkaTridentSpoutManager<K, V> implements Serializable {
     }
 
     @Override
-    public String toString() {
+    public final String toString() {
         return super.toString() +
                 "{kafkaConsumer=" + kafkaConsumer +
                 ", kafkaSpoutConfig=" + kafkaSpoutConfig +

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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 0f7f0af..ecc9219 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
@@ -18,7 +18,8 @@
 
 package org.apache.storm.kafka.spout.trident;
 
-import org.apache.kafka.common.TopicPartition;
+import java.util.List;
+import java.util.Map;
 import org.apache.storm.kafka.spout.KafkaSpoutConfig;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.trident.spout.IOpaquePartitionedTridentSpout;
@@ -26,10 +27,8 @@ import org.apache.storm.tuple.Fields;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.List;
-import java.util.Map;
-
-public class KafkaTridentSpoutOpaque<K,V> implements IOpaquePartitionedTridentSpout<List<TopicPartition>, KafkaTridentSpoutTopicPartition, KafkaTridentSpoutBatchMetadata<K,V>> {
+public class KafkaTridentSpoutOpaque<K,V> implements IOpaquePartitionedTridentSpout<List<Map<String, Object>>,
+        KafkaTridentSpoutTopicPartition, Map<String, Object>> {
     private static final long serialVersionUID = -8003272486566259640L;
 
     private static final Logger LOG = LoggerFactory.getLogger(KafkaTridentSpoutOpaque.class);
@@ -42,16 +41,17 @@ public class KafkaTridentSpoutOpaque<K,V> implements IOpaquePartitionedTridentSp
     
     public KafkaTridentSpoutOpaque(KafkaTridentSpoutManager<K, V> kafkaManager) {
         this.kafkaManager = kafkaManager;
-        LOG.debug("Created {}", this);
+        LOG.debug("Created {}", this.toString());
     }
 
     @Override
-    public Emitter<List<TopicPartition>, KafkaTridentSpoutTopicPartition, KafkaTridentSpoutBatchMetadata<K,V>> getEmitter(Map conf, TopologyContext context) {
+    public Emitter<List<Map<String, Object>>, KafkaTridentSpoutTopicPartition, Map<String, Object>> getEmitter(
+            Map conf, TopologyContext context) {
         return new KafkaTridentSpoutEmitter<>(kafkaManager, context);
     }
 
     @Override
-    public Coordinator<List<TopicPartition>> getCoordinator(Map conf, TopologyContext context) {
+    public Coordinator<List<Map<String, Object>>> getCoordinator(Map conf, TopologyContext context) {
         return new KafkaTridentSpoutOpaqueCoordinator<>(kafkaManager);
     }
 
@@ -68,7 +68,7 @@ public class KafkaTridentSpoutOpaque<K,V> implements IOpaquePartitionedTridentSp
     }
 
     @Override
-    public String toString() {
+    public final String toString() {
         return super.toString() +
                 "{kafkaManager=" + kafkaManager + '}';
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaqueCoordinator.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaqueCoordinator.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaqueCoordinator.java
index 7898b6e..449e24b 100644
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaqueCoordinator.java
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaqueCoordinator.java
@@ -18,23 +18,25 @@
 
 package org.apache.storm.kafka.spout.trident;
 
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.storm.trident.spout.IOpaquePartitionedTridentSpout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-public class KafkaTridentSpoutOpaqueCoordinator<K,V> implements IOpaquePartitionedTridentSpout.Coordinator<List<TopicPartition>>, Serializable {
+public class KafkaTridentSpoutOpaqueCoordinator<K,V> implements IOpaquePartitionedTridentSpout.Coordinator<List<Map<String, Object>>>,
+        Serializable {
     private static final Logger LOG = LoggerFactory.getLogger(KafkaTridentSpoutOpaqueCoordinator.class);
 
-    private KafkaTridentSpoutManager<K,V> kafkaManager;
+    private final TopicPartitionSerializer tpSerializer = new TopicPartitionSerializer();
+    private final KafkaTridentSpoutManager<K,V> kafkaManager;
 
     public KafkaTridentSpoutOpaqueCoordinator(KafkaTridentSpoutManager<K, V> kafkaManager) {
         this.kafkaManager = kafkaManager;
-        LOG.debug("Created {}", this);
+        LOG.debug("Created {}", this.toString());
     }
 
     @Override
@@ -44,10 +46,14 @@ public class KafkaTridentSpoutOpaqueCoordinator<K,V> implements IOpaquePartition
     }
 
     @Override
-    public List<TopicPartition> getPartitionsForBatch() {
+    public List<Map<String, Object>> getPartitionsForBatch() {
         final ArrayList<TopicPartition> topicPartitions = new ArrayList<>(kafkaManager.getTopicPartitions());
         LOG.debug("TopicPartitions for batch {}", topicPartitions);
-        return topicPartitions;
+        List<Map<String, Object>> tps = new ArrayList<>();
+        for(TopicPartition tp : topicPartitions) {
+            tps.add(tpSerializer.toMap(tp));
+        }
+        return tps;
     }
 
     @Override
@@ -56,7 +62,7 @@ public class KafkaTridentSpoutOpaqueCoordinator<K,V> implements IOpaquePartition
     }
 
     @Override
-    public String toString() {
+    public final String toString() {
         return super.toString() +
                 "{kafkaManager=" + kafkaManager +
                 '}';

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/TopicPartitionSerializer.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/TopicPartitionSerializer.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/TopicPartitionSerializer.java
new file mode 100644
index 0000000..50e78f0
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/TopicPartitionSerializer.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2017 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.trident;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.kafka.common.TopicPartition;
+
+public class TopicPartitionSerializer {
+
+    public static final String TOPIC_PARTITION_TOPIC_KEY = "topic";
+    public static final String TOPIC_PARTITION_PARTITION_KEY = "partition";
+
+    /**
+     * Serializes the given TopicPartition to Map so Trident can serialize it to JSON.
+     */
+    public Map<String, Object> toMap(TopicPartition topicPartition) {
+        Map<String, Object> topicPartitionMap = new HashMap<>();
+        topicPartitionMap.put(TOPIC_PARTITION_TOPIC_KEY, topicPartition.topic());
+        topicPartitionMap.put(TOPIC_PARTITION_PARTITION_KEY, topicPartition.partition());
+        return topicPartitionMap;
+    }
+
+    /**
+     * Deserializes the given map into a TopicPartition. The map keys are expected to be those produced by
+     * {@link #toMap(org.apache.kafka.common.TopicPartition)}.
+     */
+    public TopicPartition fromMap(Map<String, Object> map) {
+        return new TopicPartition((String) map.get(TOPIC_PARTITION_TOPIC_KEY),
+            ((Number) map.get(TOPIC_PARTITION_PARTITION_KEY)).intValue());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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
index 76e0f02..2e2c13b 100644
--- 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
@@ -37,21 +37,20 @@ import java.util.Properties;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 
-public class TridentKafkaState implements State {
+public class TridentKafkaState<K, V> implements State {
     private static final Logger LOG = LoggerFactory.getLogger(TridentKafkaState.class);
 
-    private KafkaProducer producer;
-    private OutputCollector collector;
+    private KafkaProducer<K, V> producer;
 
-    private TridentTupleToKafkaMapper mapper;
+    private TridentTupleToKafkaMapper<K, V> mapper;
     private KafkaTopicSelector topicSelector;
 
-    public TridentKafkaState withTridentTupleToKafkaMapper(TridentTupleToKafkaMapper mapper) {
+    public TridentKafkaState<K, V> withTridentTupleToKafkaMapper(TridentTupleToKafkaMapper<K, V> mapper) {
         this.mapper = mapper;
         return this;
     }
 
-    public TridentKafkaState withKafkaTopicSelector(KafkaTopicSelector selector) {
+    public TridentKafkaState<K, V> withKafkaTopicSelector(KafkaTopicSelector selector) {
         this.topicSelector = selector;
         return this;
     }
@@ -66,36 +65,45 @@ public class TridentKafkaState implements State {
         LOG.debug("commit is Noop.");
     }
 
+    /**
+     * Prepare this State.
+     * @param options The KafkaProducer config.
+     */
     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);
+        producer = new KafkaProducer<>(options);
     }
 
+    /**
+     * Write the given tuples to Kafka.
+     * @param tuples The tuples to write.
+     * @param collector The Trident collector.
+     */
     public void updateState(List<TridentTuple> tuples, TridentCollector collector) {
         String topic = null;
         try {
             long startTime = System.currentTimeMillis();
-	     int numberOfRecords = tuples.size();
-	     List<Future<RecordMetadata>> futures = new ArrayList<>(numberOfRecords);
+            int numberOfRecords = tuples.size();
+            List<Future<RecordMetadata>> futures = new ArrayList<>(numberOfRecords);
             for (TridentTuple tuple : tuples) {
                 topic = topicSelector.getTopic(tuple);
-                Object messageFromTuple = mapper.getMessageFromTuple(tuple);
-		 Object keyFromTuple = mapper.getKeyFromTuple(tuple);
-				
+                V messageFromTuple = mapper.getMessageFromTuple(tuple);
+                K keyFromTuple = mapper.getKeyFromTuple(tuple);
+
                 if (topic != null) {
-                   if (messageFromTuple != null) {
-		      Future<RecordMetadata> result = producer.send(new ProducerRecord(topic,keyFromTuple, messageFromTuple));
-		      futures.add(result);
-		   } else {
-		      LOG.warn("skipping Message with Key "+ keyFromTuple +" as message was null");
-		   }
-			
+                    if (messageFromTuple != null) {
+                        Future<RecordMetadata> result = producer.send(new ProducerRecord<>(topic, keyFromTuple, messageFromTuple));
+                        futures.add(result);
+                    } else {
+                        LOG.warn("skipping Message with Key {} as message was null", keyFromTuple);
+                    }
+
                 } else {
-                      LOG.warn("skipping key = " + keyFromTuple + ", topic selector returned null.");
+                    LOG.warn("skipping key = {}, topic selector returned null.", keyFromTuple);
                 }
             }
-            
+
             int emittedRecords = futures.size();
             List<ExecutionException> exceptions = new ArrayList<>(emittedRecords);
             for (Future<RecordMetadata> future : futures) {
@@ -106,20 +114,21 @@ public class TridentKafkaState implements State {
                 }
             }
 
-            if (exceptions.size() > 0){
-		StringBuilder errorMsg = new StringBuilder("Could not retrieve result for messages " + tuples + " from topic = " + topic 
-				+ " because of the following exceptions:" + System.lineSeparator());
-				
-		for (ExecutionException exception : exceptions) {
-			errorMsg = errorMsg.append(exception.getMessage()).append(System.lineSeparator()); ;
-		}
-		String message = errorMsg.toString();
-		LOG.error(message);
-		throw new FailedException(message);
-	    }
-	    long latestTime = System.currentTimeMillis();
-	    LOG.info("Emitted record {} sucessfully in {} ms to topic {} ", emittedRecords, latestTime-startTime, topic);
-			
+            if (exceptions.size() > 0) {
+                StringBuilder errorMsg = new StringBuilder("Could not retrieve result for messages ");
+                errorMsg.append(tuples).append(" from topic = ").append(topic)
+                        .append(" because of the following exceptions:").append(System.lineSeparator());
+
+                for (ExecutionException exception : exceptions) {
+                    errorMsg = errorMsg.append(exception.getMessage()).append(System.lineSeparator());
+                }
+                String message = errorMsg.toString();
+                LOG.error(message);
+                throw new FailedException(message);
+            }
+            long latestTime = System.currentTimeMillis();
+            LOG.info("Emitted record {} sucessfully in {} ms to topic {} ", emittedRecords, latestTime - startTime, topic);
+
         } catch (Exception ex) {
             String errorMsg = "Could not send messages " + tuples + " to topic = " + topic;
             LOG.warn(errorMsg, ex);

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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
index f564510..35620de 100644
--- 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
@@ -28,25 +28,26 @@ import org.apache.storm.trident.state.StateFactory;
 import java.util.Map;
 import java.util.Properties;
 
-public class TridentKafkaStateFactory implements StateFactory {
+public class TridentKafkaStateFactory<K, V> implements StateFactory {
 
+    private static final long serialVersionUID = -3613240970062343385L;
     private static final Logger LOG = LoggerFactory.getLogger(TridentKafkaStateFactory.class);
 
-    private TridentTupleToKafkaMapper mapper;
+    private TridentTupleToKafkaMapper<K, V> mapper;
     private KafkaTopicSelector topicSelector;
     private Properties producerProperties = new Properties();
 
-    public TridentKafkaStateFactory withTridentTupleToKafkaMapper(TridentTupleToKafkaMapper mapper) {
+    public TridentKafkaStateFactory<K, V> withTridentTupleToKafkaMapper(TridentTupleToKafkaMapper<K, V> mapper) {
         this.mapper = mapper;
         return this;
     }
 
-    public TridentKafkaStateFactory withKafkaTopicSelector(KafkaTopicSelector selector) {
+    public TridentKafkaStateFactory<K, V> withKafkaTopicSelector(KafkaTopicSelector selector) {
         this.topicSelector = selector;
         return this;
     }
 
-    public TridentKafkaStateFactory withProducerProperties(Properties props) {
+    public TridentKafkaStateFactory<K, V> withProducerProperties(Properties props) {
         this.producerProperties = props;
         return this;
     }
@@ -54,9 +55,9 @@ public class TridentKafkaStateFactory implements StateFactory {
     @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);
+        TridentKafkaState<K, V> state = new TridentKafkaState<>();
+        state.withKafkaTopicSelector(this.topicSelector)
+            .withTridentTupleToKafkaMapper(this.mapper);
         state.prepare(producerProperties);
         return state;
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaStateUpdater.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaStateUpdater.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaStateUpdater.java
index 89535c6..19e3d33 100644
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaStateUpdater.java
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/trident/TridentKafkaStateUpdater.java
@@ -23,10 +23,12 @@ import org.apache.storm.trident.operation.TridentCollector;
 import org.apache.storm.trident.state.BaseStateUpdater;
 import org.apache.storm.trident.tuple.TridentTuple;
 
-public class TridentKafkaStateUpdater extends BaseStateUpdater<TridentKafkaState> {
+public class TridentKafkaStateUpdater<K, V> extends BaseStateUpdater<TridentKafkaState<K, V>> {
+
+    private static final long serialVersionUID = 3352659585225274402L;
 
     @Override
-    public void updateState(TridentKafkaState state, List<TridentTuple> tuples, TridentCollector collector) {
+    public void updateState(TridentKafkaState<K, V> state, List<TridentTuple> tuples, TridentCollector collector) {
         state.updateState(tuples, collector);
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutAbstractTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutAbstractTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutAbstractTest.java
new file mode 100644
index 0000000..0467383
--- /dev/null
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutAbstractTest.java
@@ -0,0 +1,179 @@
+/*
+ * 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.KafkaConsumer;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.storm.kafka.KafkaUnitRule;
+import org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
+import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
+import org.apache.storm.kafka.spout.internal.KafkaConsumerFactoryDefault;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Time;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.MockitoAnnotations;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+
+public abstract class KafkaSpoutAbstractTest {
+    @Rule
+    public KafkaUnitRule kafkaUnitRule = new KafkaUnitRule();
+
+    final TopologyContext topologyContext = mock(TopologyContext.class);
+    final Map<String, Object> conf = new HashMap<>();
+    final SpoutOutputCollector collectorMock = mock(SpoutOutputCollector.class);
+    final long commitOffsetPeriodMs;
+
+    KafkaConsumer<String, String> consumerSpy;
+    KafkaSpout<String, String> spout;
+
+    @Captor
+    ArgumentCaptor<Map<TopicPartition, OffsetAndMetadata>> commitCapture;
+    private Time.SimulatedTime simulatedTime;
+    private KafkaSpoutConfig<String, String> spoutConfig;
+
+    /**
+     * This constructor should be called by the subclass' default constructor with the desired value
+     * @param commitOffsetPeriodMs commit offset period to be used in commit and verification of messages committed
+     */
+    protected KafkaSpoutAbstractTest(long commitOffsetPeriodMs) {
+        this.commitOffsetPeriodMs = commitOffsetPeriodMs;
+    }
+
+    @Before
+    public void setUp() {
+        MockitoAnnotations.initMocks(this);
+
+        spoutConfig = createSpoutConfig();
+
+        consumerSpy = createConsumerSpy();
+
+        spout = new KafkaSpout<>(spoutConfig, createConsumerFactory());
+
+        simulatedTime = new Time.SimulatedTime();
+    }
+
+    private KafkaConsumerFactory<String, String> createConsumerFactory() {
+
+        return new KafkaConsumerFactory<String, String>() {
+            @Override
+            public KafkaConsumer<String, String> createConsumer(KafkaSpoutConfig<String, String> kafkaSpoutConfig) {
+                return consumerSpy;
+            }
+
+        };
+    }
+
+    KafkaConsumer<String, String> createConsumerSpy() {
+        return spy(new KafkaConsumerFactoryDefault<String, String>().createConsumer(spoutConfig));
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        simulatedTime.close();
+    }
+
+    abstract KafkaSpoutConfig<String, String> createSpoutConfig();
+
+    void prepareSpout(int messageCount) throws Exception {
+        SingleTopicKafkaUnitSetupHelper.populateTopicData(kafkaUnitRule.getKafkaUnit(), SingleTopicKafkaSpoutConfiguration.TOPIC, messageCount);
+        SingleTopicKafkaUnitSetupHelper.initializeSpout(spout, conf, topologyContext, collectorMock);
+    }
+
+    /**
+     * Helper method to in sequence do:
+     * <li>
+     *     <ul>spout.nexTuple()</ul>
+     *     <ul>verify messageId</ul>
+     *     <ul>spout.ack(msgId)</ul>
+     *     <ul>reset(collector) to be able to reuse mock</ul>
+     * </li>
+     *
+     * @param offset offset of message to be verified
+     * @return {@link ArgumentCaptor} of the messageId verified
+     */
+    ArgumentCaptor<Object> nextTuple_verifyEmitted_ack_resetCollector(int offset) {
+        spout.nextTuple();
+
+        ArgumentCaptor<Object> messageId = verifyMessageEmitted(offset);
+
+        spout.ack(messageId.getValue());
+
+        reset(collectorMock);
+
+        return messageId;
+    }
+
+    // offset and messageId are used interchangeably
+    ArgumentCaptor<Object> verifyMessageEmitted(int offset) {
+        final ArgumentCaptor<Object> messageId = ArgumentCaptor.forClass(Object.class);
+
+        verify(collectorMock).emit(
+            eq(SingleTopicKafkaSpoutConfiguration.STREAM),
+            eq(new Values(SingleTopicKafkaSpoutConfiguration.TOPIC,
+                Integer.toString(offset),
+                Integer.toString(offset))),
+            messageId.capture());
+
+        return messageId;
+    }
+
+    void commitAndVerifyAllMessagesCommitted(long msgCount) {
+        // reset commit timer such that commit happens on next call to nextTuple()
+        Time.advanceTime(commitOffsetPeriodMs + KafkaSpout.TIMER_DELAY_MS);
+
+        //Commit offsets
+        spout.nextTuple();
+
+        verifyAllMessagesCommitted(msgCount);
+    }
+
+    /*
+     * Asserts that commitSync has been called once,
+     * that there are only commits on one topic,
+     * and that the committed offset covers messageCount messages
+     */
+    void verifyAllMessagesCommitted(long messageCount) {
+        verify(consumerSpy).commitSync(commitCapture.capture());
+
+        final Map<TopicPartition, OffsetAndMetadata> commits = commitCapture.getValue();
+        assertThat("Expected commits for only one topic partition", commits.entrySet().size(), is(1));
+
+        OffsetAndMetadata offset = commits.entrySet().iterator().next().getValue();
+        assertThat("Expected committed offset to cover all emitted messages", offset.offset(), is(messageCount));
+
+        reset(consumerSpy);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutCommitTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutCommitTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutCommitTest.java
deleted file mode 100644
index 3981a9a..0000000
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutCommitTest.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Copyright 2017 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 static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfigBuilder;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.*;
-import static org.mockito.Mockito.*;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.kafka.clients.consumer.*;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
-import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
-import org.apache.storm.spout.SpoutOutputCollector;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.utils.Time;
-import org.apache.storm.utils.Time.SimulatedTime;
-import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-import org.mockito.Captor;
-import org.mockito.InOrder;
-import org.mockito.MockitoAnnotations;
-
-public class KafkaSpoutCommitTest {
-
-    private final long offsetCommitPeriodMs = 2_000;
-    private final TopologyContext contextMock = mock(TopologyContext.class);
-    private final SpoutOutputCollector collectorMock = mock(SpoutOutputCollector.class);
-    private final Map<String, Object> conf = new HashMap<>();
-    private final TopicPartition partition = new TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 1);
-    private KafkaConsumer<String, String> consumerMock;
-    private KafkaSpout<String, String> spout;
-    private KafkaSpoutConfig spoutConfig;
-
-    @Captor
-    private ArgumentCaptor<Map<TopicPartition, OffsetAndMetadata>> commitCapture;
-
-    private void setupSpout(Set<TopicPartition> assignedPartitions) {
-        MockitoAnnotations.initMocks(this);
-        spoutConfig = getKafkaSpoutConfigBuilder(-1)
-                .setOffsetCommitPeriodMs(offsetCommitPeriodMs)
-                .build();
-
-        consumerMock = mock(KafkaConsumer.class);
-        KafkaConsumerFactory<String, String> consumerFactory = new KafkaConsumerFactory<String, String>() {
-            @Override
-            public KafkaConsumer<String, String> createConsumer(KafkaSpoutConfig<String, String> kafkaSpoutConfig) {
-                return consumerMock;
-            }
-        };
-
-        //Set up a spout listening to 1 topic partition
-        spout = new KafkaSpout<>(spoutConfig, consumerFactory);
-
-        spout.open(conf, contextMock, collectorMock);
-        spout.activate();
-
-        ArgumentCaptor<ConsumerRebalanceListener> rebalanceListenerCapture = ArgumentCaptor.forClass(ConsumerRebalanceListener.class);
-        verify(consumerMock).subscribe(anyCollection(), rebalanceListenerCapture.capture());
-
-        //Assign partitions to the spout
-        ConsumerRebalanceListener consumerRebalanceListener = rebalanceListenerCapture.getValue();
-        consumerRebalanceListener.onPartitionsAssigned(assignedPartitions);
-    }
-
-    @Test
-    public void testCommitSuccessWithOffsetVoids() {
-        //Verify that the commit logic can handle offset voids
-        try (SimulatedTime simulatedTime = new SimulatedTime()) {
-            setupSpout(Collections.singleton(partition));
-            Map<TopicPartition, List<ConsumerRecord<String, String>>> records = new HashMap<>();
-            List<ConsumerRecord<String, String>> recordsForPartition = new ArrayList<>();
-            // Offsets emitted are 0,1,2,3,4,<void>,8,9
-            for (int i = 0; i < 5; i++) {
-                recordsForPartition.add(new ConsumerRecord(partition.topic(), partition.partition(), i, "key", "value"));
-            }
-            for (int i = 8; i < 10; i++) {
-                recordsForPartition.add(new ConsumerRecord(partition.topic(), partition.partition(), i, "key", "value"));
-            }
-            records.put(partition, recordsForPartition);
-
-            when(consumerMock.poll(anyLong()))
-                    .thenReturn(new ConsumerRecords(records));
-
-            for (int i = 0; i < recordsForPartition.size(); i++) {
-                spout.nextTuple();
-            }
-
-            ArgumentCaptor<KafkaSpoutMessageId> messageIds = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
-            verify(collectorMock, times(recordsForPartition.size())).emit(anyString(), anyList(), messageIds.capture());
-
-            for (KafkaSpoutMessageId messageId : messageIds.getAllValues()) {
-                spout.ack(messageId);
-            }
-
-            // Advance time and then trigger first call to kafka consumer commit; the commit must progress to offset 9
-            Time.advanceTime(KafkaSpout.TIMER_DELAY_MS + offsetCommitPeriodMs);
-            Map<TopicPartition, List<ConsumerRecord<String, String>>> emptyConsumerRecords = Collections.emptyMap();
-            when(consumerMock.poll(anyLong()))
-                    .thenReturn(new ConsumerRecords<>(emptyConsumerRecords));
-            spout.nextTuple();
-
-            InOrder inOrder = inOrder(consumerMock);
-            inOrder.verify(consumerMock).commitSync(commitCapture.capture());
-            inOrder.verify(consumerMock).poll(anyLong());
-
-            //verify that Offset 10 was last committed offset, since this is the offset the spout should resume at
-            Map<TopicPartition, OffsetAndMetadata> commits = commitCapture.getValue();
-            assertTrue(commits.containsKey(partition));
-            assertEquals(10, commits.get(partition).offset());
-        }
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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
index 9f62b90..90e906b 100644
--- 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
@@ -17,17 +17,30 @@
  */
 package org.apache.storm.kafka.spout;
 
-import org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy;
-import org.junit.Test;
-
-import java.util.HashMap;
-
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.nullValue;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy;
+import org.hamcrest.CoreMatchers;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
 public class KafkaSpoutConfigTest {
 
+    @Rule
+    public ExpectedException expectedException = ExpectedException.none();
+    
     @Test
     public void testBasic() {
         KafkaSpoutConfig<String, String> conf = KafkaSpoutConfig.builder("localhost:1234", "topic").build();
@@ -35,17 +48,194 @@ public class KafkaSpoutConfigTest {
         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");
+        expected.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:1234");
+        expected.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false);
+        expected.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+        expected.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
+        expected.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
         assertEquals(expected, conf.getKafkaProps());
+        assertEquals(KafkaSpoutConfig.DEFAULT_METRICS_TIME_BUCKET_SIZE_SECONDS, conf.getMetricsTimeBucketSizeInSecs());
     }
 
     @Test
-    public void test_setEmitNullTuples_true_true() {
+    public void testSetEmitNullTuplesToTrue() {
         final KafkaSpoutConfig<String, String> conf = KafkaSpoutConfig.builder("localhost:1234", "topic")
                 .setEmitNullTuples(true)
                 .build();
 
         assertTrue("Failed to set emit null tuples to true", conf.isEmitNullTuples());
     }
+    
+    @Test
+    public void testShouldNotChangeAutoOffsetResetPolicyWhenNotUsingAtLeastOnce() {
+        KafkaSpoutConfig<String, String> conf = KafkaSpoutConfig.builder("localhost:1234", "topic")
+            .setProcessingGuarantee(KafkaSpoutConfig.ProcessingGuarantee.AT_MOST_ONCE)
+            .build();
+        
+        assertThat("When at-least-once is not specified, the spout should use the Kafka default auto offset reset policy",
+            conf.getKafkaProps().get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG), nullValue());
+    }
+    
+    @Test
+    public void testWillRespectExplicitAutoOffsetResetPolicy() {
+        KafkaSpoutConfig<String, String> conf = KafkaSpoutConfig.builder("localhost:1234", "topic")
+            .setProp(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none")
+            .build();
+        
+        assertThat("Should allow users to pick a different auto offset reset policy than the one recommended for the at-least-once processing guarantee",
+            (String)conf.getKafkaProps().get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG), is("none"));
+    }
+    
+    @Test
+    public void testCanConfigureWithExplicitTrueBooleanAutoCommitMode() {
+        /*
+         * Since adding setProcessingGuarantee to KafkaSpoutConfig we don't want users to set "enable.auto.commit" in the consumer config,
+         * because setting the processing guarantee will do it automatically. For backward compatibility we need to be able to handle the 
+         * property being set anyway for a few releases, and try to set a processing guarantee that corresponds to the property.
+         */
+        
+        KafkaSpoutConfig<String, String> conf = KafkaSpoutConfig.builder("localhost:1234", "topic")
+            .setProp(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, true)
+            .build();
+        
+        assertThat("When setting enable auto commit to true explicitly the spout should use the 'none' processing guarantee",
+            conf.getProcessingGuarantee(), is(KafkaSpoutConfig.ProcessingGuarantee.NO_GUARANTEE));
+    }
+    
+    @Test
+    public void testCanConfigureWithExplicitFalseBooleanAutoCommitMode() {
+        KafkaSpoutConfig<String, String> conf = KafkaSpoutConfig.builder("localhost:1234", "topic")
+            .setProp(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false)
+            .build();
+        
+        assertThat("When setting enable auto commit to false explicitly the spout should use the 'at-least-once' processing guarantee",
+            conf.getProcessingGuarantee(), is(KafkaSpoutConfig.ProcessingGuarantee.AT_LEAST_ONCE));
+    }
+    
+    @Test
+    public void testCanConfigureWithExplicitTrueStringAutoCommitMode() {
+        KafkaSpoutConfig<String, String> conf = KafkaSpoutConfig.builder("localhost:1234", "topic")
+            .setProp(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true")
+            .build();
+        
+        assertThat("When setting enable auto commit to true explicitly the spout should use the 'none' processing guarantee",
+            conf.getProcessingGuarantee(), is(KafkaSpoutConfig.ProcessingGuarantee.NO_GUARANTEE));
+    }
+    
+    @Test
+    public void testCanConfigureWithExplicitFalseStringAutoCommitMode() {
+        KafkaSpoutConfig<String, String> conf = KafkaSpoutConfig.builder("localhost:1234", "topic")
+            .setProp(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false")
+            .build();
+        
+        assertThat("When setting enable auto commit explicitly to false the spout should use the 'at-least-once' processing guarantee",
+            conf.getProcessingGuarantee(), is(KafkaSpoutConfig.ProcessingGuarantee.AT_LEAST_ONCE));
+    }
+    
+    @Test
+    public void testCanGetKeyDeserializerWhenUsingDefaultBuilder() {
+        KafkaSpoutConfig<String, String> conf = KafkaSpoutConfig.builder("localhost:1234", "topic")
+            .build();
+        
+        assertThat("When using the default builder methods, the key deserializer should default to StringDeserializer",
+            conf.getKeyDeserializer(), instanceOf(StringDeserializer.class));
+    }
+    
+    @Test
+    public void testCanGetValueDeserializerWhenUsingDefaultBuilder() {
+        KafkaSpoutConfig<String, String> conf = KafkaSpoutConfig.builder("localhost:1234", "topic")
+            .build();
+        
+        assertThat("When using the default builder methods, the value deserializer should default to StringDeserializer",
+            conf.getValueDeserializer(), instanceOf(StringDeserializer.class));
+    }
+    
+    @Test
+    public void testCanOverrideDeprecatedDeserializerClassWithKafkaProps() {
+        KafkaSpoutConfig<String, String> conf = KafkaSpoutConfig.builder("localhost:1234", "topic")
+            .setKey(StringDeserializer.class)
+            .setValue(StringDeserializer.class)
+            .setProp(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class)
+            .setProp(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class)
+            .build();
+        
+        assertThat("The last set key deserializer should be used, regardless of how it is set",
+            conf.getKafkaProps().get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG), CoreMatchers.<Object>equalTo(ByteArrayDeserializer.class));
+        assertThat("The last set value deserializer should be used, regardless of how it is set",
+            conf.getKafkaProps().get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG), CoreMatchers.<Object>equalTo(ByteArrayDeserializer.class));
+    }
+    
+    private static class SerializableStringDeserializer implements SerializableDeserializer {
+
+        private final StringDeserializer delegate = new StringDeserializer();
+
+        @Override
+        public void configure(Map configs, boolean isKey) {
+            delegate.configure(configs, isKey);
+        }
+
+        @Override
+        public Object deserialize(String topic, byte[] data) {
+            return delegate.deserialize(topic, data);
+        }
+
+        @Override
+        public void close() {
+            delegate.close();
+        }
+    }
+    
+    @Test
+    public void testCanOverrideDeprecatedDeserializerInstanceWithKafkaProps() {
+        KafkaSpoutConfig<String, String> conf = KafkaSpoutConfig.builder("localhost:1234", "topic")
+            .setKey(new SerializableStringDeserializer())
+            .setValue(new SerializableStringDeserializer())
+            .setProp(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class)
+            .setProp(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class)
+            .build();
+        
+        assertThat("The last set key deserializer should be used, regardless of how it is set",
+            conf.getKafkaProps().get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG), CoreMatchers.<Object>equalTo(ByteArrayDeserializer.class));
+        assertThat("The last set value deserializer should be used, regardless of how it is set",
+            conf.getKafkaProps().get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG), CoreMatchers.<Object>equalTo(ByteArrayDeserializer.class));
+    }
+    
+    @Test
+    public void testCanOverrideKafkaPropsWithDeprecatedDeserializerSetter() {
+        KafkaSpoutConfig<String, String> conf = KafkaSpoutConfig.builder("localhost:1234", "topic")
+            .setProp(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class)
+            .setProp(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class)
+            .setKey(new SerializableStringDeserializer())
+            .setValue(new SerializableStringDeserializer())
+            .build();
+        
+        assertThat("The last set key deserializer should be used, regardless of how it is set",
+            conf.getKafkaProps().get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG), CoreMatchers.<Object>equalTo(SerializableStringDeserializer.class));
+        assertThat("The last set value deserializer should be used, regardless of how it is set",
+            conf.getKafkaProps().get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG), CoreMatchers.<Object>equalTo(SerializableStringDeserializer.class));
+    }
+    
+    @Test
+    public void testCanMixOldAndNewDeserializerSetter() {
+        KafkaSpoutConfig<String, String> conf = KafkaSpoutConfig.builder("localhost:1234", "topic")
+            .setProp(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class)
+            .setKey(new SerializableStringDeserializer())
+            .setProp(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class)
+            .setValue(new SerializableStringDeserializer())
+            .build();
+        
+        assertThat("The last set key deserializer should be used, regardless of how it is set",
+            conf.getKafkaProps().get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG), CoreMatchers.<Object>equalTo(ByteArrayDeserializer.class));
+        assertThat("The last set value deserializer should be used, regardless of how it is set",
+            conf.getKafkaProps().get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG), CoreMatchers.<Object>equalTo(SerializableStringDeserializer.class));
+    }
+
+    @Test
+    public void testMetricsTimeBucketSizeInSecs() {
+        KafkaSpoutConfig<String, String> conf = KafkaSpoutConfig.builder("localhost:1234", "topic")
+             .setMetricsTimeBucketSizeInSecs(100)
+            .build();
+
+        assertEquals(100, conf.getMetricsTimeBucketSizeInSecs());
+    }
 }


[2/7] storm git commit: STORM-2936 Overwrite latest storm-kafka-client 1.x-branch into 1.1.x-branch

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutSingleTopicTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutSingleTopicTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutSingleTopicTest.java
new file mode 100644
index 0000000..0bf9219
--- /dev/null
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutSingleTopicTest.java
@@ -0,0 +1,379 @@
+/*
+ * 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.OffsetAndMetadata;
+import org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
+import org.apache.storm.tuple.Values;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.util.Map;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Matchers.anyListOf;
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.storm.utils.Time;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyList;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.verify;
+import static org.junit.Assert.assertEquals;
+
+import java.util.regex.Pattern;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.hamcrest.Matchers;
+
+public class KafkaSpoutSingleTopicTest extends KafkaSpoutAbstractTest {
+    private final int maxPollRecords = 10;
+    private final int maxRetries = 3;
+
+    public KafkaSpoutSingleTopicTest() {
+        super(2_000);
+    }
+
+    @Override
+    KafkaSpoutConfig<String, String> createSpoutConfig() {
+        return SingleTopicKafkaSpoutConfiguration.setCommonSpoutConfig(
+            KafkaSpoutConfig.builder("127.0.0.1:" + kafkaUnitRule.getKafkaUnit().getKafkaPort(),
+                Pattern.compile(SingleTopicKafkaSpoutConfiguration.TOPIC)))
+            .setOffsetCommitPeriodMs(commitOffsetPeriodMs)
+            .setRetry(new KafkaSpoutRetryExponentialBackoff(KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(0), KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(0),
+                maxRetries, KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(0)))
+            .setProp(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords)
+            .build();
+    }
+
+    @Test
+    public void testSeekToCommittedOffsetIfConsumerPositionIsBehindWhenCommitting() throws Exception {
+        final int messageCount = maxPollRecords * 2;
+        prepareSpout(messageCount);
+
+        //Emit all messages and fail the first one while acking the rest
+        for (int i = 0; i < messageCount; i++) {
+            spout.nextTuple();
+        }
+        ArgumentCaptor<KafkaSpoutMessageId> messageIdCaptor = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
+        verify(collectorMock, times(messageCount)).emit(anyString(), anyList(), messageIdCaptor.capture());
+        List<KafkaSpoutMessageId> messageIds = messageIdCaptor.getAllValues();
+        for (int i = 1; i < messageIds.size(); i++) {
+            spout.ack(messageIds.get(i));
+        }
+        KafkaSpoutMessageId failedTuple = messageIds.get(0);
+        spout.fail(failedTuple);
+
+        //Advance the time and replay the failed tuple. 
+        reset(collectorMock);
+        spout.nextTuple();
+        ArgumentCaptor<KafkaSpoutMessageId> failedIdReplayCaptor = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
+        verify(collectorMock).emit(anyString(), anyList(), failedIdReplayCaptor.capture());
+
+        assertThat("Expected replay of failed tuple", failedIdReplayCaptor.getValue(), is(failedTuple));
+
+        /* Ack the tuple, and commit.
+         * Since the tuple is more than max poll records behind the most recent emitted tuple, the consumer won't catch up in this poll.
+         */
+        reset(collectorMock);
+        Time.advanceTime(KafkaSpout.TIMER_DELAY_MS + commitOffsetPeriodMs);
+        spout.ack(failedIdReplayCaptor.getValue());
+        spout.nextTuple();
+        verify(consumerSpy).commitSync(commitCapture.capture());
+
+        Map<TopicPartition, OffsetAndMetadata> capturedCommit = commitCapture.getValue();
+        TopicPartition expectedTp = new TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 0);
+        assertThat("Should have committed to the right topic", capturedCommit, Matchers.hasKey(expectedTp));
+        assertThat("Should have committed all the acked messages", capturedCommit.get(expectedTp).offset(), is((long)messageCount));
+
+            /* Verify that the following acked (now committed) tuples are not emitted again
+             * Since the consumer position was somewhere in the middle of the acked tuples when the commit happened,
+             * this verifies that the spout keeps the consumer position ahead of the committed offset when committing
+             */
+        //Just do a few polls to check that nothing more is emitted
+        for(int i = 0; i < 3; i++) {
+            spout.nextTuple();
+        }
+        verify(collectorMock, never()).emit(anyString(), anyList(), anyObject());
+    }
+
+    @Test
+    public void testShouldContinueWithSlowDoubleAcks() throws Exception {
+        final int messageCount = 20;
+        prepareSpout(messageCount);
+
+        //play 1st tuple
+        ArgumentCaptor<Object> messageIdToDoubleAck = ArgumentCaptor.forClass(Object.class);
+        spout.nextTuple();
+        verify(collectorMock).emit(anyString(), anyList(), messageIdToDoubleAck.capture());
+        spout.ack(messageIdToDoubleAck.getValue());
+
+        //Emit some more messages
+        for(int i = 0; i < messageCount / 2; i++) {
+            spout.nextTuple();
+        }
+
+        spout.ack(messageIdToDoubleAck.getValue());
+
+        //Emit any remaining messages
+        for(int i = 0; i < messageCount; i++) {
+            spout.nextTuple();
+        }
+
+        //Verify that all messages are emitted, ack all the messages
+        ArgumentCaptor<Object> messageIds = ArgumentCaptor.forClass(Object.class);
+        verify(collectorMock, times(messageCount)).emit(eq(SingleTopicKafkaSpoutConfiguration.STREAM),
+            anyList(),
+            messageIds.capture());
+        for(Object id : messageIds.getAllValues()) {
+            spout.ack(id);
+        }
+
+        Time.advanceTime(commitOffsetPeriodMs + KafkaSpout.TIMER_DELAY_MS);
+        //Commit offsets
+        spout.nextTuple();
+
+        verifyAllMessagesCommitted(messageCount);
+    }
+
+    @Test
+    public void testShouldEmitAllMessages() throws Exception {
+        final int messageCount = 10;
+        prepareSpout(messageCount);
+
+        //Emit all messages and check that they are emitted. Ack the messages too
+        for(int i = 0; i < messageCount; i++) {
+            spout.nextTuple();
+            ArgumentCaptor<Object> messageId = ArgumentCaptor.forClass(Object.class);
+            verify(collectorMock).emit(
+                eq(SingleTopicKafkaSpoutConfiguration.STREAM),
+                eq(new Values(SingleTopicKafkaSpoutConfiguration.TOPIC,
+                    Integer.toString(i),
+                    Integer.toString(i))),
+                messageId.capture());
+            spout.ack(messageId.getValue());
+            reset(collectorMock);
+        }
+
+        Time.advanceTime(commitOffsetPeriodMs + KafkaSpout.TIMER_DELAY_MS);
+        //Commit offsets
+        spout.nextTuple();
+
+        verifyAllMessagesCommitted(messageCount);
+    }
+
+    @Test
+    public void testShouldReplayInOrderFailedMessages() throws Exception {
+        final int messageCount = 10;
+        prepareSpout(messageCount);
+
+        //play and ack 1 tuple
+        ArgumentCaptor<Object> messageIdAcked = ArgumentCaptor.forClass(Object.class);
+        spout.nextTuple();
+        verify(collectorMock).emit(anyString(), anyList(), messageIdAcked.capture());
+        spout.ack(messageIdAcked.getValue());
+        reset(collectorMock);
+
+        //play and fail 1 tuple
+        ArgumentCaptor<Object> messageIdFailed = ArgumentCaptor.forClass(Object.class);
+        spout.nextTuple();
+        verify(collectorMock).emit(anyString(), anyList(), messageIdFailed.capture());
+        spout.fail(messageIdFailed.getValue());
+        reset(collectorMock);
+
+        //Emit all remaining messages. Failed tuples retry immediately with current configuration, so no need to wait.
+        for(int i = 0; i < messageCount; i++) {
+            spout.nextTuple();
+        }
+
+        ArgumentCaptor<Object> remainingMessageIds = ArgumentCaptor.forClass(Object.class);
+        //All messages except the first acked message should have been emitted
+        verify(collectorMock, times(messageCount - 1)).emit(
+            eq(SingleTopicKafkaSpoutConfiguration.STREAM),
+            anyList(),
+            remainingMessageIds.capture());
+        for(Object id : remainingMessageIds.getAllValues()) {
+            spout.ack(id);
+        }
+
+        Time.advanceTime(commitOffsetPeriodMs + KafkaSpout.TIMER_DELAY_MS);
+        //Commit offsets
+        spout.nextTuple();
+
+        verifyAllMessagesCommitted(messageCount);
+    }
+
+    @Test
+    public void testShouldReplayFirstTupleFailedOutOfOrder() throws Exception {
+        final int messageCount = 10;
+        prepareSpout(messageCount);
+
+        //play 1st tuple
+        ArgumentCaptor<Object> messageIdToFail = ArgumentCaptor.forClass(Object.class);
+        spout.nextTuple();
+        verify(collectorMock).emit(anyString(), anyList(), messageIdToFail.capture());
+        reset(collectorMock);
+
+        //play 2nd tuple
+        ArgumentCaptor<Object> messageIdToAck = ArgumentCaptor.forClass(Object.class);
+        spout.nextTuple();
+        verify(collectorMock).emit(anyString(), anyList(), messageIdToAck.capture());
+        reset(collectorMock);
+
+        //ack 2nd tuple
+        spout.ack(messageIdToAck.getValue());
+        //fail 1st tuple
+        spout.fail(messageIdToFail.getValue());
+
+        //Emit all remaining messages. Failed tuples retry immediately with current configuration, so no need to wait.
+        for(int i = 0; i < messageCount; i++) {
+            spout.nextTuple();
+        }
+
+        ArgumentCaptor<Object> remainingIds = ArgumentCaptor.forClass(Object.class);
+        //All messages except the first acked message should have been emitted
+        verify(collectorMock, times(messageCount - 1)).emit(
+            eq(SingleTopicKafkaSpoutConfiguration.STREAM),
+            anyList(),
+            remainingIds.capture());
+        for(Object id : remainingIds.getAllValues()) {
+            spout.ack(id);
+        }
+
+        Time.advanceTime(commitOffsetPeriodMs + KafkaSpout.TIMER_DELAY_MS);
+        //Commit offsets
+        spout.nextTuple();
+
+        verifyAllMessagesCommitted(messageCount);
+    }
+
+    @Test
+    public void testShouldReplayAllFailedTuplesWhenFailedOutOfOrder() throws Exception {
+        //The spout must reemit retriable tuples, even if they fail out of order.
+        //The spout should be able to skip tuples it has already emitted when retrying messages, even if those tuples are also retries.
+        final int messageCount = 10;
+        prepareSpout(messageCount);
+
+        //play all tuples
+        for (int i = 0; i < messageCount; i++) {
+            spout.nextTuple();
+        }
+        ArgumentCaptor<KafkaSpoutMessageId> messageIds = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
+        verify(collectorMock, times(messageCount)).emit(anyString(), anyList(), messageIds.capture());
+        reset(collectorMock);
+        //Fail tuple 5 and 3, call nextTuple, then fail tuple 2
+        List<KafkaSpoutMessageId> capturedMessageIds = messageIds.getAllValues();
+        spout.fail(capturedMessageIds.get(5));
+        spout.fail(capturedMessageIds.get(3));
+        spout.nextTuple();
+        spout.fail(capturedMessageIds.get(2));
+
+        //Check that the spout will reemit all 3 failed tuples and no other tuples
+        ArgumentCaptor<KafkaSpoutMessageId> reemittedMessageIds = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
+        for (int i = 0; i < messageCount; i++) {
+            spout.nextTuple();
+        }
+        verify(collectorMock, times(3)).emit(anyString(), anyList(), reemittedMessageIds.capture());
+        Set<KafkaSpoutMessageId> expectedReemitIds = new HashSet<>();
+        expectedReemitIds.add(capturedMessageIds.get(5));
+        expectedReemitIds.add(capturedMessageIds.get(3));
+        expectedReemitIds.add(capturedMessageIds.get(2));
+        assertThat("Expected reemits to be the 3 failed tuples", new HashSet<>(reemittedMessageIds.getAllValues()), is(expectedReemitIds));
+    }
+
+    @Test
+    public void testShouldDropMessagesAfterMaxRetriesAreReached() throws Exception {
+        //Check that if one message fails repeatedly, the retry cap limits how many times the message can be reemitted
+        final int messageCount = 1;
+        prepareSpout(messageCount);
+
+        //Emit and fail the same tuple until we've reached retry limit
+        for (int i = 0; i <= maxRetries; i++) {
+            ArgumentCaptor<KafkaSpoutMessageId> messageIdFailed = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
+            spout.nextTuple();
+            verify(collectorMock).emit(anyString(), anyListOf(Object.class), messageIdFailed.capture());
+            KafkaSpoutMessageId msgId = messageIdFailed.getValue();
+            spout.fail(msgId);
+            assertThat("Expected message id number of failures to match the number of times the message has failed", msgId.numFails(), is(i + 1));
+            reset(collectorMock);
+        }
+
+        //Verify that the tuple is not emitted again
+        spout.nextTuple();
+        verify(collectorMock, never()).emit(anyString(), anyListOf(Object.class), anyObject());
+    }
+
+    @Test
+    public void testSpoutMustRefreshPartitionsEvenIfNotPolling() throws Exception {
+        SingleTopicKafkaUnitSetupHelper.initializeSpout(spout, conf, topologyContext, collectorMock);
+
+        //Nothing is assigned yet, should emit nothing
+        spout.nextTuple();
+        verify(collectorMock, never()).emit(anyString(), anyList(), any(KafkaSpoutMessageId.class));
+
+        SingleTopicKafkaUnitSetupHelper.populateTopicData(kafkaUnitRule.getKafkaUnit(), SingleTopicKafkaSpoutConfiguration.TOPIC, 1);
+        Time.advanceTime(KafkaSpoutConfig.DEFAULT_PARTITION_REFRESH_PERIOD_MS + KafkaSpout.TIMER_DELAY_MS);
+
+        //The new partition should be discovered and the message should be emitted
+        spout.nextTuple();
+        verify(collectorMock).emit(anyString(), anyList(), any(KafkaSpoutMessageId.class));
+    }
+
+    @Test
+    public void testOffsetMetrics() throws Exception {
+        final int messageCount = 10;
+        prepareSpout(messageCount);
+
+        Map<String, Long> offsetMetric  = (Map<String, Long>) spout.getKafkaOffsetMetric().getValueAndReset();
+        assertEquals(0, offsetMetric.get(SingleTopicKafkaSpoutConfiguration.TOPIC+"/totalEarliestTimeOffset").longValue());
+        // the offset of the last available message + 1.
+        assertEquals(10, offsetMetric.get(SingleTopicKafkaSpoutConfiguration.TOPIC+"/totalLatestTimeOffset").longValue());
+        assertEquals(10, offsetMetric.get(SingleTopicKafkaSpoutConfiguration.TOPIC+"/totalRecordsInPartitions").longValue());
+        assertEquals(0, offsetMetric.get(SingleTopicKafkaSpoutConfiguration.TOPIC+"/totalLatestEmittedOffset").longValue());
+        assertEquals(0, offsetMetric.get(SingleTopicKafkaSpoutConfiguration.TOPIC+"/totalLatestCompletedOffset").longValue());
+        //totalSpoutLag = totalLatestTimeOffset-totalLatestCompletedOffset
+        assertEquals(10, offsetMetric.get(SingleTopicKafkaSpoutConfiguration.TOPIC+"/totalSpoutLag").longValue());
+
+        //Emit all messages and check that they are emitted. Ack the messages too
+        for (int i = 0; i < messageCount; i++) {
+            nextTuple_verifyEmitted_ack_resetCollector(i);
+        }
+
+        commitAndVerifyAllMessagesCommitted(messageCount);
+
+        offsetMetric  = (Map<String, Long>) spout.getKafkaOffsetMetric().getValueAndReset();
+        assertEquals(0, offsetMetric.get(SingleTopicKafkaSpoutConfiguration.TOPIC+"/totalEarliestTimeOffset").longValue());
+        assertEquals(10, offsetMetric.get(SingleTopicKafkaSpoutConfiguration.TOPIC+"/totalLatestTimeOffset").longValue());
+        //latest offset
+        assertEquals(9, offsetMetric.get(SingleTopicKafkaSpoutConfiguration.TOPIC+"/totalLatestEmittedOffset").longValue());
+        // offset where processing will resume upon spout restart
+        assertEquals(10, offsetMetric.get(SingleTopicKafkaSpoutConfiguration.TOPIC+"/totalLatestCompletedOffset").longValue());
+        assertEquals(0, offsetMetric.get(SingleTopicKafkaSpoutConfiguration.TOPIC+"/totalSpoutLag").longValue());
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutTopologyDeployActivateDeactivateTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutTopologyDeployActivateDeactivateTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutTopologyDeployActivateDeactivateTest.java
new file mode 100644
index 0000000..a860cef
--- /dev/null
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutTopologyDeployActivateDeactivateTest.java
@@ -0,0 +1,116 @@
+/*
+ * 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.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
+import org.junit.Test;
+
+import java.util.regex.Pattern;
+
+import static org.mockito.Mockito.when;
+
+public class KafkaSpoutTopologyDeployActivateDeactivateTest extends KafkaSpoutAbstractTest {
+
+    public KafkaSpoutTopologyDeployActivateDeactivateTest() {
+        super(2_000);
+    }
+
+    @Override
+    KafkaSpoutConfig<String, String> createSpoutConfig() {
+        return SingleTopicKafkaSpoutConfiguration.setCommonSpoutConfig(
+            KafkaSpoutConfig.builder("127.0.0.1:" + kafkaUnitRule.getKafkaUnit().getKafkaPort(),
+                Pattern.compile(SingleTopicKafkaSpoutConfiguration.TOPIC)))
+            .setOffsetCommitPeriodMs(commitOffsetPeriodMs)
+            .setFirstPollOffsetStrategy(KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST)
+            .build();
+    }
+
+    @Test
+    public void test_FirstPollStrategy_Earliest_NotEnforced_OnTopologyActivateDeactivate() throws Exception {
+        final int messageCount = 2;
+        prepareSpout(messageCount);
+
+        nextTuple_verifyEmitted_ack_resetCollector(0);
+
+        //Commits offsets during deactivation
+        spout.deactivate();
+
+        verifyAllMessagesCommitted(1);
+
+        consumerSpy = createConsumerSpy();
+
+        spout.activate();
+
+        nextTuple_verifyEmitted_ack_resetCollector(1);
+
+        commitAndVerifyAllMessagesCommitted(messageCount);
+    }
+
+    @Test
+    public void test_FirstPollStrategy_Earliest_NotEnforced_OnPartitionReassignment() throws Exception {
+        when(topologyContext.getStormId()).thenReturn("topology-1");
+
+        final int messageCount = 2;
+        prepareSpout(messageCount);
+
+        nextTuple_verifyEmitted_ack_resetCollector(0);
+
+        //Commits offsets during deactivation
+        spout.deactivate();
+
+        verifyAllMessagesCommitted(1);
+
+        // Restart topology with the same topology id, which mimics the behavior of partition reassignment
+        setUp();
+        // Initialize spout using the same populated data (i.e same kafkaUnitRule)
+        SingleTopicKafkaUnitSetupHelper.initializeSpout(spout, conf, topologyContext, collectorMock);
+
+        nextTuple_verifyEmitted_ack_resetCollector(1);
+
+        commitAndVerifyAllMessagesCommitted(messageCount);
+    }
+
+    @Test
+    public void test_FirstPollStrategy_Earliest_Enforced_OnlyOnTopologyDeployment() throws Exception {
+        when(topologyContext.getStormId()).thenReturn("topology-1");
+
+        final int messageCount = 2;
+        prepareSpout(messageCount);
+
+        nextTuple_verifyEmitted_ack_resetCollector(0);
+
+        //Commits offsets during deactivation
+        spout.deactivate();
+
+        verifyAllMessagesCommitted(1);
+
+        // Restart topology with a different topology id
+        setUp();
+        when(topologyContext.getStormId()).thenReturn("topology-2");
+        // Initialize spout using the same populated data (i.e same kafkaUnitRule)
+        SingleTopicKafkaUnitSetupHelper.initializeSpout(spout, conf, topologyContext, collectorMock);
+
+        //Emit all messages and check that they are emitted. Ack the messages too
+        for (int i = 0; i < messageCount; i++) {
+            nextTuple_verifyEmitted_ack_resetCollector(i);
+        }
+
+        commitAndVerifyAllMessagesCommitted(messageCount);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/MaxUncommittedOffsetTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/MaxUncommittedOffsetTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/MaxUncommittedOffsetTest.java
index a7ad4c2..b90a49d 100755
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/MaxUncommittedOffsetTest.java
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/MaxUncommittedOffsetTest.java
@@ -15,7 +15,7 @@
  */
 package org.apache.storm.kafka.spout;
 
-import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfigBuilder;
+import static org.hamcrest.CoreMatchers.either;
 import static org.hamcrest.CoreMatchers.everyItem;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.Matchers.greaterThanOrEqualTo;
@@ -34,7 +34,8 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import org.apache.kafka.clients.producer.ProducerRecord;
+
+import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.storm.kafka.KafkaUnitRule;
 import org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
 import org.apache.storm.spout.SpoutOutputCollector;
@@ -46,6 +47,10 @@ import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 import org.mockito.MockitoAnnotations;
 
+import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.createKafkaSpoutConfigBuilder;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.never;
+
 public class MaxUncommittedOffsetTest {
 
     @Rule
@@ -59,15 +64,17 @@ public class MaxUncommittedOffsetTest {
     private final int maxUncommittedOffsets = 10;
     private final int maxPollRecords = 5;
     private final int initialRetryDelaySecs = 60;
-    private final KafkaSpoutConfig spoutConfig = getKafkaSpoutConfigBuilder(kafkaUnitRule.getKafkaUnit().getKafkaPort())
+    private final KafkaSpoutConfig<String, String> spoutConfig = createKafkaSpoutConfigBuilder(kafkaUnitRule.getKafkaUnit().getKafkaPort())
         .setOffsetCommitPeriodMs(commitOffsetPeriodMs)
-        .setMaxPollRecords(maxPollRecords)
+        .setProp(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords)
         .setMaxUncommittedOffsets(maxUncommittedOffsets)
         .setRetry(new KafkaSpoutRetryExponentialBackoff(KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(initialRetryDelaySecs), KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(0),
             1, KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(initialRetryDelaySecs))) //Retry once after a minute
         .build();
     private KafkaSpout<String, String> spout;
 
+
+
     @Before
     public void setUp() {
         //This is because the tests are checking that a hard cap of maxUncommittedOffsets + maxPollRecords - 1 uncommitted offsets exists
@@ -77,37 +84,24 @@ public class MaxUncommittedOffsetTest {
         //The spout must be able to reemit all retriable tuples, even if the maxPollRecords is set to a low value compared to maxUncommittedOffsets.
         assertThat("Current tests require maxPollRecords < maxUncommittedOffsets", maxPollRecords, lessThanOrEqualTo(maxUncommittedOffsets));
         MockitoAnnotations.initMocks(this);
-        this.spout = new KafkaSpout<>(spoutConfig);
+        spout = new KafkaSpout<>(spoutConfig);
     }
 
-    private void populateTopicData(String topicName, int msgCount) throws Exception {
-        kafkaUnitRule.getKafkaUnit().createTopic(topicName);
-
-        for (int i = 0; i < msgCount; i++) {
-            ProducerRecord<String, String> producerRecord = new ProducerRecord<>(
-                topicName, Integer.toString(i),
-                Integer.toString(i));
-
-            kafkaUnitRule.getKafkaUnit().sendMessage(producerRecord);
-        }
-    }
-
-    private void initializeSpout(int msgCount) throws Exception {
-        populateTopicData(SingleTopicKafkaSpoutConfiguration.TOPIC, msgCount);
-        spout.open(conf, topologyContext, collector);
-        spout.activate();
+    private void prepareSpout(int msgCount) throws Exception {
+        SingleTopicKafkaUnitSetupHelper.populateTopicData(kafkaUnitRule.getKafkaUnit(), SingleTopicKafkaSpoutConfiguration.TOPIC, msgCount);
+        SingleTopicKafkaUnitSetupHelper.initializeSpout(spout, conf, topologyContext, collector);
     }
 
     private ArgumentCaptor<KafkaSpoutMessageId> emitMaxUncommittedOffsetsMessagesAndCheckNoMoreAreEmitted(int messageCount) throws Exception {
         assertThat("The message count is less than maxUncommittedOffsets. This test is not meaningful with this configuration.", messageCount, greaterThanOrEqualTo(maxUncommittedOffsets));
         //The spout must respect maxUncommittedOffsets when requesting/emitting tuples
-        initializeSpout(messageCount);
+        prepareSpout(messageCount);
 
         //Try to emit all messages. Ensure only maxUncommittedOffsets are emitted
         ArgumentCaptor<KafkaSpoutMessageId> messageIds = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
         for (int i = 0; i < messageCount; i++) {
             spout.nextTuple();
-        };
+        }
         verify(collector, times(maxUncommittedOffsets)).emit(
             anyString(),
             anyList(),
@@ -128,6 +122,7 @@ public class MaxUncommittedOffsetTest {
                 spout.ack(messageId);
             }
             Time.advanceTime(commitOffsetPeriodMs + KafkaSpout.TIMER_DELAY_MS);
+
             spout.nextTuple();
 
             //Now check that the spout will emit another maxUncommittedOffsets messages
@@ -183,8 +178,60 @@ public class MaxUncommittedOffsetTest {
 
     @Test
     public void testNextTupleWillNotEmitMoreThanMaxUncommittedOffsetsPlusMaxPollRecordsMessages() throws Exception {
-        //The upper bound on uncommitted offsets should be maxUncommittedOffsets + maxPollRecords - 1
-        //This is reachable by emitting maxUncommittedOffsets messages, acking the first message, then polling.
+        /*
+        For each partition the spout is allowed to retry all tuples between the committed offset, and maxUncommittedOffsets ahead.
+        It is not allowed to retry tuples past that limit.
+        This makes the actual limit per partition maxUncommittedOffsets + maxPollRecords - 1,
+        reached if the tuple at the maxUncommittedOffsets limit is the earliest retriable tuple,
+        or if the spout is 1 tuple below the limit, and receives a full maxPollRecords tuples in the poll.
+         */
+
+        try (Time.SimulatedTime simulatedTime = new Time.SimulatedTime()) {
+            //First check that maxUncommittedOffsets is respected when emitting from scratch
+            ArgumentCaptor<KafkaSpoutMessageId> messageIds = emitMaxUncommittedOffsetsMessagesAndCheckNoMoreAreEmitted(numMessages);
+            reset(collector);
+
+            //Fail only the last tuple
+            List<KafkaSpoutMessageId> messageIdList = messageIds.getAllValues();
+            KafkaSpoutMessageId failedMessageId = messageIdList.get(messageIdList.size() - 1);
+            spout.fail(failedMessageId);
+
+            //Offset 0 to maxUncommittedOffsets - 2 are pending, maxUncommittedOffsets - 1 is failed but not retriable
+            //The spout should not emit any more tuples.
+            spout.nextTuple();
+            verify(collector, never()).emit(
+                anyString(),
+                anyList(),
+                any(KafkaSpoutMessageId.class));
+
+            //Allow the failed record to retry
+            Time.advanceTimeSecs(initialRetryDelaySecs);
+            for (int i = 0; i < maxPollRecords; i++) {
+                spout.nextTuple();
+            }
+            ArgumentCaptor<KafkaSpoutMessageId> secondRunMessageIds = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
+            verify(collector, times(maxPollRecords)).emit(
+                anyString(),
+                anyList(),
+                secondRunMessageIds.capture());
+            reset(collector);
+            assertThat(secondRunMessageIds.getAllValues().get(0), is(failedMessageId));
+            
+            //There should now be maxUncommittedOffsets + maxPollRecords emitted in all.
+            //Fail the last emitted tuple and verify that the spout won't retry it because it's above the emit limit.
+            spout.fail(secondRunMessageIds.getAllValues().get(secondRunMessageIds.getAllValues().size() - 1));
+            Time.advanceTimeSecs(initialRetryDelaySecs);
+            spout.nextTuple();
+            verify(collector, never()).emit(anyString(), anyList(), any(KafkaSpoutMessageId.class));
+        }
+    }
+
+    @Test
+    public void testNextTupleWillAllowRetryForTuplesBelowEmitLimit() throws Exception {
+        /*
+        For each partition the spout is allowed to retry all tuples between the committed offset, and maxUncommittedOffsets ahead.
+        It must retry tuples within that limit, even if more tuples were emitted.
+         */
         try (Time.SimulatedTime simulatedTime = new Time.SimulatedTime()) {
             //First check that maxUncommittedOffsets is respected when emitting from scratch
             ArgumentCaptor<KafkaSpoutMessageId> messageIds = emitMaxUncommittedOffsetsMessagesAndCheckNoMoreAreEmitted(numMessages);
@@ -192,9 +239,9 @@ public class MaxUncommittedOffsetTest {
 
             failAllExceptTheFirstMessageThenCommit(messageIds);
 
-            //Offset 0 is acked, 1 to maxUncommittedOffsets - 1 are failed
+            //Offset 0 is committed, 1 to maxUncommittedOffsets - 1 are failed but not retriable
             //The spout should now emit another maxPollRecords messages
-            //This is allowed because the acked message brings the numUncommittedOffsets below the cap
+            //This is allowed because the committed message brings the numUncommittedOffsets below the cap
             for (int i = 0; i < maxUncommittedOffsets; i++) {
                 spout.nextTuple();
             }
@@ -216,18 +263,20 @@ public class MaxUncommittedOffsetTest {
             }
             assertThat("Expected the newly emitted messages to have no overlap with the first batch", secondRunOffsets.removeAll(firstRunOffsets), is(false));
 
-            //Offset 0 is acked, 1 to maxUncommittedOffsets-1 are failed, maxUncommittedOffsets to maxUncommittedOffsets + maxPollRecords-1 are emitted
-            //There are now maxUncommittedOffsets-1 + maxPollRecords records emitted past the last committed offset
-            //Advance time so the failed tuples become ready for retry, and check that the spout will emit retriable tuples as long as numNonRetriableEmittedTuples < maxUncommittedOffsets
-            
-            int numNonRetriableEmittedTuples = maxPollRecords; //The other tuples were failed and are becoming retriable
-            int allowedPolls = (int)Math.ceil((maxUncommittedOffsets - numNonRetriableEmittedTuples)/(double)maxPollRecords);
+            //Offset 0 is committed, 1 to maxUncommittedOffsets-1 are failed, maxUncommittedOffsets to maxUncommittedOffsets + maxPollRecords-1 are emitted
+            //Fail the last tuples so only offset 0 is not failed.
+            //Advance time so the failed tuples become ready for retry, and check that the spout will emit retriable tuples
+            //for all the failed tuples that are within maxUncommittedOffsets tuples of the committed offset
+            //This means 1 to maxUncommitteddOffsets, but not maxUncommittedOffsets+1...maxUncommittedOffsets+maxPollRecords-1
+            for(KafkaSpoutMessageId msgId : secondRunMessageIds.getAllValues()) {
+                spout.fail(msgId);
+            }
             Time.advanceTimeSecs(initialRetryDelaySecs);
             for (int i = 0; i < numMessages; i++) {
                 spout.nextTuple();
             }
             ArgumentCaptor<KafkaSpoutMessageId> thirdRunMessageIds = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
-            verify(collector, times(allowedPolls*maxPollRecords)).emit(
+            verify(collector, times(maxUncommittedOffsets)).emit(
                 anyString(),
                 anyList(),
                 thirdRunMessageIds.capture());
@@ -238,8 +287,7 @@ public class MaxUncommittedOffsetTest {
                 thirdRunOffsets.add(msgId.offset());
             }
 
-            assertThat("Expected the emitted messages to be retries of the failed tuples from the first batch", thirdRunOffsets, everyItem(isIn(firstRunOffsets)));
+            assertThat("Expected the emitted messages to be retries of the failed tuples from the first batch, plus the first failed tuple from the second batch", thirdRunOffsets, everyItem(either(isIn(firstRunOffsets)).or(is(secondRunMessageIds.getAllValues().get(0).offset()))));
         }
     }
-
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/NamedTopicFilterTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/NamedTopicFilterTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/NamedTopicFilterTest.java
index e97c7e1..fe3325c 100644
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/NamedTopicFilterTest.java
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/NamedTopicFilterTest.java
@@ -16,10 +16,11 @@
 
 package org.apache.storm.kafka.spout;
 
+import org.apache.storm.kafka.spout.NamedTopicFilter;
+
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.mock;
-
 import static org.mockito.Mockito.when;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/PatternTopicFilterTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/PatternTopicFilterTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/PatternTopicFilterTest.java
index 877efdc..335ab31 100644
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/PatternTopicFilterTest.java
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/PatternTopicFilterTest.java
@@ -16,6 +16,8 @@
 
 package org.apache.storm.kafka.spout;
 
+import org.apache.storm.kafka.spout.PatternTopicFilter;
+
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.mock;

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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
deleted file mode 100644
index 436d052..0000000
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaSpoutTest.java
+++ /dev/null
@@ -1,401 +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 static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfigBuilder;
-
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.storm.kafka.KafkaUnitRule;
-import org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
-import org.apache.storm.spout.SpoutOutputCollector;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.tuple.Values;
-import org.junit.Rule;
-import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-
-import java.util.Map;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeoutException;
-
-import static org.hamcrest.CoreMatchers.is;
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.mockito.Matchers.anyList;
-import static org.mockito.Matchers.anyListOf;
-import static org.mockito.Matchers.anyObject;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.reset;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
-import org.apache.storm.kafka.spout.internal.KafkaConsumerFactoryDefault;
-import org.apache.storm.utils.Time;
-import org.apache.storm.utils.Time.SimulatedTime;
-import org.hamcrest.Matchers;
-import org.junit.Before;
-import org.mockito.Captor;
-import org.mockito.MockitoAnnotations;
-
-public class SingleTopicKafkaSpoutTest {
-
-    @Rule
-    public KafkaUnitRule kafkaUnitRule = new KafkaUnitRule();
-
-    @Captor
-    private ArgumentCaptor<Map<TopicPartition, OffsetAndMetadata>> commitCapture;
-
-    private final TopologyContext topologyContext = mock(TopologyContext.class);
-    private final Map<String, Object> conf = new HashMap<>();
-    private final SpoutOutputCollector collector = mock(SpoutOutputCollector.class);
-    private final long commitOffsetPeriodMs = 2_000;
-    private KafkaConsumer<String, String> consumerSpy;
-    private KafkaConsumerFactory<String, String> consumerFactory;
-    private KafkaSpout<String, String> spout;
-    private final int maxPollRecords = 10;
-    private final int maxRetries = 3;
-
-    @Before
-    public void setUp() {
-        MockitoAnnotations.initMocks(this);
-        KafkaSpoutConfig spoutConfig = getKafkaSpoutConfigBuilder(kafkaUnitRule.getKafkaUnit().getKafkaPort())
-            .setOffsetCommitPeriodMs(commitOffsetPeriodMs)
-            .setRetry(new KafkaSpoutRetryExponentialBackoff(KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(0), KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(0),
-                maxRetries, KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(0)))
-            .setProp(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords)
-            .build();
-        this.consumerSpy = spy(new KafkaConsumerFactoryDefault().createConsumer(spoutConfig));
-        this.consumerFactory = new KafkaConsumerFactory<String, String>() {
-            @Override
-            public KafkaConsumer<String, String> createConsumer(KafkaSpoutConfig<String, String> kafkaSpoutConfig) {
-                return consumerSpy;
-            }
-        
-        };
-        this.spout = new KafkaSpout<>(spoutConfig, consumerFactory);
-    }
-
-    void populateTopicData(String topicName, int msgCount) throws InterruptedException, ExecutionException, TimeoutException {
-        kafkaUnitRule.getKafkaUnit().createTopic(topicName);
-
-        for (int i = 0; i < msgCount; i++) {
-            ProducerRecord<String, String> producerRecord = new ProducerRecord<>(
-                topicName, Integer.toString(i),
-                Integer.toString(i));
-            kafkaUnitRule.getKafkaUnit().sendMessage(producerRecord);
-        }
-    }
-
-    private void initializeSpout(int msgCount) throws InterruptedException, ExecutionException, TimeoutException {
-        populateTopicData(SingleTopicKafkaSpoutConfiguration.TOPIC, msgCount);
-        spout.open(conf, topologyContext, collector);
-        spout.activate();
-    }
-
-    /*
-     * Asserts that commitSync has been called once, 
-     * that there are only commits on one topic,
-     * and that the committed offset covers messageCount messages
-     */
-    private void verifyAllMessagesCommitted(long messageCount) {
-        verify(consumerSpy, times(1)).commitSync(commitCapture.capture());
-        Map<TopicPartition, OffsetAndMetadata> commits = commitCapture.getValue();
-        assertThat("Expected commits for only one topic partition", commits.entrySet().size(), is(1));
-        OffsetAndMetadata offset = commits.entrySet().iterator().next().getValue();
-        assertThat("Expected committed offset to cover all emitted messages", offset.offset(), is(messageCount));
-    }
-
-    @Test
-    public void testSeekToCommittedOffsetIfConsumerPositionIsBehindWhenCommitting() throws Exception {
-        try (SimulatedTime simulatedTime = new SimulatedTime()) {
-            int messageCount = maxPollRecords * 2;
-            initializeSpout(messageCount);
-
-            //Emit all messages and fail the first one while acking the rest
-            for (int i = 0; i < messageCount; i++) {
-                spout.nextTuple();
-            }
-            ArgumentCaptor<KafkaSpoutMessageId> messageIdCaptor = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
-            verify(collector, times(messageCount)).emit(anyString(), anyList(), messageIdCaptor.capture());
-            List<KafkaSpoutMessageId> messageIds = messageIdCaptor.getAllValues();
-            for (int i = 1; i < messageIds.size(); i++) {
-                spout.ack(messageIds.get(i));
-            }
-            KafkaSpoutMessageId failedTuple = messageIds.get(0);
-            spout.fail(failedTuple);
-
-            //Advance the time and replay the failed tuple. 
-            reset(collector);
-            spout.nextTuple();
-            ArgumentCaptor<KafkaSpoutMessageId> failedIdReplayCaptor = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
-            verify(collector).emit(anyString(), anyList(), failedIdReplayCaptor.capture());
-
-            assertThat("Expected replay of failed tuple", failedIdReplayCaptor.getValue(), is(failedTuple));
-
-            /* Ack the tuple, and commit.
-             * Since the tuple is more than max poll records behind the most recent emitted tuple, the consumer won't catch up in this poll.
-             */
-            Time.advanceTime(KafkaSpout.TIMER_DELAY_MS + commitOffsetPeriodMs);
-            spout.ack(failedIdReplayCaptor.getValue());
-            spout.nextTuple();
-            verify(consumerSpy).commitSync(commitCapture.capture());
-            
-            Map<TopicPartition, OffsetAndMetadata> capturedCommit = commitCapture.getValue();
-            TopicPartition expectedTp = new TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 0);
-            assertThat("Should have committed to the right topic", capturedCommit, Matchers.hasKey(expectedTp));
-            assertThat("Should have committed all the acked messages", capturedCommit.get(expectedTp).offset(), is((long)messageCount));
-
-            /* Verify that the following acked (now committed) tuples are not emitted again
-             * Since the consumer position was somewhere in the middle of the acked tuples when the commit happened,
-             * this verifies that the spout keeps the consumer position ahead of the committed offset when committing
-             */
-            reset(collector);
-            //Just do a few polls to check that nothing more is emitted
-            for(int i = 0; i < 3; i++) {
-                spout.nextTuple();
-            }
-            verify(collector, never()).emit(anyString(), anyList(), anyObject());
-        }
-    }
-
-    @Test
-    public void shouldContinueWithSlowDoubleAcks() throws Exception {
-        try (SimulatedTime simulatedTime = new SimulatedTime()) {
-            int messageCount = 20;
-            initializeSpout(messageCount);
-
-            //play 1st tuple
-            ArgumentCaptor<Object> messageIdToDoubleAck = ArgumentCaptor.forClass(Object.class);
-            spout.nextTuple();
-            verify(collector).emit(anyString(), anyList(), messageIdToDoubleAck.capture());
-            spout.ack(messageIdToDoubleAck.getValue());
-
-            //Emit some more messages
-            for(int i = 0; i < messageCount / 2; i++) {
-                spout.nextTuple();
-            }
-
-            spout.ack(messageIdToDoubleAck.getValue());
-
-            //Emit any remaining messages
-            for(int i = 0; i < messageCount; i++) {
-                spout.nextTuple();
-            }
-
-            //Verify that all messages are emitted, ack all the messages
-            ArgumentCaptor<Object> messageIds = ArgumentCaptor.forClass(Object.class);
-            verify(collector, times(messageCount)).emit(eq(SingleTopicKafkaSpoutConfiguration.STREAM),
-                anyList(),
-                messageIds.capture());
-            for(Object id : messageIds.getAllValues()) {
-                spout.ack(id);
-            }
-
-            Time.advanceTime(commitOffsetPeriodMs + KafkaSpout.TIMER_DELAY_MS);
-            //Commit offsets
-            spout.nextTuple();
-
-            verifyAllMessagesCommitted(messageCount);
-        }
-    }
-
-    @Test
-    public void shouldEmitAllMessages() throws Exception {
-        try (SimulatedTime simulatedTime = new SimulatedTime()) {
-            int messageCount = 10;
-            initializeSpout(messageCount);
-
-            //Emit all messages and check that they are emitted. Ack the messages too
-            for(int i = 0; i < messageCount; i++) {
-                spout.nextTuple();
-                ArgumentCaptor<Object> messageId = ArgumentCaptor.forClass(Object.class);
-                verify(collector).emit(
-                    eq(SingleTopicKafkaSpoutConfiguration.STREAM),
-                    eq(new Values(SingleTopicKafkaSpoutConfiguration.TOPIC,
-                        Integer.toString(i),
-                        Integer.toString(i))),
-                    messageId.capture());
-                spout.ack(messageId.getValue());
-                reset(collector);
-            }
-
-            Time.advanceTime(commitOffsetPeriodMs + KafkaSpout.TIMER_DELAY_MS);
-            //Commit offsets
-            spout.nextTuple();
-
-            verifyAllMessagesCommitted(messageCount);
-        }
-    }
-
-    @Test
-    public void shouldReplayInOrderFailedMessages() throws Exception {
-        try (SimulatedTime simulatedTime = new SimulatedTime()) {
-            int messageCount = 10;
-            initializeSpout(messageCount);
-
-            //play and ack 1 tuple
-            ArgumentCaptor<Object> messageIdAcked = ArgumentCaptor.forClass(Object.class);
-            spout.nextTuple();
-            verify(collector).emit(anyString(), anyList(), messageIdAcked.capture());
-            spout.ack(messageIdAcked.getValue());
-            reset(collector);
-
-            //play and fail 1 tuple
-            ArgumentCaptor<Object> messageIdFailed = ArgumentCaptor.forClass(Object.class);
-            spout.nextTuple();
-            verify(collector).emit(anyString(), anyList(), messageIdFailed.capture());
-            spout.fail(messageIdFailed.getValue());
-            reset(collector);
-
-            //Emit all remaining messages. Failed tuples retry immediately with current configuration, so no need to wait.
-            for(int i = 0; i < messageCount; i++) {
-                spout.nextTuple();
-            }
-
-            ArgumentCaptor<Object> remainingMessageIds = ArgumentCaptor.forClass(Object.class);
-            //All messages except the first acked message should have been emitted
-            verify(collector, times(messageCount - 1)).emit(
-                eq(SingleTopicKafkaSpoutConfiguration.STREAM),
-                anyList(),
-                remainingMessageIds.capture());
-            for(Object id : remainingMessageIds.getAllValues()) {
-                spout.ack(id);
-            }
-
-            Time.advanceTime(commitOffsetPeriodMs + KafkaSpout.TIMER_DELAY_MS);
-            //Commit offsets
-            spout.nextTuple();
-
-            verifyAllMessagesCommitted(messageCount);
-        }
-    }
-
-    @Test
-    public void shouldReplayFirstTupleFailedOutOfOrder() throws Exception {
-        try (SimulatedTime simulatedTime = new SimulatedTime()) {
-            int messageCount = 10;
-            initializeSpout(messageCount);
-
-            //play 1st tuple
-            ArgumentCaptor<Object> messageIdToFail = ArgumentCaptor.forClass(Object.class);
-            spout.nextTuple();
-            verify(collector).emit(anyString(), anyList(), messageIdToFail.capture());
-            reset(collector);
-
-            //play 2nd tuple
-            ArgumentCaptor<Object> messageIdToAck = ArgumentCaptor.forClass(Object.class);
-            spout.nextTuple();
-            verify(collector).emit(anyString(), anyList(), messageIdToAck.capture());
-            reset(collector);
-
-            //ack 2nd tuple
-            spout.ack(messageIdToAck.getValue());
-            //fail 1st tuple
-            spout.fail(messageIdToFail.getValue());
-
-            //Emit all remaining messages. Failed tuples retry immediately with current configuration, so no need to wait.
-            for(int i = 0; i < messageCount; i++) {
-                spout.nextTuple();
-            }
-
-            ArgumentCaptor<Object> remainingIds = ArgumentCaptor.forClass(Object.class);
-            //All messages except the first acked message should have been emitted
-            verify(collector, times(messageCount - 1)).emit(
-                eq(SingleTopicKafkaSpoutConfiguration.STREAM),
-                anyList(),
-                remainingIds.capture());
-            for(Object id : remainingIds.getAllValues()) {
-                spout.ack(id);
-            }
-
-            Time.advanceTime(commitOffsetPeriodMs + KafkaSpout.TIMER_DELAY_MS);
-            //Commit offsets
-            spout.nextTuple();
-
-            verifyAllMessagesCommitted(messageCount);
-        }
-    }
-
-    @Test
-    public void shouldReplayAllFailedTuplesWhenFailedOutOfOrder() throws Exception {
-        //The spout must reemit retriable tuples, even if they fail out of order.
-        //The spout should be able to skip tuples it has already emitted when retrying messages, even if those tuples are also retries.
-        int messageCount = 10;
-        initializeSpout(messageCount);
-
-        //play all tuples
-        for (int i = 0; i < messageCount; i++) {
-            spout.nextTuple();
-        }
-        ArgumentCaptor<KafkaSpoutMessageId> messageIds = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
-        verify(collector, times(messageCount)).emit(anyString(), anyList(), messageIds.capture());
-        reset(collector);
-        //Fail tuple 5 and 3, call nextTuple, then fail tuple 2
-        List<KafkaSpoutMessageId> capturedMessageIds = messageIds.getAllValues();
-        spout.fail(capturedMessageIds.get(5));
-        spout.fail(capturedMessageIds.get(3));
-        spout.nextTuple();
-        spout.fail(capturedMessageIds.get(2));
-
-        //Check that the spout will reemit all 3 failed tuples and no other tuples
-        ArgumentCaptor<KafkaSpoutMessageId> reemittedMessageIds = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
-        for (int i = 0; i < messageCount; i++) {
-            spout.nextTuple();
-        }
-        verify(collector, times(3)).emit(anyString(), anyList(), reemittedMessageIds.capture());
-        Set<KafkaSpoutMessageId> expectedReemitIds = new HashSet<>();
-        expectedReemitIds.add(capturedMessageIds.get(5));
-        expectedReemitIds.add(capturedMessageIds.get(3));
-        expectedReemitIds.add(capturedMessageIds.get(2));
-        assertThat("Expected reemits to be the 3 failed tuples", new HashSet<>(reemittedMessageIds.getAllValues()), is(expectedReemitIds));
-    }
-
-    @Test
-    public void shouldDropMessagesAfterMaxRetriesAreReached() throws Exception {
-        //Check that if one message fails repeatedly, the retry cap limits how many times the message can be reemitted
-        int messageCount = 1;
-        initializeSpout(messageCount);
-
-        //Emit and fail the same tuple until we've reached retry limit
-        for (int i = 0; i <= maxRetries; i++) {
-            ArgumentCaptor<KafkaSpoutMessageId> messageIdFailed = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
-            spout.nextTuple();
-            verify(collector).emit(anyString(), anyListOf(Object.class), messageIdFailed.capture());
-            KafkaSpoutMessageId msgId = messageIdFailed.getValue();
-            spout.fail(msgId);
-            assertThat("Expected message id number of failures to match the number of times the message has failed", msgId.numFails(), is(i + 1));
-            reset(collector);
-        }
-
-        //Verify that the tuple is not emitted again
-        spout.nextTuple();
-        verify(collector, never()).emit(anyString(), anyListOf(Object.class), anyObject());
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaUnitSetupHelper.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaUnitSetupHelper.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaUnitSetupHelper.java
new file mode 100644
index 0000000..f5b9423
--- /dev/null
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaUnitSetupHelper.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2017 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 static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.Collections;
+import java.util.Map;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.storm.kafka.KafkaUnit;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.mockito.ArgumentCaptor;
+
+public class SingleTopicKafkaUnitSetupHelper {
+
+    /**
+     * Using the given KafkaUnit instance, put some messages in the specified topic.
+     *
+     * @param kafkaUnit The KafkaUnit instance to use
+     * @param topicName The topic to produce messages for
+     * @param msgCount The number of messages to produce
+     */
+    public static void populateTopicData(KafkaUnit kafkaUnit, String topicName, int msgCount) throws Exception {
+        kafkaUnit.createTopic(topicName);
+        
+        for (int i = 0; i < msgCount; i++) {
+            ProducerRecord<String, String> producerRecord = new ProducerRecord<>(
+                topicName, Integer.toString(i),
+                Integer.toString(i));
+            kafkaUnit.sendMessage(producerRecord);
+        }
+    }
+
+    /*
+     * Asserts that commitSync has been called once, 
+     * that there are only commits on one topic,
+     * and that the committed offset covers messageCount messages
+     */
+    public static <K, V> void verifyAllMessagesCommitted(KafkaConsumer<K, V> consumerSpy,
+        ArgumentCaptor<Map<TopicPartition, OffsetAndMetadata>> commitCapture, long messageCount) {
+        verify(consumerSpy, times(1)).commitSync(commitCapture.capture());
+        Map<TopicPartition, OffsetAndMetadata> commits = commitCapture.getValue();
+        assertThat("Expected commits for only one topic partition", commits.entrySet().size(), is(1));
+        OffsetAndMetadata offset = commits.entrySet().iterator().next().getValue();
+        assertThat("Expected committed offset to cover all emitted messages", offset.offset(), is(messageCount));
+    }
+
+    /**
+     * Open and activate a KafkaSpout that acts as a single-task/executor spout.
+     *
+     * @param <K> Kafka key type
+     * @param <V> Kafka value type
+     * @param spout The spout to prepare
+     * @param topoConf The topoConf
+     * @param topoContextMock The TopologyContext mock
+     * @param collectorMock The output collector mock
+     */
+    public static <K, V> void initializeSpout(KafkaSpout<K, V> spout, Map<String, Object> topoConf, TopologyContext topoContextMock,
+        SpoutOutputCollector collectorMock) throws Exception {
+        when(topoContextMock.getThisTaskIndex()).thenReturn(0);
+        when(topoContextMock.getComponentTasks(anyString())).thenReturn(Collections.singletonList(0));
+        spout.open(topoConf, topoContextMock, collectorMock);
+        spout.activate();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SpoutWithMockedConsumerSetupHelper.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SpoutWithMockedConsumerSetupHelper.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SpoutWithMockedConsumerSetupHelper.java
new file mode 100644
index 0000000..3aad61e
--- /dev/null
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SpoutWithMockedConsumerSetupHelper.java
@@ -0,0 +1,171 @@
+/*
+ * Copyright 2017 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 static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.anyList;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mockingDetails;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+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.common.TopicPartition;
+import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.mockito.ArgumentCaptor;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+public class SpoutWithMockedConsumerSetupHelper {
+
+    /**
+     * Creates, opens and activates a KafkaSpout using a mocked consumer. The subscription should be a mock object, since this method skips
+     * the subscription and instead just configures the mocked consumer to act as if the specified partitions are assigned to it.
+     *
+     * @param <K> The Kafka key type
+     * @param <V> The Kafka value type
+     * @param spoutConfig The spout config to use
+     * @param topoConf The topo conf to pass to the spout
+     * @param contextMock The topo context to pass to the spout
+     * @param collectorMock The mocked collector to pass to the spout
+     * @param consumerMock The mocked consumer
+     * @param assignedPartitions The partitions to assign to this spout. The consumer will act like these partitions are assigned to it.
+     * @return The spout
+     */
+    public static <K, V> KafkaSpout<K, V> setupSpout(KafkaSpoutConfig<K, V> spoutConfig, Map<String, Object> topoConf,
+        TopologyContext contextMock, SpoutOutputCollector collectorMock, final KafkaConsumer<K, V> consumerMock, TopicPartition... assignedPartitions) {
+        Subscription subscriptionMock = spoutConfig.getSubscription();
+        if (!mockingDetails(subscriptionMock).isMock()) {
+            throw new IllegalStateException("Use a mocked subscription when using this method, it helps avoid complex stubbing");
+        }
+
+        final Set<TopicPartition> assignedPartitionsSet = new HashSet<>(Arrays.asList(assignedPartitions));
+
+        when(consumerMock.assignment()).thenReturn(assignedPartitionsSet);
+        doAnswer(new Answer() {
+            @Override
+            public Object answer(InvocationOnMock invocation) throws Throwable {
+                ConsumerRebalanceListener listener = (ConsumerRebalanceListener) invocation.getArguments()[1];
+                listener.onPartitionsAssigned(assignedPartitionsSet);
+                return null;
+            }
+
+        }).when(subscriptionMock).subscribe(any(KafkaConsumer.class), any(ConsumerRebalanceListener.class), any(TopologyContext.class));
+
+        KafkaConsumerFactory<K, V> consumerFactory = new KafkaConsumerFactory<K, V>() {
+            @Override
+            public KafkaConsumer<K, V> createConsumer(KafkaSpoutConfig<K, V> kafkaSpoutConfig) {
+                return consumerMock;
+            }
+        };
+        KafkaSpout<K, V> spout = new KafkaSpout<>(spoutConfig, consumerFactory);
+
+        spout.open(topoConf, contextMock, collectorMock);
+        spout.activate();
+
+        return spout;
+    }
+
+    /**
+     * Creates sequential dummy records
+     *
+     * @param <K> The Kafka key type
+     * @param <V> The Kafka value type
+     * @param topic The topic partition to create records for
+     * @param startingOffset The starting offset of the records
+     * @param numRecords The number of records to create
+     * @return The dummy records
+     */
+    public static <K, V> List<ConsumerRecord<K, V>> createRecords(TopicPartition topic, long startingOffset, int numRecords) {
+        List<ConsumerRecord<K, V>> recordsForPartition = new ArrayList<>();
+        for (int i = 0; i < numRecords; i++) {
+            recordsForPartition.add(new ConsumerRecord<K, V>(topic.topic(), topic.partition(), startingOffset + i, null, null));
+        }
+        return recordsForPartition;
+    }
+
+    /**
+     * Creates messages for the input offsets, emits the messages by calling nextTuple once per offset and returns the captured message ids
+     *
+     * @param <K> The Kafka key type
+     * @param <V> The Kafka value type
+     * @param spout The spout
+     * @param consumerMock The consumer used by the spout
+     * @param expectedEmits The number of expected emits
+     * @param collectorMock The collector used by the spout
+     * @param partition The partition to emit messages on
+     * @param offsetsToEmit The offsets to emit
+     * @return The message ids emitted by the spout during the nextTuple calls
+     */
+    public static <K, V> List<KafkaSpoutMessageId> pollAndEmit(KafkaSpout<K, V> spout, KafkaConsumer<K, V> consumerMock, int expectedEmits, SpoutOutputCollector collectorMock, TopicPartition partition, int... offsetsToEmit) {
+        return pollAndEmit(spout, consumerMock, expectedEmits, collectorMock, Collections.singletonMap(partition, offsetsToEmit));
+    }
+
+    /**
+     * Creates messages for the input offsets, emits the messages by calling nextTuple once per offset and returns the captured message ids
+     *
+     * @param <K> The Kafka key type
+     * @param <V> The Kafka value type
+     * @param spout The spout
+     * @param consumerMock The consumer used by the spout
+     * @param collectorMock The collector used by the spout
+     * @param offsetsToEmit The offsets to emit per partition
+     * @return The message ids emitted by the spout during the nextTuple calls
+     */
+    public static <K, V> List<KafkaSpoutMessageId> pollAndEmit(KafkaSpout<K, V> spout, KafkaConsumer<K, V> consumerMock, int expectedEmits, SpoutOutputCollector collectorMock, Map<TopicPartition, int[]> offsetsToEmit) {
+        int totalOffsets = 0;
+        Map<TopicPartition, List<ConsumerRecord<K, V>>> records = new HashMap<>();
+        for (Entry<TopicPartition, int[]> entry : offsetsToEmit.entrySet()) {
+            TopicPartition tp = entry.getKey();
+            List<ConsumerRecord<K, V>> tpRecords = new ArrayList<>();
+            for (Integer offset : entry.getValue()) {
+                tpRecords.add(new ConsumerRecord<K, V>(tp.topic(), tp.partition(), offset, null, null));
+                totalOffsets++;
+            }
+            records.put(tp, tpRecords);
+        }
+
+        when(consumerMock.poll(anyLong()))
+            .thenReturn(new ConsumerRecords<>(records));
+
+        for (int i = 0; i < totalOffsets; i++) {
+            spout.nextTuple();
+        }
+
+        ArgumentCaptor<KafkaSpoutMessageId> messageIds = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
+        verify(collectorMock, times(expectedEmits)).emit(anyString(), anyList(), messageIds.capture());
+        return messageIds.getAllValues();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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 d5c052b..b178687 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,20 @@
  */
 package org.apache.storm.kafka.spout.builders;
 
+import static org.apache.storm.kafka.spout.KafkaSpoutConfig.DEFAULT_MAX_RETRIES;
 import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
 
 import java.util.List;
-
+import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.storm.Config;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.kafka.spout.Func;
 import org.apache.storm.kafka.spout.KafkaSpout;
 import org.apache.storm.kafka.spout.KafkaSpoutConfig;
+import org.apache.storm.kafka.spout.KafkaSpoutRetryExponentialBackoff;
 import org.apache.storm.kafka.spout.KafkaSpoutRetryService;
+import org.apache.storm.kafka.spout.Subscription;
 import org.apache.storm.kafka.spout.test.KafkaSpoutTestBolt;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.tuple.Fields;
@@ -46,7 +49,7 @@ public class SingleTopicKafkaSpoutConfiguration {
 
     public static StormTopology getTopologyKafkaSpout(int port) {
         final TopologyBuilder tp = new TopologyBuilder();
-        tp.setSpout("kafka_spout", new KafkaSpout<>(getKafkaSpoutConfigBuilder(port).build()), 1);
+        tp.setSpout("kafka_spout", new KafkaSpout<>(SingleTopicKafkaSpoutConfiguration.createKafkaSpoutConfigBuilder(port).build()), 1);
         tp.setBolt("kafka_bolt", new KafkaSpoutTestBolt()).shuffleGrouping("kafka_spout", STREAM);
         return tp.createTopology();
     }
@@ -57,21 +60,33 @@ public class SingleTopicKafkaSpoutConfiguration {
             return new Values(r.topic(), r.key(), r.value());
         }
     };
-    
-    public static KafkaSpoutConfig.Builder<String,String> getKafkaSpoutConfigBuilder(int port) {
-        return KafkaSpoutConfig.builder("127.0.0.1:" + port, TOPIC)
-                .setRecordTranslator(TOPIC_KEY_VALUE_FUNC,
-                        new Fields("topic", "key", "value"), STREAM)
-                .setGroupId("kafkaSpoutTestGroup")
-                .setMaxPollRecords(5)
-                .setRetry(getRetryService())
-                .setOffsetCommitPeriodMs(10_000)
-                .setFirstPollOffsetStrategy(EARLIEST)
-                .setMaxUncommittedOffsets(250)
-                .setPollTimeoutMs(1000);
+
+    public static KafkaSpoutConfig.Builder<String, String> createKafkaSpoutConfigBuilder(int port) {
+        return setCommonSpoutConfig(KafkaSpoutConfig.builder("127.0.0.1:" + port, TOPIC));
+    }
+
+    public static KafkaSpoutConfig.Builder<String, String> createKafkaSpoutConfigBuilder(Subscription subscription, int port) {
+        return setCommonSpoutConfig(new KafkaSpoutConfig.Builder<String, String>("127.0.0.1:" + port, subscription));
     }
-        
-    protected static KafkaSpoutRetryService getRetryService() {
-        return KafkaSpoutConfig.UNIT_TEST_RETRY_SERVICE;
+
+    public static KafkaSpoutConfig.Builder<String, String> setCommonSpoutConfig(KafkaSpoutConfig.Builder<String, String> config) {
+        return config
+            .setRecordTranslator(TOPIC_KEY_VALUE_FUNC,
+                new Fields("topic", "key", "value"), STREAM)
+            .setProp(ConsumerConfig.GROUP_ID_CONFIG, "kafkaSpoutTestGroup")
+            .setProp(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 5)
+            .setRetry(getNoDelayRetryService())
+            .setOffsetCommitPeriodMs(10_000)
+            .setFirstPollOffsetStrategy(EARLIEST)
+            .setMaxUncommittedOffsets(250)
+            .setPollTimeoutMs(1000);
+    }
+
+    protected static KafkaSpoutRetryService getNoDelayRetryService() {
+        /**
+         * Retry in a tight loop (keep unit tests fasts).
+         */
+        return new KafkaSpoutRetryExponentialBackoff(KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(0), KafkaSpoutRetryExponentialBackoff.TimeInterval.milliSeconds(0),
+            DEFAULT_MAX_RETRIES, KafkaSpoutRetryExponentialBackoff.TimeInterval.milliSeconds(0));
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/internal/OffsetManagerTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/internal/OffsetManagerTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/internal/OffsetManagerTest.java
index abbacf9..9972d4c 100644
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/internal/OffsetManagerTest.java
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/internal/OffsetManagerTest.java
@@ -18,10 +18,12 @@ package org.apache.storm.kafka.spout.internal;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.nullValue;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
 
 import java.util.NoSuchElementException;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.storm.kafka.spout.KafkaSpoutMessageId;
@@ -30,6 +32,7 @@ import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
 public class OffsetManagerTest {
+    private static final String COMMIT_METADATA = "{\"topologyId\":\"tp1\",\"taskId\":3,\"threadName\":\"Thread-20\"}";
 
     @Rule
     public ExpectedException expect = ExpectedException.none();
@@ -56,12 +59,12 @@ public class OffsetManagerTest {
         manager.addToAckMsgs(getMessageId(initialFetchOffset + 2));
         manager.addToAckMsgs(getMessageId(initialFetchOffset + 6));
         
-        assertThat("The offset manager should not skip past offset 5 which is still pending", manager.findNextCommitOffset().offset(), is(initialFetchOffset + 3));
+        assertThat("The offset manager should not skip past offset 5 which is still pending", manager.findNextCommitOffset(COMMIT_METADATA).offset(), is(initialFetchOffset + 3));
         
         manager.addToAckMsgs(getMessageId(initialFetchOffset + 5));
         
-        assertThat("The offset manager should skip past the gap in acked messages, since the messages were not emitted", 
-            manager.findNextCommitOffset().offset(), is(initialFetchOffset + 7));
+        assertThat("The offset manager should skip past the gap in acked messages, since the messages were not emitted",
+            manager.findNextCommitOffset(COMMIT_METADATA), is(new OffsetAndMetadata(initialFetchOffset + 7, COMMIT_METADATA)));
     }
     
     @Test
@@ -71,17 +74,17 @@ public class OffsetManagerTest {
         manager.addToEmitMsgs(initialFetchOffset + 6);
         manager.addToAckMsgs(getMessageId(initialFetchOffset + 6));
         
-        assertThat("The offset manager should not skip past offset 5 which is still pending", manager.findNextCommitOffset(), is(nullValue()));
+        assertThat("The offset manager should not skip past offset 5 which is still pending", manager.findNextCommitOffset(COMMIT_METADATA), is(nullValue()));
         
         manager.addToAckMsgs(getMessageId(initialFetchOffset + 5));
         
         assertThat("The offset manager should skip past the gap in acked messages, since the messages were not emitted", 
-            manager.findNextCommitOffset().offset(), is(initialFetchOffset + 7));
+            manager.findNextCommitOffset(COMMIT_METADATA), is(new OffsetAndMetadata(initialFetchOffset + 7, COMMIT_METADATA)));
     }
 
     @Test
     public void testFindNextCommittedOffsetWithNoAcks() {
-        OffsetAndMetadata nextCommitOffset = manager.findNextCommitOffset();
+        OffsetAndMetadata nextCommitOffset = manager.findNextCommitOffset(COMMIT_METADATA);
         assertThat("There shouldn't be a next commit offset when nothing has been acked", nextCommitOffset, is(nullValue()));
     }
 
@@ -92,7 +95,7 @@ public class OffsetManagerTest {
          * lastProcessedMessageOffset + 1. "
          */
         emitAndAckMessage(getMessageId(initialFetchOffset));
-        OffsetAndMetadata nextCommitOffset = manager.findNextCommitOffset();
+        OffsetAndMetadata nextCommitOffset = manager.findNextCommitOffset(COMMIT_METADATA);
         assertThat("The next commit offset should be one past the processed message offset", nextCommitOffset.offset(), is(initialFetchOffset + 1));
     }
 
@@ -100,7 +103,7 @@ public class OffsetManagerTest {
     public void testFindNextCommitOffsetWithMultipleOutOfOrderAcks() {
         emitAndAckMessage(getMessageId(initialFetchOffset + 1));
         emitAndAckMessage(getMessageId(initialFetchOffset));
-        OffsetAndMetadata nextCommitOffset = manager.findNextCommitOffset();
+        OffsetAndMetadata nextCommitOffset = manager.findNextCommitOffset(COMMIT_METADATA);
         assertThat("The next commit offset should be one past the processed message offset", nextCommitOffset.offset(), is(initialFetchOffset + 2));
     }
 
@@ -109,7 +112,7 @@ public class OffsetManagerTest {
         emitAndAckMessage(getMessageId(initialFetchOffset + 2));
         manager.addToEmitMsgs(initialFetchOffset + 1);
         emitAndAckMessage(getMessageId(initialFetchOffset));
-        OffsetAndMetadata nextCommitOffset = manager.findNextCommitOffset();
+        OffsetAndMetadata nextCommitOffset = manager.findNextCommitOffset(COMMIT_METADATA);
         assertThat("The next commit offset should cover the sequential acked offsets", nextCommitOffset.offset(), is(initialFetchOffset + 1));
     }
 
@@ -123,7 +126,7 @@ public class OffsetManagerTest {
          */
         emitAndAckMessage(getMessageId(initialFetchOffset + 2));
         emitAndAckMessage(getMessageId(initialFetchOffset));
-        OffsetAndMetadata nextCommitOffset = manager.findNextCommitOffset();
+        OffsetAndMetadata nextCommitOffset = manager.findNextCommitOffset(COMMIT_METADATA);
         assertThat("The next commit offset should cover all the acked offsets, since the offset in the gap hasn't been emitted and doesn't exist",
             nextCommitOffset.offset(), is(initialFetchOffset + 3));
     }
@@ -132,7 +135,7 @@ public class OffsetManagerTest {
     public void testFindNextCommitOffsetWithUnackedOffsetGap() {
         manager.addToEmitMsgs(initialFetchOffset + 1);
         emitAndAckMessage(getMessageId(initialFetchOffset));
-        OffsetAndMetadata nextCommitOffset = manager.findNextCommitOffset();
+        OffsetAndMetadata nextCommitOffset = manager.findNextCommitOffset(COMMIT_METADATA);
         assertThat("The next commit offset should cover the contiguously acked offsets", nextCommitOffset.offset(), is(initialFetchOffset + 1));
     }
     
@@ -140,7 +143,7 @@ public class OffsetManagerTest {
     public void testFindNextCommitOffsetWhenTooLowOffsetIsAcked() {
         OffsetManager startAtHighOffsetManager = new OffsetManager(testTp, 10);
         emitAndAckMessage(getMessageId(0));
-        OffsetAndMetadata nextCommitOffset = startAtHighOffsetManager.findNextCommitOffset();
+        OffsetAndMetadata nextCommitOffset = startAtHighOffsetManager.findNextCommitOffset(COMMIT_METADATA);
         assertThat("Acking an offset earlier than the committed offset should have no effect", nextCommitOffset, is(nullValue()));
     }
     
@@ -170,4 +173,24 @@ public class OffsetManagerTest {
         manager.addToAckMsgs(msgId);
     }
 
+    @Test
+    public void testGetNthUncommittedOffsetAfterCommittedOffset() { 
+        manager.addToEmitMsgs(initialFetchOffset + 1);
+        manager.addToEmitMsgs(initialFetchOffset + 2);
+        manager.addToEmitMsgs(initialFetchOffset + 5);
+        manager.addToEmitMsgs(initialFetchOffset + 30);
+        
+        assertThat("The third uncommitted offset should be 5", manager.getNthUncommittedOffsetAfterCommittedOffset(3), is(initialFetchOffset + 5L));
+        assertThat("The fourth uncommitted offset should be 30", manager.getNthUncommittedOffsetAfterCommittedOffset(4), is(initialFetchOffset + 30L));
+        
+        expect.expect(NoSuchElementException.class);
+        manager.getNthUncommittedOffsetAfterCommittedOffset(5);
+    }
+
+    @Test
+    public void testCommittedFlagSetOnCommit() throws Exception {
+        assertFalse(manager.hasCommitted());
+        manager.commit(mock(OffsetAndMetadata.class));
+        assertTrue(manager.hasCommitted());
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/subscription/ManualPartitionSubscriptionTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/subscription/ManualPartitionSubscriptionTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/subscription/ManualPartitionSubscriptionTest.java
new file mode 100644
index 0000000..9a2a682
--- /dev/null
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/subscription/ManualPartitionSubscriptionTest.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2017 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.subscription;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyList;
+import static org.mockito.Mockito.inOrder;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.when;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+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.ManualPartitionSubscription;
+import org.apache.storm.kafka.spout.ManualPartitioner;
+import org.apache.storm.kafka.spout.TopicFilter;
+import org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
+import org.apache.storm.task.TopologyContext;
+import org.junit.Test;
+import org.mockito.InOrder;
+
+public class ManualPartitionSubscriptionTest {
+
+    @Test
+    public void testCanReassignPartitions() {
+        ManualPartitioner partitionerMock = mock(ManualPartitioner.class);
+        TopicFilter filterMock = mock(TopicFilter.class);
+        KafkaConsumer<String, String> consumerMock = mock(KafkaConsumer.class);
+        ConsumerRebalanceListener listenerMock = mock(ConsumerRebalanceListener.class);
+        TopologyContext contextMock = mock(TopologyContext.class);
+        ManualPartitionSubscription subscription = new ManualPartitionSubscription(partitionerMock, filterMock);
+        
+        List<TopicPartition> onePartition = Collections.singletonList(new TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 0));
+        List<TopicPartition> twoPartitions = new ArrayList<>();
+        twoPartitions.add(new TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 0));
+        twoPartitions.add(new TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 1));
+        when(partitionerMock.partition(anyList(), any(TopologyContext.class)))
+            .thenReturn(onePartition)
+            .thenReturn(twoPartitions);
+        
+        //Set the first assignment
+        subscription.subscribe(consumerMock, listenerMock, contextMock);
+        
+        InOrder inOrder = inOrder(consumerMock, listenerMock);
+        inOrder.verify(consumerMock).assign(new HashSet<>(onePartition));
+        inOrder.verify(listenerMock).onPartitionsAssigned(new HashSet<>(onePartition));
+        
+        reset(consumerMock, listenerMock);
+        
+        when(consumerMock.assignment()).thenReturn(new HashSet<>(onePartition));
+        
+        //Update to set the second assignment
+        subscription.refreshAssignment();
+        
+        //The partition revocation hook must be called before the new partitions are assigned to the consumer,
+        //to allow the revocation hook to commit offsets for the revoked partitions.
+        inOrder.verify(listenerMock).onPartitionsRevoked(new HashSet<>(onePartition));
+        inOrder.verify(consumerMock).assign(new HashSet<>(twoPartitions));
+        inOrder.verify(listenerMock).onPartitionsAssigned(new HashSet<>(twoPartitions));
+    }
+    
+}


[6/7] storm git commit: STORM-2936 Overwrite latest storm-kafka-client 1.x-branch into 1.1.x-branch

Posted by ka...@apache.org.
STORM-2936 Overwrite latest storm-kafka-client 1.x-branch into 1.1.x-branch

* origin commit sha (1.x-branch): 74ca79596c9533b05294ca5de9565ba777b772e9
* overwriting requires small change of storm-core
  * added InterfaceStability
* also overwrote doc and storm-kafka-client-examples as well


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

Branch: refs/heads/1.1.x-branch
Commit: 324bc95b18ce3a477096842a364728cbf932831d
Parents: e906180
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Tue Feb 6 08:17:52 2018 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Wed Feb 7 07:26:22 2018 +0900

----------------------------------------------------------------------
 docs/storm-kafka-client.md                      | 181 ++---
 .../TridentKafkaClientWordCountNamedTopics.java |  27 +-
 ...identKafkaClientWordCountWildcardTopics.java |   5 +-
 external/storm-kafka-client/pom.xml             |   8 +
 .../kafka/spout/EmptyKafkaTupleListener.java    |  53 ++
 .../apache/storm/kafka/spout/KafkaSpout.java    | 532 +++++++++----
 .../storm/kafka/spout/KafkaSpoutConfig.java     | 785 ++++++++++++++-----
 .../storm/kafka/spout/KafkaSpoutMessageId.java  |  26 +-
 .../storm/kafka/spout/KafkaTupleListener.java   |  83 ++
 .../spout/ManualPartitionSubscription.java      |   8 +-
 .../storm/kafka/spout/ManualPartitioner.java    |   1 +
 .../storm/kafka/spout/NamedSubscription.java    |   4 +-
 .../storm/kafka/spout/PatternSubscription.java  |   4 +-
 .../kafka/spout/SerializableDeserializer.java   |   6 +-
 .../apache/storm/kafka/spout/Subscription.java  |   5 +-
 .../kafka/spout/internal/CommitMetadata.java    |  63 ++
 .../spout/internal/CommitMetadataManager.java   |  91 +++
 .../internal/KafkaConsumerFactoryDefault.java   |   3 +-
 .../kafka/spout/internal/OffsetManager.java     | 111 ++-
 .../kafka/spout/metrics/KafkaOffsetMetric.java  | 141 ++++
 .../trident/KafkaTridentSpoutBatchMetadata.java |  78 +-
 .../spout/trident/KafkaTridentSpoutEmitter.java | 108 +--
 .../spout/trident/KafkaTridentSpoutManager.java |  13 +-
 .../spout/trident/KafkaTridentSpoutOpaque.java  |  18 +-
 .../KafkaTridentSpoutOpaqueCoordinator.java     |  26 +-
 .../spout/trident/TopicPartitionSerializer.java |  47 ++
 .../storm/kafka/trident/TridentKafkaState.java  |  79 +-
 .../kafka/trident/TridentKafkaStateFactory.java |  17 +-
 .../kafka/trident/TridentKafkaStateUpdater.java |   6 +-
 .../kafka/spout/KafkaSpoutAbstractTest.java     | 179 +++++
 .../storm/kafka/spout/KafkaSpoutCommitTest.java | 135 ----
 .../storm/kafka/spout/KafkaSpoutConfigTest.java | 206 ++++-
 .../storm/kafka/spout/KafkaSpoutEmitTest.java   | 204 ++---
 .../KafkaSpoutLogCompactionSupportTest.java     | 223 ++++++
 .../spout/KafkaSpoutMessagingGuaranteeTest.java | 259 ++++++
 .../kafka/spout/KafkaSpoutReactivationTest.java | 145 ++++
 .../kafka/spout/KafkaSpoutRebalanceTest.java    |  63 +-
 .../kafka/spout/KafkaSpoutRetryLimitTest.java   | 104 +--
 .../kafka/spout/KafkaSpoutSingleTopicTest.java  | 379 +++++++++
 ...outTopologyDeployActivateDeactivateTest.java | 116 +++
 .../kafka/spout/MaxUncommittedOffsetTest.java   | 120 ++-
 .../storm/kafka/spout/NamedTopicFilterTest.java |   3 +-
 .../kafka/spout/PatternTopicFilterTest.java     |   2 +
 .../kafka/spout/SingleTopicKafkaSpoutTest.java  | 401 ----------
 .../spout/SingleTopicKafkaUnitSetupHelper.java  |  89 +++
 .../SpoutWithMockedConsumerSetupHelper.java     | 171 ++++
 .../SingleTopicKafkaSpoutConfiguration.java     |  49 +-
 .../kafka/spout/internal/OffsetManagerTest.java |  49 +-
 .../ManualPartitionSubscriptionTest.java        |  81 ++
 .../test/KafkaSpoutTopologyMainNamedTopics.java |   4 +-
 .../KafkaSpoutTopologyMainWildcardTopics.java   |   4 +-
 .../KafkaTridentSpoutBatchMetadataTest.java     |  66 ++
 pom.xml                                         |   2 +-
 .../storm/annotation/InterfaceStability.java    |  54 ++
 54 files changed, 4112 insertions(+), 1525 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/docs/storm-kafka-client.md
----------------------------------------------------------------------
diff --git a/docs/storm-kafka-client.md b/docs/storm-kafka-client.md
index 93f8d16..2992dd6 100644
--- a/docs/storm-kafka-client.md
+++ b/docs/storm-kafka-client.md
@@ -21,8 +21,8 @@ You need to provide implementations for the following 2 interfaces
 These interfaces have 2 methods defined:
 
 ```java
-    K getKeyFromTuple(Tuple/TridentTuple tuple);
-    V getMessageFromTuple(Tuple/TridentTuple tuple);
+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
@@ -58,10 +58,9 @@ 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);
-
+```java
+Config config = new Config();
+config.put("kafka.topic.wildcard.match",true);
 ```
 
 After this you can specify a wildcard topic for matching e.g. clickstream.*.log.  This will match all streams matching clickstream.my.log, clickstream.cart.log etc
@@ -72,65 +71,65 @@ After this you can specify a wildcard topic for matching e.g. clickstream.*.log.
 For the bolt :
 
 ```java
-        TopologyBuilder builder = new TopologyBuilder();
-
-        Fields fields = new Fields("key", "message");
-        FixedBatchSpout spout = new FixedBatchSpout(fields, 4,
-                    new Values("storm", "1"),
-                    new Values("trident", "1"),
-                    new Values("needs", "1"),
-                    new Values("javadoc", "1")
-        );
-        spout.setCycle(true);
-        builder.setSpout("spout", spout, 5);
-        //set producer properties.
-        Properties props = new Properties();
-        props.put("bootstrap.servers", "localhost:9092");
-        props.put("acks", "1");
-        props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-        props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-
-        KafkaBolt bolt = new KafkaBolt()
-                .withProducerProperties(props)
-                .withTopicSelector(new DefaultTopicSelector("test"))
-                .withTupleToKafkaMapper(new FieldNameBasedTupleToKafkaMapper());
-        builder.setBolt("forwardToKafka", bolt, 8).shuffleGrouping("spout");
-
-        Config conf = new Config();
-
-        StormSubmitter.submitTopology("kafkaboltTest", conf, builder.createTopology());
+TopologyBuilder builder = new TopologyBuilder();
+
+Fields fields = new Fields("key", "message");
+FixedBatchSpout spout = new FixedBatchSpout(fields, 4,
+            new Values("storm", "1"),
+            new Values("trident", "1"),
+            new Values("needs", "1"),
+            new Values("javadoc", "1")
+);
+spout.setCycle(true);
+builder.setSpout("spout", spout, 5);
+//set producer properties.
+Properties props = new Properties();
+props.put("bootstrap.servers", "localhost:9092");
+props.put("acks", "1");
+props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+
+KafkaBolt bolt = new KafkaBolt()
+        .withProducerProperties(props)
+        .withTopicSelector(new DefaultTopicSelector("test"))
+        .withTupleToKafkaMapper(new FieldNameBasedTupleToKafkaMapper());
+builder.setBolt("forwardToKafka", bolt, 8).shuffleGrouping("spout");
+
+Config conf = new Config();
+
+StormSubmitter.submitTopology("kafkaboltTest", conf, builder.createTopology());
 ```
 
 For Trident:
 
 ```java
-        Fields fields = new Fields("word", "count");
-        FixedBatchSpout spout = new FixedBatchSpout(fields, 4,
-                new Values("storm", "1"),
-                new Values("trident", "1"),
-                new Values("needs", "1"),
-                new Values("javadoc", "1")
-        );
-        spout.setCycle(true);
-
-        TridentTopology topology = new TridentTopology();
-        Stream stream = topology.newStream("spout1", spout);
-
-        //set producer properties.
-        Properties props = new Properties();
-        props.put("bootstrap.servers", "localhost:9092");
-        props.put("acks", "1");
-        props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-        props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-
-        TridentKafkaStateFactory stateFactory = new TridentKafkaStateFactory()
-                .withProducerProperties(props)
-                .withKafkaTopicSelector(new DefaultTopicSelector("test"))
-                .withTridentTupleToKafkaMapper(new FieldNameBasedTupleToKafkaMapper("word", "count"));
-        stream.partitionPersist(stateFactory, fields, new TridentKafkaStateUpdater(), new Fields());
-
-        Config conf = new Config();
-        StormSubmitter.submitTopology("kafkaTridentTest", conf, topology.build());
+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 TridentKafkaStateUpdater(), new Fields());
+
+Config conf = new Config();
+StormSubmitter.submitTopology("kafkaTridentTest", conf, topology.build());
 ```
 
 ## Reading From kafka (Spouts)
@@ -147,30 +146,21 @@ a spout.
 `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.
+If you are using the Builder Constructors instead of one of the `builder` methods, you will 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 deserializers can be specified via the consumer properties set with `setProp`. See the KafkaConsumer configuration documentation for details.
 
 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
+for the first time. The allowed values are listed in the [FirstPollOffsetStrategy javadocs](javadocs/org/apache/storm/kafka/spout/KafkaSpoutConfig.FirstPollOffsetStrategy.html).
 
- * `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`.
+`setProcessingGuarantee` lets you configure what processing guarantees the spout will provide. This affects how soon consumed offsets can be committed, and the frequency of commits. See the [ProcessingGuarantee javadoc](javadocs/org/apache/storm/kafka/spout/KafkaSpoutConfig.ProcessingGuarantee.html) for details.
 
 `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.
+`setProp` and `setProps` can be used to set KafkaConsumer properties. The list of these properties can be found in the KafkaConsumer configuration documentation on the [Kafka website](http://kafka.apache.org/documentation.html#consumerconfigs). Note that KafkaConsumer autocommit is unsupported. The KafkaSpoutConfig constructor will throw an exception if the "enable.auto.commit" property is set, and the consumer used by the spout will always have that property set to false. You can configure similar behavior to autocommit through the `setProcessingGuarantee` method on the KafkaSpoutConfig builder.
 
 ### Usage Examples
 
@@ -236,7 +226,7 @@ output topic it will throw an exception and not continue.
 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
+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.
 
@@ -250,16 +240,13 @@ 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,
+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)
+### Manual Partition Assigment (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.
+By default the KafkaSpout instances will be assigned partitions using a round robin strategy. If you need to customize partition assignment, you must implement the `ManualPartitioner` interface. The implementation can be passed to the `ManualPartitionSubscription` constructor, and the `Subscription` can then be set in the `KafkaSpoutConfig` via the `KafkaSpoutConfig.Builder` constructor. Please take care when supplying a custom implementation, since an incorrect `ManualPartitioner` implementation could leave some partitions unread, or concurrently read by multiple spout instances. See the `RoundRobinManualPartitioner` for an example of how to implement this functionality.
 
 ## Use the Maven Shade Plugin to Build the Uber Jar
 
@@ -326,7 +313,7 @@ use Kafka-clients 0.10.0.0, you would use the following dependency in your `pom.
 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 - 
+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 
@@ -334,7 +321,7 @@ When selecting a kafka client version, you should ensure -
 
 # 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). 
+The Kafka spout provides two internal parameters to control its performance. The parameters can be set using the [setOffsetCommitPeriodMs](javadocs/org/apache/storm/kafka/spout/KafkaSpoutConfig.Builder.html#setOffsetCommitPeriodMs-long-) and [setMaxUncommittedOffsets](javadocs/org/apache/storm/kafka/spout/KafkaSpoutConfig.Builder.html#setMaxUncommittedOffsets-int-) methods. 
 
 * "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
@@ -344,39 +331,31 @@ The [Kafka consumer config] (http://kafka.apache.org/documentation.html#consumer
 
 * “fetch.min.bytes”
 * “fetch.max.wait.ms”
-* [Kafka Consumer] (http://kafka.apache.org/090/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html) instance poll timeout, which is specified for each Kafka spout using the [KafkaSpoutConfig] (https://github.com/apache/storm/blob/1.0.x-branch/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java) method [setPollTimeoutMs] (https://github.com/apache/storm/blob/1.0.x-branch/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java#L180-L184)
+* [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 [setPollTimeoutMs](javadocs/org/apache/storm/kafka/spout/KafkaSpoutConfig.Builder.html#setPollTimeoutMs-long-) method.
 <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/)
+Currently the Kafka spout has has the following default values, which have been 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/>
 
-# Kafka AutoCommitMode 
+# Tuple Tracking
 
-If reliability isn't important to you -- that is, you don't care about losing tuples in failure situations --, and want to remove the overhead of tuple tracking, then you can run a KafkaSpout with AutoCommitMode.
-
-To enable it, you need to:
-* set Config.TOPOLOGY_ACKERS to 0;
-* enable *AutoCommitMode* in Kafka consumer configuration; 
-
-Here's one example to set AutoCommitMode in KafkaSpout:
+By default the spout only tracks emitted tuples when the processing guarantee is AT_LEAST_ONCE. It may be necessary to track
+emitted tuples with other processing guarantees to benefit from Storm features such as showing complete latency in the UI,
+or enabling backpressure with Config.TOPOLOGY_MAX_SPOUT_PENDING.
 
 ```java
 KafkaSpoutConfig<String, String> kafkaConf = KafkaSpoutConfig
-		.builder(String bootstrapServers, String ... topics)
-		.setProp(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true")
-		.setFirstPollOffsetStrategy(FirstPollOffsetStrategy.EARLIEST)
-		.build();
+  .builder(String bootstrapServers, String ... topics)
+  .setProcessingGuarantee(ProcessingGuarantee.AT_MOST_ONCE)
+  .setTupleTrackingEnforced(true)
 ```
 
-*Note that it's not exactly At-Most-Once in Storm, as offset is committed periodically by Kafka consumer, some tuples could be replayed when KafkaSpout is crashed.*
-
-
-
+Note: This setting has no effect with AT_LEAST_ONCE processing guarantee, where tuple tracking is required and therefore always enabled.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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 83d6884..1e6f297 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,6 +18,14 @@
 
 package org.apache.storm.kafka.trident;
 
+import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.storm.Config;
 import org.apache.storm.StormSubmitter;
@@ -33,19 +41,6 @@ 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.List;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
-
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.List;
-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";
@@ -65,12 +60,12 @@ public class TridentKafkaClientWordCountNamedTopics {
         public List<Object> apply(ConsumerRecord<String, String> record) {
             return new Values(record.value());
         }
-    };
+    }
 
     protected KafkaSpoutConfig<String,String> newKafkaSpoutConfig() {
         return KafkaSpoutConfig.builder(KAFKA_LOCAL_BROKER, TOPIC_1, TOPIC_2)
-                .setGroupId("kafkaSpoutTestGroup_" + System.nanoTime())
-                .setMaxPartitionFectchBytes(200)
+                .setProp(ConsumerConfig.GROUP_ID_CONFIG, "kafkaSpoutTestGroup_" + System.nanoTime())
+                .setProp(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 200)
                 .setRecordTranslator(JUST_VALUE_FUNC, new Fields("str"))
                 .setRetry(newRetryService())
                 .setOffsetCommitPeriodMs(10_000)

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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 94a9765..5131a21 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
@@ -22,6 +22,7 @@ import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrat
 
 import java.util.List;
 import java.util.regex.Pattern;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
 
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.storm.kafka.spout.Func;
@@ -41,8 +42,8 @@ public class TridentKafkaClientWordCountWildcardTopics extends TridentKafkaClien
     
     protected KafkaSpoutConfig<String,String> newKafkaSpoutConfig() {
         return KafkaSpoutConfig.builder("127.0.0.1:9092", TOPIC_WILDCARD_PATTERN)
-                .setGroupId("kafkaSpoutTestGroup")
-                .setMaxPartitionFectchBytes(200)
+                .setProp(ConsumerConfig.GROUP_ID_CONFIG, "kafkaSpoutTestGroup")
+                .setProp(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 200)
                 .setRecordTranslator(JUST_VALUE_FUNC, new Fields("str"))
                 .setRetry(newRetryService())
                 .setOffsetCommitPeriodMs(10_000)

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/pom.xml b/external/storm-kafka-client/pom.xml
index 304e703..0ba7d4d 100644
--- a/external/storm-kafka-client/pom.xml
+++ b/external/storm-kafka-client/pom.xml
@@ -71,6 +71,14 @@
                 </exclusion>
             </exclusions>
         </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
         <!--test dependencies -->
         <dependency>
             <groupId>org.mockito</groupId>

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/EmptyKafkaTupleListener.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/EmptyKafkaTupleListener.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/EmptyKafkaTupleListener.java
new file mode 100644
index 0000000..621fecd
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/EmptyKafkaTupleListener.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.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.storm.task.TopologyContext;
+
+public final class EmptyKafkaTupleListener implements KafkaTupleListener {
+
+    @Override
+    public void open(Map<String, Object> conf, TopologyContext context) { }
+
+    @Override
+    public void onEmit(List<Object> tuple, KafkaSpoutMessageId msgId) { }
+
+    @Override
+    public void onAck(KafkaSpoutMessageId msgId) { }
+
+    @Override
+    public void onPartitionsReassigned(Collection<TopicPartition> topicPartitions) { }
+
+    @Override
+    public void onRetry(KafkaSpoutMessageId msgId) { }
+
+    @Override
+    public void onMaxRetryReached(KafkaSpoutMessageId msgId) { }
+
+    @Override
+    public String toString() {
+        return "EmptyKafkaTupleListener";
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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 207e9b4..4464e68 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
@@ -23,18 +23,23 @@ import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrat
 import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST;
 import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_LATEST;
 
+import com.google.common.annotations.VisibleForTesting;
+
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Supplier;
 import org.apache.commons.lang.Validate;
+
 import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.ConsumerRecords;
@@ -43,19 +48,25 @@ 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.KafkaSpoutConfig.ProcessingGuarantee;
+import org.apache.storm.kafka.spout.internal.CommitMetadataManager;
 import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
 import org.apache.storm.kafka.spout.internal.KafkaConsumerFactoryDefault;
 import org.apache.storm.kafka.spout.internal.OffsetManager;
 import org.apache.storm.kafka.spout.internal.Timer;
+import org.apache.storm.kafka.spout.metrics.KafkaOffsetMetric;
 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.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class KafkaSpout<K, V> extends BaseRichSpout {
+
     private static final long serialVersionUID = 4151921085047987154L;
     //Initial delay for the commit and subscription refresh timers
     public static final long TIMER_DELAY_MS = 500;
@@ -66,30 +77,39 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
 
     // Kafka
     private final KafkaSpoutConfig<K, V> kafkaSpoutConfig;
-    private KafkaConsumerFactory kafkaConsumerFactory;
+    private KafkaConsumerFactory<K, V> kafkaConsumerFactory;
     private transient KafkaConsumer<K, V> kafkaConsumer;
-    private transient boolean consumerAutoCommitMode;
 
     // Bookkeeping
-    private transient FirstPollOffsetStrategy firstPollOffsetStrategy;  // Strategy to determine the fetch offset of the first realized by the spout upon activation
-    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.
+    // Strategy to determine the fetch offset of the first realized by the spout upon activation
+    private transient FirstPollOffsetStrategy firstPollOffsetStrategy;
+    // Class that has the logic to handle tuple failure.
+    private transient KafkaSpoutRetryService retryService;
+    // Handles tuple events (emit, ack etc.)
+    private transient KafkaTupleListener tupleListener;
+    // timer == null only if the processing guarantee is at-most-once
+    private transient Timer commitTimer;
     // Initialization is only complete after the first call to  KafkaSpoutConsumerRebalanceListener.onPartitionsAssigned()
 
-    private transient Map<TopicPartition, OffsetManager> offsetManagers;// Tuples that were successfully acked/emitted. These tuples will be committed periodically when the commit timer expires, or after a consumer rebalance, or during 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. Not used if it's AutoCommitMode
-    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. Not used if auto commit mode is enabled.
-    private transient Timer refreshSubscriptionTimer;                   // Triggers when a subscription should be refreshed
+    // Tuples that were successfully acked/emitted. These tuples will be committed periodically when the commit timer expires,
+    // or after a consumer rebalance, or during close/deactivate. Always empty if processing guarantee is none or at-most-once.
+    private transient Map<TopicPartition, OffsetManager> offsetManagers;
+    // Tuples that have been emitted but that are "on the wire", i.e. pending being acked or failed.
+    // Always empty if processing guarantee is none or at-most-once
+    private transient Set<KafkaSpoutMessageId> emitted;
+    // Records that have been polled and are queued to be emitted in the nextTuple() call. One record is emitted per nextTuple()
+    private transient Map<TopicPartition, List<ConsumerRecord<K, V>>> waitingToEmit;
+    // Triggers when a subscription should be refreshed
+    private transient Timer refreshSubscriptionTimer;
     private transient TopologyContext context;
-
+    private transient CommitMetadataManager commitMetadataManager;
+    private transient KafkaOffsetMetric kafkaOffsetMetric;
 
     public KafkaSpout(KafkaSpoutConfig<K, V> kafkaSpoutConfig) {
         this(kafkaSpoutConfig, new KafkaConsumerFactoryDefault<K, V>());
     }
 
-    //This constructor is here for testing
+    @VisibleForTesting
     KafkaSpout(KafkaSpoutConfig<K, V> kafkaSpoutConfig, KafkaConsumerFactory<K, V> kafkaConsumerFactory) {
         this.kafkaConsumerFactory = kafkaConsumerFactory;
         this.kafkaSpoutConfig = kafkaSpoutConfig;
@@ -97,65 +117,103 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
 
     @Override
     public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
-        initialized = false;
         this.context = context;
 
         // Spout internals
         this.collector = collector;
-        numUncommittedOffsets = 0;
 
         // Offset management
         firstPollOffsetStrategy = kafkaSpoutConfig.getFirstPollOffsetStrategy();
-        // with AutoCommitMode, offset will be periodically committed in the background by Kafka consumer
-        consumerAutoCommitMode = kafkaSpoutConfig.isConsumerAutoCommitMode();
 
         // Retries management
         retryService = kafkaSpoutConfig.getRetryService();
 
-        if (!consumerAutoCommitMode) {     // If it is auto commit, no need to commit offsets manually
+        tupleListener = kafkaSpoutConfig.getTupleListener();
+
+        if (kafkaSpoutConfig.getProcessingGuarantee() != KafkaSpoutConfig.ProcessingGuarantee.AT_MOST_ONCE) {
+            // In at-most-once mode the offsets are committed after every poll, and not periodically as controlled by the timer
             commitTimer = new Timer(TIMER_DELAY_MS, kafkaSpoutConfig.getOffsetsCommitPeriodMs(), TimeUnit.MILLISECONDS);
         }
         refreshSubscriptionTimer = new Timer(TIMER_DELAY_MS, kafkaSpoutConfig.getPartitionRefreshPeriodMs(), TimeUnit.MILLISECONDS);
 
         offsetManagers = new HashMap<>();
         emitted = new HashSet<>();
-        waitingToEmit = Collections.emptyListIterator();
+        waitingToEmit = new HashMap<>();
+        commitMetadataManager = new CommitMetadataManager(context, kafkaSpoutConfig.getProcessingGuarantee());
+
+        tupleListener.open(conf, context);
+        if (canRegisterMetrics()) {
+            registerMetric();
+        }
 
         LOG.info("Kafka Spout opened with the following configuration: {}", kafkaSpoutConfig);
     }
 
-    // =========== Consumer Rebalance Listener - On the same thread as the caller ===========
+    private void registerMetric() {
+        LOG.info("Registering Spout Metrics");
+        kafkaOffsetMetric = new KafkaOffsetMetric(new Supplier() {
+            @Override
+            public Object get() {
+                return Collections.unmodifiableMap(offsetManagers);
+            }
+        }, new Supplier() {
+            @Override
+            public Object get() {
+                return kafkaConsumer;
+            }
+        });
+        context.registerMetric("kafkaOffset", kafkaOffsetMetric, kafkaSpoutConfig.getMetricsTimeBucketSizeInSecs());
+    }
 
+    private boolean canRegisterMetrics() {
+        try {
+            KafkaConsumer.class.getDeclaredMethod("beginningOffsets", Collection.class);
+        } catch (NoSuchMethodException e) {
+            LOG.warn("Minimum required kafka-clients library version to enable metrics is 0.10.1.0. Disabling spout metrics.");
+            return false;
+        }
+        return true;
+    }
+
+    private boolean isAtLeastOnceProcessing() {
+        return kafkaSpoutConfig.getProcessingGuarantee() == KafkaSpoutConfig.ProcessingGuarantee.AT_LEAST_ONCE;
+    }
+
+    // =========== Consumer Rebalance Listener - On the same thread as the caller ===========
     private class KafkaSpoutConsumerRebalanceListener implements ConsumerRebalanceListener {
+
         private Collection<TopicPartition> previousAssignment = new HashSet<>();
-        
+
         @Override
         public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
-            LOG.info("Partitions revoked. [consumer-group={}, consumer={}, topic-partitions={}]",
-                    kafkaSpoutConfig.getConsumerGroupId(), kafkaConsumer, partitions);
             previousAssignment = partitions;
-            if (!consumerAutoCommitMode && initialized) {
-                initialized = false;
-                commitOffsetsForAckedTuples();
+
+            LOG.info("Partitions revoked. [consumer-group={}, consumer={}, topic-partitions={}]",
+                kafkaSpoutConfig.getConsumerGroupId(), kafkaConsumer, partitions);
+
+            if (isAtLeastOnceProcessing()) {
+                commitOffsetsForAckedTuples(new HashSet<>(partitions));
             }
         }
 
         @Override
         public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
-            LOG.info("Partitions reassignment. [consumer-group={}, consumer={}, topic-partitions={}]",
-                    kafkaSpoutConfig.getConsumerGroupId(), kafkaConsumer, partitions);
+            LOG.info("Partitions reassignment. [task-ID={}, consumer-group={}, consumer={}, topic-partitions={}]",
+                context.getThisTaskId(), kafkaSpoutConfig.getConsumerGroupId(), kafkaConsumer, partitions);
 
             initialize(partitions);
+            tupleListener.onPartitionsReassigned(partitions);
         }
 
         private void initialize(Collection<TopicPartition> partitions) {
-            if (!consumerAutoCommitMode) {
-                offsetManagers.keySet().retainAll(partitions);   // remove from acked all partitions that are no longer assigned to this spout
+            if (isAtLeastOnceProcessing()) {
+                // remove offsetManagers for all partitions that are no longer assigned to this spout
+                offsetManagers.keySet().retainAll(partitions);
                 retryService.retainAll(partitions);
 
                 /*
-                 * Emitted messages for partitions that are no longer assigned to this spout can't
-                 * be acked and should not be retried, hence remove them from emitted collection.
+                 * Emitted messages for partitions that are no longer assigned to this spout can't be acked and should not be retried, hence
+                 * remove them from emitted collection.
                  */
                 Iterator<KafkaSpoutMessageId> msgIdIterator = emitted.iterator();
                 while (msgIdIterator.hasNext()) {
@@ -165,76 +223,89 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
                     }
                 }
             }
+            waitingToEmit.keySet().retainAll(partitions);
 
             Set<TopicPartition> newPartitions = new HashSet<>(partitions);
             newPartitions.removeAll(previousAssignment);
-            for (TopicPartition tp : newPartitions) {
-                final OffsetAndMetadata committedOffset = kafkaConsumer.committed(tp);
-                final long fetchOffset = doSeek(tp, committedOffset);
+            for (TopicPartition newTp : newPartitions) {
+                final OffsetAndMetadata committedOffset = kafkaConsumer.committed(newTp);
+                final long fetchOffset = doSeek(newTp, committedOffset);
+                LOG.debug("Set consumer position to [{}] for topic-partition [{}] with [{}] and committed offset [{}]",
+                    fetchOffset, newTp, firstPollOffsetStrategy, committedOffset);
                 // If this partition was previously assigned to this spout, leave the acked offsets as they were to resume where it left off
-                if (!consumerAutoCommitMode && !offsetManagers.containsKey(tp)) {
-                    offsetManagers.put(tp, new OffsetManager(tp, fetchOffset));
+                if (isAtLeastOnceProcessing() && !offsetManagers.containsKey(newTp)) {
+                    offsetManagers.put(newTp, new OffsetManager(newTp, fetchOffset));
                 }
             }
-            initialized = true;
             LOG.info("Initialization complete");
         }
 
         /**
          * Sets the cursor to the location dictated by the first poll strategy and returns the fetch offset.
          */
-        private long doSeek(TopicPartition tp, OffsetAndMetadata committedOffset) {
-            if (committedOffset != null) {             // offset was committed for this TopicPartition
-                if (firstPollOffsetStrategy.equals(EARLIEST)) {
-                    kafkaConsumer.seekToBeginning(Collections.singleton(tp));
-                } else if (firstPollOffsetStrategy.equals(LATEST)) {
-                    kafkaConsumer.seekToEnd(Collections.singleton(tp));
+        private long doSeek(TopicPartition newTp, OffsetAndMetadata committedOffset) {
+            LOG.trace("Seeking offset for topic-partition [{}] with [{}] and committed offset [{}]",
+                newTp, firstPollOffsetStrategy, committedOffset);
+
+            if (committedOffset != null) {
+                // offset was previously committed for this consumer group and topic-partition, either by this or another topology.
+                if (commitMetadataManager.isOffsetCommittedByThisTopology(newTp, committedOffset, Collections.unmodifiableMap(offsetManagers))) {
+                    // Another KafkaSpout instance (of this topology) already committed, therefore FirstPollOffsetStrategy does not apply.
+                    kafkaConsumer.seek(newTp, committedOffset.offset());
                 } else {
-                    // By default polling starts at the last committed offset, i.e. the first offset that was not marked as processed.
-                    kafkaConsumer.seek(tp, committedOffset.offset());
+                    // offset was not committed by this topology, therefore FirstPollOffsetStrategy applies
+                    // (only when the topology is first deployed).
+                    if (firstPollOffsetStrategy.equals(EARLIEST)) {
+                        kafkaConsumer.seekToBeginning(Collections.singleton(newTp));
+                    } else if (firstPollOffsetStrategy.equals(LATEST)) {
+                        kafkaConsumer.seekToEnd(Collections.singleton(newTp));
+                    } else {
+                        // Resume polling at the last committed offset, i.e. the first offset that is not marked as processed.
+                        kafkaConsumer.seek(newTp, committedOffset.offset());
+                    }
                 }
-            } else {    // no commits have ever been done, so start at the beginning or end depending on the strategy
+            } else {
+                // no offset commits have ever been done for this consumer group and topic-partition,
+                // so start at the beginning or end depending on FirstPollOffsetStrategy
                 if (firstPollOffsetStrategy.equals(EARLIEST) || firstPollOffsetStrategy.equals(UNCOMMITTED_EARLIEST)) {
-                    kafkaConsumer.seekToBeginning(Collections.singleton(tp));
+                    kafkaConsumer.seekToBeginning(Collections.singleton(newTp));
                 } else if (firstPollOffsetStrategy.equals(LATEST) || firstPollOffsetStrategy.equals(UNCOMMITTED_LATEST)) {
-                    kafkaConsumer.seekToEnd(Collections.singleton(tp));
+                    kafkaConsumer.seekToEnd(Collections.singleton(newTp));
                 }
             }
-            return kafkaConsumer.position(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 && !offsetManagers.containsKey(tp)) {
-            offsetManagers.put(tp, new OffsetManager(tp, fetchOffset));
+            return kafkaConsumer.position(newTp);
         }
     }
 
     // ======== Next Tuple =======
-
     @Override
     public void nextTuple() {
         try {
-            if (initialized) {             
-                if (commit()) {
-                    commitOffsetsForAckedTuples();
-                }
+            if (refreshSubscriptionTimer.isExpiredResetOnTrue()) {
+                kafkaSpoutConfig.getSubscription().refreshAssignment();
+            }
 
-                if (poll()) {
-                    try {
-                        setWaitingToEmit(pollKafkaBroker());
-                    } catch (RetriableException e) {
-                        LOG.error("Failed to poll from kafka.", e);
-                    }
+            if (commitTimer != null && commitTimer.isExpiredResetOnTrue()) {
+                if (isAtLeastOnceProcessing()) {
+                    commitOffsetsForAckedTuples(kafkaConsumer.assignment());
+                } else if (kafkaSpoutConfig.getProcessingGuarantee() == ProcessingGuarantee.NO_GUARANTEE) {
+                    Map<TopicPartition, OffsetAndMetadata> offsetsToCommit = 
+                        createFetchedOffsetsMetadata(kafkaConsumer.assignment());
+                    kafkaConsumer.commitAsync(offsetsToCommit, null);
+                    LOG.debug("Committed offsets {} to Kafka", offsetsToCommit);
                 }
+            }
 
-                if (waitingToEmit()) {
-                    emit();
+            PollablePartitionsInfo pollablePartitionsInfo = getPollablePartitionsInfo();
+            if (pollablePartitionsInfo.shouldPoll()) {
+                try {
+                    setWaitingToEmit(pollKafkaBroker(pollablePartitionsInfo));
+                } catch (RetriableException e) {
+                    LOG.error("Failed to poll from kafka.", e);
                 }
-            } else {
-                LOG.debug("Spout not initialized. Not sending tuples until initialization completes");
             }
+
+            emitIfWaitingNotEmitted();
         } catch (InterruptException e) {
             throwKafkaConsumerInterruptedException();
         }
@@ -246,115 +317,186 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
         throw new RuntimeException(new InterruptedException("Kafka consumer was interrupted"));
     }
 
-    private boolean commit() {
-        return !consumerAutoCommitMode && commitTimer.isExpiredResetOnTrue();    // timer != null for non auto commit mode
-    }
+    private PollablePartitionsInfo getPollablePartitionsInfo() {
+        if (isWaitingToEmit()) {
+            LOG.debug("Not polling. Tuples waiting to be emitted.");
+            return new PollablePartitionsInfo(Collections.<TopicPartition>emptySet(), Collections.<TopicPartition, Long>emptyMap());
+        }
 
-    private boolean poll() {
-        final int maxUncommittedOffsets = kafkaSpoutConfig.getMaxUncommittedOffsets();
-        final int readyMessageCount = retryService.readyMessageCount();
-        final boolean poll = !waitingToEmit() &&
-            //Check that the number of uncommitted, nonretriable tuples is less than the maxUncommittedOffsets limit
-            //Accounting for retriable tuples this way still guarantees that the limit is followed on a per partition basis, and prevents locking up the spout when there are too many retriable tuples
-            (numUncommittedOffsets - readyMessageCount < maxUncommittedOffsets ||
-            consumerAutoCommitMode);
-        
-        if (!poll) {
-            if (waitingToEmit()) {
-                LOG.debug("Not polling. Tuples waiting to be emitted. [{}] uncommitted offsets across all topic partitions", numUncommittedOffsets);
-            }
+        Set<TopicPartition> assignment = kafkaConsumer.assignment();
+        if (!isAtLeastOnceProcessing()) {
+            return new PollablePartitionsInfo(assignment, Collections.<TopicPartition, Long>emptyMap());
+        }
 
-            if (numUncommittedOffsets >= maxUncommittedOffsets && !consumerAutoCommitMode) {
-                LOG.debug("Not polling. [{}] uncommitted offsets across all topic partitions has reached the threshold of [{}]", numUncommittedOffsets, maxUncommittedOffsets);
+        Map<TopicPartition, Long> earliestRetriableOffsets = retryService.earliestRetriableOffsets();
+        Set<TopicPartition> pollablePartitions = new HashSet<>();
+        final int maxUncommittedOffsets = kafkaSpoutConfig.getMaxUncommittedOffsets();
+        for (TopicPartition tp : assignment) {
+            OffsetManager offsetManager = offsetManagers.get(tp);
+            int numUncommittedOffsets = offsetManager.getNumUncommittedOffsets();
+            if (numUncommittedOffsets < maxUncommittedOffsets) {
+                //Allow poll if the partition is not at the maxUncommittedOffsets limit
+                pollablePartitions.add(tp);
+            } else {
+                long offsetAtLimit = offsetManager.getNthUncommittedOffsetAfterCommittedOffset(maxUncommittedOffsets);
+                Long earliestRetriableOffset = earliestRetriableOffsets.get(tp);
+                if (earliestRetriableOffset != null && earliestRetriableOffset <= offsetAtLimit) {
+                    //Allow poll if there are retriable tuples within the maxUncommittedOffsets limit
+                    pollablePartitions.add(tp);
+                } else {
+                    LOG.debug("Not polling on partition [{}]. It has [{}] uncommitted offsets, which exceeds the limit of [{}]. ", tp,
+                        numUncommittedOffsets, maxUncommittedOffsets);
+                }
             }
         }
-        return poll;
+        return new PollablePartitionsInfo(pollablePartitions, earliestRetriableOffsets);
     }
 
-    private boolean waitingToEmit() {
-        return waitingToEmit != null && waitingToEmit.hasNext();
+    private boolean isWaitingToEmit() {
+        for (List<ConsumerRecord<K, V>> value : waitingToEmit.values()) {
+            if (!value.isEmpty()) {
+                return true;
+            }
+        }
+        return false;
     }
 
-    public void setWaitingToEmit(ConsumerRecords<K,V> consumerRecords) {
-        List<ConsumerRecord<K,V>> waitingToEmitList = new LinkedList<>();
+    private void setWaitingToEmit(ConsumerRecords<K, V> consumerRecords) {
         for (TopicPartition tp : consumerRecords.partitions()) {
-            waitingToEmitList.addAll(consumerRecords.records(tp));
+            waitingToEmit.put(tp, new ArrayList<>(consumerRecords.records(tp)));
         }
-        waitingToEmit = waitingToEmitList.iterator();
     }
 
     // ======== poll =========
-    private ConsumerRecords<K, V> pollKafkaBroker() {
-        doSeekRetriableTopicPartitions();
-        if (refreshSubscriptionTimer.isExpiredResetOnTrue()) {
-            kafkaSpoutConfig.getSubscription().refreshAssignment();
+    private ConsumerRecords<K, V> pollKafkaBroker(PollablePartitionsInfo pollablePartitionsInfo) {
+        doSeekRetriableTopicPartitions(pollablePartitionsInfo.pollableEarliestRetriableOffsets);
+        Set<TopicPartition> pausedPartitions = new HashSet<>(kafkaConsumer.assignment());
+        Iterator<TopicPartition> pausedIter = pausedPartitions.iterator();
+        while (pausedIter.hasNext()) {
+            if (pollablePartitionsInfo.pollablePartitions.contains(pausedIter.next())) {
+                pausedIter.remove();
+            }
+        }
+        try {
+            kafkaConsumer.pause(pausedPartitions);
+            final ConsumerRecords<K, V> consumerRecords = kafkaConsumer.poll(kafkaSpoutConfig.getPollTimeoutMs());
+            ackRetriableOffsetsIfCompactedAway(pollablePartitionsInfo.pollableEarliestRetriableOffsets, consumerRecords);
+            final int numPolledRecords = consumerRecords.count();
+            LOG.debug("Polled [{}] records from Kafka",
+                numPolledRecords);
+            if (kafkaSpoutConfig.getProcessingGuarantee() == KafkaSpoutConfig.ProcessingGuarantee.AT_MOST_ONCE) {
+                //Commit polled records immediately to ensure delivery is at-most-once.
+                Map<TopicPartition, OffsetAndMetadata> offsetsToCommit = 
+                    createFetchedOffsetsMetadata(kafkaConsumer.assignment());
+                kafkaConsumer.commitSync(offsetsToCommit);
+                LOG.debug("Committed offsets {} to Kafka", offsetsToCommit);
+            }
+            return consumerRecords;
+        } finally {
+            kafkaConsumer.resume(pausedPartitions);
         }
-        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;
     }
 
-    private void doSeekRetriableTopicPartitions() {
-        final Map<TopicPartition, Long> retriableTopicPartitions = retryService.earliestRetriableOffsets();
-
-        for (Entry<TopicPartition, Long> retriableTopicPartitionAndOffset : retriableTopicPartitions.entrySet()) {
+    private void doSeekRetriableTopicPartitions(Map<TopicPartition, Long> pollableEarliestRetriableOffsets) {
+        for (Entry<TopicPartition, Long> retriableTopicPartitionAndOffset : pollableEarliestRetriableOffsets.entrySet()) {
             //Seek directly to the earliest retriable message for each retriable topic partition
             kafkaConsumer.seek(retriableTopicPartitionAndOffset.getKey(), retriableTopicPartitionAndOffset.getValue());
         }
     }
 
+    private void ackRetriableOffsetsIfCompactedAway(Map<TopicPartition, Long> earliestRetriableOffsets,
+        ConsumerRecords<K, V> consumerRecords) {
+        for (Entry<TopicPartition, Long> entry : earliestRetriableOffsets.entrySet()) {
+            TopicPartition tp = entry.getKey();
+            List<ConsumerRecord<K, V>> records = consumerRecords.records(tp);
+            if (!records.isEmpty()) {
+                ConsumerRecord<K, V> record = records.get(0);
+                long seekOffset = entry.getValue();
+                long earliestReceivedOffset = record.offset();
+                if (seekOffset < earliestReceivedOffset) {
+                    //Since we asked for tuples starting at seekOffset, some retriable records must have been compacted away.
+                    //Ack up to the first offset received if the record is not already acked or currently in the topology
+                    for (long i = seekOffset; i < earliestReceivedOffset; i++) {
+                        KafkaSpoutMessageId msgId = retryService.getMessageId(new ConsumerRecord<>(tp.topic(), tp.partition(), i, null, null));
+                        if (!offsetManagers.get(tp).contains(msgId) && !emitted.contains(msgId)) {
+                            LOG.debug("Record at offset [{}] appears to have been compacted away from topic [{}], marking as acked", i, tp);
+                            retryService.remove(msgId);
+                            emitted.add(msgId);
+                            ack(msgId);
+                        }
+                    }
+                }
+            }
+        }
+    }
+
     // ======== emit  =========
-    private void emit() {
-        while (!emitTupleIfNotEmitted(waitingToEmit.next()) && waitingToEmit.hasNext()) {
-            waitingToEmit.remove();
+    private void emitIfWaitingNotEmitted() {
+        Iterator<List<ConsumerRecord<K, V>>> waitingToEmitIter = waitingToEmit.values().iterator();
+        outerLoop:
+        while (waitingToEmitIter.hasNext()) {
+            List<ConsumerRecord<K, V>> waitingToEmitForTp = waitingToEmitIter.next();
+            while (!waitingToEmitForTp.isEmpty()) {
+                final boolean emittedTuple = emitOrRetryTuple(waitingToEmitForTp.remove(0));
+                if (emittedTuple) {
+                    break outerLoop;
+                }
+            }
+            waitingToEmitIter.remove();
         }
     }
 
     /**
-     * Creates a tuple from the kafka record and emits it if it was not yet emitted
+     * Creates a tuple from the kafka record and emits it if it was never emitted or it is ready to be retried.
      *
      * @param record to be emitted
      * @return true if tuple was emitted. False if tuple has been acked or has been emitted and is pending ack or fail
      */
-    private boolean emitTupleIfNotEmitted(ConsumerRecord<K, V> record) {
+    private boolean emitOrRetryTuple(ConsumerRecord<K, V> record) {
         final TopicPartition tp = new TopicPartition(record.topic(), record.partition());
         final KafkaSpoutMessageId msgId = retryService.getMessageId(record);
+
         if (offsetManagers.containsKey(tp) && offsetManagers.get(tp).contains(msgId)) {   // has been acked
             LOG.trace("Tuple for record [{}] has already been acked. Skipping", record);
-        } else if (emitted.contains(msgId)) {   // has been emitted and it's pending ack or fail
+        } else if (emitted.contains(msgId)) {   // has been emitted and it is pending ack or fail
             LOG.trace("Tuple for record [{}] has already been emitted. Skipping", record);
         } else {
-            Validate.isTrue(kafkaConsumer.committed(tp) == null || kafkaConsumer.committed(tp).offset() < kafkaConsumer.position(tp),
-                "The spout is about to emit a message that has already been committed."
-                + " This should never occur, and indicates a bug in the spout");
+            final OffsetAndMetadata committedOffset = kafkaConsumer.committed(tp);
+            if (isAtLeastOnceProcessing()
+                && committedOffset != null 
+                && committedOffset.offset() > record.offset()
+                && commitMetadataManager.isOffsetCommittedByThisTopology(tp, committedOffset, Collections.unmodifiableMap(offsetManagers))) {
+                // Ensures that after a topology with this id is started, the consumer fetch
+                // position never falls behind the committed offset (STORM-2844)
+                throw new IllegalStateException("Attempting to emit a message that has already been committed."
+                    + " This should never occur when using the at-least-once processing guarantee.");
+            }
+
             final List<Object> tuple = kafkaSpoutConfig.getTranslator().apply(record);
             if (isEmitTuple(tuple)) {
                 final boolean isScheduled = retryService.isScheduled(msgId);
                 // not scheduled <=> never failed (i.e. never emitted), or scheduled and ready to be retried
                 if (!isScheduled || retryService.isReady(msgId)) {
-                    if (consumerAutoCommitMode) {
-                        if (tuple instanceof KafkaTuple) {
-                            collector.emit(((KafkaTuple) tuple).getStream(), tuple);
+                    final String stream = tuple instanceof KafkaTuple ? ((KafkaTuple) tuple).getStream() : Utils.DEFAULT_STREAM_ID;
+
+                    if (!isAtLeastOnceProcessing()) {
+                        if (kafkaSpoutConfig.isTupleTrackingEnforced()) {
+                            collector.emit(stream, tuple, msgId);
+                            LOG.trace("Emitted tuple [{}] for record [{}] with msgId [{}]", tuple, record, msgId);
                         } else {
-                            collector.emit(tuple);
+                            collector.emit(stream, tuple);
+                            LOG.trace("Emitted tuple [{}] for record [{}]", tuple, record);
                         }
                     } else {
                         emitted.add(msgId);
                         offsetManagers.get(tp).addToEmitMsgs(msgId.offset());
                         if (isScheduled) {  // Was scheduled for retry and re-emitted, so remove from schedule.
                             retryService.remove(msgId);
-                        } else {            //New tuple, hence increment the uncommitted offset counter
-                            numUncommittedOffsets++;
-                        }
-
-                        if (tuple instanceof KafkaTuple) {
-                            collector.emit(((KafkaTuple) tuple).getStream(), tuple, msgId);
-                        } else {
-                            collector.emit(tuple, msgId);
                         }
+                        collector.emit(stream, tuple, msgId);
+                        tupleListener.onEmit(tuple, msgId);
+                        LOG.trace("Emitted tuple [{}] for record [{}] with msgId [{}]", tuple, record, msgId);
                     }
-                    LOG.trace("Emitted tuple [{}] for record [{}] with msgId [{}]", tuple, record, msgId);
                     return true;
                 }
             } else {
@@ -367,17 +509,32 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
     }
 
     /**
-     * Emits a tuple if it is not a null tuple, or if the spout is configured to emit null tuples
+     * Emits a tuple if it is not a null tuple, or if the spout is configured to emit null tuples.
      */
     private boolean isEmitTuple(List<Object> tuple) {
         return tuple != null || kafkaSpoutConfig.isEmitNullTuples();
     }
 
-    private void commitOffsetsForAckedTuples() {
-        // Find offsets that are ready to be committed for every topic partition
+    private Map<TopicPartition, OffsetAndMetadata> createFetchedOffsetsMetadata(Set<TopicPartition> assignedPartitions) {
+        Map<TopicPartition, OffsetAndMetadata> offsetsToCommit = new HashMap<>();
+        for (TopicPartition tp : assignedPartitions) {
+            offsetsToCommit.put(tp, new OffsetAndMetadata(kafkaConsumer.position(tp), commitMetadataManager.getCommitMetadata()));
+        }
+        return offsetsToCommit;
+    }
+    
+    private void commitOffsetsForAckedTuples(Set<TopicPartition> assignedPartitions) {
+        // Find offsets that are ready to be committed for every assigned topic partition
+        final Map<TopicPartition, OffsetManager> assignedOffsetManagers = new HashMap<>();
+        for (Entry<TopicPartition, OffsetManager> entry : offsetManagers.entrySet()) {
+            if (assignedPartitions.contains(entry.getKey())) {
+                assignedOffsetManagers.put(entry.getKey(), entry.getValue());
+            }
+        }
+
         final Map<TopicPartition, OffsetAndMetadata> nextCommitOffsets = new HashMap<>();
-        for (Map.Entry<TopicPartition, OffsetManager> tpOffset : offsetManagers.entrySet()) {
-            final OffsetAndMetadata nextCommitOffset = tpOffset.getValue().findNextCommitOffset();
+        for (Map.Entry<TopicPartition, OffsetManager> tpOffset : assignedOffsetManagers.entrySet()) {
+            final OffsetAndMetadata nextCommitOffset = tpOffset.getValue().findNextCommitOffset(commitMetadataManager.getCommitMetadata());
             if (nextCommitOffset != null) {
                 nextCommitOffsets.put(tpOffset.getKey(), nextCommitOffset);
             }
@@ -396,24 +553,30 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
                 long committedOffset = tpOffset.getValue().offset();
                 if (position < committedOffset) {
                     /*
-                     * The position is behind the committed offset. This can happen in some cases, e.g. if a message failed,
-                     * lots of (more than max.poll.records) later messages were acked, and the failed message then gets acked. 
-                     * The consumer may only be part way through "catching up" to where it was when it went back to retry the failed tuple. 
-                     * Skip the consumer forward to the committed offset drop the current waiting to emit list,
-                     * since it'll likely contain committed offsets.
+                     * The position is behind the committed offset. This can happen in some cases, e.g. if a message failed, lots of (more
+                     * than max.poll.records) later messages were acked, and the failed message then gets acked. The consumer may only be
+                     * part way through "catching up" to where it was when it went back to retry the failed tuple. Skip the consumer forward
+                     * to the committed offset and drop the current waiting to emit list, since it'll likely contain committed offsets.
                      */
                     LOG.debug("Consumer fell behind committed offset. Catching up. Position was [{}], skipping to [{}]",
                         position, committedOffset);
                     kafkaConsumer.seek(tp, committedOffset);
-                    waitingToEmit = null;
+                    List<ConsumerRecord<K, V>> waitingToEmitForTp = waitingToEmit.get(tp);
+                    if (waitingToEmitForTp != null) {
+                        //Discard the pending records that are already committed
+                        List<ConsumerRecord<K, V>> filteredRecords = new ArrayList<>();
+                        for (ConsumerRecord<K, V> record : waitingToEmitForTp) {
+                            if (record.offset() >= committedOffset) {
+                                filteredRecords.add(record);
+                            }
+                        }
+                        waitingToEmit.put(tp, filteredRecords);
+                    }
                 }
-                
-                
-                final OffsetManager offsetManager = offsetManagers.get(tp);
-                long numCommittedOffsets = offsetManager.commit(tpOffset.getValue());
-                numUncommittedOffsets -= numCommittedOffsets;
-                LOG.debug("[{}] uncommitted offsets across all topic partitions",
-                    numUncommittedOffsets);
+
+                final OffsetManager offsetManager = assignedOffsetManagers.get(tp);
+                offsetManager.commit(tpOffset.getValue());
+                LOG.debug("[{}] uncommitted offsets for partition [{}] after commit", offsetManager.getNumUncommittedOffsets(), tp);
             }
         } else {
             LOG.trace("No offsets to commit. {}", this);
@@ -421,52 +584,64 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
     }
 
     // ======== Ack =======
-
     @Override
     public void ack(Object messageId) {
+        if (!isAtLeastOnceProcessing()) {
+            return;
+        }
+
+        // Only need to keep track of acked tuples if commits to Kafka are controlled by
+        // tuple acks, which happens only for at-least-once processing semantics
         final KafkaSpoutMessageId msgId = (KafkaSpoutMessageId) messageId;
         if (!emitted.contains(msgId)) {
             if (msgId.isEmitted()) {
-                LOG.debug("Received ack for message [{}], associated with tuple emitted for a ConsumerRecord that " +
-                        "came from a topic-partition that this consumer group instance is no longer tracking " +
-                        "due to rebalance/partition reassignment. No action taken.", msgId);
+                LOG.debug("Received ack for message [{}], associated with tuple emitted for a ConsumerRecord that "
+                    + "came from a topic-partition that this consumer group instance is no longer tracking "
+                    + "due to rebalance/partition reassignment. No action taken.", msgId);
             } else {
                 LOG.debug("Received direct ack for message [{}], associated with null tuple", msgId);
             }
         } else {
             Validate.isTrue(!retryService.isScheduled(msgId), "The message id " + msgId + " is queued for retry while being acked."
                 + " This should never occur barring errors in the RetryService implementation or the spout code.");
-            if (!consumerAutoCommitMode) {  // Only need to keep track of acked tuples if commits are not done automatically
-                offsetManagers.get(msgId.getTopicPartition()).addToAckMsgs(msgId);
-            }
+            offsetManagers.get(msgId.getTopicPartition()).addToAckMsgs(msgId);
             emitted.remove(msgId);
         }
+        tupleListener.onAck(msgId);
     }
 
     // ======== Fail =======
-
     @Override
     public void fail(Object messageId) {
+        if (!isAtLeastOnceProcessing()) {
+            return;
+        }
+        // Only need to keep track of failed tuples if commits to Kafka are controlled by
+        // tuple acks, which happens only for at-least-once processing semantics
         final KafkaSpoutMessageId msgId = (KafkaSpoutMessageId) messageId;
         if (!emitted.contains(msgId)) {
-            LOG.debug("Received fail for tuple this spout is no longer tracking. Partitions may have been reassigned. Ignoring message [{}]", msgId);
+            LOG.debug("Received fail for tuple this spout is no longer tracking."
+                + " Partitions may have been reassigned. Ignoring message [{}]", msgId);
             return;
         }
         Validate.isTrue(!retryService.isScheduled(msgId), "The message id " + msgId + " is queued for retry while being failed."
             + " This should never occur barring errors in the RetryService implementation or the spout code.");
+
         msgId.incrementNumFails();
+
         if (!retryService.schedule(msgId)) {
             LOG.debug("Reached maximum number of retries. Message [{}] being marked as acked.", msgId);
             // this tuple should be removed from emitted only inside the ack() method. This is to ensure
             // that the OffsetManager for that TopicPartition is updated and allows commit progression
+            tupleListener.onMaxRetryReached(msgId);
             ack(msgId);
         } else {
+            tupleListener.onRetry(msgId);
             emitted.remove(msgId);
         }
     }
 
     // ======== Activate / Deactivate / Close / Declare Outputs =======
-
     @Override
     public void activate() {
         try {
@@ -502,8 +677,8 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
 
     private void shutdown() {
         try {
-            if (!consumerAutoCommitMode) {
-                commitOffsetsForAckedTuples();
+            if (isAtLeastOnceProcessing()) {
+                commitOffsetsForAckedTuples(kafkaConsumer.assignment());
             }
         } finally {
             //remove resources
@@ -521,10 +696,10 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
 
     @Override
     public String toString() {
-        return "KafkaSpout{" +
-                "offsetManagers =" + offsetManagers +
-                ", emitted=" + emitted +
-                "}";
+        return "KafkaSpout{"
+            + "offsetManagers =" + offsetManagers
+            + ", emitted=" + emitted
+            + "}";
     }
 
     @Override
@@ -546,7 +721,30 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
     private String getTopicsString() {
         return kafkaSpoutConfig.getSubscription().getTopicsString();
     }
-}
 
+    private static class PollablePartitionsInfo {
+
+        private final Set<TopicPartition> pollablePartitions;
+        //The subset of earliest retriable offsets that are on pollable partitions
+        private final Map<TopicPartition, Long> pollableEarliestRetriableOffsets;
+
+        public PollablePartitionsInfo(Set<TopicPartition> pollablePartitions, Map<TopicPartition, Long> earliestRetriableOffsets) {
+            this.pollablePartitions = pollablePartitions;
+            this.pollableEarliestRetriableOffsets = new HashMap<>();
+            for (TopicPartition tp : earliestRetriableOffsets.keySet()) {
+                if (this.pollablePartitions.contains(tp)) {
+                    this.pollableEarliestRetriableOffsets.put(tp, earliestRetriableOffsets.get(tp));
+                }
+            }
+        }
 
+        public boolean shouldPoll() {
+            return !this.pollablePartitions.isEmpty();
+        }
+    }
 
+    @VisibleForTesting
+    KafkaOffsetMetric getKafkaOffsetMetric() {
+        return kafkaOffsetMetric;
+    }
+}


[7/7] storm git commit: Merge branch 'overwrite-latest-storm-kafka-client-1.1.x' of https://github.com/HeartSaVioR/storm into STORM-2936-1.1.x-merge

Posted by ka...@apache.org.
Merge branch 'overwrite-latest-storm-kafka-client-1.1.x' of https://github.com/HeartSaVioR/storm into STORM-2936-1.1.x-merge


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

Branch: refs/heads/1.1.x-branch
Commit: c270ff4bde557e66798d68301142b5b47be41ee4
Parents: 8ed22cf 324bc95
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Thu Feb 8 06:05:10 2018 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Thu Feb 8 06:05:10 2018 +0900

----------------------------------------------------------------------
 docs/storm-kafka-client.md                      | 181 ++---
 .../TridentKafkaClientWordCountNamedTopics.java |  27 +-
 ...identKafkaClientWordCountWildcardTopics.java |   5 +-
 external/storm-kafka-client/pom.xml             |   8 +
 .../kafka/spout/EmptyKafkaTupleListener.java    |  53 ++
 .../apache/storm/kafka/spout/KafkaSpout.java    | 532 +++++++++----
 .../storm/kafka/spout/KafkaSpoutConfig.java     | 785 ++++++++++++++-----
 .../storm/kafka/spout/KafkaSpoutMessageId.java  |  26 +-
 .../storm/kafka/spout/KafkaTupleListener.java   |  83 ++
 .../spout/ManualPartitionSubscription.java      |   8 +-
 .../storm/kafka/spout/ManualPartitioner.java    |   1 +
 .../storm/kafka/spout/NamedSubscription.java    |   4 +-
 .../storm/kafka/spout/PatternSubscription.java  |   4 +-
 .../kafka/spout/SerializableDeserializer.java   |   6 +-
 .../apache/storm/kafka/spout/Subscription.java  |   5 +-
 .../kafka/spout/internal/CommitMetadata.java    |  63 ++
 .../spout/internal/CommitMetadataManager.java   |  91 +++
 .../internal/KafkaConsumerFactoryDefault.java   |   3 +-
 .../kafka/spout/internal/OffsetManager.java     | 111 ++-
 .../kafka/spout/metrics/KafkaOffsetMetric.java  | 141 ++++
 .../trident/KafkaTridentSpoutBatchMetadata.java |  78 +-
 .../spout/trident/KafkaTridentSpoutEmitter.java | 108 +--
 .../spout/trident/KafkaTridentSpoutManager.java |  13 +-
 .../spout/trident/KafkaTridentSpoutOpaque.java  |  18 +-
 .../KafkaTridentSpoutOpaqueCoordinator.java     |  26 +-
 .../spout/trident/TopicPartitionSerializer.java |  47 ++
 .../storm/kafka/trident/TridentKafkaState.java  |  79 +-
 .../kafka/trident/TridentKafkaStateFactory.java |  17 +-
 .../kafka/trident/TridentKafkaStateUpdater.java |   6 +-
 .../kafka/spout/KafkaSpoutAbstractTest.java     | 179 +++++
 .../storm/kafka/spout/KafkaSpoutCommitTest.java | 135 ----
 .../storm/kafka/spout/KafkaSpoutConfigTest.java | 206 ++++-
 .../storm/kafka/spout/KafkaSpoutEmitTest.java   | 204 ++---
 .../KafkaSpoutLogCompactionSupportTest.java     | 223 ++++++
 .../spout/KafkaSpoutMessagingGuaranteeTest.java | 259 ++++++
 .../kafka/spout/KafkaSpoutReactivationTest.java | 145 ++++
 .../kafka/spout/KafkaSpoutRebalanceTest.java    |  63 +-
 .../kafka/spout/KafkaSpoutRetryLimitTest.java   | 104 +--
 .../kafka/spout/KafkaSpoutSingleTopicTest.java  | 379 +++++++++
 ...outTopologyDeployActivateDeactivateTest.java | 116 +++
 .../kafka/spout/MaxUncommittedOffsetTest.java   | 120 ++-
 .../storm/kafka/spout/NamedTopicFilterTest.java |   3 +-
 .../kafka/spout/PatternTopicFilterTest.java     |   2 +
 .../kafka/spout/SingleTopicKafkaSpoutTest.java  | 401 ----------
 .../spout/SingleTopicKafkaUnitSetupHelper.java  |  89 +++
 .../SpoutWithMockedConsumerSetupHelper.java     | 171 ++++
 .../SingleTopicKafkaSpoutConfiguration.java     |  49 +-
 .../kafka/spout/internal/OffsetManagerTest.java |  49 +-
 .../ManualPartitionSubscriptionTest.java        |  81 ++
 .../test/KafkaSpoutTopologyMainNamedTopics.java |   4 +-
 .../KafkaSpoutTopologyMainWildcardTopics.java   |   4 +-
 .../KafkaTridentSpoutBatchMetadataTest.java     |  66 ++
 pom.xml                                         |   2 +-
 .../storm/annotation/InterfaceStability.java    |  54 ++
 54 files changed, 4112 insertions(+), 1525 deletions(-)
----------------------------------------------------------------------



[5/7] storm git commit: STORM-2936 Overwrite latest storm-kafka-client 1.x-branch into 1.1.x-branch

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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 e1c6c96..7aa836c 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,83 +18,169 @@
 
 package org.apache.storm.kafka.spout;
 
-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;
-
 import java.io.Serializable;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 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.Config;
+import org.apache.storm.annotation.InterfaceStability;
+import org.apache.storm.kafka.spout.KafkaSpoutRetryExponentialBackoff.TimeInterval;
+import org.apache.storm.tuple.Fields;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 /**
  * 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 Logger LOG = LoggerFactory.getLogger(KafkaSpoutConfig.class);
     private static final long serialVersionUID = 141902646130682494L;
-    public static final long DEFAULT_POLL_TIMEOUT_MS = 200;            // 200ms
-    public static final long DEFAULT_OFFSET_COMMIT_PERIOD_MS = 30_000;   // 30s
-    public static final int DEFAULT_MAX_RETRIES = Integer.MAX_VALUE;     // Retry forever
-    public static final int DEFAULT_MAX_UNCOMMITTED_OFFSETS = 10_000_000;    // 10,000,000 records => 80MBs of memory footprint in the worst case
-    public static final long DEFAULT_PARTITION_REFRESH_PERIOD_MS = 2_000; // 2s
-    public static final KafkaSpoutRetryService DEFAULT_RETRY_SERVICE =  
-            new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
-                    DEFAULT_MAX_RETRIES, TimeInterval.seconds(10));
+    // 200ms
+    public static final long DEFAULT_POLL_TIMEOUT_MS = 200;
+    // 30s
+    public static final long DEFAULT_OFFSET_COMMIT_PERIOD_MS = 30_000;
+    // Retry forever
+    public static final int DEFAULT_MAX_RETRIES = Integer.MAX_VALUE;
+    // 10,000,000 records => 80MBs of memory footprint in the worst case
+    public static final int DEFAULT_MAX_UNCOMMITTED_OFFSETS = 10_000_000;
+    // 2s
+    public static final long DEFAULT_PARTITION_REFRESH_PERIOD_MS = 2_000;
+
+    public static final FirstPollOffsetStrategy DEFAULT_FIRST_POLL_OFFSET_STRATEGY = FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST;
+
+    public static final KafkaSpoutRetryService DEFAULT_RETRY_SERVICE =
+        new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(2),
+            DEFAULT_MAX_RETRIES, TimeInterval.seconds(10));
+
+    public static final ProcessingGuarantee DEFAULT_PROCESSING_GUARANTEE = ProcessingGuarantee.AT_LEAST_ONCE;
+
+    public static final KafkaTupleListener DEFAULT_TUPLE_LISTENER = new EmptyKafkaTupleListener();
+
+    public static final int DEFAULT_METRICS_TIME_BUCKET_SIZE_SECONDS = 60;
+
+
+    // Kafka consumer configuration
+    private final Map<String, Object> kafkaProps;
+    private final Subscription subscription;
+    private final long pollTimeoutMs;
+
+    // Kafka spout configuration
+    private final RecordTranslator<K, V> translator;
+    private final long offsetCommitPeriodMs;
+    private final int maxUncommittedOffsets;
+    private final FirstPollOffsetStrategy firstPollOffsetStrategy;
+    private final KafkaSpoutRetryService retryService;
+    private final KafkaTupleListener tupleListener;
+    private final long partitionRefreshPeriodMs;
+    private final boolean emitNullTuples;
+    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 ProcessingGuarantee processingGuarantee;
+    private final boolean tupleTrackingEnforced;
+    private final int metricsTimeBucketSizeInSecs;
+
     /**
-     * Retry in a tight loop (keep unit tests fasts) do not use in production.
+     * Creates a new KafkaSpoutConfig using a Builder.
+     *
+     * @param builder The Builder to construct the KafkaSpoutConfig from
      */
-    public static final KafkaSpoutRetryService UNIT_TEST_RETRY_SERVICE = 
-    new KafkaSpoutRetryExponentialBackoff(TimeInterval.seconds(0), TimeInterval.milliSeconds(0),
-            DEFAULT_MAX_RETRIES, TimeInterval.milliSeconds(0));
+    public KafkaSpoutConfig(Builder<K, V> builder) {
+        setKafkaPropsForProcessingGuarantee(builder);
+        this.kafkaProps = builder.kafkaProps;
+        this.subscription = builder.subscription;
+        this.translator = builder.translator;
+        this.pollTimeoutMs = builder.pollTimeoutMs;
+        this.offsetCommitPeriodMs = builder.offsetCommitPeriodMs;
+        this.firstPollOffsetStrategy = builder.firstPollOffsetStrategy;
+        this.maxUncommittedOffsets = builder.maxUncommittedOffsets;
+        this.retryService = builder.retryService;
+        this.tupleListener = builder.tupleListener;
+        this.partitionRefreshPeriodMs = builder.partitionRefreshPeriodMs;
+        this.emitNullTuples = builder.emitNullTuples;
+        this.keyDes = builder.keyDes;
+        this.keyDesClazz = builder.keyDesClazz;
+        this.valueDes = builder.valueDes;
+        this.valueDesClazz = builder.valueDesClazz;
+        this.processingGuarantee = builder.processingGuarantee;
+        this.tupleTrackingEnforced = builder.tupleTrackingEnforced;
+        this.metricsTimeBucketSizeInSecs = builder.metricsTimeBucketSizeInSecs;
+    }
 
     /**
-     * The offset used by the Kafka spout in the first poll to Kafka broker. The choice of this parameter will
-     * affect the number of consumer records returned in the first poll. By default this parameter is set to UNCOMMITTED_EARLIEST. <br/><br/>
-     * The allowed values are EARLIEST, LATEST, UNCOMMITTED_EARLIEST, UNCOMMITTED_LATEST. <br/>
-     * <ul>
-     * <li>EARLIEST means that the kafka spout polls records starting in the first offset of the partition, regardless of previous commits</li>
-     * <li>LATEST means that the kafka spout polls records with offsets greater than the last offset in the partition, regardless of previous commits</li>
-     * <li>UNCOMMITTED_EARLIEST means that the kafka spout polls records from the last committed offset, if any.
-     * If no offset has been committed, it behaves as EARLIEST.</li>
-     * <li>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.</li>
-     * </ul>
-     * */
-    public static enum FirstPollOffsetStrategy {
+     * Defines how the {@link KafkaSpout} seeks the offset to be used in the first poll to Kafka upon topology deployment.
+     * By default this parameter is set to UNCOMMITTED_EARLIEST. 
+     */
+    public enum FirstPollOffsetStrategy {
+        /**
+         * The kafka spout polls records starting in the first offset of the partition, regardless of previous commits. This setting only
+         * takes effect on topology deployment
+         */
         EARLIEST,
+        /**
+         * The kafka spout polls records with offsets greater than the last offset in the partition, regardless of previous commits. This
+         * setting only takes effect on topology deployment
+         */
         LATEST,
+        /**
+         * The kafka spout polls records from the last committed offset, if any. If no offset has been committed it behaves as EARLIEST
+         */
         UNCOMMITTED_EARLIEST,
-        UNCOMMITTED_LATEST }
+        /**
+         * The kafka spout polls records from the last committed offset, if any. If no offset has been committed it behaves as LATEST
+         */
+        UNCOMMITTED_LATEST;
 
-    public static Builder<String, String> builder(String bootstrapServers, String ... topics) {
-        return new Builder<>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics);
-    }
-    
-    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(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) {
-            kafkaProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
+        @Override
+        public String toString() {
+            return "FirstPollOffsetStrategy{" + super.toString() + "}";
         }
-        return kafkaProps;
     }
-    
-    public static class Builder<K,V> {
+
+    /**
+     * This enum controls when the tuple with the {@link ConsumerRecord} for an offset is marked as processed,
+     * i.e. when the offset can be committed to Kafka. The default value is AT_LEAST_ONCE.
+     * The commit interval is controlled by {@link KafkaSpoutConfig#getOffsetsCommitPeriodMs() }, if the mode commits on an interval.
+     * NO_GUARANTEE may be removed in a later release without warning, we're still evaluating whether it makes sense to keep.
+     */
+    @InterfaceStability.Unstable
+    public enum ProcessingGuarantee {
+        /**
+         * An offset is ready to commit only after the corresponding tuple has been processed and acked (at least once). If a tuple fails or
+         * times out it will be re-emitted, as controlled by the {@link KafkaSpoutRetryService}. Commits synchronously on the defined
+         * interval.
+         */
+        AT_LEAST_ONCE,
+        /**
+         * Every offset will be synchronously committed to Kafka right after being polled but before being emitted to the downstream
+         * components of the topology. The commit interval is ignored. This mode guarantees that the offset is processed at most once by
+         * ensuring the spout won't retry tuples that fail or time out after the commit to Kafka has been done
+         */
+        AT_MOST_ONCE,
+        /**
+         * The polled offsets are ready to commit immediately after being polled. The offsets are committed periodically, i.e. a message may
+         * be processed 0, 1 or more times. This behavior is similar to setting enable.auto.commit=true in the consumer, but allows the
+         * spout to control when commits occur. Commits asynchronously on the defined interval.
+         */
+        NO_GUARANTEE,
+    }
+
+    public static class Builder<K, V> {
+
         private final Map<String, Object> kafkaProps;
-        private Subscription subscription;
+        private final Subscription subscription;
         private final SerializableDeserializer<K> keyDes;
         private final Class<? extends Deserializer<K>> keyDesClazz;
         private final SerializableDeserializer<V> valueDes;
@@ -102,232 +188,362 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
         private RecordTranslator<K, V> translator;
         private long pollTimeoutMs = DEFAULT_POLL_TIMEOUT_MS;
         private long offsetCommitPeriodMs = DEFAULT_OFFSET_COMMIT_PERIOD_MS;
-        private FirstPollOffsetStrategy firstPollOffsetStrategy = FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST;
+        private FirstPollOffsetStrategy firstPollOffsetStrategy = DEFAULT_FIRST_POLL_OFFSET_STRATEGY;
         private int maxUncommittedOffsets = DEFAULT_MAX_UNCOMMITTED_OFFSETS;
         private KafkaSpoutRetryService retryService = DEFAULT_RETRY_SERVICE;
+        private KafkaTupleListener tupleListener = DEFAULT_TUPLE_LISTENER;
         private long partitionRefreshPeriodMs = DEFAULT_PARTITION_REFRESH_PERIOD_MS;
         private boolean emitNullTuples = false;
+        private ProcessingGuarantee processingGuarantee = DEFAULT_PROCESSING_GUARANTEE;
+        private boolean tupleTrackingEnforced = false;
+        private int metricsTimeBucketSizeInSecs = DEFAULT_METRICS_TIME_BUCKET_SIZE_SECONDS;
+
+        public Builder(String bootstrapServers, String... topics) {
+            this(bootstrapServers, (SerializableDeserializer) null, (SerializableDeserializer) null, new ManualPartitionSubscription(new RoundRobinManualPartitioner(), new NamedTopicFilter(topics)));
+        }
 
-        public Builder(String bootstrapServers, String ... topics) {
-            this(bootstrapServers, (SerializableDeserializer) null, (SerializableDeserializer) null, new NamedSubscription(topics));
+        public Builder(String bootstrapServers, Collection<String> topics) {
+            this(bootstrapServers, (SerializableDeserializer) null, (SerializableDeserializer) null, new ManualPartitionSubscription(new RoundRobinManualPartitioner(),
+                new NamedTopicFilter(new HashSet<String>(topics))));
         }
 
-        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, String ... topics) {
-            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
+        public Builder(String bootstrapServers, Pattern topics) {
+            this(bootstrapServers, (SerializableDeserializer) null, (SerializableDeserializer) null, new ManualPartitionSubscription(new RoundRobinManualPartitioner(), new PatternTopicFilter(topics)));
         }
-        
+
+        /**
+         * @deprecated Please use {@link #Builder(java.lang.String, java.lang.String...)} instead, and set the deserializer with
+         * {@link #setProp(java.lang.String, java.lang.Object)}, {@link ConsumerConfig#KEY_DESERIALIZER_CLASS_CONFIG} and
+         * {@link ConsumerConfig#VALUE_DESERIALIZER_CLASS_CONFIG}
+         */
+        @Deprecated
+        public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, String... topics) {
+            this(bootstrapServers, keyDes, valDes, new ManualPartitionSubscription(new RoundRobinManualPartitioner(), new NamedTopicFilter(topics)));
+        }
+
+        /**
+         * @deprecated Please use {@link #Builder(java.lang.String, java.util.Collection) } instead, and set the deserializer with
+         * {@link #setProp(java.lang.String, java.lang.Object)}, {@link ConsumerConfig#KEY_DESERIALIZER_CLASS_CONFIG} and
+         * {@link ConsumerConfig#VALUE_DESERIALIZER_CLASS_CONFIG}
+         */
+        @Deprecated
         public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Collection<String> topics) {
-            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
+            this(bootstrapServers, keyDes, valDes, new ManualPartitionSubscription(new RoundRobinManualPartitioner(), new NamedTopicFilter(new HashSet<String>(topics))));
         }
-        
+
+        /**
+         * @deprecated Please use {@link #Builder(java.lang.String, java.util.regex.Pattern) } instead, and set the deserializer with
+         * {@link #setProp(java.lang.String, java.lang.Object)}, {@link ConsumerConfig#KEY_DESERIALIZER_CLASS_CONFIG} and
+         * {@link ConsumerConfig#VALUE_DESERIALIZER_CLASS_CONFIG}
+         */
+        @Deprecated
         public Builder(String bootstrapServers, SerializableDeserializer<K> keyDes, SerializableDeserializer<V> valDes, Pattern topics) {
-            this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics));
+            this(bootstrapServers, keyDes, valDes, new ManualPartitionSubscription(new RoundRobinManualPartitioner(), new PatternTopicFilter(topics)));
         }
-        
+
+        /**
+         * @deprecated Please use {@link #Builder(java.lang.String, org.apache.storm.kafka.spout.Subscription) } instead, and set the
+         * deserializer with {@link #setProp(java.lang.String, java.lang.Object)}, {@link ConsumerConfig#KEY_DESERIALIZER_CLASS_CONFIG} and
+         * {@link ConsumerConfig#VALUE_DESERIALIZER_CLASS_CONFIG}
+         */
+        @Deprecated
         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));
+
+        /**
+         * @deprecated Please use {@link #Builder(java.lang.String, java.lang.String...)} instead, and set the deserializer with {@link #setProp(java.lang.String, java.lang.Object)}, {@link ConsumerConfig#KEY_DESERIALIZER_CLASS_CONFIG} and {@link ConsumerConfig#VALUE_DESERIALIZER_CLASS_CONFIG}
+         */
+        @Deprecated
+        public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, String... topics) {
+            this(bootstrapServers, keyDes, valDes, new ManualPartitionSubscription(new RoundRobinManualPartitioner(), new NamedTopicFilter(topics)));
         }
-        
+
+        /**
+         * @deprecated Please use {@link #Builder(java.lang.String, java.util.Collection) } instead, and set the deserializer with {@link #setProp(java.lang.String, java.lang.Object)}, {@link ConsumerConfig#KEY_DESERIALIZER_CLASS_CONFIG} and {@link ConsumerConfig#VALUE_DESERIALIZER_CLASS_CONFIG}
+         */
+        @Deprecated
         public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Collection<String> topics) {
-            this(bootstrapServers, keyDes, valDes, new NamedSubscription(topics));
+            this(bootstrapServers, keyDes, valDes, new ManualPartitionSubscription(new RoundRobinManualPartitioner(), new NamedTopicFilter(new HashSet<String>(topics))));
         }
-        
+
+        /**
+         * @deprecated Please use {@link #Builder(java.lang.String, java.util.regex.Pattern) } instead, and set the deserializer with {@link #setProp(java.lang.String, java.lang.Object)}, {@link ConsumerConfig#KEY_DESERIALIZER_CLASS_CONFIG} and {@link ConsumerConfig#VALUE_DESERIALIZER_CLASS_CONFIG}
+         */
+        @Deprecated
         public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Pattern topics) {
-            this(bootstrapServers, keyDes, valDes, new PatternSubscription(topics));
+            this(bootstrapServers, keyDes, valDes, new ManualPartitionSubscription(new RoundRobinManualPartitioner(), new PatternTopicFilter(topics)));
         }
-        
+
+        /**
+         * @deprecated Please use {@link #Builder(java.lang.String, org.apache.storm.kafka.spout.Subscription) } instead, and set the deserializer with {@link #setProp(java.lang.String, java.lang.Object)}, {@link ConsumerConfig#KEY_DESERIALIZER_CLASS_CONFIG} and {@link ConsumerConfig#VALUE_DESERIALIZER_CLASS_CONFIG}
+         */
+        @Deprecated
         public Builder(String bootstrapServers, Class<? extends Deserializer<K>> keyDes, Class<? extends Deserializer<V>> valDes, Subscription subscription) {
             this(bootstrapServers, null, keyDes, null, valDes, subscription);
         }
-        
+
+        /**
+         * Create a KafkaSpoutConfig builder with default property values and no key/value deserializers.
+         *
+         * @param bootstrapServers The bootstrap servers the consumer will use
+         * @param subscription The subscription defining which topics and partitions each spout instance will read.
+         */
+        public Builder(String bootstrapServers, Subscription subscription) {
+            this(bootstrapServers, null, null, null, null, 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<>();
+            SerializableDeserializer<V> valDes, Class<? extends Deserializer<V>> valDesClazz, Subscription subscription) {
+
+            this(keyDes, keyDesClazz, valDes, valDesClazz, subscription,
+                    new DefaultRecordTranslator<K, V>(), new HashMap<String, Object>());
+
             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>();
+
+            setNonNullSerDeKafkaProp(keyDes, keyDesClazz, valueDes, valueDesClazz);
         }
 
-        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 constructor will always be called by one of the methods {@code setKey} or {@code setVal}, which implies
+         * that only one of its SerDe parameters will be non null, for which the corresponding Kafka property will be set
+         */
+        @SuppressWarnings("unchecked")
+        private Builder(final Builder<?, ?> builder, SerializableDeserializer<K> keyDes, Class<? extends Deserializer<K>> keyDesClazz,
+                        SerializableDeserializer<V> valueDes, Class<? extends Deserializer<V>> valueDesClazz) {
+
+            this(keyDes, keyDesClazz, valueDes, valueDesClazz, builder.subscription,
+                    (RecordTranslator<K, V>) builder.translator, new HashMap<>(builder.kafkaProps));
+
             this.pollTimeoutMs = builder.pollTimeoutMs;
             this.offsetCommitPeriodMs = builder.offsetCommitPeriodMs;
             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;
+
+            setNonNullSerDeKafkaProp(keyDes, keyDesClazz, valueDes, valueDesClazz);
+        }
+
+        private Builder(SerializableDeserializer<K> keyDes, Class<? extends Deserializer<K>> keyDesClazz,
+               SerializableDeserializer<V> valueDes, Class<? extends Deserializer<V>> valueDesClazz,
+               Subscription subscription, RecordTranslator<K, V> translator, Map<String, Object> kafkaProps) {
             this.keyDes = keyDes;
             this.keyDesClazz = keyDesClazz;
             this.valueDes = valueDes;
             this.valueDesClazz = valueDesClazz;
+            this.subscription = subscription;
+            this.translator = translator;
+            this.kafkaProps = kafkaProps;
+        }
+
+        private void setNonNullSerDeKafkaProp(SerializableDeserializer<K> keyDes, Class<? extends Deserializer<K>> keyDesClazz,
+                SerializableDeserializer<V> valueDes, Class<? extends Deserializer<V>> valueDesClazz) {
+            if (keyDesClazz != null) {
+                kafkaProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDesClazz);
+            }
+            if (keyDes != null) {
+                kafkaProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDes.getClass());
+            }
+            if (valueDesClazz != null) {
+                kafkaProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDesClazz);
+            }
+            if (valueDes != null) {
+                kafkaProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDes.getClass());
+            }
         }
 
         /**
-         * 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.
+         * 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.
+         *
+         * @deprecated Please use {@link #setProp(java.lang.String, java.lang.Object) } with {@link ConsumerConfig#KEY_DESERIALIZER_CLASS_CONFIG} instead
          */
-        public <NK> Builder<NK,V> setKey(SerializableDeserializer<NK> keyDeserializer) {
-            return new Builder<>(this, keyDeserializer, null, valueDes, valueDesClazz);
+        @Deprecated
+        public <NK> Builder<NK, V> setKey(SerializableDeserializer<NK> keyDeserializer) {
+            return new Builder<>(this, keyDeserializer, null, null, null);
         }
-        
+
         /**
-         * 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.
+         * 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.
+         *
+         * @deprecated Please use {@link #setProp(java.lang.String, java.lang.Object) } with {@link ConsumerConfig#VALUE_DESERIALIZER_CLASS_CONFIG} instead
          */
+        @Deprecated
         public <NK> Builder<NK, V> setKey(Class<? extends Deserializer<NK>> clazz) {
-            return new Builder<>(this, null, clazz, valueDes, valueDesClazz);
+            return new Builder<>(this, null, clazz, null, null);
         }
 
         /**
-         * 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.
+         * 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.
+         *
+         * @deprecated Please use {@link #setProp(java.lang.String, java.lang.Object) } with {@link ConsumerConfig#KEY_DESERIALIZER_CLASS_CONFIG} instead
          */
-        public <NV> Builder<K,NV> setValue(SerializableDeserializer<NV> valueDeserializer) {
-            return new Builder<>(this, keyDes, keyDesClazz, valueDeserializer, null);
+        @Deprecated
+        public <NV> Builder<K, NV> setValue(SerializableDeserializer<NV> valueDeserializer) {
+            return new Builder<>(this, null, null, 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.
+         * 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.
+         *
+         * @deprecated Please use {@link #setProp(java.lang.String, java.lang.Object) } with {@link ConsumerConfig#VALUE_DESERIALIZER_CLASS_CONFIG} instead
          */
-        public <NV> Builder<K,NV> setValue(Class<? extends Deserializer<NV>> clazz) {
-            return new Builder<>(this, keyDes, keyDesClazz, null, clazz);
+        @Deprecated
+        public <NV> Builder<K, NV> setValue(Class<? extends Deserializer<NV>> clazz) {
+            return new Builder<>(this, null, null, null, clazz);
         }
-        
+
         /**
-         * Set a Kafka property config
+         * Set a {@link KafkaConsumer} property.
          */
-        public Builder<K,V> setProp(String key, Object value) {
+        public Builder<K, V> setProp(String key, Object value) {
             kafkaProps.put(key, value);
             return this;
         }
-        
+
         /**
-         * Set multiple Kafka property configs
+         * Set multiple {@link KafkaConsumer} properties.
          */
-        public Builder<K,V> setProp(Map<String, Object> props) {
+        public Builder<K, V> setProp(Map<String, Object> props) {
             kafkaProps.putAll(props);
             return this;
         }
-        
+
         /**
-         * Set multiple Kafka property configs
+         * Set multiple {@link KafkaConsumer} properties.
          */
-        public Builder<K,V> setProp(Properties props) {
-            for (String name: props.stringPropertyNames()) {
-                kafkaProps.put(name, props.get(name));
+        public Builder<K, V> setProp(Properties props) {
+            for(Entry<Object, Object> entry : props.entrySet()) {
+            if (entry.getKey() instanceof String) {
+                    kafkaProps.put((String) entry.getKey(), entry.getValue());
+                } else {
+                    throw new IllegalArgumentException("Kafka Consumer property keys must be Strings");
+                }
             }
             return this;
         }
-        
+
         /**
          * Set the group.id for the consumers
+         *
+         * @deprecated Please use {@link #setProp(java.lang.String, java.lang.Object) } with {@link ConsumerConfig#GROUP_ID_CONFIG} instead
          */
-        public Builder<K,V> setGroupId(String id) {
+        @Deprecated
+        public Builder<K, V> setGroupId(String id) {
             return setProp("group.id", id);
         }
-        
+
         /**
          * reset the bootstrap servers for the Consumer
+         *
+         * @deprecated Please use {@link #setProp(java.lang.String, java.lang.Object) } with {@link ConsumerConfig#BOOTSTRAP_SERVERS_CONFIG} instead
          */
-        public Builder<K,V> setBootstrapServers(String servers) {
+        @Deprecated
+        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.
+         *
+         * @deprecated Please use {@link #setProp(java.lang.String, java.lang.Object) } with {@link ConsumerConfig#FETCH_MIN_BYTES_CONFIG} instead
          */
-        public Builder<K,V> setFetchMinBytes(int bytes) {
+        @Deprecated
+        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.
+         *
+         * @deprecated Please use {@link #setProp(java.lang.String, java.lang.Object) } with {@link ConsumerConfig#MAX_PARTITION_FETCH_BYTES_CONFIG} instead
          */
-        public Builder<K,V> setMaxPartitionFectchBytes(int bytes) {
+        @Deprecated
+        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.
+         *
+         * @deprecated Please use {@link #setProp(java.lang.String, java.lang.Object) } with {@link ConsumerConfig#MAX_POLL_RECORDS_CONFIG} instead
          */
-        public Builder<K,V> setMaxPollRecords(int records) {
+        @Deprecated
+        public Builder<K, V> setMaxPollRecords(int records) {
             return setProp(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, records);
         }
-        
+
         //Security Related Configs
-        
         /**
          * Configure the SSL Keystore for mutual authentication
+         *
+         * @deprecated Please use {@link #setProp(java.lang.String, java.lang.Object) } with "ssl.keystore.location" and "ssl.keystore.password" instead
          */
-        public Builder<K,V> setSSLKeystore(String location, String password) {
+        @Deprecated
+        public Builder<K, V> setSSLKeystore(String location, String password) {
             return setProp("ssl.keystore.location", location)
-                    .setProp("ssl.keystore.password", password);
+                .setProp("ssl.keystore.password", password);
         }
-       
+
         /**
          * Configure the SSL Keystore for mutual authentication
+         *
+         * @deprecated Please use {@link #setProp(java.lang.String, java.lang.Object) } with "ssl.keystore.location", "ssl.keystore.password" and "ssl.key.password" instead
          */
-        public Builder<K,V> setSSLKeystore(String location, String password, String keyPassword) {
+        @Deprecated
+        public Builder<K, V> setSSLKeystore(String location, String password, String keyPassword) {
             return setProp("ssl.key.password", keyPassword)
-                    .setSSLKeystore(location, password);
+                .setSSLKeystore(location, password);
         }
-        
+
         /**
          * Configure the SSL Truststore to authenticate with the brokers
+         *
+         * @deprecated Please use {@link #setProp(java.lang.String, java.lang.Object) } with "security.protocol", "ssl.truststore.location" and "ssl.truststore.password" instead
          */
-        public Builder<K,V> setSSLTruststore(String location, String password) {
+        @Deprecated
+        public Builder<K, V> setSSLTruststore(String location, String password) {
             return setSecurityProtocol("SSL")
-                    .setProp("ssl.truststore.location", location)
-                    .setProp("ssl.truststore.password", password);
+                .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.
+         * Protocol used to communicate with brokers. Valid values are: PLAINTEXT, SSL, SASL_PLAINTEXT, SASL_SSL.
+         *
+         * @deprecated Please use {@link #setProp(java.lang.String, java.lang.Object) } with "security.protocol" instead
          */
+        @Deprecated
         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
+         * Specifies the time, in milliseconds, spent waiting in poll if data is not available. Default is 2s.
+         *
          * @param pollTimeoutMs time in ms
          */
-        public Builder<K,V> setPollTimeoutMs(long pollTimeoutMs) {
+        public Builder<K, V> setPollTimeoutMs(long pollTimeoutMs) {
             this.pollTimeoutMs = pollTimeoutMs;
             return this;
         }
 
         /**
          * Specifies the period, in milliseconds, the offset commit task is periodically called. Default is 15s.
+         *
+         * <p>This setting only has an effect if the configured {@link ProcessingGuarantee} is {@link ProcessingGuarantee#AT_LEAST_ONCE} or
+         * {@link ProcessingGuarantee#NO_GUARANTEE}.
+         *
          * @param offsetCommitPeriodMs time in ms
          */
-        public Builder<K,V> setOffsetCommitPeriodMs(long offsetCommitPeriodMs) {
+        public Builder<K, V> setOffsetCommitPeriodMs(long offsetCommitPeriodMs) {
             this.offsetCommitPeriodMs = offsetCommitPeriodMs;
             return this;
         }
@@ -336,26 +552,34 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
          * Defines the max number of polled offsets (records) that can be pending commit, before another poll can take place.
          * Once this limit is reached, no more offsets (records) can be polled until the next successful commit(s) sets the number
          * of pending offsets below the threshold. The default is {@link #DEFAULT_MAX_UNCOMMITTED_OFFSETS}.
-         * Note that this limit can in some cases be exceeded, but no partition will exceed this limit by more than maxPollRecords - 1.
+         * This limit is per partition and may in some cases be exceeded,
+         * but each partition cannot exceed this limit by more than maxPollRecords - 1.
+         * 
+         * <p>This setting only has an effect if the configured {@link ProcessingGuarantee} is {@link ProcessingGuarantee#AT_LEAST_ONCE}.
+         *
          * @param maxUncommittedOffsets max number of records that can be be pending commit
          */
-        public Builder<K,V> setMaxUncommittedOffsets(int maxUncommittedOffsets) {
+        public Builder<K, V> setMaxUncommittedOffsets(int maxUncommittedOffsets) {
             this.maxUncommittedOffsets = maxUncommittedOffsets;
             return this;
         }
 
         /**
-         * Sets the offset used by the Kafka spout in the first poll to Kafka broker upon process start.
-         * Please refer to to the documentation in {@link FirstPollOffsetStrategy}
+         * Sets the offset used by the Kafka spout in the first poll to Kafka broker upon process start. Please refer to to the
+         * documentation in {@link FirstPollOffsetStrategy}
+         *
          * @param firstPollOffsetStrategy Offset used by Kafka spout first poll
-         * */
+         */
         public Builder<K, V> setFirstPollOffsetStrategy(FirstPollOffsetStrategy firstPollOffsetStrategy) {
             this.firstPollOffsetStrategy = firstPollOffsetStrategy;
             return this;
         }
-        
+
         /**
          * Sets the retry service for the spout to use.
+         *
+         * <p>This setting only has an effect if the configured {@link ProcessingGuarantee} is {@link ProcessingGuarantee#AT_LEAST_ONCE}.
+         *
          * @param retryService the new retry service
          * @return the builder (this).
          */
@@ -367,13 +591,28 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
             return this;
         }
 
+        /**
+         * Sets the tuple listener for the spout to use.
+         *
+         * @param tupleListener the tuple listener
+         * @return the builder (this).
+         */
+        public Builder<K, V> setTupleListener(KafkaTupleListener tupleListener) {
+            if (tupleListener == null) {
+                throw new NullPointerException("KafkaTupleListener cannot be null");
+            }
+            this.tupleListener = tupleListener;
+            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
@@ -381,9 +620,10 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
         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
@@ -392,12 +632,12 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
         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
+         * 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)
          */
@@ -407,8 +647,9 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
         }
 
         /**
-         * Specifies if the spout should emit null tuples to the component downstream, or rather not emit and directly
-         * ack them. By default this parameter is set to false, which means that null tuples are not emitted.
+         * Specifies if the spout should emit null tuples to the component downstream, or rather not emit and directly ack them. By default
+         * this parameter is set to false, which means that null tuples are not emitted.
+         *
          * @param emitNullTuples sets if null tuples should or not be emitted downstream
          */
         public Builder<K, V> setEmitNullTuples(boolean emitNullTuples) {
@@ -416,50 +657,139 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
             return this;
         }
 
-        public KafkaSpoutConfig<K,V> build() {
+        /**
+         * Specifies which processing guarantee the spout should offer. Refer to the documentation for {@link ProcessingGuarantee}.
+         *
+         * @param processingGuarantee The processing guarantee the spout should offer.
+         */
+        public Builder<K, V> setProcessingGuarantee(ProcessingGuarantee processingGuarantee) {
+            this.processingGuarantee = processingGuarantee;
+            return this;
+        }
+
+        /**
+         * Specifies whether the spout should require Storm to track emitted tuples when using a {@link ProcessingGuarantee} other than
+         * {@link ProcessingGuarantee#AT_LEAST_ONCE}. The spout will always track emitted tuples when offering at-least-once guarantees
+         * regardless of this setting. This setting is false by default.
+         *
+         * <p>Enabling tracking can be useful even in cases where reliability is not a concern, because it allows
+         * {@link Config#TOPOLOGY_MAX_SPOUT_PENDING} to have an effect, and enables some spout metrics (e.g. complete-latency) that would
+         * otherwise be disabled.
+         *
+         * @param tupleTrackingEnforced true if Storm should track emitted tuples, false otherwise
+         */
+        public Builder<K, V> setTupleTrackingEnforced(boolean tupleTrackingEnforced) {
+            this.tupleTrackingEnforced = tupleTrackingEnforced;
+            return this;
+        }
+
+        /**
+         * The time period that metrics data in bucketed into.
+         * @param metricsTimeBucketSizeInSecs time in seconds
+         */
+        public Builder<K, V> setMetricsTimeBucketSizeInSecs(int metricsTimeBucketSizeInSecs) {
+            this.metricsTimeBucketSizeInSecs = metricsTimeBucketSizeInSecs;
+            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;
+    /**
+     * Factory method that creates a Builder with String key/value deserializers.
+     *
+     * @param bootstrapServers The bootstrap servers for the consumer
+     * @param topics The topics to subscribe to
+     * @return The new builder
+     */
+    public static Builder<String, String> builder(String bootstrapServers, String... topics) {
+        return setStringDeserializers(new Builder<String, String>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics));
+    }
 
-    // Kafka spout configuration
-    private final RecordTranslator<K, V> translator;
-    private final long offsetCommitPeriodMs;
-    private final int maxUncommittedOffsets;
-    private final FirstPollOffsetStrategy firstPollOffsetStrategy;
-    private final KafkaSpoutRetryService retryService;
-    private final long partitionRefreshPeriodMs;
-    private final boolean emitNullTuples;
+    /**
+     * Factory method that creates a Builder with String key/value deserializers.
+     *
+     * @param bootstrapServers The bootstrap servers for the consumer
+     * @param topics The topics to subscribe to
+     * @return The new builder
+     */
+    public static Builder<String, String> builder(String bootstrapServers, Collection<String> topics) {
+        return setStringDeserializers(new Builder<String, String>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics));
+    }
 
-    public 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.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;
-        this.emitNullTuples = builder.emitNullTuples;
+    /**
+     * Factory method that creates a Builder with String key/value deserializers.
+     *
+     * @param bootstrapServers The bootstrap servers for the consumer
+     * @param topics The topic pattern to subscribe to
+     * @return The new builder
+     */
+    public static Builder<String, String> builder(String bootstrapServers, Pattern topics) {
+        return setStringDeserializers(new Builder<String, String>(bootstrapServers, StringDeserializer.class, StringDeserializer.class, topics));
     }
 
+    private static Builder<String, String> setStringDeserializers(Builder<String, String> builder) {
+        builder.setProp(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
+        builder.setProp(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
+        return builder;
+    }
+
+    private static void setKafkaPropsForProcessingGuarantee(Builder<?, ?> builder) {
+        if (builder.kafkaProps.containsKey(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) {
+            LOG.warn("The KafkaConsumer " + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG
+                + " setting is not supported. You can configure similar behavior through KafkaSpoutConfig.Builder.setProcessingGuarantee."
+                + "This will be treated as an error in the next major release.");
+
+            final boolean enableAutoCommit = Boolean.parseBoolean(builder.kafkaProps.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG).toString());
+            if (enableAutoCommit) {
+                builder.processingGuarantee = ProcessingGuarantee.NO_GUARANTEE;
+            } else {
+                builder.processingGuarantee = ProcessingGuarantee.AT_LEAST_ONCE;
+            }
+        }
+        String autoOffsetResetPolicy = (String) builder.kafkaProps.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG);
+        if (builder.processingGuarantee == ProcessingGuarantee.AT_LEAST_ONCE) {
+            if (autoOffsetResetPolicy == null) {
+                /*
+                 * If the user wants to explicitly set an auto offset reset policy, we should respect it, but when the spout is configured
+                 * for at-least-once processing we should default to seeking to the earliest offset in case there's an offset out of range
+                 * error, rather than seeking to the latest (Kafka's default). This type of error will typically happen when the consumer
+                 * requests an offset that was deleted.
+                 */
+                LOG.info("Setting Kafka consumer property '{}' to 'earliest' to ensure at-least-once processing",
+                    ConsumerConfig.AUTO_OFFSET_RESET_CONFIG);
+                builder.kafkaProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+            } else if (!autoOffsetResetPolicy.equals("earliest") && !autoOffsetResetPolicy.equals("none")) {
+                LOG.warn("Cannot guarantee at-least-once processing with auto.offset.reset.policy other than 'earliest' or 'none'."
+                    + " Some messages may be skipped.");
+            }
+        } else if (builder.processingGuarantee == ProcessingGuarantee.AT_MOST_ONCE) {
+            if (autoOffsetResetPolicy != null
+                && (!autoOffsetResetPolicy.equals("latest") && !autoOffsetResetPolicy.equals("none"))) {
+                LOG.warn("Cannot guarantee at-most-once processing with auto.offset.reset.policy other than 'latest' or 'none'."
+                    + " Some messages may be processed more than once.");
+            }
+        }
+        LOG.info("Setting Kafka consumer property '{}' to 'false', because the spout does not support auto-commit",
+            ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG);
+        builder.kafkaProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false);
+    }
+
+    /**
+     * Gets the properties that will be passed to the KafkaConsumer.
+     *
+     * @return The Kafka properties map
+     */
     public Map<String, Object> getKafkaProps() {
         return kafkaProps;
     }
 
+    /**
+     * @deprecated Please use {@link #getKafkaProps() } and look up the entry for {@link ConsumerConfig#KEY_DESERIALIZER_CLASS_CONFIG} instead
+     */
+    @Deprecated
     public Deserializer<K> getKeyDeserializer() {
         if (keyDesClazz != null) {
             try {
@@ -471,6 +801,10 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
         return keyDes;
     }
 
+    /**
+     * @deprecated Please use {@link #getKafkaProps() } and look up the entry for {@link ConsumerConfig#VALUE_DESERIALIZER_CLASS_CONFIG} instead
+     */
+    @Deprecated
     public Deserializer<V> getValueDeserializer() {
         if (valueDesClazz != null) {
             try {
@@ -481,12 +815,12 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
         }
         return valueDes;
     }
-    
+
     public Subscription getSubscription() {
         return subscription;
     }
-    
-    public RecordTranslator<K,V> getTranslator() {
+
+    public RecordTranslator<K, V> getTranslator() {
         return translator;
     }
 
@@ -498,9 +832,21 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
         return offsetCommitPeriodMs;
     }
 
+    /**
+     * @deprecated Use {@link #getProcessingGuarantee()} instead.
+     */
+    @Deprecated
     public boolean isConsumerAutoCommitMode() {
-        return kafkaProps.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG) == null     // default is true
-                || Boolean.valueOf((String)kafkaProps.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG));
+        return kafkaProps.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG) == null // default is false
+            || Boolean.valueOf((String) kafkaProps.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG));
+    }
+    
+    public ProcessingGuarantee getProcessingGuarantee() {
+        return processingGuarantee;
+    }
+
+    public boolean isTupleTrackingEnforced() {
+        return tupleTrackingEnforced;
     }
 
     public String getConsumerGroupId() {
@@ -518,7 +864,11 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
     public KafkaSpoutRetryService getRetryService() {
         return retryService;
     }
-    
+
+    public KafkaTupleListener getTupleListener() {
+        return tupleListener;
+    }
+
     public long getPartitionRefreshPeriodMs() {
         return partitionRefreshPeriodMs;
     }
@@ -527,19 +877,26 @@ public class KafkaSpoutConfig<K, V> implements Serializable {
         return emitNullTuples;
     }
 
+    public int getMetricsTimeBucketSizeInSecs() {
+        return metricsTimeBucketSizeInSecs;
+    }
+
     @Override
     public String toString() {
-        return "KafkaSpoutConfig{" +
-                "kafkaProps=" + kafkaProps +
-                ", key=" + getKeyDeserializer() +
-                ", value=" + getValueDeserializer() +
-                ", pollTimeoutMs=" + pollTimeoutMs +
-                ", offsetCommitPeriodMs=" + offsetCommitPeriodMs +
-                ", maxUncommittedOffsets=" + maxUncommittedOffsets +
-                ", firstPollOffsetStrategy=" + firstPollOffsetStrategy +
-                ", subscription=" + subscription +
-                ", translator=" + translator +
-                ", retryService=" + retryService +
-                '}';
+        return "KafkaSpoutConfig{"
+            + "kafkaProps=" + kafkaProps
+            + ", key=" + getKeyDeserializer()
+            + ", value=" + getValueDeserializer()
+            + ", pollTimeoutMs=" + pollTimeoutMs
+            + ", offsetCommitPeriodMs=" + offsetCommitPeriodMs
+            + ", maxUncommittedOffsets=" + maxUncommittedOffsets
+            + ", firstPollOffsetStrategy=" + firstPollOffsetStrategy
+            + ", subscription=" + subscription
+            + ", translator=" + translator
+            + ", retryService=" + retryService
+            + ", tupleListener=" + tupleListener
+            + ", processingGuarantee=" + processingGuarantee
+            + ", metricsTimeBucketSizeInSecs=" + metricsTimeBucketSizeInSecs
+            + '}';
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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 59a25c2..1626fee 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
@@ -44,6 +44,12 @@ public class KafkaSpoutMessageId implements Serializable {
         this(topicPart, offset, true);
     }
 
+    /**
+     * Creates a new KafkaSpoutMessageId.
+     * @param topicPart The topic partition this message belongs to
+     * @param offset The offset of this message
+     * @param emitted True iff this message is not being skipped as a null tuple
+     */
     public KafkaSpoutMessageId(TopicPartition topicPart, long offset, boolean emitted) {
         this.topicPart = topicPart;
         this.offset = offset;
@@ -82,22 +88,14 @@ public class KafkaSpoutMessageId implements Serializable {
         this.emitted = emitted;
     }
 
-    public String getMetadata(Thread currThread) {
-        return "{" +
-                "topic-partition=" + topicPart +
-                ", offset=" + offset +
-                ", numFails=" + numFails +
-                ", thread='" + currThread.getName() + "'" +
-                '}';
-    }
-
     @Override
     public String toString() {
-        return "{" +
-                "topic-partition=" + topicPart +
-                ", offset=" + offset +
-                ", numFails=" + numFails +
-                '}';
+        return "{"
+            + "topic-partition=" + topicPart
+            + ", offset=" + offset
+            + ", numFails=" + numFails
+            + ", emitted=" + emitted
+            + '}';
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaTupleListener.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaTupleListener.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaTupleListener.java
new file mode 100644
index 0000000..3f16220
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaTupleListener.java
@@ -0,0 +1,83 @@
+/*
+ * 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.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.storm.task.TopologyContext;
+
+
+/**
+ * The KafkaTupleListener handles state changes of a kafka tuple inside a KafkaSpout.
+ */
+public interface KafkaTupleListener extends Serializable {
+
+
+    /**
+     * Called during the initialization of the kafka spout.
+     *
+     * @param conf The storm configuration.
+     * @param context The {@link TopologyContext}
+     */
+    void open(Map<String, Object> conf, TopologyContext context);
+
+    /**
+     * Called when the tuple is emitted and auto commit is disabled.
+     * If kafka auto commit is enabled, the kafka consumer will periodically (depending on the commit interval)
+     * commit the offsets. Therefore, storm disables anchoring for tuples when auto commit is enabled and the spout will
+     * not receive acks and fails for those tuples.
+     *
+     * @param tuple the storm tuple.
+     * @param msgId The id of the tuple in the spout.
+     */
+    void onEmit(List<Object> tuple, KafkaSpoutMessageId msgId);
+
+    /**
+     * Called when a tuple is acked.
+     *
+     * @param msgId The id of the tuple in the spout.
+     */
+    void onAck(KafkaSpoutMessageId msgId);
+
+    /**
+     * Called when kafka partitions are rebalanced.
+     *
+     * @param partitions The list of partitions that are now assigned to the consumer (may include partitions previously
+     *                   assigned to the consumer)
+     */
+    void onPartitionsReassigned(Collection<TopicPartition> partitions);
+
+    /**
+     * Called when the Kafka spout sets a record for retry.
+     *
+     * @param msgId The id of the tuple in the spout.
+     */
+    void onRetry(KafkaSpoutMessageId msgId);
+
+    /**
+     * Called when the maximum number of retries have been reached.
+     *
+     * @param msgId The id of the tuple in the spout.
+     */
+    void onMaxRetryReached(KafkaSpoutMessageId msgId);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionSubscription.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionSubscription.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionSubscription.java
index 61b98a8..05cd361 100644
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionSubscription.java
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/ManualPartitionSubscription.java
@@ -25,13 +25,13 @@ import java.util.Set;
 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.TopicPartitionComparator;
 import org.apache.storm.task.TopologyContext;
 
 public class ManualPartitionSubscription extends Subscription {
     private static final long serialVersionUID = 5633018073527583826L;
     private final ManualPartitioner partitioner;
     private final TopicFilter partitionFilter;
-    private transient Set<TopicPartition> currentAssignment = null;
     private transient KafkaConsumer<?, ?> consumer = null;
     private transient ConsumerRebalanceListener listener = null;
     private transient TopologyContext context = null;
@@ -54,12 +54,10 @@ public class ManualPartitionSubscription extends Subscription {
         List<TopicPartition> allPartitions = partitionFilter.getFilteredTopicPartitions(consumer);
         Collections.sort(allPartitions, TopicPartitionComparator.INSTANCE);
         Set<TopicPartition> newAssignment = new HashSet<>(partitioner.partition(allPartitions, context));
+        Set<TopicPartition> currentAssignment = consumer.assignment();
         if (!newAssignment.equals(currentAssignment)) {
+            listener.onPartitionsRevoked(currentAssignment);
             consumer.assign(newAssignment);
-            if (currentAssignment != null) {
-                listener.onPartitionsRevoked(currentAssignment);
-            }
-            currentAssignment = newAssignment;
             listener.onPartitionsAssigned(newAssignment);
         }
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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
index 0abd6c8..f9a6869 100644
--- 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
@@ -15,6 +15,7 @@
  *   See the License for the specific language governing permissions and
  *   limitations under the License.
  */
+
 package org.apache.storm.kafka.spout;
 
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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
index 3409184..3bb7152 100644
--- 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
@@ -30,8 +30,10 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Subscribe to all topics that follow a given list of values
+ * Subscribe to all topics that follow a given list of values.
+ * @deprecated Please use {@link ManualPartitionSubscription} with {@link NamedTopicFilter} instead
  */
+@Deprecated
 public class NamedSubscription extends Subscription {
     private static final Logger LOG = LoggerFactory.getLogger(NamedSubscription.class);
     private static final long serialVersionUID = 3438543305215813839L;

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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
index 9a8de0f..dc9f9e3 100644
--- 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
@@ -26,8 +26,10 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Subscribe to all topics that match a given pattern
+ * Subscribe to all topics that match a given pattern.
+ * @deprecated Please use {@link ManualPartitionSubscription} with {@link PatternTopicFilter} instead
  */
+@Deprecated
 public class PatternSubscription extends Subscription {
     private static final Logger LOG = LoggerFactory.getLogger(PatternSubscription.class);
     private static final long serialVersionUID = 3438543305215813839L;

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/SerializableDeserializer.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/SerializableDeserializer.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/SerializableDeserializer.java
index eb76a90..cc37348 100644
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/SerializableDeserializer.java
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/SerializableDeserializer.java
@@ -16,10 +16,14 @@
 package org.apache.storm.kafka.spout;
 
 import java.io.Serializable;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.common.serialization.Deserializer;
 
 /**
  * @param <T> The type this deserializer deserializes to.
+ * @deprecated Avoid using this class. Use {@link KafkaSpoutConfig.Builder#setProp(java.lang.String, java.lang.Object) } with
+ * {@link ConsumerConfig#KEY_DESERIALIZER_CLASS_CONFIG} and {@link ConsumerConfig#VALUE_DESERIALIZER_CLASS_CONFIG} instead
  */
-public interface SerializableDeserializer<T> extends Deserializer<T>, Serializable { 
+@Deprecated
+public interface SerializableDeserializer<T> extends Deserializer<T>, Serializable {
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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
index eb20f4f..722039d 100644
--- 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
@@ -30,7 +30,10 @@ public abstract class Subscription implements Serializable {
     private static final long serialVersionUID = -216136367240198716L;
 
     /**
-     * Subscribe the KafkaConsumer to the proper topics
+     * Subscribe the KafkaConsumer to the proper topics. Implementations must ensure that a given topic partition is always assigned to the
+     * same spout task. Adding and removing partitions as necessary is fine, but partitions must not move from one task to another. This
+     * constraint is only important for use with the Trident spout.
+     *
      * @param consumer the Consumer to get.
      * @param listener the rebalance listener to include in the subscription
      */

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/CommitMetadata.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/CommitMetadata.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/CommitMetadata.java
new file mode 100644
index 0000000..b7fd1a6
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/CommitMetadata.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.spout.internal;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Object representing metadata committed to Kafka.
+ */
+public class CommitMetadata {
+    private final String topologyId;
+    private final int taskId;
+    private final String threadName;
+
+    /** Kafka metadata. */
+    @JsonCreator
+    public CommitMetadata(@JsonProperty("topologyId") String topologyId,
+                          @JsonProperty("taskId") int taskId,
+                          @JsonProperty("threadName") String threadName) {
+
+        this.topologyId = topologyId;
+        this.taskId = taskId;
+        this.threadName = threadName;
+    }
+
+    public String getTopologyId() {
+        return topologyId;
+    }
+
+    public int getTaskId() {
+        return taskId;
+    }
+
+    public String getThreadName() {
+        return threadName;
+    }
+
+    @Override
+    public String toString() {
+        return "CommitMetadata{"
+            + "topologyId='" + topologyId + '\''
+            + ", taskId=" + taskId
+            + ", threadName='" + threadName + '\''
+            + '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/CommitMetadataManager.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/CommitMetadataManager.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/CommitMetadataManager.java
new file mode 100644
index 0000000..a63619c
--- /dev/null
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/CommitMetadataManager.java
@@ -0,0 +1,91 @@
+/*
+ * Copyright 2018 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.internal;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.storm.kafka.spout.KafkaSpout;
+import org.apache.storm.kafka.spout.KafkaSpoutConfig.ProcessingGuarantee;
+import org.apache.storm.task.TopologyContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Generates and reads commit metadata.
+ */
+public final class CommitMetadataManager {
+
+    private static final ObjectMapper JSON_MAPPER = new ObjectMapper();
+    private static final Logger LOG = LoggerFactory.getLogger(CommitMetadataManager.class);
+    // Metadata information to commit to Kafka. It is unique per spout instance.
+    private final String commitMetadata;
+    private final ProcessingGuarantee processingGuarantee;
+    private final TopologyContext context;
+
+    /**
+     * Create a manager with the given context.
+     */
+    public CommitMetadataManager(TopologyContext context, ProcessingGuarantee processingGuarantee) {
+        this.context = context;
+        try {
+            commitMetadata = JSON_MAPPER.writeValueAsString(new CommitMetadata(
+                context.getStormId(), context.getThisTaskId(), Thread.currentThread().getName()));
+            this.processingGuarantee = processingGuarantee;
+        } catch (JsonProcessingException e) {
+            LOG.error("Failed to create Kafka commit metadata due to JSON serialization error", e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Checks if {@link OffsetAndMetadata} was committed by a {@link KafkaSpout} instance in this topology.
+     *
+     * @param tp The topic partition the commit metadata belongs to.
+     * @param committedOffset {@link OffsetAndMetadata} info committed to Kafka
+     * @param offsetManagers The offset managers.
+     * @return true if this topology committed this {@link OffsetAndMetadata}, false otherwise
+     */
+    public boolean isOffsetCommittedByThisTopology(TopicPartition tp, OffsetAndMetadata committedOffset,
+        Map<TopicPartition, OffsetManager> offsetManagers) {
+        try {
+            if (processingGuarantee == ProcessingGuarantee.AT_LEAST_ONCE
+                && offsetManagers.containsKey(tp)
+                && offsetManagers.get(tp).hasCommitted()) {
+                return true;
+            }
+
+            final CommitMetadata committedMetadata = JSON_MAPPER.readValue(committedOffset.metadata(), CommitMetadata.class);
+            return committedMetadata.getTopologyId().equals(context.getStormId());
+        } catch (IOException e) {
+            LOG.warn("Failed to deserialize expected commit metadata [{}]."
+                + " This error is expected to occur once per partition, if the last commit to each partition"
+                + " was by an earlier version of the KafkaSpout, or by a process other than the KafkaSpout. "
+                + "Defaulting to behavior compatible with earlier version", committedOffset);
+            LOG.trace("", e);
+            return false;
+        }
+    }
+
+    public String getCommitMetadata() {
+        return commitMetadata;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/KafkaConsumerFactoryDefault.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/KafkaConsumerFactoryDefault.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/KafkaConsumerFactoryDefault.java
index 7900388..ec2fbac 100644
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/KafkaConsumerFactoryDefault.java
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/KafkaConsumerFactoryDefault.java
@@ -22,8 +22,7 @@ public class KafkaConsumerFactoryDefault<K, V> implements KafkaConsumerFactory<K
 
     @Override
     public KafkaConsumer<K, V> createConsumer(KafkaSpoutConfig<K, V> kafkaSpoutConfig) {
-        return new KafkaConsumer<>(kafkaSpoutConfig.getKafkaProps(),
-                kafkaSpoutConfig.getKeyDeserializer(), kafkaSpoutConfig.getValueDeserializer());
+        return new KafkaConsumer<>(kafkaSpoutConfig.getKafkaProps());
     }
     
 }


[3/7] storm git commit: STORM-2936 Overwrite latest storm-kafka-client 1.x-branch into 1.1.x-branch

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutEmitTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutEmitTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutEmitTest.java
index 447f8c4..dbba04b 100755
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutEmitTest.java
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutEmitTest.java
@@ -15,9 +15,6 @@
  */
 package org.apache.storm.kafka.spout;
 
-import static org.hamcrest.CoreMatchers.is;
-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;
@@ -26,35 +23,36 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
-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.common.TopicPartition;
 import org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
-import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 
-import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfigBuilder;
 import static org.mockito.Matchers.anyList;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.inOrder;
 import static org.mockito.Mockito.never;
 
-import java.util.HashSet;
 import org.apache.storm.utils.Time;
 import org.apache.storm.utils.Time.SimulatedTime;
+import org.junit.Before;
 import org.mockito.InOrder;
 
+import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.createKafkaSpoutConfigBuilder;
+import static org.mockito.Matchers.eq;
+
 public class KafkaSpoutEmitTest {
 
     private final long offsetCommitPeriodMs = 2_000;
@@ -63,50 +61,26 @@ public class KafkaSpoutEmitTest {
     private final Map<String, Object> conf = new HashMap<>();
     private final TopicPartition partition = new TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 1);
     private KafkaConsumer<String, String> consumerMock;
-    private KafkaSpout<String, String> spout;
-    private KafkaSpoutConfig spoutConfig;
+    private KafkaSpoutConfig<String, String> spoutConfig;
 
-    private void setupSpout(Set<TopicPartition> assignedPartitions) {
-        spoutConfig = getKafkaSpoutConfigBuilder(-1)
+    @Before
+    public void setUp() {
+        spoutConfig = createKafkaSpoutConfigBuilder(mock(Subscription.class), -1)
             .setOffsetCommitPeriodMs(offsetCommitPeriodMs)
             .build();
-
         consumerMock = mock(KafkaConsumer.class);
-        KafkaConsumerFactory<String, String> consumerFactory = new KafkaConsumerFactory<String, String>() {
-            @Override
-            public KafkaConsumer<String, String> createConsumer(KafkaSpoutConfig<String, String> kafkaSpoutConfig) {
-                return consumerMock;
-            }
-        };
-
-        //Set up a spout listening to 1 topic partition
-        spout = new KafkaSpout<>(spoutConfig, consumerFactory);
-
-        spout.open(conf, contextMock, collectorMock);
-        spout.activate();
-
-        ArgumentCaptor<ConsumerRebalanceListener> rebalanceListenerCapture = ArgumentCaptor.forClass(ConsumerRebalanceListener.class);
-        verify(consumerMock).subscribe(anyCollection(), rebalanceListenerCapture.capture());
-
-        //Assign partitions to the spout
-        ConsumerRebalanceListener consumerRebalanceListener = rebalanceListenerCapture.getValue();
-        consumerRebalanceListener.onPartitionsAssigned(assignedPartitions);
     }
 
     @Test
     public void testNextTupleEmitsAtMostOneTuple() {
         //The spout should emit at most one message per call to nextTuple
         //This is necessary for Storm to be able to throttle the spout according to maxSpoutPending
-        setupSpout(Collections.singleton(partition));
+        KafkaSpout<String, String> spout = SpoutWithMockedConsumerSetupHelper.setupSpout(spoutConfig, conf, contextMock, collectorMock, consumerMock, partition);
         Map<TopicPartition, List<ConsumerRecord<String, String>>> records = new HashMap<>();
-        List<ConsumerRecord<String, String>> recordsForPartition = new ArrayList<>();
-        for (int i = 0; i < 10; i++) {
-            recordsForPartition.add(new ConsumerRecord(partition.topic(), partition.partition(), i, "key", "value"));
-        }
-        records.put(partition, recordsForPartition);
+        records.put(partition, SpoutWithMockedConsumerSetupHelper.<String, String>createRecords(partition, 0, 10));
 
         when(consumerMock.poll(anyLong()))
-            .thenReturn(new ConsumerRecords(records));
+            .thenReturn(new ConsumerRecords<>(records));
 
         spout.nextTuple();
 
@@ -114,29 +88,26 @@ public class KafkaSpoutEmitTest {
     }
 
     @Test
-    public void testNextTupleEmitsFailedMessagesEvenWhenMaxUncommittedOffsetsIsExceeded() {
+    public void testNextTupleEmitsFailedMessagesEvenWhenMaxUncommittedOffsetsIsExceeded() throws IOException {
         //The spout must reemit failed messages waiting for retry even if it is not allowed to poll for new messages due to maxUncommittedOffsets being exceeded
-        
+
         //Emit maxUncommittedOffsets messages, and fail all of them. Then ensure that the spout will retry them when the retry backoff has passed
         try (SimulatedTime simulatedTime = new SimulatedTime()) {
-            setupSpout(Collections.singleton(partition));
+            KafkaSpout<String, String> spout = SpoutWithMockedConsumerSetupHelper.setupSpout(spoutConfig, conf, contextMock, collectorMock, consumerMock, partition);
             Map<TopicPartition, List<ConsumerRecord<String, String>>> records = new HashMap<>();
-            List<ConsumerRecord<String, String>> recordsForPartition = new ArrayList<>();
-            for (int i = 0; i < spoutConfig.getMaxUncommittedOffsets(); i++) {
-                //This is cheating a bit since maxPollRecords would normally spread this across multiple polls
-                recordsForPartition.add(new ConsumerRecord(partition.topic(), partition.partition(), i, "key", "value"));
-            }
-            records.put(partition, recordsForPartition);
+            int numRecords = spoutConfig.getMaxUncommittedOffsets();
+            //This is cheating a bit since maxPollRecords would normally spread this across multiple polls
+            records.put(partition, SpoutWithMockedConsumerSetupHelper.<String, String>createRecords(partition, 0, numRecords));
 
             when(consumerMock.poll(anyLong()))
-                .thenReturn(new ConsumerRecords(records));
+                .thenReturn(new ConsumerRecords<>(records));
 
-            for (int i = 0; i < recordsForPartition.size(); i++) {
+            for (int i = 0; i < numRecords; i++) {
                 spout.nextTuple();
             }
 
             ArgumentCaptor<KafkaSpoutMessageId> messageIds = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
-            verify(collectorMock, times(recordsForPartition.size())).emit(anyString(), anyList(), messageIds.capture());
+            verify(collectorMock, times(numRecords)).emit(anyString(), anyList(), messageIds.capture());
 
             for (KafkaSpoutMessageId messageId : messageIds.getAllValues()) {
                 spout.fail(messageId);
@@ -146,16 +117,16 @@ public class KafkaSpoutEmitTest {
 
             Time.advanceTime(50);
             //No backoff for test retry service, just check that messages will retry immediately
-            for (int i = 0; i < recordsForPartition.size(); i++) {
+            for (int i = 0; i < numRecords; i++) {
                 spout.nextTuple();
             }
 
             ArgumentCaptor<KafkaSpoutMessageId> retryMessageIds = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
-            verify(collectorMock, times(recordsForPartition.size())).emit(anyString(), anyList(), retryMessageIds.capture());
+            verify(collectorMock, times(numRecords)).emit(anyString(), anyList(), retryMessageIds.capture());
 
             //Verify that the poll started at the earliest retriable tuple offset
             List<Long> failedOffsets = new ArrayList<>();
-            for(KafkaSpoutMessageId msgId : messageIds.getAllValues()) {
+            for (KafkaSpoutMessageId msgId : messageIds.getAllValues()) {
                 failedOffsets.add(msgId.offset());
             }
             InOrder inOrder = inOrder(consumerMock);
@@ -163,93 +134,80 @@ public class KafkaSpoutEmitTest {
             inOrder.verify(consumerMock).poll(anyLong());
         }
     }
-    
+
     @Test
-    public void testNextTupleEmitsAtMostMaxUncommittedOffsetsPlusMaxPollRecordsWhenRetryingTuples() {
-        /*
-        The spout must reemit failed messages waiting for retry even if it is not allowed to poll for new messages due to maxUncommittedOffsets being exceeded.
-        numUncommittedOffsets is equal to numNonRetriableEmittedTuples + numRetriableTuples.
-        The spout will only emit if numUncommittedOffsets - numRetriableTuples < maxUncommittedOffsets (i.e. numNonRetriableEmittedTuples < maxUncommittedOffsets)
-        This means that the latest offset a poll can start at for a retriable partition,
-        counting from the last committed offset, is maxUncommittedOffsets,
-        where there are maxUncommittedOffsets - 1 uncommitted tuples "to the left".
-        If the retry poll starts at that offset, it at most emits the retried tuple plus maxPollRecords - 1 new tuples.
-        The limit on uncommitted offsets for one partition is therefore maxUncommittedOffsets + maxPollRecords - 1.
-        
-        It is only necessary to test this for a single partition, because partitions can't contribute negatively to numNonRetriableEmittedTuples,
-        so if the limit holds for one partition, it will also hold for each individual partition when multiple are involved.
-        
-        This makes the actual limit numPartitions * (maxUncommittedOffsets + maxPollRecords - 1)
-         */
-        
-        //Emit maxUncommittedOffsets messages, and fail only the last. Then ensure that the spout will allow no more than maxUncommittedOffsets + maxPollRecords - 1 uncommitted offsets when retrying
+    public void testSpoutWillSkipPartitionsAtTheMaxUncommittedOffsetsLimit() {
+        //This verifies that partitions can't prevent each other from retrying tuples due to the maxUncommittedOffsets limit.
         try (SimulatedTime simulatedTime = new SimulatedTime()) {
-            setupSpout(Collections.singleton(partition));
-            
-            Map<TopicPartition, List<ConsumerRecord<String, String>>> firstPollRecords = new HashMap<>();
-            List<ConsumerRecord<String, String>> firstPollRecordsForPartition = new ArrayList<>();
-            for (int i = 0; i < spoutConfig.getMaxUncommittedOffsets(); i++) {
-                //This is cheating a bit since maxPollRecords would normally spread this across multiple polls
-                firstPollRecordsForPartition.add(new ConsumerRecord(partition.topic(), partition.partition(), i, "key", "value"));
-            }
-            firstPollRecords.put(partition, firstPollRecordsForPartition);
-            
-            int maxPollRecords = 5;
-            Map<TopicPartition, List<ConsumerRecord<String, String>>> secondPollRecords = new HashMap<>();
-            List<ConsumerRecord<String, String>> secondPollRecordsForPartition = new ArrayList<>();
-            for(int i = 0; i < maxPollRecords; i++) {
-                secondPollRecordsForPartition.add(new ConsumerRecord(partition.topic(), partition.partition(), spoutConfig.getMaxUncommittedOffsets() + i, "key", "value"));
-            }
-            secondPollRecords.put(partition, secondPollRecordsForPartition);
+            TopicPartition partitionTwo = new TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 2);
+            KafkaSpout<String, String> spout = SpoutWithMockedConsumerSetupHelper.setupSpout(spoutConfig, conf, contextMock, collectorMock, consumerMock, partition, partitionTwo);
+            Map<TopicPartition, List<ConsumerRecord<String, String>>> records = new HashMap<>();
+            //This is cheating a bit since maxPollRecords would normally spread this across multiple polls
+            records.put(partition, SpoutWithMockedConsumerSetupHelper.<String, String>createRecords(partition, 0, spoutConfig.getMaxUncommittedOffsets()));
+            records.put(partitionTwo, SpoutWithMockedConsumerSetupHelper.<String, String>createRecords(partitionTwo, 0, spoutConfig.getMaxUncommittedOffsets() + 1));
+            int numMessages = spoutConfig.getMaxUncommittedOffsets()*2 + 1;
 
             when(consumerMock.poll(anyLong()))
-                .thenReturn(new ConsumerRecords(firstPollRecords))
-                .thenReturn(new ConsumerRecords(secondPollRecords));
+                .thenReturn(new ConsumerRecords<>(records));
 
-            for (int i = 0; i < spoutConfig.getMaxUncommittedOffsets() + maxPollRecords; i++) {
+            for (int i = 0; i < numMessages; i++) {
                 spout.nextTuple();
             }
 
             ArgumentCaptor<KafkaSpoutMessageId> messageIds = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
-            verify(collectorMock, times(firstPollRecordsForPartition.size())).emit(anyString(), anyList(), messageIds.capture());
-
-            KafkaSpoutMessageId failedMessageId = messageIds.getAllValues().get(messageIds.getAllValues().size() - 1);
-            spout.fail(failedMessageId);
-
-            reset(collectorMock);
-
-            //Now make the single failed tuple retriable
-            Time.advanceTime(50);
-            //The spout should allow another poll since there are now only maxUncommittedOffsets - 1 nonretriable tuples
-            for (int i = 0; i < firstPollRecordsForPartition.size() + maxPollRecords; i++) {
-                spout.nextTuple();
+            verify(collectorMock, times(numMessages)).emit(anyString(), anyList(), messageIds.capture());
+            
+            //Now fail a tuple on partition one and verify that it is allowed to retry, because the failed tuple is below the maxUncommittedOffsets limit
+            KafkaSpoutMessageId failedMessageIdPartitionOne = null;
+            for (KafkaSpoutMessageId msgId : messageIds.getAllValues()) {
+                if (msgId.partition() == partition.partition()) {
+                    failedMessageIdPartitionOne = msgId;
+                    break;
+                }
+            }
+            
+            spout.fail(failedMessageIdPartitionOne);
+
+            //Also fail the last tuple from partition two. Since the failed tuple is beyond the maxUncommittedOffsets limit, it should not be retried until earlier messages are acked.
+            KafkaSpoutMessageId failedMessageIdPartitionTwo = null;
+            for (KafkaSpoutMessageId msgId: messageIds.getAllValues()) {
+                if (msgId.partition() == partitionTwo.partition()) {
+                    if (failedMessageIdPartitionTwo != null) {
+                        if (msgId.offset() >= failedMessageIdPartitionTwo.offset()) {
+                            failedMessageIdPartitionTwo = msgId;
+                        }
+                    } else {
+                        failedMessageIdPartitionTwo = msgId;
+                    }
+                }
             }
 
-            ArgumentCaptor<KafkaSpoutMessageId> retryBatchMessageIdsCaptor = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
-            verify(collectorMock, times(maxPollRecords)).emit(anyString(), anyList(), retryBatchMessageIdsCaptor.capture());
+            spout.fail(failedMessageIdPartitionTwo);
+            
             reset(collectorMock);
             
-            //Check that the consumer started polling at the failed tuple offset
+            Time.advanceTime(50);
+            when(consumerMock.poll(anyLong()))
+                .thenReturn(new ConsumerRecords<>(Collections.singletonMap(partition, SpoutWithMockedConsumerSetupHelper.<String, String>createRecords(partition, failedMessageIdPartitionOne.offset(), 1))));
+            
+            spout.nextTuple();
+            
+            verify(collectorMock, times(1)).emit(anyString(), anyList(), anyObject());
+            
             InOrder inOrder = inOrder(consumerMock);
-            inOrder.verify(consumerMock).seek(partition, failedMessageId.offset());
+            inOrder.verify(consumerMock).seek(partition, failedMessageIdPartitionOne.offset());
+            //Should not seek on the paused partition
+            inOrder.verify(consumerMock, never()).seek(eq(partitionTwo), anyLong());
+            inOrder.verify(consumerMock).pause(Collections.singleton(partitionTwo));
             inOrder.verify(consumerMock).poll(anyLong());
+            inOrder.verify(consumerMock).resume(Collections.singleton(partitionTwo));
             
-            //Now fail all except one of the last batch, and check that the spout won't reemit any tuples because there are more than maxUncommittedOffsets nonretriable tuples
-            Time.advanceTime(50);
-            List<KafkaSpoutMessageId> retryBatchMessageIds = retryBatchMessageIdsCaptor.getAllValues();
-            KafkaSpoutMessageId firstTupleFromRetryBatch = retryBatchMessageIds.remove(0);
-            for(KafkaSpoutMessageId msgId : retryBatchMessageIds) {
-                spout.fail(msgId);
-            }
-            for (int i = 0; i < firstPollRecordsForPartition.size() + maxPollRecords; i++) {
-                spout.nextTuple();
-            }
-            verify(collectorMock, never()).emit(anyString(), anyList(), anyObject());
+            reset(collectorMock);
             
-            //Fail the last tuple, which brings the number of nonretriable tuples back under the limit, and check that the spout polls again
-            spout.fail(firstTupleFromRetryBatch);
+            //Now also check that no more tuples are polled for, since both partitions are at their limits
             spout.nextTuple();
-            verify(collectorMock, times(1)).emit(anyString(), anyList(), anyObject());
+
+            verify(collectorMock, never()).emit(anyString(), anyList(), anyObject());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutLogCompactionSupportTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutLogCompactionSupportTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutLogCompactionSupportTest.java
new file mode 100644
index 0000000..09f7fc5
--- /dev/null
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutLogCompactionSupportTest.java
@@ -0,0 +1,223 @@
+/*
+ * Copyright 2017 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 static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.createKafkaSpoutConfigBuilder;
+
+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.*;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Time.SimulatedTime;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.InOrder;
+import org.mockito.MockitoAnnotations;
+
+import static org.hamcrest.CoreMatchers.is;
+
+import static org.hamcrest.Matchers.hasKey;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.anyList;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.inOrder;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
+
+public class KafkaSpoutLogCompactionSupportTest {
+
+    private final long offsetCommitPeriodMs = 2_000;
+    private final TopologyContext contextMock = mock(TopologyContext.class);
+    private final SpoutOutputCollector collectorMock = mock(SpoutOutputCollector.class);
+    private final Map<String, Object> conf = new HashMap<>();
+    private final TopicPartition partition = new TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 1);
+    private KafkaConsumer<String, String> consumerMock;
+    private KafkaSpoutConfig<String, String> spoutConfig;
+
+    @Captor
+    private ArgumentCaptor<Map<TopicPartition, OffsetAndMetadata>> commitCapture;
+
+    @Before
+    public void setUp() {
+        MockitoAnnotations.initMocks(this);
+        spoutConfig = createKafkaSpoutConfigBuilder(mock(Subscription.class), -1)
+            .setOffsetCommitPeriodMs(offsetCommitPeriodMs)
+            .build();
+        consumerMock = mock(KafkaConsumer.class);
+    }
+
+    @Test
+    public void testCommitSuccessWithOffsetVoids() {
+        //Verify that the commit logic can handle offset voids due to log compaction
+        try (SimulatedTime simulatedTime = new SimulatedTime()) {
+            KafkaSpout<String, String> spout = SpoutWithMockedConsumerSetupHelper.setupSpout(spoutConfig, conf, contextMock, collectorMock, consumerMock, partition);
+            Map<TopicPartition, List<ConsumerRecord<String, String>>> records = new HashMap<>();
+            List<ConsumerRecord<String, String>> recordsForPartition = new ArrayList<>();
+            // Offsets emitted are 0,1,2,3,4,<void>,8,9
+            recordsForPartition.addAll(SpoutWithMockedConsumerSetupHelper.<String, String>createRecords(partition, 0, 5));
+            recordsForPartition.addAll(SpoutWithMockedConsumerSetupHelper.<String, String>createRecords(partition, 8, 2));
+            records.put(partition, recordsForPartition);
+
+            when(consumerMock.poll(anyLong()))
+                    .thenReturn(new ConsumerRecords<>(records));
+
+            for (int i = 0; i < recordsForPartition.size(); i++) {
+                spout.nextTuple();
+            }
+
+            ArgumentCaptor<KafkaSpoutMessageId> messageIds = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
+            verify(collectorMock, times(recordsForPartition.size())).emit(anyString(), anyList(), messageIds.capture());
+
+            for (KafkaSpoutMessageId messageId : messageIds.getAllValues()) {
+                spout.ack(messageId);
+            }
+
+            // Advance time and then trigger first call to kafka consumer commit; the commit must progress to offset 9
+            Time.advanceTime(KafkaSpout.TIMER_DELAY_MS + offsetCommitPeriodMs);
+            when(consumerMock.poll(anyLong()))
+                    .thenReturn(new ConsumerRecords<String, String>(Collections.<TopicPartition, List<ConsumerRecord<String, String>>>emptyMap()));
+            spout.nextTuple();
+
+            InOrder inOrder = inOrder(consumerMock);
+            inOrder.verify(consumerMock).commitSync(commitCapture.capture());
+            inOrder.verify(consumerMock).poll(anyLong());
+
+            //verify that Offset 10 was last committed offset, since this is the offset the spout should resume at
+            Map<TopicPartition, OffsetAndMetadata> commits = commitCapture.getValue();
+            assertTrue(commits.containsKey(partition));
+            assertEquals(10, commits.get(partition).offset());
+        }
+    }
+    
+    @Test
+    public void testWillSkipRetriableTuplesIfOffsetsAreCompactedAway() {
+        /*
+          Verify that failed offsets will only retry if the corresponding message exists. 
+          When log compaction is enabled in Kafka it is possible that a tuple can fail, 
+          and then be impossible to retry because the message in Kafka has been deleted.
+          The spout needs to quietly ack such tuples to allow commits to progress past the deleted offset.
+         */
+        try (SimulatedTime simulatedTime = new SimulatedTime()) {
+            TopicPartition partitionTwo = new TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 2);
+            KafkaSpout<String, String> spout = SpoutWithMockedConsumerSetupHelper.setupSpout(spoutConfig, conf, contextMock, collectorMock, consumerMock, partition, partitionTwo);
+            
+            List<KafkaSpoutMessageId> firstPartitionMsgIds = SpoutWithMockedConsumerSetupHelper
+                .pollAndEmit(spout, consumerMock, 3, collectorMock, partition, 0, 1, 2);
+            reset(collectorMock);
+            List<KafkaSpoutMessageId> secondPartitionMsgIds = SpoutWithMockedConsumerSetupHelper
+                .pollAndEmit(spout, consumerMock, 3, collectorMock, partitionTwo, 0, 1, 2);
+            reset(collectorMock);
+            
+            for(int i = 0; i < 3; i++) {
+                spout.fail(firstPartitionMsgIds.get(i));
+                spout.fail(secondPartitionMsgIds.get(i));
+            }
+            
+            Time.advanceTime(50);
+            
+            //The failed tuples are ready for retry. Make it appear like 0 and 1 on the first partition were compacted away.
+            //In this case the second partition acts as control to verify that we only skip past offsets that are no longer present.
+            Map<TopicPartition, int[]> retryOffsets = new HashMap<>();
+            retryOffsets.put(partition, new int[] {2});
+            retryOffsets.put(partitionTwo, new int[] {0, 1, 2});
+            int expectedEmits = 4; //2 on first partition, 0-2 on second partition
+            List<KafkaSpoutMessageId> retryMessageIds = SpoutWithMockedConsumerSetupHelper.pollAndEmit(spout, consumerMock, expectedEmits, collectorMock, retryOffsets);
+            
+            Time.advanceTime(KafkaSpout.TIMER_DELAY_MS + offsetCommitPeriodMs);
+            spout.nextTuple();
+            
+            verify(consumerMock).commitSync(commitCapture.capture());
+            Map<TopicPartition, OffsetAndMetadata> committed = commitCapture.getValue();
+            assertThat(committed.keySet(), is(Collections.singleton(partition)));
+            assertThat("The first partition should have committed up to the first retriable tuple that is not missing", committed.get(partition).offset(), is(2L));
+            
+            for(KafkaSpoutMessageId msgId : retryMessageIds) {
+                spout.ack(msgId);
+            }
+            
+            //The spout should now commit all the offsets, since all offsets are either acked or were missing when retrying
+            Time.advanceTime(KafkaSpout.TIMER_DELAY_MS + offsetCommitPeriodMs);
+            spout.nextTuple();
+
+            verify(consumerMock, times(2)).commitSync(commitCapture.capture());
+            committed = commitCapture.getValue();
+            assertThat(committed, hasKey(partition));
+            assertThat(committed, hasKey(partitionTwo));
+            assertThat(committed.get(partition).offset(), is(3L));
+            assertThat(committed.get(partitionTwo).offset(), is(3L));
+        }
+    }
+    
+    @Test
+    public void testWillSkipRetriableTuplesIfOffsetsAreCompactedAwayWithoutAckingPendingTuples() {
+        //Demonstrate that the spout doesn't ack pending tuples when skipping compacted tuples. The pending tuples should be allowed to finish normally.
+        try (SimulatedTime simulatedTime = new SimulatedTime()) {
+            KafkaSpout<String, String> spout = SpoutWithMockedConsumerSetupHelper.setupSpout(spoutConfig, conf, contextMock, collectorMock, consumerMock, partition);
+            
+            List<KafkaSpoutMessageId> firstPartitionMsgIds = SpoutWithMockedConsumerSetupHelper
+                .pollAndEmit(spout, consumerMock, 3, collectorMock, partition, 0, 1, 2);
+            reset(collectorMock);
+            
+            spout.fail(firstPartitionMsgIds.get(0));            
+            spout.fail(firstPartitionMsgIds.get(2));
+            
+            Time.advanceTime(50);
+            
+            //The failed tuples are ready for retry. Make it appear like 0 and 1 were compacted away.
+            List<KafkaSpoutMessageId> retryMessageIds = SpoutWithMockedConsumerSetupHelper.pollAndEmit(spout, consumerMock, 1, collectorMock, partition, 2);
+            for(KafkaSpoutMessageId msgId : retryMessageIds) {
+                spout.ack(msgId);
+            }
+            
+            Time.advanceTime(KafkaSpout.TIMER_DELAY_MS + offsetCommitPeriodMs);
+            spout.nextTuple();
+            
+            verify(consumerMock).commitSync(commitCapture.capture());
+            Map<TopicPartition, OffsetAndMetadata> committed = commitCapture.getValue();
+            assertThat(committed.keySet(), is(Collections.singleton(partition)));
+            assertThat("The first partition should have committed the missing offset, but no further since the next tuple is pending",
+                committed.get(partition).offset(), is(1L));
+            
+            spout.ack(firstPartitionMsgIds.get(1));
+            
+            Time.advanceTime(KafkaSpout.TIMER_DELAY_MS + offsetCommitPeriodMs);
+            spout.nextTuple();
+            
+            verify(consumerMock, times(2)).commitSync(commitCapture.capture());
+            committed = commitCapture.getValue();
+            assertThat(committed.keySet(), is(Collections.singleton(partition)));
+            assertThat("The first partition should have committed all offsets", committed.get(partition).offset(), is(3L));
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutMessagingGuaranteeTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutMessagingGuaranteeTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutMessagingGuaranteeTest.java
new file mode 100644
index 0000000..082cc58
--- /dev/null
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutMessagingGuaranteeTest.java
@@ -0,0 +1,259 @@
+/*
+ * Copyright 2017 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 static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.createKafkaSpoutConfigBuilder;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Matchers.anyList;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Matchers.isNull;
+import static org.mockito.Mockito.inOrder;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+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.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
+import org.apache.storm.kafka.spout.internal.CommitMetadataManager;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Time.SimulatedTime;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.ArgumentMatcher;
+import org.mockito.Captor;
+import org.mockito.InOrder;
+import org.mockito.runners.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.class)
+public class KafkaSpoutMessagingGuaranteeTest {
+
+    @Captor
+    private ArgumentCaptor<Map<TopicPartition, OffsetAndMetadata>> commitCapture;
+
+    private final TopologyContext contextMock = mock(TopologyContext.class);
+    private final SpoutOutputCollector collectorMock = mock(SpoutOutputCollector.class);
+    private final Map<String, Object> conf = new HashMap<>();
+    private final TopicPartition partition = new TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 1);
+    private KafkaConsumer<String, String> consumerMock;
+
+    @Before
+    public void setUp() {
+        consumerMock = mock(KafkaConsumer.class);
+    }
+
+    @Test
+    public void testAtMostOnceModeCommitsBeforeEmit() throws Exception {
+        //At-most-once mode must commit tuples before they are emitted to the topology to ensure that a spout crash won't cause replays.
+        KafkaSpoutConfig<String, String> spoutConfig = createKafkaSpoutConfigBuilder(mock(Subscription.class), -1)
+            .setProcessingGuarantee(KafkaSpoutConfig.ProcessingGuarantee.AT_MOST_ONCE)
+            .build();
+        KafkaSpout<String, String> spout = SpoutWithMockedConsumerSetupHelper.setupSpout(spoutConfig, conf, contextMock, collectorMock, consumerMock, partition);
+
+        when(consumerMock.poll(anyLong())).thenReturn(new ConsumerRecords<>(Collections.singletonMap(partition,
+            SpoutWithMockedConsumerSetupHelper.<String, String>createRecords(partition, 0, 1))));
+
+        spout.nextTuple();
+
+        when(consumerMock.position(partition)).thenReturn(1L);
+
+        //The spout should have emitted the tuple, and must have committed it before emit
+        InOrder inOrder = inOrder(consumerMock, collectorMock);
+        inOrder.verify(consumerMock).poll(anyLong());
+        inOrder.verify(consumerMock).commitSync(commitCapture.capture());
+        inOrder.verify(collectorMock).emit(eq(SingleTopicKafkaSpoutConfiguration.STREAM), anyList());
+
+        CommitMetadataManager metadataManager = new CommitMetadataManager(contextMock, KafkaSpoutConfig.ProcessingGuarantee.AT_MOST_ONCE);
+        Map<TopicPartition, OffsetAndMetadata> committedOffsets = commitCapture.getValue();
+        assertThat(committedOffsets.get(partition).offset(), is(0L));
+        assertThat(committedOffsets.get(partition).metadata(), is(metadataManager.getCommitMetadata()));
+    }
+
+    private void doTestModeDisregardsMaxUncommittedOffsets(KafkaSpoutConfig<String, String> spoutConfig) {
+        KafkaSpout<String, String> spout = SpoutWithMockedConsumerSetupHelper.setupSpout(spoutConfig, conf, contextMock, collectorMock, consumerMock, partition);
+
+        when(consumerMock.poll(anyLong()))
+            .thenReturn(new ConsumerRecords<>(Collections.singletonMap(partition,
+                SpoutWithMockedConsumerSetupHelper.<String, String>createRecords(partition, 0, spoutConfig.getMaxUncommittedOffsets()))))
+            .thenReturn(new ConsumerRecords<>(Collections.singletonMap(partition,
+                SpoutWithMockedConsumerSetupHelper.<String, String>createRecords(partition, spoutConfig.getMaxUncommittedOffsets() - 1, spoutConfig.getMaxUncommittedOffsets()))));
+
+        for (int i = 0; i < spoutConfig.getMaxUncommittedOffsets() * 2; i++) {
+            spout.nextTuple();
+        }
+
+        verify(consumerMock, times(2)).poll(anyLong());
+        verify(collectorMock, times(spoutConfig.getMaxUncommittedOffsets() * 2)).emit(eq(SingleTopicKafkaSpoutConfiguration.STREAM), anyList());
+    }
+
+    @Test
+    public void testAtMostOnceModeDisregardsMaxUncommittedOffsets() throws Exception {
+        //The maxUncommittedOffsets limit should not be enforced, since it is only meaningful in at-least-once mode
+        KafkaSpoutConfig<String, String> spoutConfig = createKafkaSpoutConfigBuilder(mock(Subscription.class), -1)
+            .setProcessingGuarantee(KafkaSpoutConfig.ProcessingGuarantee.AT_MOST_ONCE)
+            .build();
+        doTestModeDisregardsMaxUncommittedOffsets(spoutConfig);
+    }
+
+    @Test
+    public void testNoGuaranteeModeDisregardsMaxUncommittedOffsets() throws Exception {
+        //The maxUncommittedOffsets limit should not be enforced, since it is only meaningful in at-least-once mode
+        KafkaSpoutConfig<String, String> spoutConfig = createKafkaSpoutConfigBuilder(mock(Subscription.class), -1)
+            .setProcessingGuarantee(KafkaSpoutConfig.ProcessingGuarantee.NO_GUARANTEE)
+            .build();
+        doTestModeDisregardsMaxUncommittedOffsets(spoutConfig);
+    }
+
+    private void doTestModeCannotReplayTuples(KafkaSpoutConfig<String, String> spoutConfig) {
+        KafkaSpout<String, String> spout = SpoutWithMockedConsumerSetupHelper.setupSpout(spoutConfig, conf, contextMock, collectorMock, consumerMock, partition);
+
+        when(consumerMock.poll(anyLong())).thenReturn(new ConsumerRecords<>(Collections.singletonMap(partition,
+            SpoutWithMockedConsumerSetupHelper.<String, String>createRecords(partition, 0, 1))));
+
+        spout.nextTuple();
+
+        ArgumentCaptor<KafkaSpoutMessageId> msgIdCaptor = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
+        verify(collectorMock).emit(eq(SingleTopicKafkaSpoutConfiguration.STREAM), anyList(), msgIdCaptor.capture());
+        assertThat("Should have captured a message id", msgIdCaptor.getValue(), not(nullValue()));
+
+        spout.fail(msgIdCaptor.getValue());
+
+        reset(consumerMock);
+
+        when(consumerMock.poll(anyLong())).thenReturn(new ConsumerRecords<>(Collections.singletonMap(partition,
+            SpoutWithMockedConsumerSetupHelper.<String, String>createRecords(partition, 1, 1))));
+
+        spout.nextTuple();
+
+        //The consumer should not be seeking to retry the failed tuple, it should just be continuing from the current position
+        verify(consumerMock, never()).seek(eq(partition), anyLong());
+    }
+
+    @Test
+    public void testAtMostOnceModeCannotReplayTuples() throws Exception {
+        //When tuple tracking is enabled, the spout must not replay tuples in at-most-once mode
+        KafkaSpoutConfig<String, String> spoutConfig = createKafkaSpoutConfigBuilder(mock(Subscription.class), -1)
+            .setProcessingGuarantee(KafkaSpoutConfig.ProcessingGuarantee.AT_MOST_ONCE)
+            .setTupleTrackingEnforced(true)
+            .build();
+        doTestModeCannotReplayTuples(spoutConfig);
+    }
+
+    @Test
+    public void testNoGuaranteeModeCannotReplayTuples() throws Exception {
+        //When tuple tracking is enabled, the spout must not replay tuples in no guarantee mode
+        KafkaSpoutConfig<String, String> spoutConfig = createKafkaSpoutConfigBuilder(mock(Subscription.class), -1)
+            .setProcessingGuarantee(KafkaSpoutConfig.ProcessingGuarantee.NO_GUARANTEE)
+            .setTupleTrackingEnforced(true)
+            .build();
+        doTestModeCannotReplayTuples(spoutConfig);
+    }
+
+    @Test
+    public void testAtMostOnceModeDoesNotCommitAckedTuples() throws Exception {
+        //When tuple tracking is enabled, the spout must not commit acked tuples in at-most-once mode because they were committed before being emitted
+        KafkaSpoutConfig<String, String> spoutConfig = createKafkaSpoutConfigBuilder(mock(Subscription.class), -1)
+            .setProcessingGuarantee(KafkaSpoutConfig.ProcessingGuarantee.AT_MOST_ONCE)
+            .setTupleTrackingEnforced(true)
+            .build();
+        try (SimulatedTime time = new SimulatedTime()) {
+            KafkaSpout<String, String> spout = SpoutWithMockedConsumerSetupHelper.setupSpout(spoutConfig, conf, contextMock, collectorMock, consumerMock, partition);
+
+            when(consumerMock.poll(anyLong())).thenReturn(new ConsumerRecords<>(Collections.singletonMap(partition,
+                SpoutWithMockedConsumerSetupHelper.<String, String>createRecords(partition, 0, 1))));
+
+            spout.nextTuple();
+            reset(consumerMock);
+
+            ArgumentCaptor<KafkaSpoutMessageId> msgIdCaptor = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
+            verify(collectorMock).emit(eq(SingleTopicKafkaSpoutConfiguration.STREAM), anyList(), msgIdCaptor.capture());
+            assertThat("Should have captured a message id", msgIdCaptor.getValue(), not(nullValue()));
+
+            spout.ack(msgIdCaptor.getValue());
+
+            Time.advanceTime(KafkaSpout.TIMER_DELAY_MS + spoutConfig.getOffsetsCommitPeriodMs());
+
+            when(consumerMock.poll(anyLong())).thenReturn(new ConsumerRecords<>(Collections.<TopicPartition, List<ConsumerRecord<String, String>>>emptyMap()));
+
+            spout.nextTuple();
+
+            verify(consumerMock, never()).commitSync(argThat(new ArgumentMatcher<Map<TopicPartition, OffsetAndMetadata>>() {
+                @Override
+                public boolean matches(Object arg) {
+                    Map<TopicPartition, OffsetAndMetadata> castArg = (Map<TopicPartition, OffsetAndMetadata>) arg;
+                    return !castArg.containsKey(partition);
+                }
+            }));
+        }
+    }
+
+    @Test
+    public void testNoGuaranteeModeCommitsPolledTuples() throws Exception {
+        //When using the no guarantee mode, the spout must commit tuples periodically, regardless of whether they've been acked
+        KafkaSpoutConfig<String, String> spoutConfig = createKafkaSpoutConfigBuilder(mock(Subscription.class), -1)
+            .setProcessingGuarantee(KafkaSpoutConfig.ProcessingGuarantee.NO_GUARANTEE)
+            .setTupleTrackingEnforced(true)
+            .build();
+
+        try (SimulatedTime time = new SimulatedTime()) {
+            KafkaSpout<String, String> spout = SpoutWithMockedConsumerSetupHelper.setupSpout(spoutConfig, conf, contextMock, collectorMock, consumerMock, partition);
+
+            when(consumerMock.poll(anyLong())).thenReturn(new ConsumerRecords<>(Collections.singletonMap(partition,
+                SpoutWithMockedConsumerSetupHelper.<String, String>createRecords(partition, 0, 1))));
+
+            spout.nextTuple();
+
+            when(consumerMock.position(partition)).thenReturn(1L);
+
+            ArgumentCaptor<KafkaSpoutMessageId> msgIdCaptor = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
+            verify(collectorMock).emit(eq(SingleTopicKafkaSpoutConfiguration.STREAM), anyList(), msgIdCaptor.capture());
+            assertThat("Should have captured a message id", msgIdCaptor.getValue(), not(nullValue()));
+
+            Time.advanceTime(KafkaSpout.TIMER_DELAY_MS + spoutConfig.getOffsetsCommitPeriodMs());
+
+            spout.nextTuple();
+
+            verify(consumerMock).commitAsync(commitCapture.capture(), isNull(OffsetCommitCallback.class));
+
+            CommitMetadataManager metadataManager = new CommitMetadataManager(contextMock, KafkaSpoutConfig.ProcessingGuarantee.NO_GUARANTEE);
+            Map<TopicPartition, OffsetAndMetadata> committedOffsets = commitCapture.getValue();
+            assertThat(committedOffsets.get(partition).offset(), is(1L));
+            assertThat(committedOffsets.get(partition).metadata(), is(metadataManager.getCommitMetadata()));
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutReactivationTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutReactivationTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutReactivationTest.java
new file mode 100644
index 0000000..c2c46b5
--- /dev/null
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutReactivationTest.java
@@ -0,0 +1,145 @@
+/*
+ * Copyright 2017 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 static org.apache.storm.kafka.spout.KafkaSpout.TIMER_DELAY_MS;
+import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyList;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+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.KafkaUnitRule;
+import org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
+import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
+import org.apache.storm.kafka.spout.internal.KafkaConsumerFactoryDefault;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.utils.Time;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.runners.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.class)
+public class KafkaSpoutReactivationTest {
+
+    @Rule
+    public KafkaUnitRule kafkaUnitRule = new KafkaUnitRule();
+
+    @Captor
+    private ArgumentCaptor<Map<TopicPartition, OffsetAndMetadata>> commitCapture;
+
+    private final TopologyContext topologyContext = mock(TopologyContext.class);
+    private final Map<String, Object> conf = new HashMap<>();
+    private final SpoutOutputCollector collector = mock(SpoutOutputCollector.class);
+    private final long commitOffsetPeriodMs = 2_000;
+    private KafkaConsumer<String, String> consumerSpy;
+    private KafkaConsumer<String, String> postReactivationConsumerSpy;
+    private KafkaSpout<String, String> spout;
+    private final int maxPollRecords = 10;
+
+    @Before
+    public void setUp() {
+        KafkaSpoutConfig<String, String> spoutConfig =
+            SingleTopicKafkaSpoutConfiguration.setCommonSpoutConfig(
+                KafkaSpoutConfig.builder("127.0.0.1:" + kafkaUnitRule.getKafkaUnit().getKafkaPort(),
+                    SingleTopicKafkaSpoutConfiguration.TOPIC))
+                .setFirstPollOffsetStrategy(UNCOMMITTED_EARLIEST)
+                .setOffsetCommitPeriodMs(commitOffsetPeriodMs)
+                .setProp(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords)
+                .build();
+        KafkaConsumerFactory<String, String> consumerFactory = new KafkaConsumerFactoryDefault<>();
+        this.consumerSpy = spy(consumerFactory.createConsumer(spoutConfig));
+        this.postReactivationConsumerSpy = spy(consumerFactory.createConsumer(spoutConfig));
+        KafkaConsumerFactory<String, String> consumerFactoryMock = mock(KafkaConsumerFactory.class);
+        when(consumerFactoryMock.createConsumer(any(KafkaSpoutConfig.class)))
+            .thenReturn(consumerSpy)
+            .thenReturn(postReactivationConsumerSpy);
+        this.spout = new KafkaSpout<>(spoutConfig, consumerFactoryMock);
+    }
+
+    private void prepareSpout(int messageCount) throws Exception {
+        SingleTopicKafkaUnitSetupHelper.populateTopicData(kafkaUnitRule.getKafkaUnit(), SingleTopicKafkaSpoutConfiguration.TOPIC, messageCount);
+        SingleTopicKafkaUnitSetupHelper.initializeSpout(spout, conf, topologyContext, collector);
+    }
+
+    private KafkaSpoutMessageId emitOne() {
+        ArgumentCaptor<KafkaSpoutMessageId> messageId = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
+        spout.nextTuple();
+        verify(collector).emit(anyString(), anyList(), messageId.capture());
+        reset(collector);
+        return messageId.getValue();
+    }
+
+    @Test
+    public void testSpoutMustHandleReactivationGracefully() throws Exception {
+        try (Time.SimulatedTime time = new Time.SimulatedTime()) {
+            int messageCount = maxPollRecords * 2;
+            prepareSpout(messageCount);
+
+            //Emit and ack some tuples, ensure that some polled tuples remain cached in the spout by emitting less than maxPollRecords
+            int beforeReactivationEmits = maxPollRecords - 3;
+            for (int i = 0; i < beforeReactivationEmits - 1; i++) {
+                KafkaSpoutMessageId msgId = emitOne();
+                spout.ack(msgId);
+            }
+
+            KafkaSpoutMessageId ackAfterDeactivateMessageId = emitOne();
+            
+            //Cycle spout activation
+            spout.deactivate();
+            SingleTopicKafkaUnitSetupHelper.verifyAllMessagesCommitted(consumerSpy, commitCapture, beforeReactivationEmits - 1);
+            //Tuples may be acked/failed after the spout deactivates, so we have to be able to handle this too
+            spout.ack(ackAfterDeactivateMessageId);
+            spout.activate();
+
+            //Emit and ack the rest
+            for (int i = beforeReactivationEmits; i < messageCount; i++) {
+                KafkaSpoutMessageId msgId = emitOne();
+                spout.ack(msgId);
+            }
+
+            //Commit
+            Time.advanceTime(TIMER_DELAY_MS + commitOffsetPeriodMs);
+            spout.nextTuple();
+
+            //Verify that no more tuples are emitted and all tuples are committed
+            SingleTopicKafkaUnitSetupHelper.verifyAllMessagesCommitted(postReactivationConsumerSpy, commitCapture, messageCount);
+
+            reset(collector);
+            spout.nextTuple();
+            verify(collector, never()).emit(anyString(), anyList(), any(KafkaSpoutMessageId.class));
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/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 1033e83..29d2a22 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,12 +15,12 @@
  */
 package org.apache.storm.kafka.spout;
 
-import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfigBuilder;
 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.any;
 import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.reset;
@@ -31,10 +31,12 @@ import static org.mockito.Mockito.when;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-
+import java.util.Set;
 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;
@@ -52,12 +54,10 @@ import org.mockito.Captor;
 import org.mockito.Mockito;
 import org.mockito.MockitoAnnotations;
 
+import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.createKafkaSpoutConfigBuilder;
+import static org.mockito.Matchers.anyObject;
 import static org.mockito.Matchers.eq;
 
-import java.util.HashSet;
-import java.util.Set;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-
 public class KafkaSpoutRebalanceTest {
 
     @Captor
@@ -85,30 +85,24 @@ public class KafkaSpoutRebalanceTest {
     }
 
     //Returns messageIds in order of emission
-    private List<KafkaSpoutMessageId> emitOneMessagePerPartitionThenRevokeOnePartition(KafkaSpout<String, String> spout, TopicPartition partitionThatWillBeRevoked, TopicPartition assignedPartition) {
-        //Setup spout with mock consumer so we can get at the rebalance listener
+    private List<KafkaSpoutMessageId> emitOneMessagePerPartitionThenRevokeOnePartition(KafkaSpout<String, String> spout, TopicPartition partitionThatWillBeRevoked, TopicPartition assignedPartition, ArgumentCaptor<ConsumerRebalanceListener> rebalanceListenerCapture) {
+        //Setup spout with mock consumer so we can get at the rebalance listener   
         spout.open(conf, contextMock, collectorMock);
         spout.activate();
 
-        ArgumentCaptor<ConsumerRebalanceListener> rebalanceListenerCapture = ArgumentCaptor.forClass(ConsumerRebalanceListener.class);
-        verify(consumerMock).subscribe(anyCollection(), rebalanceListenerCapture.capture());
-
         //Assign partitions to the spout
         ConsumerRebalanceListener consumerRebalanceListener = rebalanceListenerCapture.getValue();
-        List<TopicPartition> assignedPartitions = new ArrayList<>();
+        Set<TopicPartition> assignedPartitions = new HashSet<>();
         assignedPartitions.add(partitionThatWillBeRevoked);
         assignedPartitions.add(assignedPartition);
         consumerRebalanceListener.onPartitionsAssigned(assignedPartitions);
+        when(consumerMock.assignment()).thenReturn(assignedPartitions);
 
         //Make the consumer return a single message for each partition
-        Map<TopicPartition, List<ConsumerRecord<String, String>>> firstPartitionRecords = new HashMap<>();
-        firstPartitionRecords.put(partitionThatWillBeRevoked, Collections.singletonList(new ConsumerRecord<>(partitionThatWillBeRevoked.topic(), partitionThatWillBeRevoked.partition(), 0L, "key", "value")));
-        Map<TopicPartition, List<ConsumerRecord<String, String>>> secondPartitionRecords = new HashMap<>();
-        secondPartitionRecords.put(assignedPartition, Collections.singletonList(new ConsumerRecord<>(assignedPartition.topic(), assignedPartition.partition(), 0L, "key", "value")));
         when(consumerMock.poll(anyLong()))
-            .thenReturn(new ConsumerRecords(firstPartitionRecords))
-            .thenReturn(new ConsumerRecords(secondPartitionRecords))
-            .thenReturn(new ConsumerRecords(Collections.emptyMap()));
+            .thenReturn(new ConsumerRecords<>(Collections.singletonMap(partitionThatWillBeRevoked, SpoutWithMockedConsumerSetupHelper.<String, String>createRecords(partitionThatWillBeRevoked, 0, 1))))
+            .thenReturn(new ConsumerRecords<>(Collections.singletonMap(assignedPartition, SpoutWithMockedConsumerSetupHelper.<String, String>createRecords(assignedPartition, 0, 1))))
+            .thenReturn(new ConsumerRecords<>(new HashMap<TopicPartition, List<ConsumerRecord<String, String>>>()));
 
         //Emit the messages
         spout.nextTuple();
@@ -122,6 +116,7 @@ public class KafkaSpoutRebalanceTest {
         //Now rebalance
         consumerRebalanceListener.onPartitionsRevoked(assignedPartitions);
         consumerRebalanceListener.onPartitionsAssigned(Collections.singleton(assignedPartition));
+        when(consumerMock.assignment()).thenReturn(Collections.singleton(assignedPartition));
 
         List<KafkaSpoutMessageId> emittedMessageIds = new ArrayList<>();
         emittedMessageIds.add(messageIdForRevokedPartition.getValue());
@@ -133,7 +128,12 @@ public class KafkaSpoutRebalanceTest {
     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
         try (SimulatedTime simulatedTime = new SimulatedTime()) {
-            KafkaSpout<String, String> spout = new KafkaSpout<>(getKafkaSpoutConfigBuilder(-1)
+            ArgumentCaptor<ConsumerRebalanceListener> rebalanceListenerCapture = ArgumentCaptor.forClass(ConsumerRebalanceListener.class);
+            Subscription subscriptionMock = mock(Subscription.class);
+            doNothing()
+                .when(subscriptionMock)
+                .subscribe(any(KafkaConsumer.class), rebalanceListenerCapture.capture(), any(TopologyContext.class));
+            KafkaSpout<String, String> spout = new KafkaSpout<>(createKafkaSpoutConfigBuilder(subscriptionMock, -1)
                 .setOffsetCommitPeriodMs(offsetCommitPeriodMs)
                 .build(), consumerFactoryMock);
             String topic = SingleTopicKafkaSpoutConfiguration.TOPIC;
@@ -141,7 +141,8 @@ public class KafkaSpoutRebalanceTest {
             TopicPartition assignedPartition = new TopicPartition(topic, 2);
 
             //Emit a message on each partition and revoke the first partition
-            List<KafkaSpoutMessageId> emittedMessageIds = emitOneMessagePerPartitionThenRevokeOnePartition(spout, partitionThatWillBeRevoked, assignedPartition);
+            List<KafkaSpoutMessageId> emittedMessageIds = emitOneMessagePerPartitionThenRevokeOnePartition(
+                spout, partitionThatWillBeRevoked, assignedPartition, rebalanceListenerCapture);
 
             //Ack both emitted tuples
             spout.ack(emittedMessageIds.get(0));
@@ -163,8 +164,13 @@ public class KafkaSpoutRebalanceTest {
     @Test
     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
+        ArgumentCaptor<ConsumerRebalanceListener> rebalanceListenerCapture = ArgumentCaptor.forClass(ConsumerRebalanceListener.class);
+            Subscription subscriptionMock = mock(Subscription.class);
+            doNothing()
+                .when(subscriptionMock)
+                .subscribe(any(KafkaConsumer.class), rebalanceListenerCapture.capture(), any(TopologyContext.class));
         KafkaSpoutRetryService retryServiceMock = mock(KafkaSpoutRetryService.class);
-        KafkaSpout<String, String> spout = new KafkaSpout<>(getKafkaSpoutConfigBuilder(-1)
+        KafkaSpout<String, String> spout = new KafkaSpout<>(createKafkaSpoutConfigBuilder(subscriptionMock, -1)
             .setOffsetCommitPeriodMs(10)
             .setRetry(retryServiceMock)
             .build(), consumerFactoryMock);
@@ -177,7 +183,8 @@ public class KafkaSpoutRebalanceTest {
             .thenReturn(new KafkaSpoutMessageId(assignedPartition, 0));
 
         //Emit a message on each partition and revoke the first partition
-        List<KafkaSpoutMessageId> emittedMessageIds = emitOneMessagePerPartitionThenRevokeOnePartition(spout, partitionThatWillBeRevoked, assignedPartition);
+        List<KafkaSpoutMessageId> emittedMessageIds = emitOneMessagePerPartitionThenRevokeOnePartition(
+            spout, partitionThatWillBeRevoked, assignedPartition, rebalanceListenerCapture);
 
         //Check that only two message ids were generated
         verify(retryServiceMock, times(2)).getMessageId(Mockito.any(ConsumerRecord.class));
@@ -199,7 +206,11 @@ public class KafkaSpoutRebalanceTest {
          */
 
         ArgumentCaptor<ConsumerRebalanceListener> rebalanceListenerCapture = ArgumentCaptor.forClass(ConsumerRebalanceListener.class);
-        KafkaSpout<String, String> spout = new KafkaSpout<>(getKafkaSpoutConfigBuilder(-1)
+        Subscription subscriptionMock = mock(Subscription.class);
+        doNothing()
+            .when(subscriptionMock)
+            .subscribe(any(KafkaConsumer.class), rebalanceListenerCapture.capture(), any(TopologyContext.class));
+        KafkaSpout<String, String> spout = new KafkaSpout<>(createKafkaSpoutConfigBuilder(subscriptionMock, -1)
             .setFirstPollOffsetStrategy(KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST)
             .build(), consumerFactoryMock);
         String topic = SingleTopicKafkaSpoutConfiguration.TOPIC;
@@ -209,8 +220,6 @@ public class KafkaSpoutRebalanceTest {
         //Setup spout with mock consumer so we can get at the rebalance listener   
         spout.open(conf, contextMock, collectorMock);
         spout.activate();
-        
-        verify(consumerMock).subscribe(anyCollection(), rebalanceListenerCapture.capture());
 
         //Assign partitions to the spout
         ConsumerRebalanceListener consumerRebalanceListener = rebalanceListenerCapture.getValue();

http://git-wip-us.apache.org/repos/asf/storm/blob/324bc95b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRetryLimitTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRetryLimitTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRetryLimitTest.java
index dac4bff..569becf 100644
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRetryLimitTest.java
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRetryLimitTest.java
@@ -15,103 +15,84 @@
  */
 package org.apache.storm.kafka.spout;
 
-import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfigBuilder;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.anyCollection;
-import static org.mockito.Matchers.anyList;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.inOrder;
-import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 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 java.util.Set;
 
 import org.apache.kafka.clients.consumer.*;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
-import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.utils.Time;
 import org.apache.storm.utils.Time.SimulatedTime;
+import org.junit.Before;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
 import org.mockito.InOrder;
+import org.mockito.MockitoAnnotations;
 
-public class KafkaSpoutRetryLimitTest {
+import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.createKafkaSpoutConfigBuilder;
+import static org.mockito.Matchers.anyList;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.mock;
 
+public class KafkaSpoutRetryLimitTest {
+    
     private final long offsetCommitPeriodMs = 2_000;
     private final TopologyContext contextMock = mock(TopologyContext.class);
     private final SpoutOutputCollector collectorMock = mock(SpoutOutputCollector.class);
     private final Map<String, Object> conf = new HashMap<>();
     private final TopicPartition partition = new TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 1);
     private KafkaConsumer<String, String> consumerMock;
-    private KafkaSpout<String, String> spout;
-    private KafkaSpoutConfig spoutConfig;
-
+    private KafkaSpoutConfig<String, String> spoutConfig;
+    
     public static final KafkaSpoutRetryService ZERO_RETRIES_RETRY_SERVICE =
-            new KafkaSpoutRetryExponentialBackoff(KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(0), KafkaSpoutRetryExponentialBackoff.TimeInterval.milliSeconds(0),
-                    0, KafkaSpoutRetryExponentialBackoff.TimeInterval.milliSeconds(0));
-
-    private void setupSpoutWithNoRetry(Set<TopicPartition> assignedPartitions) {
-        spoutConfig = getKafkaSpoutConfigBuilder(-1)
-                .setOffsetCommitPeriodMs(offsetCommitPeriodMs)
-                .setRetry(ZERO_RETRIES_RETRY_SERVICE)
-                .build();
-
+        new KafkaSpoutRetryExponentialBackoff(KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(0), KafkaSpoutRetryExponentialBackoff.TimeInterval.milliSeconds(0),
+            0, KafkaSpoutRetryExponentialBackoff.TimeInterval.milliSeconds(0));
+    
+    @Captor
+    private ArgumentCaptor<Map<TopicPartition, OffsetAndMetadata>> commitCapture;
+    
+    @Before
+    public void setUp() {
+        MockitoAnnotations.initMocks(this);
+        spoutConfig = createKafkaSpoutConfigBuilder(mock(Subscription.class), -1)
+            .setOffsetCommitPeriodMs(offsetCommitPeriodMs)
+            .setRetry(ZERO_RETRIES_RETRY_SERVICE)
+            .build();
         consumerMock = mock(KafkaConsumer.class);
-        KafkaConsumerFactory<String, String> consumerFactory = new KafkaConsumerFactory<String, String>() {
-            @Override
-            public KafkaConsumer<String, String> createConsumer(KafkaSpoutConfig<String, String> kafkaSpoutConfig) {
-                return consumerMock;
-            }
-        };
-
-        spout = new KafkaSpout<>(spoutConfig, consumerFactory);
-
-        spout.open(conf, contextMock, collectorMock);
-        spout.activate();
-
-        ArgumentCaptor<ConsumerRebalanceListener> rebalanceListenerCapture = ArgumentCaptor.forClass(ConsumerRebalanceListener.class);
-        verify(consumerMock).subscribe(anyCollection(), rebalanceListenerCapture.capture());
-
-        //Assign partitions to the spout
-        ConsumerRebalanceListener consumerRebalanceListener = rebalanceListenerCapture.getValue();
-        consumerRebalanceListener.onPartitionsAssigned(assignedPartitions);
     }
-
+    
     @Test
     public void testFailingTupleCompletesAckAfterRetryLimitIsMet() {
         //Spout should ack failed messages after they hit the retry limit
         try (SimulatedTime simulatedTime = new SimulatedTime()) {
-            setupSpoutWithNoRetry(Collections.singleton(partition));
+            KafkaSpout<String, String> spout = SpoutWithMockedConsumerSetupHelper.setupSpout(spoutConfig, conf, contextMock, collectorMock, consumerMock, partition);
             Map<TopicPartition, List<ConsumerRecord<String, String>>> records = new HashMap<>();
-            List<ConsumerRecord<String, String>> recordsForPartition = new ArrayList<>();
             int lastOffset = 3;
-            for (int i = 0; i <= lastOffset; i++) {
-                recordsForPartition.add(new ConsumerRecord(partition.topic(), partition.partition(), i, "key", "value"));
-            }
-            records.put(partition, recordsForPartition);
-
+            int numRecords = lastOffset + 1;
+            records.put(partition, SpoutWithMockedConsumerSetupHelper.<String, String>createRecords(partition, 0, numRecords));
+            
             when(consumerMock.poll(anyLong()))
-                    .thenReturn(new ConsumerRecords(records));
-
-            for (int i = 0; i < recordsForPartition.size(); i++) {
+                .thenReturn(new ConsumerRecords<>(records));
+            
+            for (int i = 0; i < numRecords; i++) {
                 spout.nextTuple();
             }
-
+            
             ArgumentCaptor<KafkaSpoutMessageId> messageIds = ArgumentCaptor.forClass(KafkaSpoutMessageId.class);
-            verify(collectorMock, times(recordsForPartition.size())).emit(anyString(), anyList(), messageIds.capture());
-
+            verify(collectorMock, times(numRecords)).emit(anyString(), anyList(), messageIds.capture());
+            
             for (KafkaSpoutMessageId messageId : messageIds.getAllValues()) {
                 spout.fail(messageId);
             }
@@ -119,16 +100,15 @@ public class KafkaSpoutRetryLimitTest {
             // Advance time and then trigger call to kafka consumer commit
             Time.advanceTime(KafkaSpout.TIMER_DELAY_MS + offsetCommitPeriodMs);
             spout.nextTuple();
-
-            ArgumentCaptor<Map> committedOffsets=ArgumentCaptor.forClass(Map.class);
+            
             InOrder inOrder = inOrder(consumerMock);
-            inOrder.verify(consumerMock).commitSync(committedOffsets.capture());
+            inOrder.verify(consumerMock).commitSync(commitCapture.capture());
             inOrder.verify(consumerMock).poll(anyLong());
 
             //verify that offset 4 was committed for the given TopicPartition, since processing should resume at 4.
-            assertTrue(committedOffsets.getValue().containsKey(partition));
-            assertEquals(lastOffset + 1, ((OffsetAndMetadata) (committedOffsets.getValue().get(partition))).offset());
+            assertTrue(commitCapture.getValue().containsKey(partition));
+            assertEquals(lastOffset + 1, ((OffsetAndMetadata) (commitCapture.getValue().get(partition))).offset());
         }
     }
-
-}
\ No newline at end of file
+    
+}