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/07/19 03:34:57 UTC

[1/7] storm git commit: STORM-2953: Remove storm-kafka

Repository: storm
Updated Branches:
  refs/heads/master 3883bf74c -> a7e817bcd


http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 503924c..ea5b616 100644
--- a/pom.xml
+++ b/pom.xml
@@ -315,14 +315,9 @@
         <rocketmq.version>4.2.0</rocketmq.version>
 
         <jackson.version>2.9.4</jackson.version>
-        <!-- Kafka version used by old storm-kafka spout code -->
-        <storm.kafka.version>0.8.2.2</storm.kafka.version>
-        <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.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, org.apache.storm.testing.PerformanceTest</java.unit.test.exclude>
         <java.unit.test.include>**/Test*.java, **/*Test.java, **/*TestCase.java</java.unit.test.include>    <!--maven surefire plugin default test list-->
@@ -365,7 +360,6 @@
 
         <!-- externals -->
         <module>external/storm-autocreds</module>
-        <module>external/storm-kafka</module>
         <module>external/storm-hdfs</module>
         <module>external/storm-hdfs-blobstore</module>
         <module>external/storm-hbase</module>
@@ -397,7 +391,6 @@
         <module>examples/storm-redis-examples</module>
         <module>examples/storm-opentsdb-examples</module>
         <module>examples/storm-solr-examples</module>
-        <module>examples/storm-kafka-examples</module>
         <module>examples/storm-kafka-client-examples</module>
         <module>examples/storm-jdbc-examples</module>
         <module>examples/storm-hdfs-examples</module>
@@ -1085,28 +1078,6 @@
                 <artifactId>calcite-core</artifactId>
                 <version>${calcite.version}</version>
             </dependency>
-
-            <!-- kafka artifact dependency needed for storm-kafka -->
-            <dependency>
-                <groupId>org.apache.kafka</groupId>
-                <artifactId>${storm.kafka.artifact.id}</artifactId>
-                <version>${storm.kafka.version}</version>
-                <scope>provided</scope>
-                <exclusions>
-                    <exclusion>
-                        <groupId>org.apache.zookeeper</groupId>
-                        <artifactId>zookeeper</artifactId>
-                    </exclusion>
-                    <exclusion>
-                        <groupId>log4j</groupId>
-                        <artifactId>log4j</artifactId>
-                    </exclusion>
-                    <exclusion>
-                        <groupId>org.slf4j</groupId>
-                        <artifactId>slf4j-log4j12</artifactId>
-                    </exclusion>
-                </exclusions>
-            </dependency>
             <dependency>
                 <groupId>org.apache.kafka</groupId>
                 <artifactId>kafka-clients</artifactId>

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/storm-core/src/jvm/org/apache/storm/utils/TopologySpoutLag.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/utils/TopologySpoutLag.java b/storm-core/src/jvm/org/apache/storm/utils/TopologySpoutLag.java
index db4c1d7..25a5312 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/TopologySpoutLag.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/TopologySpoutLag.java
@@ -18,7 +18,6 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import org.apache.storm.Config;
 import org.apache.storm.generated.ComponentCommon;
 import org.apache.storm.generated.ComponentObject;
 import org.apache.storm.generated.SpoutSpec;
@@ -49,9 +48,7 @@ public class TopologySpoutLag {
                 className = getClassNameFromComponentObject(componentObject);
                 logger.debug("spout classname: {}", className);
                 if (className.endsWith("storm.kafka.spout.KafkaSpout")) {
-                    result.put(spout.getKey(), getLagResultForNewKafkaSpout(spout.getKey(), spoutSpec, topologyConf));
-                } else if (className.endsWith("storm.kafka.KafkaSpout")) {
-                    result.put(spout.getKey(), getLagResultForOldKafkaSpout(spout.getKey(), spoutSpec, topologyConf));
+                    result.put(spout.getKey(), getLagResultForNewKafkaSpout(spout.getKey(), spoutSpec));
                 }
             } catch (Exception e) {
                 logger.warn("Exception thrown while getting lag for spout id: " + spout.getKey() + " and spout class: " + className);
@@ -93,46 +90,8 @@ public class TopologySpoutLag {
         }
         return commands;
     }
-
-    private static List<String> getCommandLineOptionsForOldKafkaSpout(Map<String, Object> jsonConf, Map<String, Object> topologyConf) {
-        logger.debug("json configuration: {}", jsonConf);
-
-        List<String> commands = new ArrayList<>();
-        String configKeyPrefix = "config.";
-        commands.add("-o");
-        commands.add("-t");
-        commands.add((String) jsonConf.get(configKeyPrefix + "topics"));
-        commands.add("-n");
-        commands.add((String) jsonConf.get(configKeyPrefix + "zkRoot"));
-        String zkServers = (String) jsonConf.get(configKeyPrefix + "zkServers");
-        if (zkServers == null || zkServers.isEmpty()) {
-            StringBuilder zkServersBuilder = new StringBuilder();
-            Integer zkPort = ((Number) topologyConf.get(Config.STORM_ZOOKEEPER_PORT)).intValue();
-            for (String zkServer : (List<String>) topologyConf.get(Config.STORM_ZOOKEEPER_SERVERS)) {
-                zkServersBuilder.append(zkServer + ":" + zkPort + ",");
-            }
-            zkServers = zkServersBuilder.toString();
-        }
-        commands.add("-z");
-        commands.add(zkServers);
-        if (jsonConf.get(configKeyPrefix + "leaders") != null) {
-            commands.add("-p");
-            commands.add((String) jsonConf.get(configKeyPrefix + "partitions"));
-            commands.add("-l");
-            commands.add((String) jsonConf.get(configKeyPrefix + "leaders"));
-        } else {
-            commands.add("-r");
-            commands.add((String) jsonConf.get(configKeyPrefix + "zkNodeBrokers"));
-            Boolean isWildCard = (Boolean) topologyConf.get("kafka.topic.wildcard.match");
-            if (isWildCard != null && isWildCard.booleanValue()) {
-                commands.add("-w");
-            }
-        }
-        return commands;
-    }
-
-    private static Map<String, Object> getLagResultForKafka(String spoutId, SpoutSpec spoutSpec, Map<String, Object> topologyConf,
-                                                            boolean old) throws IOException {
+    
+    private static Map<String, Object> getLagResultForKafka(String spoutId, SpoutSpec spoutSpec) throws IOException {
         ComponentCommon componentCommon = spoutSpec.get_common();
         String json = componentCommon.get_json_conf();
         Map<String, Object> result = null;
@@ -150,12 +109,11 @@ public class TopologySpoutLag {
             } catch (ParseException e) {
                 throw new IOException(e);
             }
-            commands.addAll(
-                old ? getCommandLineOptionsForOldKafkaSpout(jsonMap, topologyConf) : getCommandLineOptionsForNewKafkaSpout(jsonMap));
+            commands.addAll(getCommandLineOptionsForNewKafkaSpout(jsonMap));
 
             logger.debug("Command to run: {}", commands);
 
-            // if commands contains one or more null value, spout is compiled with lower version of storm-kafka / storm-kafka-client
+            // if commands contains one or more null value, spout is compiled with lower version of storm-kafka-client
             if (!commands.contains(null)) {
                 String resultFromMonitor = new ShellCommandRunnerImpl().execCommand(commands.toArray(new String[0]));
 
@@ -182,13 +140,7 @@ public class TopologySpoutLag {
         return kafkaSpoutLagInfo;
     }
 
-    private static Map<String, Object> getLagResultForNewKafkaSpout(String spoutId, SpoutSpec spoutSpec,
-                                                                    Map<String, Object> topologyConf) throws IOException {
-        return getLagResultForKafka(spoutId, spoutSpec, topologyConf, false);
-    }
-
-    private static Map<String, Object> getLagResultForOldKafkaSpout(String spoutId, SpoutSpec spoutSpec,
-                                                                    Map<String, Object> topologyConf) throws IOException {
-        return getLagResultForKafka(spoutId, spoutSpec, topologyConf, true);
+    private static Map<String, Object> getLagResultForNewKafkaSpout(String spoutId, SpoutSpec spoutSpec) throws IOException {
+        return getLagResultForKafka(spoutId, spoutSpec);
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/storm-dist/binary/final-package/src/main/assembly/binary.xml
----------------------------------------------------------------------
diff --git a/storm-dist/binary/final-package/src/main/assembly/binary.xml b/storm-dist/binary/final-package/src/main/assembly/binary.xml
index 316ac1b..6e0d968 100644
--- a/storm-dist/binary/final-package/src/main/assembly/binary.xml
+++ b/storm-dist/binary/final-package/src/main/assembly/binary.xml
@@ -111,13 +111,6 @@
         <!-- EXTERNAL -->
         <!-- only include the README file -->
         <fileSet>
-            <directory>${project.basedir}/../../../external/storm-kafka</directory>
-            <outputDirectory>external/storm-kafka</outputDirectory>
-            <includes>
-                <include>README.*</include>
-            </includes>
-        </fileSet>
-        <fileSet>
             <directory>${project.basedir}/../../../external/storm-kinesis</directory>
             <outputDirectory>external/storm-kinesis</outputDirectory>
             <includes>


[6/7] storm git commit: STORM-2953: Remove storm-kafka

Posted by ka...@apache.org.
STORM-2953: Remove storm-kafka


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

Branch: refs/heads/master
Commit: e58ac3e033670aeb83e8543ab7b8779227ca10d8
Parents: 7f992a6
Author: Stig Rohde Døssing <sr...@apache.org>
Authored: Tue Jul 17 18:29:08 2018 +0200
Committer: Stig Rohde Døssing <sr...@apache.org>
Committed: Wed Jul 18 11:01:13 2018 +0200

----------------------------------------------------------------------
 docs/index.md                                   |   2 +-
 docs/storm-kafka-client.md                      |   7 +-
 docs/storm-kafka.md                             | 399 ------------------
 examples/storm-kafka-examples/pom.xml           | 110 -----
 .../kafka/trident/KafkaProducerTopology.java    |  75 ----
 .../trident/TridentKafkaConsumerTopology.java   |  42 --
 .../trident/TridentKafkaRandomStrings.java      |  82 ----
 .../kafka/trident/TridentKafkaTopology.java     |  83 ----
 external/storm-kafka-monitor/pom.xml            |   5 -
 .../storm/kafka/monitor/KafkaOffsetLagUtil.java | 298 +-------------
 .../kafka/monitor/OldKafkaSpoutOffsetQuery.java | 127 ------
 external/storm-kafka/README.md                  | 382 -----------------
 external/storm-kafka/pom.xml                    | 125 ------
 .../src/jvm/org/apache/storm/kafka/Broker.java  |  79 ----
 .../jvm/org/apache/storm/kafka/BrokerHosts.java |  20 -
 .../storm/kafka/ByteBufferSerializer.java       |  41 --
 .../storm/kafka/DynamicBrokersReader.java       | 208 ----------
 .../kafka/DynamicPartitionConnections.java      |  91 -----
 .../ExponentialBackoffMsgRetryManager.java      | 201 ---------
 .../storm/kafka/FailedFetchException.java       |  24 --
 .../storm/kafka/FailedMsgRetryManager.java      |  77 ----
 .../org/apache/storm/kafka/IntSerializer.java   |  42 --
 .../jvm/org/apache/storm/kafka/KafkaConfig.java |  49 ---
 .../jvm/org/apache/storm/kafka/KafkaError.java  |  38 --
 .../jvm/org/apache/storm/kafka/KafkaSpout.java  | 255 ------------
 .../jvm/org/apache/storm/kafka/KafkaUtils.java  | 288 -------------
 .../org/apache/storm/kafka/KeyValueScheme.java  |  21 -
 .../kafka/KeyValueSchemeAsMultiScheme.java      |  35 --
 .../storm/kafka/MessageMetadataScheme.java      |  21 -
 .../MessageMetadataSchemeAsMultiScheme.java     |  35 --
 .../jvm/org/apache/storm/kafka/Partition.java   |  85 ----
 .../storm/kafka/PartitionCoordinator.java       |  23 --
 .../apache/storm/kafka/PartitionManager.java    | 405 -------------------
 .../jvm/org/apache/storm/kafka/SpoutConfig.java |  58 ---
 .../apache/storm/kafka/StaticCoordinator.java   |  50 ---
 .../jvm/org/apache/storm/kafka/StaticHosts.java |  33 --
 .../storm/kafka/StaticPartitionConnections.java |  46 ---
 .../storm/kafka/StringKeyValueScheme.java       |  32 --
 .../kafka/StringMessageAndMetadataScheme.java   |  36 --
 .../storm/kafka/StringMultiSchemeWithTopic.java |  41 --
 .../org/apache/storm/kafka/StringScheme.java    |  44 --
 .../kafka/TopicOffsetOutOfRangeException.java   |  26 --
 .../org/apache/storm/kafka/ZkCoordinator.java   | 127 ------
 .../src/jvm/org/apache/storm/kafka/ZkHosts.java |  31 --
 .../src/jvm/org/apache/storm/kafka/ZkState.java | 112 -----
 .../org/apache/storm/kafka/bolt/KafkaBolt.java  | 172 --------
 .../FieldNameBasedTupleToKafkaMapper.java       |  43 --
 .../kafka/bolt/mapper/TupleToKafkaMapper.java   |  27 --
 .../bolt/selector/DefaultTopicSelector.java     |  29 --
 .../bolt/selector/FieldIndexTopicSelector.java  |  43 --
 .../bolt/selector/FieldNameTopicSelector.java   |  44 --
 .../kafka/bolt/selector/KafkaTopicSelector.java |  20 -
 .../apache/storm/kafka/trident/Coordinator.java |  46 ---
 .../storm/kafka/trident/DefaultCoordinator.java |  26 --
 .../trident/GlobalPartitionInformation.java     | 113 ------
 .../storm/kafka/trident/IBatchCoordinator.java  |  21 -
 .../storm/kafka/trident/IBrokerReader.java      |  24 --
 .../apache/storm/kafka/trident/MaxMetric.java   |  35 --
 .../kafka/trident/OpaqueTridentKafkaSpout.java  |  62 ---
 .../storm/kafka/trident/StaticBrokerReader.java |  44 --
 .../trident/TransactionalTridentKafkaSpout.java |  50 ---
 .../storm/kafka/trident/TridentKafkaConfig.java |  32 --
 .../kafka/trident/TridentKafkaEmitter.java      | 306 --------------
 .../storm/kafka/trident/TridentKafkaState.java  | 110 -----
 .../kafka/trident/TridentKafkaStateFactory.java |  57 ---
 .../kafka/trident/TridentKafkaUpdater.java      |  25 --
 .../storm/kafka/trident/ZkBrokerReader.java     |  79 ----
 .../FieldNameBasedTupleToKafkaMapper.java       |  36 --
 .../mapper/TridentTupleToKafkaMapper.java       |  22 -
 .../trident/selector/DefaultTopicSelector.java  |  29 --
 .../trident/selector/KafkaTopicSelector.java    |  20 -
 .../storm/kafka/DynamicBrokersReaderTest.java   | 245 -----------
 .../ExponentialBackoffMsgRetryManagerTest.java  | 279 -------------
 .../org/apache/storm/kafka/KafkaErrorTest.java  |  51 ---
 .../org/apache/storm/kafka/KafkaTestBroker.java | 177 --------
 .../org/apache/storm/kafka/KafkaUtilsTest.java  | 294 --------------
 .../storm/kafka/PartitionManagerTest.java       | 241 -----------
 .../storm/kafka/StringKeyValueSchemeTest.java   |  56 ---
 .../apache/storm/kafka/TestStringScheme.java    |  40 --
 .../test/org/apache/storm/kafka/TestUtils.java  |  97 -----
 .../apache/storm/kafka/TridentKafkaTest.java    |  75 ----
 .../apache/storm/kafka/ZkCoordinatorTest.java   | 191 ---------
 .../apache/storm/kafka/bolt/KafkaBoltTest.java  | 355 ----------------
 flux/flux-core/pom.xml                          |   5 +
 flux/flux-examples/pom.xml                      |   4 +
 flux/pom.xml                                    |   5 -
 pom.xml                                         |  31 +-
 .../apache/storm/utils/TopologySpoutLag.java    |  62 +--
 .../final-package/src/main/assembly/binary.xml  |   7 -
 89 files changed, 30 insertions(+), 8111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/docs/index.md
----------------------------------------------------------------------
diff --git a/docs/index.md b/docs/index.md
index 2697c47..135e563 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -91,7 +91,7 @@ But small change will not affect the user experience. We will notify the user wh
 * [Event Logging](Eventlogging.html)
 
 ### Integration With External Systems, and Other Libraries
-* [Apache Kafka Integration](storm-kafka.html), [New Kafka Consumer Integration](storm-kafka-client.html)
+* [Apache Kafka Integration](storm-kafka-client.html)
 * [Apache HBase Integration](storm-hbase.html)
 * [Apache HDFS Integration](storm-hdfs.html)
 * [Apache Hive Integration](storm-hive.html)

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/docs/storm-kafka-client.md
----------------------------------------------------------------------
diff --git a/docs/storm-kafka-client.md b/docs/storm-kafka-client.md
index cd7ad20..0b521a0 100644
--- a/docs/storm-kafka-client.md
+++ b/docs/storm-kafka-client.md
@@ -269,10 +269,9 @@ You can also override the kafka clients version while building from maven, with
 e.g. `mvn clean install -Dstorm.kafka.client.version=0.10.0.0`
 
 When selecting a kafka client version, you should ensure -
- 1. kafka api is compatible. storm-kafka-client module only supports **0.10 or newer** kafka client API. For older versions,
- you can use storm-kafka module (https://github.com/apache/storm/tree/master/external/storm-kafka).  
- 2. The kafka client selected by you should be wire compatible with the broker. e.g. 0.9.x client will not work with 
- 0.8.x broker. 
+ 1. The Kafka api must be compatible. The storm-kafka-client module only supports Kafka **0.10 or newer**. For older versions,
+ you can use the storm-kafka module (https://github.com/apache/storm/tree/1.x-branch/external/storm-kafka).  
+ 2. The Kafka client version selected by you should be wire compatible with the broker. Please see the [Kafka compatibility matrix](https://cwiki.apache.org/confluence/display/KAFKA/Compatibility+Matrix).
 
 # Kafka Spout Performance Tuning
 

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/docs/storm-kafka.md
----------------------------------------------------------------------
diff --git a/docs/storm-kafka.md b/docs/storm-kafka.md
deleted file mode 100644
index 901d795..0000000
--- a/docs/storm-kafka.md
+++ /dev/null
@@ -1,399 +0,0 @@
----
-title: Storm Kafka Integration
-layout: documentation
-documentation: true
----
-
-Provides core Storm and Trident spout implementations for consuming data from Apache Kafka 0.8.x.
-
-## Spouts
-We support both Trident and core Storm spouts. For both spout implementations, we use a BrokerHost interface that
-tracks Kafka broker host to partition mapping and kafkaConfig that controls some Kafka related parameters.
-
-### BrokerHosts
-In order to initialize your Kafka spout/emitter you need to construct an instance of the marker interface BrokerHosts.
-Currently, we support the following two implementations:
-
-#### ZkHosts
-ZkHosts is what you should use if you want to dynamically track Kafka broker to partition mapping. This class uses
-Kafka's ZooKeeper entries to track brokerHost -> partition mapping. You can instantiate an object by calling
-
-```java
-public ZkHosts(String brokerZkStr, String brokerZkPath)
-public ZkHosts(String brokerZkStr)
-```
-
-Where brokerZkStr is just ip:port (e.g. localhost:2181). brokerZkPath is the root directory under which all the topics and
-partition information is stored. By default this is /brokers which is what the default Kafka implementation uses.
-
-By default, the broker-partition mapping is refreshed every 60 seconds from ZooKeeper. If you want to change it, you
-should set host.refreshFreqSecs to your chosen value.
-
-#### StaticHosts
-This is an alternative implementation where broker -> partition information is static. In order to construct an instance
-of this class, you need to first construct an instance of GlobalPartitionInformation.
-
-```java
-Broker brokerForPartition0 = new Broker("localhost");//localhost:9092
-Broker brokerForPartition1 = new Broker("localhost", 9092);//localhost:9092 but we specified the port explicitly
-Broker brokerForPartition2 = new Broker("localhost:9092");//localhost:9092 specified as one string.
-GlobalPartitionInformation partitionInfo = new GlobalPartitionInformation();
-partitionInfo.addPartition(0, brokerForPartition0);//mapping from partition 0 to brokerForPartition0
-partitionInfo.addPartition(1, brokerForPartition1);//mapping from partition 1 to brokerForPartition1
-partitionInfo.addPartition(2, brokerForPartition2);//mapping from partition 2 to brokerForPartition2
-StaticHosts hosts = new StaticHosts(partitionInfo);
-```
-
-### KafkaConfig
-The second thing needed for constructing a kafkaSpout is an instance of KafkaConfig.
-
-```java
-public KafkaConfig(BrokerHosts hosts, String topic)
-public KafkaConfig(BrokerHosts hosts, String topic, String clientId)
-```
-
-The BrokerHosts can be any implementation of BrokerHosts interface as described above. The topic is name of Kafka topic.
-The optional ClientId is used as a part of the ZooKeeper path where the spout's current consumption offset is stored.
-
-There are 2 extensions of KafkaConfig currently in use.
-
-SpoutConfig is an extension of KafkaConfig that supports additional fields with ZooKeeper connection info and for controlling
-behavior specific to KafkaSpout.
-The clientId will be used to identify requests which are made using the Kafka Protocol.
-The zkRoot will be used as root to store your consumer's offset.
-The id should uniquely identify your spout.
-
-```java
-public SpoutConfig(BrokerHosts hosts, String topic, String clientId, String zkRoot, String id);
-public SpoutConfig(BrokerHosts hosts, String topic, String zkRoot, String id);
-```
-
-In addition to these parameters, SpoutConfig contains the following fields that control how KafkaSpout behaves:
-
-```java
-// setting for how often to save the current Kafka offset to ZooKeeper
-public long stateUpdateIntervalMs = 2000;
-
-// Retry strategy for failed messages
-public String failedMsgRetryManagerClass = ExponentialBackoffMsgRetryManager.class.getName();
-
-// Exponential back-off retry settings.  These are used by ExponentialBackoffMsgRetryManager for retrying messages after a bolt
-// calls OutputCollector.fail(). These come into effect only if ExponentialBackoffMsgRetryManager is being used.
-// Initial delay between successive retries
-public long retryInitialDelayMs = 0;
-public double retryDelayMultiplier = 1.0;
-
-// Maximum delay between successive retries    
-public long retryDelayMaxMs = 60 * 1000;
-// Failed message will be retried infinitely if retryLimit is less than zero. 
-public int retryLimit = -1;     
-```
-
-Core KafkaSpout only accepts an instance of SpoutConfig.
-
-TridentKafkaConfig is another extension of KafkaConfig.
-TridentKafkaEmitter only accepts TridentKafkaConfig.
-
-The KafkaConfig class also has bunch of public variables that controls your application's behavior. Here are defaults:
-
-```java
-public int fetchSizeBytes = 1024 * 1024;
-public int socketTimeoutMs = 10000;
-public int fetchMaxWait = 10000;
-public int bufferSizeBytes = 1024 * 1024;
-public MultiScheme scheme = new RawMultiScheme();
-public boolean ignoreZkOffsets = false;
-public long startOffsetTime = kafka.api.OffsetRequest.EarliestTime();
-public long maxOffsetBehind = Long.MAX_VALUE;
-public boolean useStartOffsetTimeIfOffsetOutOfRange = true;
-public int metricsTimeBucketSizeInSecs = 60;
-```
-
-Most of them are self explanatory except MultiScheme.
-### MultiScheme
-MultiScheme is an interface that dictates how the ByteBuffer consumed from Kafka gets transformed into a storm tuple. It
-also controls the naming of your output field.
-
-```java
-public Iterable<List<Object>> deserialize(ByteBuffer ser);
-public Fields getOutputFields();
-```
-
-The default `RawMultiScheme` just takes the `ByteBuffer` and returns a tuple with the ByteBuffer converted to a `byte[]`. The name of the outputField is "bytes". There are alternative implementations like `SchemeAsMultiScheme` and `KeyValueSchemeAsMultiScheme` which can convert the `ByteBuffer` to `String`.
-
-There is also an extension of `SchemeAsMultiScheme`, `MessageMetadataSchemeAsMultiScheme`,
-which has an additional deserialize method that accepts the message `ByteBuffer` in addition to the `Partition` and `offset` associated with the message.
-
-```java
-public Iterable<List<Object>> deserializeMessageWithMetadata(ByteBuffer message, Partition partition, long offset)
-```
-
-This is useful for auditing/replaying messages from arbitrary points on a Kafka topic, saving the partition and offset of each message of a discrete stream instead of persisting the entire message.
-
-### Failed message retry
-FailedMsgRetryManager is an interface which defines the retry strategy for a failed message. Default implementation is ExponentialBackoffMsgRetryManager which retries with exponential delays
-between consecutive retries. To use a custom implementation, set SpoutConfig.failedMsgRetryManagerClass to the full classname
-of implementation. Here is the interface 
-
-```java
-// Spout initialization can go here. This can be called multiple times during lifecycle of a worker. 
-void prepare(SpoutConfig spoutConfig, Map stormConf);
-
-// Message corresponding to offset has failed. This method is called only if retryFurther returns true for offset.
-void failed(Long offset);
-
-// Message corresponding to offset has been acked.  
-void acked(Long offset);
-
-// Message corresponding to the offset, has been re-emitted and under transit.
-void retryStarted(Long offset);
-
-/**
- * The offset of message, which is to be re-emitted. Spout will fetch messages starting from this offset
- * and resend them, except completed messages.
- */
-Long nextFailedMessageToRetry();
-
-/**
- * @return True if the message corresponding to the offset should be emitted NOW. False otherwise.
- */
-boolean shouldReEmitMsg(Long offset);
-
-/**
- * Spout will clean up the state for this offset if false is returned. If retryFurther is set to true,
- * spout will called failed(offset) in next call and acked(offset) otherwise 
- */
-boolean retryFurther(Long offset);
-
-/**
- * Spout will call this method after retryFurther returns false.
- * This gives a chance for hooking up custom logic before all clean up.
- * @param partition,offset
- */
-void cleanOffsetAfterRetries(Partition partition, Long offset);
-
-/**
- * Clear any offsets before kafkaOffset. These offsets are no longer available in kafka.
- */
-Set<Long> clearOffsetsBefore(Long kafkaOffset);
-``` 
-
-#### Version incompatibility
-In Storm versions prior to 1.0, the MultiScheme methods accepted a `byte[]` instead of `ByteBuffer`. The `MultScheme` and the related
-Scheme apis were changed in version 1.0 to accept a ByteBuffer instead of a byte[].
-
-This means that pre 1.0 kafka spouts will not work with Storm versions 1.0 and higher. While running topologies in Storm version 1.0
-and higher, it must be ensured that the storm-kafka version is at least 1.0. Pre 1.0 shaded topology jars that bundles
-storm-kafka classes must be rebuilt with storm-kafka version 1.0 for running in clusters with storm 1.0 and higher.
-
-### Examples
-
-#### Core Spout
-
-```java
-BrokerHosts hosts = new ZkHosts(zkConnString);
-SpoutConfig spoutConfig = new SpoutConfig(hosts, topicName, "/" + topicName, UUID.randomUUID().toString());
-spoutConfig.scheme = new SchemeAsMultiScheme(new StringScheme());
-KafkaSpout kafkaSpout = new KafkaSpout(spoutConfig);
-```
-
-#### Trident Spout
-
-```java
-TridentTopology topology = new TridentTopology();
-BrokerHosts zk = new ZkHosts("localhost");
-TridentKafkaConfig spoutConf = new TridentKafkaConfig(zk, "test-topic");
-spoutConf.scheme = new SchemeAsMultiScheme(new StringScheme());
-OpaqueTridentKafkaSpout spout = new OpaqueTridentKafkaSpout(spoutConf);
-```
-
-
-### How KafkaSpout stores offsets of a Kafka topic and recovers in case of failures
-
-As shown in the above KafkaConfig properties, you can control from where in the Kafka topic the spout begins to read by
-setting `KafkaConfig.startOffsetTime` as follows:
-
-1. `kafka.api.OffsetRequest.EarliestTime()`:  read from the beginning of the topic (i.e. from the oldest messages onwards)
-2. `kafka.api.OffsetRequest.LatestTime()`: read from the end of the topic (i.e. any new messsages that are being written to the topic)
-3. A Unix timestamp aka seconds since the epoch (e.g. via `System.currentTimeMillis()`):
-   see [How do I accurately get offsets of messages for a certain timestamp using OffsetRequest?](https://cwiki.apache.org/confluence/display/KAFKA/FAQ#FAQ-HowdoIaccuratelygetoffsetsofmessagesforacertaintimestampusingOffsetRequest?) in the Kafka FAQ
-
-As the topology runs the Kafka spout keeps track of the offsets it has read and emitted by storing state information
-under the ZooKeeper path `SpoutConfig.zkRoot+ "/" + SpoutConfig.id`.  In the case of failures it recovers from the last
-written offset in ZooKeeper.
-
-> **Important:**  When re-deploying a topology make sure that the settings for `SpoutConfig.zkRoot` and `SpoutConfig.id`
-> were not modified, otherwise the spout will not be able to read its previous consumer state information (i.e. the
-> offsets) from ZooKeeper -- which may lead to unexpected behavior and/or to data loss, depending on your use case.
-
-This means that when a topology has run once the setting `KafkaConfig.startOffsetTime` will not have an effect for
-subsequent runs of the topology because now the topology will rely on the consumer state information (offsets) in
-ZooKeeper to determine from where it should begin (more precisely: resume) reading.
-If you want to force the spout to ignore any consumer state information stored in ZooKeeper, then you should
-set the parameter `KafkaConfig.ignoreZkOffsets` to `true`.  If `true`, the spout will always begin reading from the
-offset defined by `KafkaConfig.startOffsetTime` as described above.
-
-
-## Using storm-kafka with different versions of kafka
-
-Storm-kafka's Kafka dependency is defined as `provided` scope in maven, meaning it will not be pulled in
-as a transitive dependency. This allows you to use a version of Kafka dependency compatible with your kafka cluster.
-
-When building a project with storm-kafka, you must explicitly add the Kafka dependency. For example, to
-use Kafka 0.8.1.1 built against Scala 2.10, you would use the following dependency in your `pom.xml`:
-
-```xml
-<dependency>
-    <groupId>org.apache.kafka</groupId>
-    <artifactId>kafka_2.10</artifactId>
-    <version>0.8.1.1</version>
-    <exclusions>
-        <exclusion>
-            <groupId>org.apache.zookeeper</groupId>
-            <artifactId>zookeeper</artifactId>
-        </exclusion>
-        <exclusion>
-            <groupId>log4j</groupId>
-            <artifactId>log4j</artifactId>
-        </exclusion>
-    </exclusions>
-</dependency>
-```
-
-Note that the ZooKeeper and log4j dependencies are excluded to prevent version conflicts with Storm's dependencies.
-
-You can also override the kafka dependency version while building from maven, with parameter `kafka.version` and `kafka.artifact.id`
-e.g. `mvn clean install -Dkafka.artifact.id=kafka_2.11 -Dkafka.version=0.9.0.1`
-
-When selecting a kafka dependency version, you should ensure - 
-
-1. kafka api is compatible with storm-kafka. Currently, only 0.9.x and 0.8.x client API is supported by storm-kafka 
-module. If you want to use a higher version, storm-kafka-client module should be used instead.
-2. The kafka client selected by you should be wire compatible with the broker. e.g. 0.9.x client will not work with 
-0.8.x broker. 
-
-
-## Writing to Kafka as part of your topology
-You can create an instance of org.apache.storm.kafka.bolt.KafkaBolt and attach it as a component to your topology or if you
-are using trident you can use org.apache.storm.kafka.trident.TridentState, org.apache.storm.kafka.trident.TridentStateFactory and
-org.apache.storm.kafka.trident.TridentKafkaUpdater.
-
-You need to provide implementation of following 2 interfaces
-
-### TupleToKafkaMapper and TridentTupleToKafkaMapper
-These interfaces have 2 methods defined:
-
-```java
-K getKeyFromTuple(Tuple/TridentTuple tuple);
-V getMessageFromTuple(Tuple/TridentTuple tuple);
-```
-
-As the name suggests, these methods are called to map a tuple to Kafka key and Kafka message. If you just want one field
-as key and one field as value, then you can use the provided FieldNameBasedTupleToKafkaMapper.java
-implementation. In the KafkaBolt, the implementation always looks for a field with field name "key" and "message" if you
-use the default constructor to construct FieldNameBasedTupleToKafkaMapper for backward compatibility
-reasons. Alternatively you could also specify a different key and message field by using the non default constructor.
-In the TridentKafkaState you must specify what is the field name for key and message as there is no default constructor.
-These should be specified while constructing and instance of FieldNameBasedTupleToKafkaMapper.
-
-### KafkaTopicSelector and trident KafkaTopicSelector
-This interface has only one method
-
-```java
-public interface KafkaTopicSelector {
-    String getTopics(Tuple/TridentTuple tuple);
-}
-```
-The implementation of this interface should return the topic to which the tuple's key/message mapping needs to be published
-You can return a null and the message will be ignored. If you have one static topic name then you can use
-DefaultTopicSelector.java and set the name of the topic in the constructor.
-`FieldNameTopicSelector` and `FieldIndexTopicSelector` use to support decided which topic should to push message from tuple.
-User could specify the field name or field index in tuple ,selector will use this value as topic name which to publish message.
-When the topic name not found , `KafkaBolt` will write messages into default topic .
-Please make sure the default topic have created .
-
-### Specifying Kafka producer properties
-You can provide all the produce properties in your Storm topology by calling `KafkaBolt.withProducerProperties()` and `TridentKafkaStateFactory.withProducerProperties()`. Please see  http://kafka.apache.org/documentation.html#newproducerconfigs
-Section "Important configuration properties for the producer" for more details.
-
-### Using wildcard kafka topic match
-You can do a wildcard topic match by adding the following config
-
-```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
-
-
-### Putting it all together
-
-For the bolt :
-
-```java
-TopologyBuilder builder = new TopologyBuilder();
-
-Fields fields = new Fields("key", "message");
-FixedBatchSpout spout = new FixedBatchSpout(fields, 4,
-            new Values("storm", "1"),
-            new Values("trident", "1"),
-            new Values("needs", "1"),
-            new Values("javadoc", "1")
-);
-spout.setCycle(true);
-builder.setSpout("spout", spout, 5);
-//set producer properties.
-Properties props = new Properties();
-props.put("bootstrap.servers", "localhost:9092");
-props.put("acks", "1");
-props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-
-KafkaBolt bolt = new KafkaBolt()
-        .withProducerProperties(props)
-        .withTopicSelector(new DefaultTopicSelector("test"))
-        .withTupleToKafkaMapper(new FieldNameBasedTupleToKafkaMapper());
-builder.setBolt("forwardToKafka", bolt, 8).shuffleGrouping("spout");
-
-Config conf = new Config();
-
-StormSubmitter.submitTopology("kafkaboltTest", conf, builder.createTopology());
-```
-
-For Trident:
-
-```java
-Fields fields = new Fields("word", "count");
-FixedBatchSpout spout = new FixedBatchSpout(fields, 4,
-        new Values("storm", "1"),
-        new Values("trident", "1"),
-        new Values("needs", "1"),
-        new Values("javadoc", "1")
-);
-spout.setCycle(true);
-
-TridentTopology topology = new TridentTopology();
-Stream stream = topology.newStream("spout1", spout);
-
-//set producer properties.
-Properties props = new Properties();
-props.put("bootstrap.servers", "localhost:9092");
-props.put("acks", "1");
-props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-
-TridentKafkaStateFactory stateFactory = new TridentKafkaStateFactory()
-        .withProducerProperties(props)
-        .withKafkaTopicSelector(new DefaultTopicSelector("test"))
-        .withTridentTupleToKafkaMapper(new FieldNameBasedTupleToKafkaMapper("word", "count"));
-stream.partitionPersist(stateFactory, fields, new TridentKafkaUpdater(), new Fields());
-
-Config conf = new Config();
-StormSubmitter.submitTopology("kafkaTridentTest", conf, topology.build());
-```
-
-## Committer Sponsors
-
- * P. Taylor Goetz ([ptgoetz@apache.org](mailto:ptgoetz@apache.org))
-

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/examples/storm-kafka-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-examples/pom.xml b/examples/storm-kafka-examples/pom.xml
deleted file mode 100644
index 13b5573..0000000
--- a/examples/storm-kafka-examples/pom.xml
+++ /dev/null
@@ -1,110 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- 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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <artifactId>storm</artifactId>
-        <groupId>org.apache.storm</groupId>
-        <version>2.0.0-SNAPSHOT</version>
-        <relativePath>../../pom.xml</relativePath>
-    </parent>
-
-    <artifactId>storm-kafka-examples</artifactId>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-client</artifactId>
-            <version>${project.version}</version>
-            <scope>${provided.scope}</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-kafka</artifactId>
-            <version>${project.version}</version>
-            <scope>compile</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.kafka</groupId>
-            <artifactId>${storm.kafka.artifact.id}</artifactId>
-            <version>${storm.kafka.version}</version>
-            <scope>compile</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.kafka</groupId>
-            <artifactId>kafka-clients</artifactId>
-            <version>${storm.kafka.version}</version>
-            <scope>compile</scope>
-        </dependency>
-    </dependencies>
-
-    <build>
-        <plugins>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-shade-plugin</artifactId>
-                <configuration>
-                    <createDependencyReducedPom>true</createDependencyReducedPom>
-                    <filters>
-                        <filter>
-                            <artifact>*:*</artifact>
-                            <excludes>
-                                <exclude>META-INF/*.SF</exclude>
-                                <exclude>META-INF/*.sf</exclude>
-                                <exclude>META-INF/*.DSA</exclude>
-                                <exclude>META-INF/*.dsa</exclude>
-                                <exclude>META-INF/*.RSA</exclude>
-                                <exclude>META-INF/*.rsa</exclude>
-                                <exclude>META-INF/*.EC</exclude>
-                                <exclude>META-INF/*.ec</exclude>
-                                <exclude>META-INF/MSFTSIG.SF</exclude>
-                                <exclude>META-INF/MSFTSIG.RSA</exclude>
-                            </excludes>
-                        </filter>
-                    </filters>
-                </configuration>
-                <executions>
-                    <execution>
-                        <phase>package</phase>
-                        <goals>
-                            <goal>shade</goal>
-                        </goals>
-                        <configuration>
-                            <transformers>
-                                <transformer
-                                        implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
-                                <transformer
-                                        implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
-                                </transformer>
-                            </transformers>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-checkstyle-plugin</artifactId>
-                <!--Note - the version would be inherited-->
-                <configuration>
-                    <maxAllowedViolations>26</maxAllowedViolations>
-                </configuration>
-            </plugin>
-        </plugins>
-    </build>
-</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/KafkaProducerTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/KafkaProducerTopology.java b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/KafkaProducerTopology.java
deleted file mode 100644
index d3f55de..0000000
--- a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/KafkaProducerTopology.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.trident;
-
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.kafka.bolt.KafkaBolt;
-import org.apache.storm.kafka.bolt.mapper.FieldNameBasedTupleToKafkaMapper;
-import org.apache.storm.kafka.bolt.selector.DefaultTopicSelector;
-import org.apache.storm.topology.TopologyBuilder;
-
-import java.util.Properties;
-import java.util.UUID;
-import org.apache.storm.lambda.LambdaSpout;
-import org.apache.storm.utils.Utils;
-
-public class KafkaProducerTopology {
-    /**
-     * Create a new topology that writes random UUIDs to Kafka.
-     *
-     * @param brokerUrl Kafka broker URL
-     * @param topicName Topic to which publish sentences
-     * @return A Storm topology that produces random UUIDs using a {@link LambdaSpout} and uses a {@link KafkaBolt} to publish the UUIDs to
-     *     the kafka topic specified
-     */
-    public static StormTopology newTopology(String brokerUrl, String topicName) {
-        final TopologyBuilder builder = new TopologyBuilder();
-        builder.setSpout("spout", () -> {
-            Utils.sleep(1000); //Throttle this spout a bit to avoid maxing out CPU
-            return UUID.randomUUID().toString();
-        });
-
-        /* The output field of the spout ("lambda") is provided as the boltMessageField
-          so that this gets written out as the message in the kafka topic.
-          The tuples have no key field, so the messages are written to Kafka without a key.*/
-        final KafkaBolt<String, String> bolt = new KafkaBolt<String, String>()
-            .withProducerProperties(newProps(brokerUrl, topicName))
-            .withTopicSelector(new DefaultTopicSelector(topicName))
-            .withTupleToKafkaMapper(new FieldNameBasedTupleToKafkaMapper<>("key", "lambda"));
-
-        builder.setBolt("forwardToKafka", bolt, 1).shuffleGrouping("spout");
-
-        return builder.createTopology();
-    }
-
-    /**
-     * @return the Storm config for the topology that publishes sentences to kafka using a kafka bolt.
-     */
-    private static Properties newProps(final String brokerUrl, final String topicName) {
-        return new Properties() {{
-            put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerUrl);
-            put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer");
-            put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer");
-            put(ProducerConfig.CLIENT_ID_CONFIG, topicName);
-        }};
-    }
-}
-
-

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaConsumerTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaConsumerTopology.java b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaConsumerTopology.java
deleted file mode 100644
index ae76cf9..0000000
--- a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaConsumerTopology.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *  
- *   http://www.apache.org/licenses/LICENSE-2.0
- *  
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.storm.kafka.trident;
-
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.trident.Stream;
-import org.apache.storm.trident.TridentTopology;
-import org.apache.storm.trident.operation.builtin.Debug;
-import org.apache.storm.trident.spout.ITridentDataSource;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TridentKafkaConsumerTopology {
-    protected static final Logger LOG = LoggerFactory.getLogger(TridentKafkaConsumerTopology.class);
-
-    /**
-     * Creates a new topology that prints inputs to stdout.
-     * @param tridentSpout The spout to use
-     */
-    public static StormTopology newTopology(ITridentDataSource tridentSpout) {
-        final TridentTopology tridentTopology = new TridentTopology();
-        final Stream spoutStream = tridentTopology.newStream("spout", tridentSpout).parallelismHint(2);
-        spoutStream.each(spoutStream.getOutputFields(), new Debug(false));
-        return tridentTopology.build();
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaRandomStrings.java
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaRandomStrings.java b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaRandomStrings.java
deleted file mode 100644
index a6c90e8..0000000
--- a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaRandomStrings.java
+++ /dev/null
@@ -1,82 +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.trident;
-
-
-import org.apache.storm.Config;
-import org.apache.storm.StormSubmitter;
-import org.apache.storm.kafka.StringScheme;
-import org.apache.storm.kafka.ZkHosts;
-import org.apache.storm.spout.SchemeAsMultiScheme;
-
-import java.io.Serializable;
-
-/**
- * This example sets up a few topologies to put random strings in the "test" Kafka topic via the KafkaBolt,
- * and shows how to set up a Trident topology that reads from the "test" topic using the KafkaSpout.
- * Please ensure you have a Kafka instance running on localhost:9092 before you deploy this example.
- */
-public class TridentKafkaRandomStrings implements Serializable {
-    public static void main(String[] args) throws Exception {
-        final String[] zkBrokerUrl = parseUrl(args);
-        final String topicName = "test";
-        Config tpConf = new Config();
-        tpConf.setMaxSpoutPending(20);
-        String prodTpName = "kafkaBolt";
-        String consTpName = "kafka-topology-random-strings";
-
-        if (args.length == 3)  {
-            prodTpName = args[2] + "-producer";
-            consTpName = args[2] + "-consumer";
-        }
-        // Producer
-        StormSubmitter.submitTopology(prodTpName, tpConf, KafkaProducerTopology.newTopology(zkBrokerUrl[1], topicName));
-        // Consumer
-        StormSubmitter.submitTopology(consTpName, tpConf, TridentKafkaConsumerTopology.newTopology(
-                new TransactionalTridentKafkaSpout(newTridentKafkaConfig(zkBrokerUrl[0]))));
-    }
-
-    private static String[] parseUrl(String[] args) {
-        String zkUrl = "localhost:2181";        // the defaults.
-        String brokerUrl = "localhost:9092";
-
-        if (args.length > 3 || (args.length == 1 && args[0].matches("^-h|--help$"))) {
-            System.out.println("Usage: TridentKafkaWordCount [kafka zookeeper url] [kafka broker url] [topology name]");
-            System.out.println("   E.g TridentKafkaWordCount [" + zkUrl + "]" + " [" + brokerUrl + "] [wordcount]");
-            System.exit(1);
-        } else if (args.length == 1) {
-            zkUrl = args[0];
-        } else if (args.length == 2) {
-            zkUrl = args[0];
-            brokerUrl = args[1];
-        }
-
-        System.out.println("Using Kafka zookeeper uHrl: " + zkUrl + " broker url: " + brokerUrl);
-        return new String[]{zkUrl, brokerUrl};
-    }
-
-    private static TridentKafkaConfig newTridentKafkaConfig(String zkUrl) {
-        ZkHosts hosts = new ZkHosts(zkUrl);
-        TridentKafkaConfig config = new TridentKafkaConfig(hosts, "test");
-        config.scheme = new SchemeAsMultiScheme(new StringScheme());
-
-        // Consume new data from the topic
-        config.startOffsetTime = kafka.api.OffsetRequest.LatestTime();
-        return config;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaTopology.java b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaTopology.java
deleted file mode 100644
index ad785b8..0000000
--- a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaTopology.java
+++ /dev/null
@@ -1,83 +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.trident;
-
-import java.util.Properties;
-
-import org.apache.storm.Config;
-import org.apache.storm.StormSubmitter;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.kafka.trident.mapper.FieldNameBasedTupleToKafkaMapper;
-import org.apache.storm.kafka.trident.selector.DefaultTopicSelector;
-import org.apache.storm.trident.Stream;
-import org.apache.storm.trident.TridentTopology;
-import org.apache.storm.trident.testing.FixedBatchSpout;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
-
-public class TridentKafkaTopology {
-
-    private static StormTopology buildTopology(String brokerConnectionString) {
-        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);
-
-        Properties props = new Properties();
-        props.put("bootstrap.servers", brokerConnectionString);
-        props.put("acks", "1");
-        props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-        props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-
-        TridentKafkaStateFactory stateFactory = new TridentKafkaStateFactory()
-            .withProducerProperties(props)
-            .withKafkaTopicSelector(new DefaultTopicSelector("test"))
-            .withTridentTupleToKafkaMapper(new FieldNameBasedTupleToKafkaMapper("word", "count"));
-        stream.partitionPersist(stateFactory, fields, new TridentKafkaUpdater(), new Fields());
-
-        return topology.build();
-    }
-
-    /**
-     * To run this topology ensure you have a kafka broker running and provide connection string to broker as argument.
-     * Create a topic test with command line,
-     * kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partition 1 --topic test
-     *
-     * run this program and run the kafka consumer:
-     * kafka-console-consumer.sh --zookeeper localhost:2181 --topic test --from-beginning
-     *
-     * you should see the messages flowing through.
-     *
-     * @param args
-     * @throws Exception
-     */
-    public static void main(String[] args) throws Exception {
-        if(args.length < 1) {
-            System.out.println("Please provide kafka broker url ,e.g. localhost:9092");
-        }
-        StormSubmitter.submitTopology("wordCounter", new Config(), buildTopology(args[0]));
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka-monitor/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-kafka-monitor/pom.xml b/external/storm-kafka-monitor/pom.xml
index 711dbea..eb9eb25 100644
--- a/external/storm-kafka-monitor/pom.xml
+++ b/external/storm-kafka-monitor/pom.xml
@@ -49,11 +49,6 @@
             <version>${storm.kafka.client.version}</version>
         </dependency>
         <dependency>
-            <groupId>org.apache.kafka</groupId>
-            <artifactId>${storm.kafka.artifact.id}</artifactId>
-            <version>${storm.kafka.version}</version>
-        </dependency>
-        <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-log4j12</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka-monitor/src/main/java/org/apache/storm/kafka/monitor/KafkaOffsetLagUtil.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-monitor/src/main/java/org/apache/storm/kafka/monitor/KafkaOffsetLagUtil.java b/external/storm-kafka-monitor/src/main/java/org/apache/storm/kafka/monitor/KafkaOffsetLagUtil.java
index ef65bcb..78b6993 100644
--- a/external/storm-kafka-monitor/src/main/java/org/apache/storm/kafka/monitor/KafkaOffsetLagUtil.java
+++ b/external/storm-kafka-monitor/src/main/java/org/apache/storm/kafka/monitor/KafkaOffsetLagUtil.java
@@ -24,19 +24,11 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
-import kafka.api.OffsetRequest;
-import kafka.api.PartitionOffsetRequestInfo;
-import kafka.common.TopicAndPartition;
-import kafka.javaapi.OffsetResponse;
-import kafka.javaapi.consumer.SimpleConsumer;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.DefaultParser;
 import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Options;
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.retry.RetryOneTime;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.common.PartitionInfo;
@@ -49,105 +41,30 @@ import org.json.simple.JSONValue;
 public class KafkaOffsetLagUtil {
     private static final String OPTION_TOPIC_SHORT = "t";
     private static final String OPTION_TOPIC_LONG = "topics";
-    private static final String OPTION_OLD_CONSUMER_SHORT = "o";
-    private static final String OPTION_OLD_CONSUMER_LONG = "old-spout";
     private static final String OPTION_BOOTSTRAP_BROKERS_SHORT = "b";
     private static final String OPTION_BOOTSTRAP_BROKERS_LONG = "bootstrap-brokers";
     private static final String OPTION_GROUP_ID_SHORT = "g";
     private static final String OPTION_GROUP_ID_LONG = "groupid";
-    private static final String OPTION_TOPIC_WILDCARD_SHORT = "w";
-    private static final String OPTION_TOPIC_WILDCARD_LONG = "wildcard-topic";
-    private static final String OPTION_PARTITIONS_SHORT = "p";
-    private static final String OPTION_PARTITIONS_LONG = "partitions";
-    private static final String OPTION_LEADERS_SHORT = "l";
-    private static final String OPTION_LEADERS_LONG = "leaders";
-    private static final String OPTION_ZK_SERVERS_SHORT = "z";
-    private static final String OPTION_ZK_SERVERS_LONG = "zk-servers";
-    private static final String OPTION_ZK_COMMITTED_NODE_SHORT = "n";
-    private static final String OPTION_ZK_COMMITTED_NODE_LONG = "zk-node";
-    private static final String OPTION_ZK_BROKERS_ROOT_SHORT = "r";
-    private static final String OPTION_ZK_BROKERS_ROOT_LONG = "zk-brokers-root-node";
     private static final String OPTION_SECURITY_PROTOCOL_SHORT = "s";
     private static final String OPTION_SECURITY_PROTOCOL_LONG = "security-protocol";
 
     public static void main(String args[]) {
         try {
-            List<KafkaOffsetLagResult> results;
             Options options = buildOptions();
             CommandLineParser parser = new DefaultParser();
             CommandLine commandLine = parser.parse(options, args);
             if (!commandLine.hasOption(OPTION_TOPIC_LONG)) {
                 printUsageAndExit(options, OPTION_TOPIC_LONG + " is required");
             }
-            if (commandLine.hasOption(OPTION_OLD_CONSUMER_LONG)) {
-                OldKafkaSpoutOffsetQuery oldKafkaSpoutOffsetQuery;
-                if (commandLine.hasOption(OPTION_GROUP_ID_LONG) || commandLine.hasOption(OPTION_BOOTSTRAP_BROKERS_LONG) ||
-                    commandLine.hasOption(OPTION_SECURITY_PROTOCOL_LONG)) {
-                    printUsageAndExit(options, OPTION_GROUP_ID_LONG + " or " + OPTION_BOOTSTRAP_BROKERS_LONG + " or " +
-                                               OPTION_SECURITY_PROTOCOL_LONG + " is " +
-                                               "not accepted with option " + OPTION_OLD_CONSUMER_LONG);
-                }
-                if (!commandLine.hasOption(OPTION_ZK_SERVERS_LONG) || !commandLine.hasOption(OPTION_ZK_COMMITTED_NODE_LONG)) {
-                    printUsageAndExit(options, OPTION_ZK_SERVERS_LONG + " and " + OPTION_ZK_COMMITTED_NODE_LONG + " are required  with " +
-                                               OPTION_OLD_CONSUMER_LONG);
-                }
-                String[] topics = commandLine.getOptionValue(OPTION_TOPIC_LONG).split(",");
-                if (topics != null && topics.length > 1) {
-                    printUsageAndExit(options, "Multiple topics not supported with option " + OPTION_OLD_CONSUMER_LONG +
-                                               ". Either a single topic or a " +
-                                               "wildcard string for matching topics is supported");
-                }
-                if (commandLine.hasOption(OPTION_ZK_BROKERS_ROOT_LONG)) {
-                    if (commandLine.hasOption(OPTION_PARTITIONS_LONG) || commandLine.hasOption(OPTION_LEADERS_LONG)) {
-                        printUsageAndExit(options, OPTION_PARTITIONS_LONG + " or " + OPTION_LEADERS_LONG + " is not accepted with " +
-                                                   OPTION_ZK_BROKERS_ROOT_LONG);
-                    }
-                    oldKafkaSpoutOffsetQuery =
-                        new OldKafkaSpoutOffsetQuery(commandLine.getOptionValue(OPTION_TOPIC_LONG), commandLine.getOptionValue
-                            (OPTION_ZK_SERVERS_LONG), commandLine.getOptionValue(OPTION_ZK_COMMITTED_NODE_LONG), commandLine.hasOption
-                            (OPTION_TOPIC_WILDCARD_LONG), commandLine.getOptionValue(OPTION_ZK_BROKERS_ROOT_LONG));
-                } else {
-                    if (commandLine.hasOption(OPTION_TOPIC_WILDCARD_LONG)) {
-                        printUsageAndExit(options, OPTION_TOPIC_WILDCARD_LONG + " is not supported without " + OPTION_ZK_BROKERS_ROOT_LONG);
-                    }
-                    if (!commandLine.hasOption(OPTION_PARTITIONS_LONG) || !commandLine.hasOption(OPTION_LEADERS_LONG)) {
-                        printUsageAndExit(options, OPTION_PARTITIONS_LONG + " and " + OPTION_LEADERS_LONG + " are required if " +
-                                                   OPTION_ZK_BROKERS_ROOT_LONG +
-                                                   " is not provided");
-                    }
-                    String[] partitions = commandLine.getOptionValue(OPTION_PARTITIONS_LONG).split(",");
-                    String[] leaders = commandLine.getOptionValue(OPTION_LEADERS_LONG).split(",");
-                    if (partitions.length != leaders.length) {
-                        printUsageAndExit(options, OPTION_PARTITIONS_LONG + " and " + OPTION_LEADERS_LONG + " need to be of same size");
-                    }
-                    oldKafkaSpoutOffsetQuery =
-                        new OldKafkaSpoutOffsetQuery(commandLine.getOptionValue(OPTION_TOPIC_LONG), commandLine.getOptionValue
-                            (OPTION_ZK_SERVERS_LONG), commandLine.getOptionValue(OPTION_ZK_COMMITTED_NODE_LONG), commandLine.getOptionValue
-                            (OPTION_PARTITIONS_LONG), commandLine.getOptionValue(OPTION_LEADERS_LONG));
-                }
-                results = getOffsetLags(oldKafkaSpoutOffsetQuery);
-            } else {
-                String securityProtocol = commandLine.getOptionValue(OPTION_SECURITY_PROTOCOL_LONG);
-                String[] oldSpoutOptions = {
-                    OPTION_TOPIC_WILDCARD_LONG, OPTION_PARTITIONS_LONG, OPTION_LEADERS_LONG, OPTION_ZK_SERVERS_LONG,
-                    OPTION_ZK_COMMITTED_NODE_LONG, OPTION_ZK_BROKERS_ROOT_LONG
-                };
-                for (String oldOption : oldSpoutOptions) {
-                    if (commandLine.hasOption(oldOption)) {
-                        printUsageAndExit(options, oldOption + " is not accepted without " + OPTION_OLD_CONSUMER_LONG);
-                    }
-                }
-                if (!commandLine.hasOption(OPTION_GROUP_ID_LONG) || !commandLine.hasOption(OPTION_BOOTSTRAP_BROKERS_LONG)) {
-                    printUsageAndExit(options, OPTION_GROUP_ID_LONG + " and " + OPTION_BOOTSTRAP_BROKERS_LONG + " are required if " +
-                                               OPTION_OLD_CONSUMER_LONG +
-                                               " is not specified");
-                }
-                NewKafkaSpoutOffsetQuery newKafkaSpoutOffsetQuery =
-                    new NewKafkaSpoutOffsetQuery(commandLine.getOptionValue(OPTION_TOPIC_LONG),
-                                                 commandLine.getOptionValue(OPTION_BOOTSTRAP_BROKERS_LONG),
-                                                 commandLine.getOptionValue(OPTION_GROUP_ID_LONG), securityProtocol);
-                results = getOffsetLags(newKafkaSpoutOffsetQuery);
+            String securityProtocol = commandLine.getOptionValue(OPTION_SECURITY_PROTOCOL_LONG);
+            if (!commandLine.hasOption(OPTION_GROUP_ID_LONG) || !commandLine.hasOption(OPTION_BOOTSTRAP_BROKERS_LONG)) {
+                printUsageAndExit(options, OPTION_GROUP_ID_LONG + " and " + OPTION_BOOTSTRAP_BROKERS_LONG + " are required");
             }
+            NewKafkaSpoutOffsetQuery newKafkaSpoutOffsetQuery =
+                new NewKafkaSpoutOffsetQuery(commandLine.getOptionValue(OPTION_TOPIC_LONG),
+                    commandLine.getOptionValue(OPTION_BOOTSTRAP_BROKERS_LONG),
+                    commandLine.getOptionValue(OPTION_GROUP_ID_LONG), securityProtocol);
+            List<KafkaOffsetLagResult> results = getOffsetLags(newKafkaSpoutOffsetQuery);
 
             Map<String, Map<Integer, KafkaPartitionOffsetLag>> keyedResult = keyByTopicAndPartition(results);
             System.out.print(JSONValue.toJSONString(keyedResult));
@@ -188,29 +105,10 @@ public class KafkaOffsetLagUtil {
         options.addOption(OPTION_TOPIC_SHORT, OPTION_TOPIC_LONG, true,
                           "REQUIRED Topics (comma separated list) for fetching log head and spout committed " +
                           "offset");
-        options.addOption(OPTION_OLD_CONSUMER_SHORT, OPTION_OLD_CONSUMER_LONG, false, "Whether request is for old spout");
         options.addOption(OPTION_BOOTSTRAP_BROKERS_SHORT, OPTION_BOOTSTRAP_BROKERS_LONG, true,
                           "Comma separated list of bootstrap broker hosts for new " +
                           "consumer/spout e.g. hostname1:9092,hostname2:9092");
-        options.addOption(OPTION_GROUP_ID_SHORT, OPTION_GROUP_ID_LONG, true, "Group id of consumer (applicable only for new kafka spout) ");
-        options.addOption(OPTION_TOPIC_WILDCARD_SHORT, OPTION_TOPIC_WILDCARD_LONG, false,
-                          "Whether topic provided is a wildcard as supported by ZkHosts in " +
-                          "old spout");
-        options.addOption(OPTION_PARTITIONS_SHORT, OPTION_PARTITIONS_LONG, true, "Comma separated list of partitions corresponding to " +
-                                                                                 OPTION_LEADERS_LONG + " for old spout with StaticHosts");
-        options.addOption(OPTION_LEADERS_SHORT, OPTION_LEADERS_LONG, true, "Comma separated list of broker leaders corresponding to " +
-                                                                           OPTION_PARTITIONS_LONG +
-                                                                           " for old spout with StaticHosts e.g. hostname1:9092," +
-                                                                           "hostname2:9092");
-        options.addOption(OPTION_ZK_SERVERS_SHORT, OPTION_ZK_SERVERS_LONG, true,
-                          "Comma separated list of zk servers for fetching spout committed offsets  " +
-                          "and/or topic metadata for ZkHosts e.g hostname1:2181,hostname2:2181");
-        options.addOption(OPTION_ZK_COMMITTED_NODE_SHORT, OPTION_ZK_COMMITTED_NODE_LONG, true,
-                          "Zk node prefix where old kafka spout stores the committed" +
-                          " offsets without the topic and partition nodes");
-        options.addOption(OPTION_ZK_BROKERS_ROOT_SHORT, OPTION_ZK_BROKERS_ROOT_LONG, true,
-                          "Zk node prefix where kafka stores broker information e.g. " +
-                          "/brokers (applicable only for old kafka spout) ");
+        options.addOption(OPTION_GROUP_ID_SHORT, OPTION_GROUP_ID_LONG, true, "Group id of consumer");
         options.addOption(OPTION_SECURITY_PROTOCOL_SHORT, OPTION_SECURITY_PROTOCOL_LONG, true, "Security protocol to connect to kafka");
         return options;
     }
@@ -265,182 +163,4 @@ public class KafkaOffsetLagUtil {
         }};
     }
 
-    /**
-     * @param oldKafkaSpoutOffsetQuery represents the information needed to query kafka for log head and spout offsets
-     * @return log head offset, spout offset and lag for each partition
-     */
-    public static List<KafkaOffsetLagResult> getOffsetLags(OldKafkaSpoutOffsetQuery oldKafkaSpoutOffsetQuery) throws Exception {
-        List<KafkaOffsetLagResult> result = new ArrayList<>();
-        Map<String, List<TopicPartition>> leaders = getLeadersAndTopicPartitions(oldKafkaSpoutOffsetQuery);
-        if (leaders != null) {
-            Map<String, Map<Integer, Long>> logHeadOffsets = getLogHeadOffsets(leaders);
-            Map<String, List<Integer>> topicPartitions = new HashMap<>();
-            for (Map.Entry<String, List<TopicPartition>> entry : leaders.entrySet()) {
-                for (TopicPartition topicPartition : entry.getValue()) {
-                    if (!topicPartitions.containsKey(topicPartition.topic())) {
-                        topicPartitions.put(topicPartition.topic(), new ArrayList<Integer>());
-                    }
-                    topicPartitions.get(topicPartition.topic()).add(topicPartition.partition());
-                }
-            }
-            Map<String, Map<Integer, Long>> oldConsumerOffsets = getOldConsumerOffsetsFromZk(topicPartitions, oldKafkaSpoutOffsetQuery);
-            for (Map.Entry<String, Map<Integer, Long>> topicOffsets : logHeadOffsets.entrySet()) {
-                for (Map.Entry<Integer, Long> partitionOffsets : topicOffsets.getValue().entrySet()) {
-                    Long consumerCommittedOffset =
-                        oldConsumerOffsets.get(topicOffsets.getKey()) != null ? (Long) oldConsumerOffsets.get(topicOffsets.getKey()).get
-                            (partitionOffsets.getKey()) : -1;
-                    consumerCommittedOffset = (consumerCommittedOffset == null ? -1 : consumerCommittedOffset);
-                    KafkaOffsetLagResult kafkaOffsetLagResult = new KafkaOffsetLagResult(topicOffsets.getKey(), partitionOffsets.getKey(),
-                                                                                         consumerCommittedOffset,
-                                                                                         partitionOffsets.getValue());
-                    result.add(kafkaOffsetLagResult);
-                }
-            }
-        }
-        return result;
-    }
-
-    private static Map<String, List<TopicPartition>> getLeadersAndTopicPartitions(OldKafkaSpoutOffsetQuery oldKafkaSpoutOffsetQuery) throws
-        Exception {
-        Map<String, List<TopicPartition>> result = new HashMap<>();
-        // this means that kafka spout was configured with StaticHosts hosts (leader for partition)
-        if (oldKafkaSpoutOffsetQuery.getPartitions() != null) {
-            String[] partitions = oldKafkaSpoutOffsetQuery.getPartitions().split(",");
-            String[] leaders = oldKafkaSpoutOffsetQuery.getLeaders().split(",");
-            for (int i = 0; i < leaders.length; ++i) {
-                if (!result.containsKey(leaders[i])) {
-                    result.put(leaders[i], new ArrayList<TopicPartition>());
-                }
-                result.get(leaders[i]).add(new TopicPartition(oldKafkaSpoutOffsetQuery.getTopic(), Integer.parseInt(partitions[i])));
-            }
-        } else {
-            // else use zk nodes to figure out partitions and leaders for topics i.e. ZkHosts
-            CuratorFramework curatorFramework = null;
-            try {
-                String brokersZkNode = oldKafkaSpoutOffsetQuery.getBrokersZkPath();
-                if (!brokersZkNode.endsWith("/")) {
-                    brokersZkNode += "/";
-                }
-                String topicsZkPath = brokersZkNode + "topics";
-                curatorFramework =
-                    CuratorFrameworkFactory.newClient(oldKafkaSpoutOffsetQuery.getZkServers(), 20000, 15000, new RetryOneTime(1000));
-                curatorFramework.start();
-                List<String> topics = new ArrayList<>();
-                if (oldKafkaSpoutOffsetQuery.isWildCardTopic()) {
-                    List<String> children = curatorFramework.getChildren().forPath(topicsZkPath);
-                    for (String child : children) {
-                        if (child.matches(oldKafkaSpoutOffsetQuery.getTopic())) {
-                            topics.add(child);
-                        }
-                    }
-                } else {
-                    topics.add(oldKafkaSpoutOffsetQuery.getTopic());
-                }
-                for (String topic : topics) {
-                    String partitionsPath = topicsZkPath + "/" + topic + "/partitions";
-                    List<String> children = curatorFramework.getChildren().forPath(partitionsPath);
-                    for (int i = 0; i < children.size(); ++i) {
-                        byte[] leaderData = curatorFramework.getData().forPath(partitionsPath + "/" + i + "/state");
-                        Map<Object, Object> value = (Map<Object, Object>) JSONValue.parseWithException(new String(leaderData, "UTF-8"));
-                        Integer leader = ((Number) value.get("leader")).intValue();
-                        byte[] brokerData = curatorFramework.getData().forPath(brokersZkNode + "ids/" + leader);
-                        Map<Object, Object> broker = (Map<Object, Object>) JSONValue.parseWithException(new String(brokerData, "UTF-8"));
-                        String host = (String) broker.get("host");
-                        Integer port = ((Long) broker.get("port")).intValue();
-                        String leaderBroker = host + ":" + port;
-                        if (!result.containsKey(leaderBroker)) {
-                            result.put(leaderBroker, new ArrayList<TopicPartition>());
-                        }
-                        result.get(leaderBroker).add(new TopicPartition(topic, i));
-                    }
-                }
-            } finally {
-                if (curatorFramework != null) {
-                    curatorFramework.close();
-                }
-            }
-        }
-        return result;
-    }
-
-    private static Map<String, Map<Integer, Long>> getLogHeadOffsets(Map<String, List<TopicPartition>> leadersAndTopicPartitions) {
-        Map<String, Map<Integer, Long>> result = new HashMap<>();
-        if (leadersAndTopicPartitions != null) {
-            PartitionOffsetRequestInfo partitionOffsetRequestInfo = new PartitionOffsetRequestInfo(OffsetRequest.LatestTime(), 1);
-            SimpleConsumer simpleConsumer = null;
-            for (Map.Entry<String, List<TopicPartition>> leader : leadersAndTopicPartitions.entrySet()) {
-                try {
-                    simpleConsumer =
-                        new SimpleConsumer(leader.getKey().split(":")[0], Integer.parseInt(leader.getKey().split(":")[1]), 10000, 64 *
-                                                                                                                                  1024,
-                                           "LogHeadOffsetRequest");
-                    Map<TopicAndPartition, PartitionOffsetRequestInfo> requestInfo =
-                        new HashMap<TopicAndPartition, PartitionOffsetRequestInfo>();
-                    for (TopicPartition topicPartition : leader.getValue()) {
-                        requestInfo
-                            .put(new TopicAndPartition(topicPartition.topic(), topicPartition.partition()), partitionOffsetRequestInfo);
-                        if (!result.containsKey(topicPartition.topic())) {
-                            result.put(topicPartition.topic(), new HashMap<Integer, Long>());
-                        }
-                    }
-                    kafka.javaapi.OffsetRequest request =
-                        new kafka.javaapi.OffsetRequest(requestInfo, kafka.api.OffsetRequest.CurrentVersion(),
-                                                        "LogHeadOffsetRequest");
-                    OffsetResponse response = simpleConsumer.getOffsetsBefore(request);
-                    for (TopicPartition topicPartition : leader.getValue()) {
-                        result.get(topicPartition.topic())
-                              .put(topicPartition.partition(), response.offsets(topicPartition.topic(), topicPartition.partition())[0]);
-                    }
-                } finally {
-                    if (simpleConsumer != null) {
-                        simpleConsumer.close();
-                    }
-                }
-            }
-        }
-        return result;
-    }
-
-    private static Map<String, Map<Integer, Long>> getOldConsumerOffsetsFromZk(Map<String, List<Integer>> topicPartitions,
-                                                                               OldKafkaSpoutOffsetQuery
-                                                                                   oldKafkaSpoutOffsetQuery) throws Exception {
-        Map<String, Map<Integer, Long>> result = new HashMap<>();
-        CuratorFramework curatorFramework =
-            CuratorFrameworkFactory.newClient(oldKafkaSpoutOffsetQuery.getZkServers(), 20000, 15000, new RetryOneTime(1000));
-        curatorFramework.start();
-        String partitionPrefix = "partition_";
-        String zkPath = oldKafkaSpoutOffsetQuery.getZkPath();
-        if (zkPath.endsWith("/")) {
-            zkPath = zkPath.substring(0, zkPath.length() - 1);
-        }
-        if (curatorFramework.checkExists().forPath(zkPath) == null) {
-            throw new IllegalArgumentException(OPTION_ZK_COMMITTED_NODE_LONG + " '" + zkPath + "' dose not exists.");
-        }
-        byte[] zkData;
-        try {
-            if (topicPartitions != null) {
-                for (Map.Entry<String, List<Integer>> topicEntry : topicPartitions.entrySet()) {
-                    Map<Integer, Long> partitionOffsets = new HashMap<>();
-                    for (Integer partition : topicEntry.getValue()) {
-                        String path =
-                            zkPath + "/" + (oldKafkaSpoutOffsetQuery.isWildCardTopic() ? topicEntry.getKey() + "/" : "") + partitionPrefix +
-                            partition;
-                        if (curatorFramework.checkExists().forPath(path) != null) {
-                            zkData = curatorFramework.getData().forPath(path);
-                            Map<Object, Object> offsetData =
-                                (Map<Object, Object>) JSONValue.parseWithException(new String(zkData, "UTF-8"));
-                            partitionOffsets.put(partition, (Long) offsetData.get("offset"));
-                        }
-                    }
-                    result.put(topicEntry.getKey(), partitionOffsets);
-                }
-            }
-        } finally {
-            if (curatorFramework != null) {
-                curatorFramework.close();
-            }
-        }
-        return result;
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka-monitor/src/main/java/org/apache/storm/kafka/monitor/OldKafkaSpoutOffsetQuery.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-monitor/src/main/java/org/apache/storm/kafka/monitor/OldKafkaSpoutOffsetQuery.java b/external/storm-kafka-monitor/src/main/java/org/apache/storm/kafka/monitor/OldKafkaSpoutOffsetQuery.java
deleted file mode 100644
index ea80b64..0000000
--- a/external/storm-kafka-monitor/src/main/java/org/apache/storm/kafka/monitor/OldKafkaSpoutOffsetQuery.java
+++ /dev/null
@@ -1,127 +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.monitor;
-
-/**
- * Class representing information for querying kafka for log head offsets, spout offsets and the difference for old kafka spout
- */
-public class OldKafkaSpoutOffsetQuery {
-    private final String topic; //single topic or a wildcard topic
-    private final String zkServers; //comma separated list of zk servers and port e.g. hostname1:2181, hostname2:2181
-    private final String zkPath; //zk node prefix without topic/partition where committed offsets are stored
-    private final boolean isWildCardTopic; //if the topic is a wildcard
-    private final String brokersZkPath; //zk node prefix where kafka stores all broker information
-    private final String partitions; //comma separated list of partitions corresponding to leaders below (for StaticHosts)
-    private final String leaders;
-        //comma separated list of leader brokers and port corresponding to the partitions above (for StaticHosts) e.g.
-    // hostname1:9092,hostname2:9092
-
-    public OldKafkaSpoutOffsetQuery(String topic, String zkServers, String zkPath, boolean isWildCardTopic, String brokersZkPath) {
-        this(topic, zkServers, zkPath, isWildCardTopic, brokersZkPath, null, null);
-    }
-
-    public OldKafkaSpoutOffsetQuery(String topic, String zkServers, String zkPath, String partitions, String leaders) {
-        this(topic, zkServers, zkPath, false, null, partitions, leaders);
-
-    }
-
-    private OldKafkaSpoutOffsetQuery(String topic, String zkServers, String zkPath, boolean isWildCardTopic, String brokersZkPath,
-                                     String partitions, String
-                                         leaders) {
-        this.topic = topic;
-        this.zkServers = zkServers;
-        this.zkPath = zkPath;
-        this.isWildCardTopic = isWildCardTopic;
-        this.brokersZkPath = brokersZkPath;
-        this.partitions = partitions;
-        this.leaders = leaders;
-    }
-
-    @Override
-    public String toString() {
-        return "OldKafkaSpoutOffsetQuery{" +
-               "topic='" + topic + '\'' +
-               ", zkServers='" + zkServers + '\'' +
-               ", zkPath='" + zkPath + '\'' +
-               ", isWildCardTopic=" + isWildCardTopic +
-               ", brokersZkPath='" + brokersZkPath + '\'' +
-               ", partitions='" + partitions + '\'' +
-               ", leaders='" + leaders + '\'' +
-               '}';
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-
-        OldKafkaSpoutOffsetQuery that = (OldKafkaSpoutOffsetQuery) o;
-
-        if (isWildCardTopic != that.isWildCardTopic) return false;
-        if (topic != null ? !topic.equals(that.topic) : that.topic != null) return false;
-        if (zkServers != null ? !zkServers.equals(that.zkServers) : that.zkServers != null) return false;
-        if (zkPath != null ? !zkPath.equals(that.zkPath) : that.zkPath != null) return false;
-        if (brokersZkPath != null ? !brokersZkPath.equals(that.brokersZkPath) : that.brokersZkPath != null) return false;
-        if (partitions != null ? !partitions.equals(that.partitions) : that.partitions != null) return false;
-        return !(leaders != null ? !leaders.equals(that.leaders) : that.leaders != null);
-
-    }
-
-    @Override
-    public int hashCode() {
-        int result = topic != null ? topic.hashCode() : 0;
-        result = 31 * result + (zkServers != null ? zkServers.hashCode() : 0);
-        result = 31 * result + (zkPath != null ? zkPath.hashCode() : 0);
-        result = 31 * result + (isWildCardTopic ? 1 : 0);
-        result = 31 * result + (brokersZkPath != null ? brokersZkPath.hashCode() : 0);
-        result = 31 * result + (partitions != null ? partitions.hashCode() : 0);
-        result = 31 * result + (leaders != null ? leaders.hashCode() : 0);
-        return result;
-    }
-
-    public String getTopic() {
-
-        return topic;
-    }
-
-    public String getZkServers() {
-        return zkServers;
-    }
-
-    public String getZkPath() {
-        return zkPath;
-    }
-
-    public boolean isWildCardTopic() {
-        return isWildCardTopic;
-    }
-
-    public String getBrokersZkPath() {
-        return brokersZkPath;
-    }
-
-    public String getPartitions() {
-        return partitions;
-    }
-
-    public String getLeaders() {
-        return leaders;
-    }
-
-}


[5/7] storm git commit: STORM-2953: Remove storm-kafka

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/README.md
----------------------------------------------------------------------
diff --git a/external/storm-kafka/README.md b/external/storm-kafka/README.md
deleted file mode 100644
index d25bf29..0000000
--- a/external/storm-kafka/README.md
+++ /dev/null
@@ -1,382 +0,0 @@
-Storm Kafka
-====================
-
-Provides core Storm and Trident spout implementations for consuming data from Apache Kafka 0.8.x.
-
-## Spouts
-We support both Trident and core Storm spouts. For both spout implementations, we use a BrokerHost interface that
-tracks Kafka broker host to partition mapping and kafkaConfig that controls some Kafka related parameters.
-
-### BrokerHosts
-In order to initialize your Kafka spout/emitter you need to construct an instance of the marker interface BrokerHosts.
-Currently, we support the following two implementations:
-
-#### ZkHosts
-ZkHosts is what you should use if you want to dynamically track Kafka broker to partition mapping. This class uses
-Kafka's ZooKeeper entries to track brokerHost -> partition mapping. You can instantiate an object by calling
-```java
-    public ZkHosts(String brokerZkStr, String brokerZkPath)
-    public ZkHosts(String brokerZkStr)
-```
-Where brokerZkStr is just ip:port (e.g. localhost:2181). brokerZkPath is the root directory under which all the topics and
-partition information is stored. By default this is /brokers which is what the default Kafka implementation uses.
-
-By default, the broker-partition mapping is refreshed every 60 seconds from ZooKeeper. If you want to change it, you
-should set host.refreshFreqSecs to your chosen value.
-
-#### StaticHosts
-This is an alternative implementation where broker -> partition information is static. In order to construct an instance
-of this class, you need to first construct an instance of GlobalPartitionInformation.
-
-```java
-    Broker brokerForPartition0 = new Broker("localhost");//localhost:9092
-    Broker brokerForPartition1 = new Broker("localhost", 9092);//localhost:9092 but we specified the port explicitly
-    Broker brokerForPartition2 = new Broker("localhost:9092");//localhost:9092 specified as one string.
-    GlobalPartitionInformation partitionInfo = new GlobalPartitionInformation();
-    partitionInfo.addPartition(0, brokerForPartition0);//mapping from partition 0 to brokerForPartition0
-    partitionInfo.addPartition(1, brokerForPartition1);//mapping from partition 1 to brokerForPartition1
-    partitionInfo.addPartition(2, brokerForPartition2);//mapping from partition 2 to brokerForPartition2
-    StaticHosts hosts = new StaticHosts(partitionInfo);
-```
-
-### KafkaConfig
-The second thing needed for constructing a kafkaSpout is an instance of KafkaConfig.
-```java
-    public KafkaConfig(BrokerHosts hosts, String topic)
-    public KafkaConfig(BrokerHosts hosts, String topic, String clientId)
-```
-
-The BrokerHosts can be any implementation of BrokerHosts interface as described above. The topic is name of Kafka topic.
-The optional ClientId is used as a part of the ZooKeeper path where the spout's current consumption offset is stored.
-
-There are 2 extensions of KafkaConfig currently in use.
-
-Spoutconfig is an extension of KafkaConfig that supports additional fields with ZooKeeper connection info and for controlling
-behavior specific to KafkaSpout. The Zkroot will be used as root to store your consumer's offset. The id should uniquely
-identify your spout.
-```java
-public SpoutConfig(BrokerHosts hosts, String topic, String zkRoot, String id);
-```
-In addition to these parameters, SpoutConfig contains the following fields that control how KafkaSpout behaves:
-```java
-    // setting for how often to save the current Kafka offset to ZooKeeper
-    public long stateUpdateIntervalMs = 2000;
-
-    // Retry strategy for failed messages
-    public String failedMsgRetryManagerClass = ExponentialBackoffMsgRetryManager.class.getName();
-
-    // Exponential back-off retry settings.  These are used by ExponentialBackoffMsgRetryManager for retrying messages after a bolt
-    // calls OutputCollector.fail(). These come into effect only if ExponentialBackoffMsgRetryManager is being used.
-    // Initial delay between successive retries
-    public long retryInitialDelayMs = 0;
-    public double retryDelayMultiplier = 1.0;
-    
-    // Maximum delay between successive retries    
-    public long retryDelayMaxMs = 60 * 1000;
-    // Failed message will be retried infinitely if retryLimit is less than zero. 
-    public int retryLimit = -1;     
-
-```
-Core KafkaSpout only accepts an instance of SpoutConfig.
-
-TridentKafkaConfig is another extension of KafkaConfig.
-TridentKafkaEmitter only accepts TridentKafkaConfig.
-
-The KafkaConfig class also has bunch of public variables that controls your application's behavior. Here are defaults:
-```java
-    public int fetchSizeBytes = 1024 * 1024;
-    public int socketTimeoutMs = 10000;
-    public int fetchMaxWait = 10000;
-    public int bufferSizeBytes = 1024 * 1024;
-    public MultiScheme scheme = new RawMultiScheme();
-    public boolean ignoreZkOffsets = false;
-    public long startOffsetTime = kafka.api.OffsetRequest.EarliestTime();
-    public long maxOffsetBehind = Long.MAX_VALUE;
-    public boolean useStartOffsetTimeIfOffsetOutOfRange = true;
-    public int metricsTimeBucketSizeInSecs = 60;
-```
-
-Most of them are self explanatory except MultiScheme.
-### MultiScheme
-MultiScheme is an interface that dictates how the ByteBuffer consumed from Kafka gets transformed into a storm tuple. It
-also controls the naming of your output field.
-
-```java
-  public Iterable<List<Object>> deserialize(ByteBuffer ser);
-  public Fields getOutputFields();
-```
-
-The default `RawMultiScheme` just takes the `ByteBuffer` and returns a tuple with the ByteBuffer converted to a `byte[]`. The name of the outputField is "bytes". There are alternative implementations like `SchemeAsMultiScheme` and `KeyValueSchemeAsMultiScheme` which can convert the `ByteBuffer` to `String`.
-
-There is also an extension of `SchemeAsMultiScheme`, `MessageMetadataSchemeAsMultiScheme`,
-which has an additional deserialize method that accepts the message `ByteBuffer` in addition to the `Partition` and `offset` associated with the message.
-
-```java
-public Iterable<List<Object>> deserializeMessageWithMetadata(ByteBuffer message, Partition partition, long offset)
-
-```
-
-This is useful for auditing/replaying messages from arbitrary points on a Kafka topic, saving the partition and offset of each message of a discrete stream instead of persisting the entire message.
-
-### Failed message retry
-FailedMsgRetryManager is an interface which defines the retry strategy for a failed message. Default implementation is ExponentialBackoffMsgRetryManager which retries with exponential delays
-between consecutive retries. To use a custom implementation, set SpoutConfig.failedMsgRetryManagerClass to the full classname
-of implementation. Here is the interface 
-
-```java
-    // Spout initialization can go here. This can be called multiple times during lifecycle of a worker. 
-    void prepare(SpoutConfig spoutConfig, Map stormConf);
-
-    // Message corresponding to offset has failed. This method is called only if retryFurther returns true for offset.
-    void failed(Long offset);
-
-    // Message corresponding to offset has been acked.  
-    void acked(Long offset);
-
-    // Message corresponding to the offset, has been re-emitted and under transit.
-    void retryStarted(Long offset);
-
-    /**
-     * The offset of message, which is to be re-emitted. Spout will fetch messages starting from this offset
-     * and resend them, except completed messages.
-     */
-    Long nextFailedMessageToRetry();
-
-    /**
-     * @return True if the message corresponding to the offset should be emitted NOW. False otherwise.
-     */
-    boolean shouldReEmitMsg(Long offset);
-
-    /**
-     * Spout will clean up the state for this offset if false is returned. If retryFurther is set to true,
-     * spout will called failed(offset) in next call and acked(offset) otherwise 
-     */
-    boolean retryFurther(Long offset);
-    
-    /**
-     * Spout will call this method after retryFurther returns false.
-     * This gives a chance for hooking up custom logic before all clean up.
-     * @param partition,offset
-     */
-    void cleanOffsetAfterRetries(Partition partition, Long offset);
-
-    /**
-     * Clear any offsets before kafkaOffset. These offsets are no longer available in kafka.
-     */
-    Set<Long> clearOffsetsBefore(Long kafkaOffset);
-``` 
-
-#### Version incompatibility
-In Storm versions prior to 1.0, the MultiScheme methods accepted a `byte[]` instead of `ByteBuffer`. The `MultScheme` and the related
-Scheme apis were changed in version 1.0 to accept a ByteBuffer instead of a byte[].
-
-This means that pre 1.0 kafka spouts will not work with Storm versions 1.0 and higher. While running topologies in Storm version 1.0
-and higher, it must be ensured that the storm-kafka version is at least 1.0. Pre 1.0 shaded topology jars that bundles
-storm-kafka classes must be rebuilt with storm-kafka version 1.0 for running in clusters with storm 1.0 and higher.
-
-### Examples
-
-#### Core Spout
-
-```java
-BrokerHosts hosts = new ZkHosts(zkConnString);
-SpoutConfig spoutConfig = new SpoutConfig(hosts, topicName, "/" + topicName, UUID.randomUUID().toString());
-spoutConfig.scheme = new SchemeAsMultiScheme(new StringScheme());
-KafkaSpout kafkaSpout = new KafkaSpout(spoutConfig);
-```
-
-#### Trident Spout
-```java
-TridentTopology topology = new TridentTopology();
-BrokerHosts zk = new ZkHosts("localhost");
-TridentKafkaConfig spoutConf = new TridentKafkaConfig(zk, "test-topic");
-spoutConf.scheme = new SchemeAsMultiScheme(new StringScheme());
-OpaqueTridentKafkaSpout spout = new OpaqueTridentKafkaSpout(spoutConf);
-```
-
-
-### How KafkaSpout stores offsets of a Kafka topic and recovers in case of failures
-
-As shown in the above KafkaConfig properties, you can control from where in the Kafka topic the spout begins to read by
-setting `KafkaConfig.startOffsetTime` as follows:
-
-1. `kafka.api.OffsetRequest.EarliestTime()`:  read from the beginning of the topic (i.e. from the oldest messages onwards)
-2. `kafka.api.OffsetRequest.LatestTime()`: read from the end of the topic (i.e. any new messsages that are being written to the topic)
-3. A Unix timestamp aka seconds since the epoch (e.g. via `System.currentTimeMillis()`):
-   see [How do I accurately get offsets of messages for a certain timestamp using OffsetRequest?](https://cwiki.apache.org/confluence/display/KAFKA/FAQ#FAQ-HowdoIaccuratelygetoffsetsofmessagesforacertaintimestampusingOffsetRequest?) in the Kafka FAQ
-
-As the topology runs the Kafka spout keeps track of the offsets it has read and emitted by storing state information
-under the ZooKeeper path `SpoutConfig.zkRoot+ "/" + SpoutConfig.id`.  In the case of failures it recovers from the last
-written offset in ZooKeeper.
-
-> **Important:**  When re-deploying a topology make sure that the settings for `SpoutConfig.zkRoot` and `SpoutConfig.id`
-> were not modified, otherwise the spout will not be able to read its previous consumer state information (i.e. the
-> offsets) from ZooKeeper -- which may lead to unexpected behavior and/or to data loss, depending on your use case.
-
-This means that when a topology has run once the setting `KafkaConfig.startOffsetTime` will not have an effect for
-subsequent runs of the topology because now the topology will rely on the consumer state information (offsets) in
-ZooKeeper to determine from where it should begin (more precisely: resume) reading.
-If you want to force the spout to ignore any consumer state information stored in ZooKeeper, then you should
-set the parameter `KafkaConfig.ignoreZkOffsets` to `true`.  If `true`, the spout will always begin reading from the
-offset defined by `KafkaConfig.startOffsetTime` as described above.
-
-
-## Using storm-kafka with different versions of kafka
-
-Storm-kafka's Kafka dependency is defined as `provided` scope in maven, meaning it will not be pulled in
-as a transitive dependency. This allows you to use a version of Kafka dependency compatible with your kafka cluster.
-
-When building a project with storm-kafka, you must explicitly add the Kafka dependency. For example, to
-use Kafka 0.8.1.1 built against Scala 2.10, you would use the following dependency in your `pom.xml`:
-
-```xml
-        <dependency>
-            <groupId>org.apache.kafka</groupId>
-            <artifactId>kafka_2.10</artifactId>
-            <version>0.8.1.1</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.apache.zookeeper</groupId>
-                    <artifactId>zookeeper</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>log4j</groupId>
-                    <artifactId>log4j</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-```
-
-Note that the ZooKeeper and log4j dependencies are excluded to prevent version conflicts with Storm's dependencies.
-
-You can also override the kafka dependency version while building from maven, with parameter `kafka.version` and `kafka.artifact.id`
-e.g. `mvn clean install -Dkafka.artifact.id=kafka_2.11 -Dkafka.version=0.9.0.1`
-
-When selecting a kafka dependency version, you should ensure - 
- 1. kafka api is compatible with storm-kafka. Currently, only 0.9.x and 0.8.x client API is supported by storm-kafka 
- module. If you want to use a higher version, storm-kafka-client module should be used instead.
- 2. The kafka client selected by you should be wire compatible with the broker. e.g. 0.9.x client will not work with 
- 0.8.x broker. 
-
-
-## Writing to Kafka as part of your topology
-You can create an instance of org.apache.storm.kafka.bolt.KafkaBolt and attach it as a component to your topology or if you
-are using trident you can use org.apache.storm.kafka.trident.TridentState, org.apache.storm.kafka.trident.TridentStateFactory and
-org.apache.storm.kafka.trident.TridentKafkaUpdater.
-
-You need to provide implementation of following 2 interfaces
-
-### TupleToKafkaMapper and TridentTupleToKafkaMapper
-These interfaces have 2 methods defined:
-
-```java
-    K getKeyFromTuple(Tuple/TridentTuple tuple);
-    V getMessageFromTuple(Tuple/TridentTuple tuple);
-```
-
-As the name suggests, these methods are called to map a tuple to Kafka key and Kafka message. If you just want one field
-as key and one field as value, then you can use the provided FieldNameBasedTupleToKafkaMapper.java
-implementation. In the KafkaBolt, the implementation always looks for a field with field name "key" and "message" if you
-use the default constructor to construct FieldNameBasedTupleToKafkaMapper for backward compatibility
-reasons. Alternatively you could also specify a different key and message field by using the non default constructor.
-In the TridentKafkaState you must specify what is the field name for key and message as there is no default constructor.
-These should be specified while constructing and instance of FieldNameBasedTupleToKafkaMapper.
-
-### KafkaTopicSelector and trident KafkaTopicSelector
-This interface has only one method
-```java
-public interface KafkaTopicSelector {
-    String getTopics(Tuple/TridentTuple tuple);
-}
-```
-The implementation of this interface should return the topic to which the tuple's key/message mapping needs to be published
-You can return a null and the message will be ignored. If you have one static topic name then you can use
-DefaultTopicSelector.java and set the name of the topic in the constructor.
-`FieldNameTopicSelector` and `FieldIndexTopicSelector` use to support decided which topic should to push message from tuple.
-User could specify the field name or field index in tuple ,selector will use this value as topic name which to publish message.
-When the topic name not found , `KafkaBolt` will write messages into default topic .
-Please make sure the default topic have created .
-
-### Specifying Kafka producer properties
-You can provide all the produce properties in your Storm topology by calling `KafkaBolt.withProducerProperties()` and `TridentKafkaStateFactory.withProducerProperties()`. Please see  http://kafka.apache.org/documentation.html#newproducerconfigs
-Section "Important configuration properties for the producer" for more details.
-
-### Using wildcard kafka topic match
-You can do a wildcard topic match by adding the following config
-```
-     Config config = new Config();
-     config.put("kafka.topic.wildcard.match",true);
-
-```
-
-After this you can specify a wildcard topic for matching e.g. clickstream.*.log.  This will match all streams matching clickstream.my.log, clickstream.cart.log etc
-
-
-### Putting it all together
-
-For the bolt :
-```java
-        TopologyBuilder builder = new TopologyBuilder();
-
-        Fields fields = new Fields("key", "message");
-        FixedBatchSpout spout = new FixedBatchSpout(fields, 4,
-                    new Values("storm", "1"),
-                    new Values("trident", "1"),
-                    new Values("needs", "1"),
-                    new Values("javadoc", "1")
-        );
-        spout.setCycle(true);
-        builder.setSpout("spout", spout, 5);
-        //set producer properties.
-        Properties props = new Properties();
-        props.put("bootstrap.servers", "localhost:9092");
-        props.put("acks", "1");
-        props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-        props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-
-        KafkaBolt bolt = new KafkaBolt()
-                .withProducerProperties(props)
-                .withTopicSelector(new DefaultTopicSelector("test"))
-                .withTupleToKafkaMapper(new FieldNameBasedTupleToKafkaMapper());
-        builder.setBolt("forwardToKafka", bolt, 8).shuffleGrouping("spout");
-
-        Config conf = new Config();
-
-        StormSubmitter.submitTopology("kafkaboltTest", conf, builder.createTopology());
-```
-
-For Trident:
-
-```java
-        Fields fields = new Fields("word", "count");
-        FixedBatchSpout spout = new FixedBatchSpout(fields, 4,
-                new Values("storm", "1"),
-                new Values("trident", "1"),
-                new Values("needs", "1"),
-                new Values("javadoc", "1")
-        );
-        spout.setCycle(true);
-
-        TridentTopology topology = new TridentTopology();
-        Stream stream = topology.newStream("spout1", spout);
-
-        //set producer properties.
-        Properties props = new Properties();
-        props.put("bootstrap.servers", "localhost:9092");
-        props.put("acks", "1");
-        props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-        props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-
-        TridentKafkaStateFactory stateFactory = new TridentKafkaStateFactory()
-                .withProducerProperties(props)
-                .withKafkaTopicSelector(new DefaultTopicSelector("test"))
-                .withTridentTupleToKafkaMapper(new FieldNameBasedTupleToKafkaMapper("word", "count"));
-        stream.partitionPersist(stateFactory, fields, new TridentKafkaUpdater(), new Fields());
-
-        Config conf = new Config();
-        StormSubmitter.submitTopology("kafkaTridentTest", conf, topology.build());
-```
-
-## Committer Sponsors
-
- * P. Taylor Goetz ([ptgoetz@apache.org](mailto:ptgoetz@apache.org))

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml
deleted file mode 100644
index 2e0f35b..0000000
--- a/external/storm-kafka/pom.xml
+++ /dev/null
@@ -1,125 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- 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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <artifactId>storm</artifactId>
-        <groupId>org.apache.storm</groupId>
-        <version>2.0.0-SNAPSHOT</version>
-        <relativePath>../../pom.xml</relativePath>
-    </parent>
-
-    <packaging>jar</packaging>
-    <artifactId>storm-kafka</artifactId>
-    <name>storm-kafka</name>
-    <description>Storm Spouts for Apache Kafka</description>
-
-    <build>
-        <plugins>
-            <plugin>
-                <groupId>org.jacoco</groupId>
-                <artifactId>jacoco-maven-plugin</artifactId>
-                <version>0.7.2.201409121644</version>
-                <executions>
-                    <execution>
-                        <id>jacoco-initialize</id>
-                        <goals>
-                            <goal>prepare-agent</goal>
-                        </goals>
-                    </execution>
-                    <execution>
-                        <id>jacoco-report</id>
-                        <phase>test</phase>
-                        <goals>
-                            <goal>report</goal>
-                        </goals>
-                    </execution>
-                </executions>
-            </plugin>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-checkstyle-plugin</artifactId>
-                <!--Note - the version would be inherited-->
-                <configuration>
-                    <maxAllowedViolations>180</maxAllowedViolations>
-                </configuration>
-            </plugin>
-        </plugins>
-        <sourceDirectory>src/jvm</sourceDirectory>
-        <testSourceDirectory>src/test</testSourceDirectory>
-    </build>
-    <dependencies>
-        <dependency>
-            <groupId>commons-io</groupId>
-            <artifactId>commons-io</artifactId>
-            <scope>compile</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.mockito</groupId>
-            <artifactId>mockito-core</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.hamcrest</groupId>
-            <artifactId>java-hamcrest</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.curator</groupId>
-            <artifactId>curator-framework</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.zookeeper</groupId>
-            <artifactId>zookeeper</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.curator</groupId>
-            <artifactId>curator-recipes</artifactId>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.curator</groupId>
-            <artifactId>curator-test</artifactId>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.kafka</groupId>
-            <artifactId>${storm.kafka.artifact.id}</artifactId>
-            <version>${storm.kafka.version}</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.kafka</groupId>
-            <artifactId>kafka-clients</artifactId>
-            <version>${storm.kafka.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-client</artifactId>
-            <version>${project.version}</version>
-            <scope>${provided.scope}</scope>
-        </dependency>
-        <dependency>
-            <groupId>commons-lang</groupId>
-            <artifactId>commons-lang</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.googlecode.json-simple</groupId>
-            <artifactId>json-simple</artifactId>
-        </dependency>
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/Broker.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/Broker.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/Broker.java
deleted file mode 100644
index b33af99..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/Broker.java
+++ /dev/null
@@ -1,79 +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;
-
-import com.google.common.base.Objects;
-import java.io.Serializable;
-
-public class Broker implements Serializable, Comparable<Broker> {
-    public String host;
-    public int port;
-
-    // for kryo compatibility
-    private Broker() {
-
-    }
-
-    public Broker(String host, int port) {
-        this.host = host;
-        this.port = port;
-    }
-
-    public Broker(String host) {
-        this(host, 9092);
-    }
-
-    public static Broker fromString(String host) {
-        Broker hp;
-        String[] spec = host.split(":");
-        if (spec.length == 1) {
-            hp = new Broker(spec[0]);
-        } else if (spec.length == 2) {
-            hp = new Broker(spec[0], Integer.parseInt(spec[1]));
-        } else {
-            throw new IllegalArgumentException("Invalid host specification: " + host);
-        }
-        return hp;
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hashCode(host, port);
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj) {
-            return true;
-        }
-        if (obj == null || getClass() != obj.getClass()) {
-            return false;
-        }
-        final Broker other = (Broker) obj;
-        return Objects.equal(this.host, other.host) && Objects.equal(this.port, other.port);
-    }
-
-    @Override
-    public String toString() {
-        return host + ":" + port;
-    }
-
-    @Override
-    public int compareTo(Broker o) {
-        if (this.host.equals(o.host)) {
-            return this.port - o.port;
-        } else {
-            return this.host.compareTo(o.host);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/BrokerHosts.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/BrokerHosts.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/BrokerHosts.java
deleted file mode 100644
index dbd6a10..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/BrokerHosts.java
+++ /dev/null
@@ -1,20 +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;
-
-import java.io.Serializable;
-
-
-public interface BrokerHosts extends Serializable {
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/ByteBufferSerializer.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/ByteBufferSerializer.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/ByteBufferSerializer.java
deleted file mode 100644
index 37986a5..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/ByteBufferSerializer.java
+++ /dev/null
@@ -1,41 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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;
-
-import java.nio.ByteBuffer;
-import java.util.Map;
-import org.apache.kafka.common.serialization.Serializer;
-import org.apache.storm.utils.Utils;
-
-public class ByteBufferSerializer implements Serializer<ByteBuffer> {
-    @Override
-    public void configure(Map<String, ?> map, boolean b) {
-
-    }
-
-    @Override
-    public void close() {
-
-    }
-
-    @Override
-    public byte[] serialize(String s, ByteBuffer b) {
-        return Utils.toByteArray(b);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicBrokersReader.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicBrokersReader.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicBrokersReader.java
deleted file mode 100644
index 49ad530..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicBrokersReader.java
+++ /dev/null
@@ -1,208 +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;
-
-import com.google.common.base.Preconditions;
-import java.net.SocketTimeoutException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.retry.RetryNTimes;
-import org.apache.storm.Config;
-import org.apache.storm.kafka.trident.GlobalPartitionInformation;
-import org.apache.storm.utils.ObjectReader;
-import org.json.simple.JSONValue;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-public class DynamicBrokersReader {
-
-    private static final Logger LOG = LoggerFactory.getLogger(DynamicBrokersReader.class);
-
-    private CuratorFramework _curator;
-    private String _zkPath;
-    private String _topic;
-    private Boolean _isWildcardTopic;
-
-    public DynamicBrokersReader(Map<String, Object> conf, String zkStr, String zkPath, String topic) {
-        // Check required parameters
-        Preconditions.checkNotNull(conf, "conf cannot be null");
-
-        validateConfig(conf);
-
-        Preconditions.checkNotNull(zkStr, "zkString cannot be null");
-        Preconditions.checkNotNull(zkPath, "zkPath cannot be null");
-        Preconditions.checkNotNull(topic, "topic cannot be null");
-
-        _zkPath = zkPath;
-        _topic = topic;
-        _isWildcardTopic = ObjectReader.getBoolean(conf.get("kafka.topic.wildcard.match"), false);
-        try {
-            _curator = CuratorFrameworkFactory.newClient(
-                zkStr,
-                ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT)),
-                ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT)),
-                new RetryNTimes(ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES)),
-                                ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL))));
-            _curator.start();
-        } catch (Exception ex) {
-            LOG.error("Couldn't connect to zookeeper", ex);
-            throw new RuntimeException(ex);
-        }
-    }
-
-    /**
-     * Get all partitions with their current leaders
-     */
-    public List<GlobalPartitionInformation> getBrokerInfo() throws SocketTimeoutException {
-        List<String> topics = getTopics();
-        List<GlobalPartitionInformation> partitions = new ArrayList<GlobalPartitionInformation>();
-
-        for (String topic : topics) {
-            GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation(topic, this._isWildcardTopic);
-            try {
-                int numPartitionsForTopic = getNumPartitions(topic);
-                String brokerInfoPath = brokerPath();
-                for (int partition = 0; partition < numPartitionsForTopic; partition++) {
-                    int leader = getLeaderFor(topic, partition);
-                    String path = brokerInfoPath + "/" + leader;
-                    try {
-                        byte[] brokerData = _curator.getData().forPath(path);
-                        Broker hp = getBrokerHost(brokerData);
-                        globalPartitionInformation.addPartition(partition, hp);
-                    } catch (org.apache.zookeeper.KeeperException.NoNodeException e) {
-                        LOG.error("Node {} does not exist ", path);
-                    }
-                }
-            } catch (SocketTimeoutException e) {
-                throw e;
-            } catch (Exception e) {
-                throw new RuntimeException(e);
-            }
-            LOG.info("Read partition info from zookeeper: " + globalPartitionInformation);
-            partitions.add(globalPartitionInformation);
-        }
-        return partitions;
-    }
-
-    private int getNumPartitions(String topic) {
-        try {
-            String topicBrokersPath = partitionPath(topic);
-            List<String> children = _curator.getChildren().forPath(topicBrokersPath);
-            return children.size();
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private List<String> getTopics() {
-        List<String> topics = new ArrayList<String>();
-        if (!_isWildcardTopic) {
-            topics.add(_topic);
-            return topics;
-        } else {
-            try {
-                List<String> children = _curator.getChildren().forPath(topicsPath());
-                for (String t : children) {
-                    if (t.matches(_topic)) {
-                        LOG.info(String.format("Found matching topic %s", t));
-                        topics.add(t);
-                    }
-                }
-                return topics;
-            } catch (Exception e) {
-                throw new RuntimeException(e);
-            }
-        }
-    }
-
-    public String topicsPath() {
-        return _zkPath + "/topics";
-    }
-
-    public String partitionPath(String topic) {
-        return topicsPath() + "/" + topic + "/partitions";
-    }
-
-    public String brokerPath() {
-        return _zkPath + "/ids";
-    }
-
-
-    /**
-     * get /brokers/topics/distributedTopic/partitions/1/state
-     * { "controller_epoch":4, "isr":[ 1, 0 ], "leader":1, "leader_epoch":1, "version":1 }
-     * @param topic
-     * @param partition
-     * @return
-     */
-    private int getLeaderFor(String topic, long partition) {
-        try {
-            String topicBrokersPath = partitionPath(topic);
-            byte[] hostPortData = _curator.getData().forPath(topicBrokersPath + "/" + partition + "/state");
-            Map<Object, Object> value = (Map<Object, Object>) JSONValue.parseWithException(new String(hostPortData, "UTF-8"));
-            Integer leader = ((Number) value.get("leader")).intValue();
-            if (leader == -1) {
-                throw new RuntimeException("No leader found for partition " + partition);
-            }
-            return leader;
-        } catch (RuntimeException e) {
-            throw e;
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public void close() {
-        _curator.close();
-    }
-
-    /**
-     * [zk: localhost:2181(CONNECTED) 56] get /brokers/ids/0
-     * { "host":"localhost", "jmx_port":9999, "port":9092, "version":1 }
-     *
-     * @param contents
-     * @return
-     */
-    private Broker getBrokerHost(byte[] contents) {
-        try {
-            Map<Object, Object> value = (Map<Object, Object>) JSONValue.parseWithException(new String(contents, "UTF-8"));
-            String host = (String) value.get("host");
-            Integer port = ((Long) value.get("port")).intValue();
-            return new Broker(host, port);
-        } catch (RuntimeException e) {
-            throw e;
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * Validate required parameters in the input configuration Map
-     * @param conf
-     */
-    private void validateConfig(final Map<String, Object> conf) {
-        Preconditions.checkNotNull(conf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT),
-                                   "%s cannot be null", Config.STORM_ZOOKEEPER_SESSION_TIMEOUT);
-        Preconditions.checkNotNull(conf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT),
-                                   "%s cannot be null", Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT);
-        Preconditions.checkNotNull(conf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES),
-                                   "%s cannot be null", Config.STORM_ZOOKEEPER_RETRY_TIMES);
-        Preconditions.checkNotNull(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL),
-                                   "%s cannot be null", Config.STORM_ZOOKEEPER_RETRY_INTERVAL);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicPartitionConnections.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicPartitionConnections.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicPartitionConnections.java
deleted file mode 100644
index 1ca7144..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/DynamicPartitionConnections.java
+++ /dev/null
@@ -1,91 +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;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import kafka.javaapi.consumer.SimpleConsumer;
-import org.apache.storm.kafka.trident.IBrokerReader;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-public class DynamicPartitionConnections {
-
-    private static final Logger LOG = LoggerFactory.getLogger(DynamicPartitionConnections.class);
-    Map<Broker, ConnectionInfo> _connections = new HashMap<>();
-    KafkaConfig _config;
-    IBrokerReader _reader;
-    public DynamicPartitionConnections(KafkaConfig config, IBrokerReader brokerReader) {
-        _config = config;
-        _reader = brokerReader;
-    }
-
-    public SimpleConsumer register(Partition partition) {
-        Broker broker = _reader.getBrokerForTopic(partition.topic).getBrokerFor(partition.partition);
-        return register(broker, partition.topic, partition.partition);
-    }
-
-    public SimpleConsumer register(Broker host, String topic, int partition) {
-        if (!_connections.containsKey(host)) {
-            _connections.put(host, new ConnectionInfo(
-                new SimpleConsumer(host.host, host.port, _config.socketTimeoutMs, _config.bufferSizeBytes, _config.clientId)));
-        }
-        ConnectionInfo info = _connections.get(host);
-        info.partitions.add(getHashKey(topic, partition));
-        return info.consumer;
-    }
-
-    public SimpleConsumer getConnection(Partition partition) {
-        ConnectionInfo info = _connections.get(partition.host);
-        if (info != null) {
-            return info.consumer;
-        }
-        return null;
-    }
-
-    public void unregister(Broker port, String topic, int partition) {
-        ConnectionInfo info = _connections.get(port);
-        info.partitions.remove(getHashKey(topic, partition));
-        if (info.partitions.isEmpty()) {
-            info.consumer.close();
-            _connections.remove(port);
-        }
-    }
-
-    public void unregister(Partition partition) {
-        unregister(partition.host, partition.topic, partition.partition);
-    }
-
-    public void clear() {
-        for (ConnectionInfo info : _connections.values()) {
-            info.consumer.close();
-        }
-        _connections.clear();
-    }
-
-    private String getHashKey(String topic, int partition) {
-        return topic + "_" + partition;
-    }
-
-    static class ConnectionInfo {
-        SimpleConsumer consumer;
-        Set<String> partitions = new HashSet<String>();
-
-        public ConnectionInfo(SimpleConsumer consumer) {
-            this.consumer = consumer;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/ExponentialBackoffMsgRetryManager.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/ExponentialBackoffMsgRetryManager.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/ExponentialBackoffMsgRetryManager.java
deleted file mode 100644
index 2651c30..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/ExponentialBackoffMsgRetryManager.java
+++ /dev/null
@@ -1,201 +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;
-
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.PriorityQueue;
-import java.util.Queue;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import org.apache.storm.utils.Time;
-
-public class ExponentialBackoffMsgRetryManager implements FailedMsgRetryManager {
-
-    private long retryInitialDelayMs;
-    private double retryDelayMultiplier;
-    private long retryDelayMaxMs;
-    private int retryLimit;
-
-    private Queue<MessageRetryRecord> waiting;
-    private Map<Long, MessageRetryRecord> records;
-
-    public ExponentialBackoffMsgRetryManager() {
-
-    }
-
-    public void prepare(SpoutConfig spoutConfig, Map<String, Object> topoConf) {
-        this.retryInitialDelayMs = spoutConfig.retryInitialDelayMs;
-        this.retryDelayMultiplier = spoutConfig.retryDelayMultiplier;
-        this.retryDelayMaxMs = spoutConfig.retryDelayMaxMs;
-        this.retryLimit = spoutConfig.retryLimit;
-        this.waiting = new PriorityQueue<MessageRetryRecord>(11, new RetryTimeComparator());
-        this.records = new ConcurrentHashMap<Long, MessageRetryRecord>();
-    }
-
-    @Override
-    public void failed(Long offset) {
-        MessageRetryRecord oldRecord = this.records.get(offset);
-        MessageRetryRecord newRecord = oldRecord == null ?
-            new MessageRetryRecord(offset) :
-            oldRecord.createNextRetryRecord();
-        this.records.put(offset, newRecord);
-        this.waiting.add(newRecord);
-    }
-
-    @Override
-    public void acked(Long offset) {
-        MessageRetryRecord record = this.records.remove(offset);
-        if (record != null) {
-            this.waiting.remove(record);
-        }
-    }
-
-    @Override
-    public void retryStarted(Long offset) {
-        MessageRetryRecord record = this.records.get(offset);
-        if (record == null || !this.waiting.contains(record)) {
-            throw new IllegalStateException("cannot retry a message that has not failed");
-        } else {
-            this.waiting.remove(record);
-        }
-    }
-
-    @Override
-    public Long nextFailedMessageToRetry() {
-        if (this.waiting.size() > 0) {
-            MessageRetryRecord first = this.waiting.peek();
-            if (Time.currentTimeMillis() >= first.retryTimeUTC) {
-                if (this.records.containsKey(first.offset)) {
-                    return first.offset;
-                } else {
-                    // defensive programming - should be impossible
-                    this.waiting.remove(first);
-                    return nextFailedMessageToRetry();
-                }
-            }
-        }
-        return null;
-    }
-
-    @Override
-    public boolean shouldReEmitMsg(Long offset) {
-        MessageRetryRecord record = this.records.get(offset);
-        return record != null &&
-               this.waiting.contains(record) &&
-               Time.currentTimeMillis() >= record.retryTimeUTC;
-    }
-
-    @Override
-    public boolean retryFurther(Long offset) {
-        MessageRetryRecord record = this.records.get(offset);
-        return !(record != null &&
-                 this.retryLimit > 0 &&
-                 this.retryLimit <= record.retryNum);
-    }
-
-    @Override
-    public void cleanOffsetAfterRetries(Partition partition, Long offset) {
-        //Do nothing..
-    }
-
-    @Override
-    public Set<Long> clearOffsetsBefore(Long kafkaOffset) {
-        Set<Long> invalidOffsets = new HashSet<Long>();
-        for (Long offset : records.keySet()) {
-            if (offset < kafkaOffset) {
-                MessageRetryRecord record = this.records.remove(offset);
-                if (record != null) {
-                    this.waiting.remove(record);
-                    invalidOffsets.add(offset);
-                }
-            }
-        }
-        return invalidOffsets;
-    }
-
-    private static class RetryTimeComparator implements Comparator<MessageRetryRecord> {
-
-        @Override
-        public int compare(MessageRetryRecord record1, MessageRetryRecord record2) {
-            return Long.valueOf(record1.retryTimeUTC).compareTo(Long.valueOf(record2.retryTimeUTC));
-        }
-
-        @Override
-        public boolean equals(Object obj) {
-            return false;
-        }
-    }
-
-    /**
-     * A MessageRetryRecord holds the data of how many times a message has
-     * failed and been retried, and when the last failure occurred.  It can
-     * determine whether it is ready to be retried by employing an exponential
-     * back-off calculation using config values stored in SpoutConfig:
-     * <ul>
-     *  <li>retryInitialDelayMs - time to delay before the first retry</li>
-     *  <li>retryDelayMultiplier - multiplier by which to increase the delay for each subsequent retry</li>
-     *  <li>retryDelayMaxMs - maximum retry delay (once this delay time is reached, subsequent retries will
-     *                        delay for this amount of time every time)
-     *  </li>
-     * </ul>
-     */
-    private class MessageRetryRecord {
-        private final long offset;
-        private final int retryNum;
-        private final long retryTimeUTC;
-
-        public MessageRetryRecord(long offset) {
-            this(offset, 1);
-        }
-
-        private MessageRetryRecord(long offset, int retryNum) {
-            this.offset = offset;
-            this.retryNum = retryNum;
-            this.retryTimeUTC = Time.currentTimeMillis() + calculateRetryDelay();
-        }
-
-        /**
-         * Create a MessageRetryRecord for the next retry that should occur after this one.
-         * @return MessageRetryRecord with the next retry time, or null to indicate that another
-         *         retry should not be performed.  The latter case can happen if we are about to
-         *         run into the org.apache.storm.Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS in the Storm
-         *         configuration.
-         */
-        public MessageRetryRecord createNextRetryRecord() {
-            return new MessageRetryRecord(this.offset, this.retryNum + 1);
-        }
-
-        private long calculateRetryDelay() {
-            double delayMultiplier = Math.pow(retryDelayMultiplier, this.retryNum - 1);
-            double delay = retryInitialDelayMs * delayMultiplier;
-            Long maxLong = Long.MAX_VALUE;
-            long delayThisRetryMs = delay >= maxLong.doubleValue()
-                ? maxLong
-                : (long) delay;
-            return Math.min(delayThisRetryMs, retryDelayMaxMs);
-        }
-
-        @Override
-        public boolean equals(Object other) {
-            return (other instanceof MessageRetryRecord
-                    && this.offset == ((MessageRetryRecord) other).offset);
-        }
-
-        @Override
-        public int hashCode() {
-            return Long.valueOf(this.offset).hashCode();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/FailedFetchException.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/FailedFetchException.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/FailedFetchException.java
deleted file mode 100644
index a1f3fe5..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/FailedFetchException.java
+++ /dev/null
@@ -1,24 +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;
-
-public class FailedFetchException extends RuntimeException {
-
-    public FailedFetchException(String message) {
-        super(message);
-    }
-
-    public FailedFetchException(Exception e) {
-        super(e);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/FailedMsgRetryManager.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/FailedMsgRetryManager.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/FailedMsgRetryManager.java
deleted file mode 100644
index b7fafdc..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/FailedMsgRetryManager.java
+++ /dev/null
@@ -1,77 +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;
-
-import java.io.Serializable;
-import java.util.Map;
-import java.util.Set;
-
-public interface FailedMsgRetryManager extends Serializable {
-
-    /**
-     * Initialization
-     */
-    void prepare(SpoutConfig spoutConfig, Map<String, Object> topoConf);
-
-    /**
-     * Message corresponding to the offset failed in kafka spout.
-     * @param offset
-     */
-    void failed(Long offset);
-
-    /**
-     * Message corresponding to the offset, was acked to kafka spout.
-     * @param offset
-     */
-    void acked(Long offset);
-
-    /**
-     * Message corresponding to the offset, has been re-emitted and under transit.
-     * @param offset
-     */
-    void retryStarted(Long offset);
-
-    /**
-     * The offset of message, which is to be re-emitted. Spout will fetch messages starting from this offset
-     * and resend them, except completed messages.
-     * @return
-     */
-    Long nextFailedMessageToRetry();
-
-    /**
-     * @param offset
-     * @return True if the message corresponding to the offset should be emitted NOW. False otherwise.
-     */
-    boolean shouldReEmitMsg(Long offset);
-
-    /**
-     * Spout will clean up the state for this offset if false is returned.
-     * @param offset
-     * @return True if the message will be retried again. False otherwise.
-     */
-    boolean retryFurther(Long offset);
-
-    /**
-     * Spout will call this method after retryFurther returns false.
-     * This gives a chance for hooking up custom logic before all clean up.
-     * @param partition,offset
-     */
-    void cleanOffsetAfterRetries(Partition partition, Long offset);
-
-    /**
-     * Clear any offsets before kafkaOffset. These offsets are no longer available in kafka.
-     * @param kafkaOffset
-     * @return Set of offsets removed.
-     */
-    Set<Long> clearOffsetsBefore(Long kafkaOffset);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/IntSerializer.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/IntSerializer.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/IntSerializer.java
deleted file mode 100644
index fef8625..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/IntSerializer.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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;
-
-import java.util.Map;
-import org.apache.kafka.common.serialization.Serializer;
-
-public class IntSerializer implements Serializer<Integer> {
-    @Override
-    public void configure(Map<String, ?> map, boolean b) {
-    }
-
-    @Override
-    public byte[] serialize(String topic, Integer val) {
-        return new byte[]{
-            (byte) (val >>> 24),
-            (byte) (val >>> 16),
-            (byte) (val >>> 8),
-            val.byteValue()
-        };
-    }
-
-    @Override
-    public void close() {
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaConfig.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaConfig.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaConfig.java
deleted file mode 100644
index a93f426..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaConfig.java
+++ /dev/null
@@ -1,49 +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;
-
-import java.io.Serializable;
-import kafka.api.FetchRequest;
-import org.apache.storm.spout.MultiScheme;
-import org.apache.storm.spout.RawMultiScheme;
-
-public class KafkaConfig implements Serializable {
-    private static final long serialVersionUID = 5276718734571623855L;
-
-    public final BrokerHosts hosts;
-    public final String topic;
-    public final String clientId;
-
-    public int fetchSizeBytes = 1024 * 1024;
-    public int socketTimeoutMs = 10000;
-    public int fetchMaxWait = 100;
-    public int bufferSizeBytes = 1024 * 1024;
-    public MultiScheme scheme = new RawMultiScheme();
-    public boolean ignoreZkOffsets = false;
-    public long startOffsetTime = kafka.api.OffsetRequest.EarliestTime();
-    public long maxOffsetBehind = Long.MAX_VALUE;
-    public boolean useStartOffsetTimeIfOffsetOutOfRange = true;
-    public int metricsTimeBucketSizeInSecs = 60;
-    public int minFetchByte = FetchRequest.DefaultMinBytes();
-
-    public KafkaConfig(BrokerHosts hosts, String topic) {
-        this(hosts, topic, kafka.api.OffsetRequest.DefaultClientId());
-    }
-
-    public KafkaConfig(BrokerHosts hosts, String topic, String clientId) {
-        this.hosts = hosts;
-        this.topic = topic;
-        this.clientId = clientId;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaError.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaError.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaError.java
deleted file mode 100644
index 4bf2ed2..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaError.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.  The ASF licenses this file to you under the Apache License, Version
- * 2.0 (the "License"); you may not use this file except in compliance with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions
- * and limitations under the License.
- */
-
-package org.apache.storm.kafka;
-
-public enum KafkaError {
-    NO_ERROR,
-    OFFSET_OUT_OF_RANGE,
-    INVALID_MESSAGE,
-    UNKNOWN_TOPIC_OR_PARTITION,
-    INVALID_FETCH_SIZE,
-    LEADER_NOT_AVAILABLE,
-    NOT_LEADER_FOR_PARTITION,
-    REQUEST_TIMED_OUT,
-    BROKER_NOT_AVAILABLE,
-    REPLICA_NOT_AVAILABLE,
-    MESSAGE_SIZE_TOO_LARGE,
-    STALE_CONTROLLER_EPOCH,
-    OFFSET_METADATA_TOO_LARGE,
-    UNKNOWN;
-
-    public static KafkaError getError(int errorCode) {
-        if (errorCode < 0 || errorCode >= UNKNOWN.ordinal()) {
-            return UNKNOWN;
-        } else {
-            return values()[errorCode];
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaSpout.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaSpout.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaSpout.java
deleted file mode 100644
index aed5986..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaSpout.java
+++ /dev/null
@@ -1,255 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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;
-
-import com.google.common.base.Strings;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.storm.Config;
-import org.apache.storm.kafka.PartitionManager.KafkaMessageId;
-import org.apache.storm.kafka.trident.GlobalPartitionInformation;
-import org.apache.storm.metric.api.IMetric;
-import org.apache.storm.spout.SpoutOutputCollector;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseRichSpout;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-// TODO: need to add blacklisting
-// TODO: need to make a best effort to not re-emit messages if don't have to
-public class KafkaSpout extends BaseRichSpout {
-    private static final Logger LOG = LoggerFactory.getLogger(KafkaSpout.class);
-    SpoutConfig _spoutConfig;
-    SpoutOutputCollector _collector;
-    PartitionCoordinator _coordinator;
-    DynamicPartitionConnections _connections;
-    ZkState _state;
-    long _lastUpdateMs = 0;
-    int _currPartitionIndex = 0;
-
-    public KafkaSpout(SpoutConfig spoutConf) {
-        _spoutConfig = spoutConf;
-    }
-
-    @Override
-    public void open(Map<String, Object> conf, final TopologyContext context, final SpoutOutputCollector collector) {
-        _collector = collector;
-        String topologyInstanceId = context.getStormId();
-        Map<String, Object> stateConf = new HashMap<>(conf);
-        List<String> zkServers = _spoutConfig.zkServers;
-        if (zkServers == null) {
-            zkServers = (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS);
-        }
-        Integer zkPort = _spoutConfig.zkPort;
-        if (zkPort == null) {
-            zkPort = ((Number) conf.get(Config.STORM_ZOOKEEPER_PORT)).intValue();
-        }
-        stateConf.put(Config.TRANSACTIONAL_ZOOKEEPER_SERVERS, zkServers);
-        stateConf.put(Config.TRANSACTIONAL_ZOOKEEPER_PORT, zkPort);
-        stateConf.put(Config.TRANSACTIONAL_ZOOKEEPER_ROOT, _spoutConfig.zkRoot);
-        _state = new ZkState(stateConf);
-
-        _connections = new DynamicPartitionConnections(_spoutConfig, KafkaUtils.makeBrokerReader(conf, _spoutConfig));
-
-        // using TransactionalState like this is a hack
-        int totalTasks = context.getComponentTasks(context.getThisComponentId()).size();
-        if (_spoutConfig.hosts instanceof StaticHosts) {
-            _coordinator = new StaticCoordinator(_connections, conf,
-                                                 _spoutConfig, _state, context.getThisTaskIndex(),
-                                                 totalTasks, context.getThisTaskId(), topologyInstanceId);
-        } else {
-            _coordinator = new ZkCoordinator(_connections, conf,
-                                             _spoutConfig, _state, context.getThisTaskIndex(),
-                                             totalTasks, context.getThisTaskId(), topologyInstanceId);
-        }
-
-        context.registerMetric("kafkaOffset", new IMetric() {
-            KafkaUtils.KafkaOffsetMetric _kafkaOffsetMetric = new KafkaUtils.KafkaOffsetMetric(_connections);
-
-            @Override
-            public Object getValueAndReset() {
-                List<PartitionManager> pms = _coordinator.getMyManagedPartitions();
-                Set<Partition> latestPartitions = new HashSet();
-                for (PartitionManager pm : pms) {
-                    latestPartitions.add(pm.getPartition());
-                }
-                _kafkaOffsetMetric.refreshPartitions(latestPartitions);
-                for (PartitionManager pm : pms) {
-                    _kafkaOffsetMetric.setOffsetData(pm.getPartition(), pm.getOffsetData());
-                }
-                return _kafkaOffsetMetric.getValueAndReset();
-            }
-        }, _spoutConfig.metricsTimeBucketSizeInSecs);
-
-        context.registerMetric("kafkaPartition", new IMetric() {
-            @Override
-            public Object getValueAndReset() {
-                List<PartitionManager> pms = _coordinator.getMyManagedPartitions();
-                Map<String, Object> concatMetricsDataMaps = new HashMap<>();
-                for (PartitionManager pm : pms) {
-                    concatMetricsDataMaps.putAll(pm.getMetricsDataMap());
-                }
-                return concatMetricsDataMaps;
-            }
-        }, _spoutConfig.metricsTimeBucketSizeInSecs);
-    }
-
-    @Override
-    public void close() {
-        _state.close();
-    }
-
-    @Override
-    public void nextTuple() {
-        List<PartitionManager> managers = _coordinator.getMyManagedPartitions();
-        for (int i = 0; i < managers.size(); i++) {
-
-            try {
-                // in case the number of managers decreased
-                _currPartitionIndex = _currPartitionIndex % managers.size();
-                EmitState state = managers.get(_currPartitionIndex).next(_collector);
-                if (state != EmitState.EMITTED_MORE_LEFT) {
-                    _currPartitionIndex = (_currPartitionIndex + 1) % managers.size();
-                }
-                if (state != EmitState.NO_EMITTED) {
-                    break;
-                }
-            } catch (FailedFetchException e) {
-                LOG.warn("Fetch failed", e);
-                _coordinator.refresh();
-            }
-        }
-
-        long diffWithNow = System.currentTimeMillis() - _lastUpdateMs;
-
-        /*
-             As far as the System.currentTimeMillis() is dependent on System clock,
-             additional check on negative value of diffWithNow in case of external changes.
-         */
-        if (diffWithNow > _spoutConfig.stateUpdateIntervalMs || diffWithNow < 0) {
-            commit();
-        }
-    }
-
-    private PartitionManager getManagerForPartition(int partition) {
-        for (PartitionManager partitionManager : _coordinator.getMyManagedPartitions()) {
-            if (partitionManager.getPartition().partition == partition) {
-                return partitionManager;
-            }
-        }
-        return null;
-    }
-
-    @Override
-    public void ack(Object msgId) {
-        KafkaMessageId id = (KafkaMessageId) msgId;
-        PartitionManager m = _coordinator.getManager(id.partition);
-        if (m != null) {
-            m.ack(id.offset);
-        } else {
-            // managers for partitions changed - try to find new manager responsible for that partition
-            PartitionManager newManager = getManagerForPartition(id.partition.partition);
-            if (newManager != null) {
-                newManager.ack(id.offset);
-            }
-        }
-    }
-
-    @Override
-    public void fail(Object msgId) {
-        KafkaMessageId id = (KafkaMessageId) msgId;
-        PartitionManager m = _coordinator.getManager(id.partition);
-        if (m != null) {
-            m.fail(id.offset);
-        } else {
-            // managers for partitions changed - try to find new manager responsible for that partition
-            PartitionManager newManager = getManagerForPartition(id.partition.partition);
-            if (newManager != null) {
-                newManager.fail(id.offset);
-            }
-        }
-    }
-
-    @Override
-    public void deactivate() {
-        commit();
-    }
-
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-        if (!Strings.isNullOrEmpty(_spoutConfig.outputStreamId)) {
-            declarer.declareStream(_spoutConfig.outputStreamId, _spoutConfig.scheme.getOutputFields());
-        } else {
-            declarer.declare(_spoutConfig.scheme.getOutputFields());
-        }
-    }
-
-    @Override
-    public Map<String, Object> getComponentConfiguration() {
-        Map<String, Object> configuration = super.getComponentConfiguration();
-        if (configuration == null) {
-            configuration = new HashMap<>();
-        }
-        String configKeyPrefix = "config.";
-        configuration.put(configKeyPrefix + "topics", this._spoutConfig.topic);
-        StringBuilder zkServers = new StringBuilder();
-        if (_spoutConfig.zkServers != null && _spoutConfig.zkServers.size() > 0) {
-            for (String zkServer : this._spoutConfig.zkServers) {
-                zkServers.append(zkServer + ":" + this._spoutConfig.zkPort + ",");
-            }
-            configuration.put(configKeyPrefix + "zkServers", zkServers.toString());
-        }
-        BrokerHosts brokerHosts = this._spoutConfig.hosts;
-        String zkRoot = this._spoutConfig.zkRoot + "/" + this._spoutConfig.id;
-        if (brokerHosts instanceof ZkHosts) {
-            ZkHosts zkHosts = (ZkHosts) brokerHosts;
-            configuration.put(configKeyPrefix + "zkNodeBrokers", zkHosts.brokerZkPath);
-        } else if (brokerHosts instanceof StaticHosts) {
-            StaticHosts staticHosts = (StaticHosts) brokerHosts;
-            GlobalPartitionInformation globalPartitionInformation = staticHosts.getPartitionInformation();
-            boolean useTopicNameForPath = globalPartitionInformation.getbUseTopicNameForPartitionPathId();
-            if (useTopicNameForPath) {
-                zkRoot += ("/" + this._spoutConfig.topic);
-            }
-            List<Partition> partitions = globalPartitionInformation.getOrderedPartitions();
-            StringBuilder staticPartitions = new StringBuilder();
-            StringBuilder leaderHosts = new StringBuilder();
-            for (Partition partition : partitions) {
-                staticPartitions.append(partition.partition + ",");
-                leaderHosts.append(partition.host.host + ":" + partition.host.port).append(",");
-            }
-            configuration.put(configKeyPrefix + "partitions", staticPartitions.toString());
-            configuration.put(configKeyPrefix + "leaders", leaderHosts.toString());
-        }
-        configuration.put(configKeyPrefix + "zkRoot", zkRoot);
-        return configuration;
-    }
-
-    private void commit() {
-        _lastUpdateMs = System.currentTimeMillis();
-        for (PartitionManager manager : _coordinator.getMyManagedPartitions()) {
-            manager.commit();
-        }
-    }
-
-    static enum EmitState {
-        EMITTED_MORE_LEFT,
-        EMITTED_END,
-        NO_EMITTED
-    }
-
-}


[7/7] storm git commit: Merge branch 'STORM-2953-new' of https://github.com/srdo/storm into STORM-2953-test

Posted by ka...@apache.org.
Merge branch 'STORM-2953-new' of https://github.com/srdo/storm into STORM-2953-test


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

Branch: refs/heads/master
Commit: a7e817bcd1424d300ab5bad06c5d9f4729d9f347
Parents: 3883bf7 e58ac3e
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Thu Jul 19 12:29:34 2018 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Thu Jul 19 12:29:34 2018 +0900

----------------------------------------------------------------------
 docs/index.md                                   |   2 +-
 docs/storm-kafka-client.md                      |   7 +-
 docs/storm-kafka.md                             | 399 ------------------
 examples/storm-kafka-examples/pom.xml           | 110 -----
 .../kafka/trident/KafkaProducerTopology.java    |  75 ----
 .../trident/TridentKafkaConsumerTopology.java   |  42 --
 .../trident/TridentKafkaRandomStrings.java      |  82 ----
 .../kafka/trident/TridentKafkaTopology.java     |  83 ----
 external/storm-kafka-monitor/pom.xml            |   5 -
 .../storm/kafka/monitor/KafkaOffsetLagUtil.java | 298 +-------------
 .../kafka/monitor/OldKafkaSpoutOffsetQuery.java | 127 ------
 external/storm-kafka/README.md                  | 382 -----------------
 external/storm-kafka/pom.xml                    | 125 ------
 .../src/jvm/org/apache/storm/kafka/Broker.java  |  79 ----
 .../jvm/org/apache/storm/kafka/BrokerHosts.java |  20 -
 .../storm/kafka/ByteBufferSerializer.java       |  41 --
 .../storm/kafka/DynamicBrokersReader.java       | 208 ----------
 .../kafka/DynamicPartitionConnections.java      |  91 -----
 .../ExponentialBackoffMsgRetryManager.java      | 201 ---------
 .../storm/kafka/FailedFetchException.java       |  24 --
 .../storm/kafka/FailedMsgRetryManager.java      |  77 ----
 .../org/apache/storm/kafka/IntSerializer.java   |  42 --
 .../jvm/org/apache/storm/kafka/KafkaConfig.java |  49 ---
 .../jvm/org/apache/storm/kafka/KafkaError.java  |  38 --
 .../jvm/org/apache/storm/kafka/KafkaSpout.java  | 255 ------------
 .../jvm/org/apache/storm/kafka/KafkaUtils.java  | 288 -------------
 .../org/apache/storm/kafka/KeyValueScheme.java  |  21 -
 .../kafka/KeyValueSchemeAsMultiScheme.java      |  35 --
 .../storm/kafka/MessageMetadataScheme.java      |  21 -
 .../MessageMetadataSchemeAsMultiScheme.java     |  35 --
 .../jvm/org/apache/storm/kafka/Partition.java   |  85 ----
 .../storm/kafka/PartitionCoordinator.java       |  23 --
 .../apache/storm/kafka/PartitionManager.java    | 405 -------------------
 .../jvm/org/apache/storm/kafka/SpoutConfig.java |  58 ---
 .../apache/storm/kafka/StaticCoordinator.java   |  50 ---
 .../jvm/org/apache/storm/kafka/StaticHosts.java |  33 --
 .../storm/kafka/StaticPartitionConnections.java |  46 ---
 .../storm/kafka/StringKeyValueScheme.java       |  32 --
 .../kafka/StringMessageAndMetadataScheme.java   |  36 --
 .../storm/kafka/StringMultiSchemeWithTopic.java |  41 --
 .../org/apache/storm/kafka/StringScheme.java    |  44 --
 .../kafka/TopicOffsetOutOfRangeException.java   |  26 --
 .../org/apache/storm/kafka/ZkCoordinator.java   | 127 ------
 .../src/jvm/org/apache/storm/kafka/ZkHosts.java |  31 --
 .../src/jvm/org/apache/storm/kafka/ZkState.java | 112 -----
 .../org/apache/storm/kafka/bolt/KafkaBolt.java  | 172 --------
 .../FieldNameBasedTupleToKafkaMapper.java       |  43 --
 .../kafka/bolt/mapper/TupleToKafkaMapper.java   |  27 --
 .../bolt/selector/DefaultTopicSelector.java     |  29 --
 .../bolt/selector/FieldIndexTopicSelector.java  |  43 --
 .../bolt/selector/FieldNameTopicSelector.java   |  44 --
 .../kafka/bolt/selector/KafkaTopicSelector.java |  20 -
 .../apache/storm/kafka/trident/Coordinator.java |  46 ---
 .../storm/kafka/trident/DefaultCoordinator.java |  26 --
 .../trident/GlobalPartitionInformation.java     | 113 ------
 .../storm/kafka/trident/IBatchCoordinator.java  |  21 -
 .../storm/kafka/trident/IBrokerReader.java      |  24 --
 .../apache/storm/kafka/trident/MaxMetric.java   |  35 --
 .../kafka/trident/OpaqueTridentKafkaSpout.java  |  62 ---
 .../storm/kafka/trident/StaticBrokerReader.java |  44 --
 .../trident/TransactionalTridentKafkaSpout.java |  50 ---
 .../storm/kafka/trident/TridentKafkaConfig.java |  32 --
 .../kafka/trident/TridentKafkaEmitter.java      | 306 --------------
 .../storm/kafka/trident/TridentKafkaState.java  | 110 -----
 .../kafka/trident/TridentKafkaStateFactory.java |  57 ---
 .../kafka/trident/TridentKafkaUpdater.java      |  25 --
 .../storm/kafka/trident/ZkBrokerReader.java     |  79 ----
 .../FieldNameBasedTupleToKafkaMapper.java       |  36 --
 .../mapper/TridentTupleToKafkaMapper.java       |  22 -
 .../trident/selector/DefaultTopicSelector.java  |  29 --
 .../trident/selector/KafkaTopicSelector.java    |  20 -
 .../storm/kafka/DynamicBrokersReaderTest.java   | 245 -----------
 .../ExponentialBackoffMsgRetryManagerTest.java  | 279 -------------
 .../org/apache/storm/kafka/KafkaErrorTest.java  |  51 ---
 .../org/apache/storm/kafka/KafkaTestBroker.java | 177 --------
 .../org/apache/storm/kafka/KafkaUtilsTest.java  | 294 --------------
 .../storm/kafka/PartitionManagerTest.java       | 241 -----------
 .../storm/kafka/StringKeyValueSchemeTest.java   |  56 ---
 .../apache/storm/kafka/TestStringScheme.java    |  40 --
 .../test/org/apache/storm/kafka/TestUtils.java  |  97 -----
 .../apache/storm/kafka/TridentKafkaTest.java    |  75 ----
 .../apache/storm/kafka/ZkCoordinatorTest.java   | 191 ---------
 .../apache/storm/kafka/bolt/KafkaBoltTest.java  | 355 ----------------
 flux/flux-core/pom.xml                          |   5 +
 flux/flux-examples/pom.xml                      |   4 +
 flux/pom.xml                                    |   5 -
 pom.xml                                         |  31 +-
 .../apache/storm/utils/TopologySpoutLag.java    |  62 +--
 .../final-package/src/main/assembly/binary.xml  |   7 -
 89 files changed, 30 insertions(+), 8111 deletions(-)
----------------------------------------------------------------------



[2/7] storm git commit: STORM-2953: Remove storm-kafka

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/test/org/apache/storm/kafka/ExponentialBackoffMsgRetryManagerTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/ExponentialBackoffMsgRetryManagerTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/ExponentialBackoffMsgRetryManagerTest.java
deleted file mode 100644
index 5fcee28..0000000
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/ExponentialBackoffMsgRetryManagerTest.java
+++ /dev/null
@@ -1,279 +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;
-
-import org.apache.storm.utils.Time;
-import org.apache.storm.utils.Time.SimulatedTime;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class ExponentialBackoffMsgRetryManagerTest {
-
-    private static final Long TEST_OFFSET = 101L;
-    private static final Long TEST_OFFSET2 = 102L;
-    private static final Long TEST_OFFSET3 = 105L;
-    private static final Long TEST_NEW_OFFSET = 103L;
-    private SimulatedTime st;
-
-    @Before
-    public void setup() throws Exception {
-        st = new SimulatedTime();
-    }
-
-    @After
-    public void cleanup() throws Exception {
-        if (st != null) {
-            st.close();
-            st = null;
-        }
-    }
-
-    @Test
-    public void testImmediateRetry() throws Exception {
-
-
-        ExponentialBackoffMsgRetryManager manager = buildExponentialBackoffMsgRetryManager(0, 0d, 0, Integer.MAX_VALUE);
-        manager.failed(TEST_OFFSET);
-        Long next = manager.nextFailedMessageToRetry();
-        assertEquals("expect test offset next available for retry", TEST_OFFSET, next);
-        assertTrue("message should be ready for retry immediately", manager.shouldReEmitMsg(TEST_OFFSET));
-
-        manager.retryStarted(TEST_OFFSET);
-
-        manager.failed(TEST_OFFSET);
-        next = manager.nextFailedMessageToRetry();
-        assertEquals("expect test offset next available for retry", TEST_OFFSET, next);
-        assertTrue("message should be ready for retry immediately", manager.shouldReEmitMsg(TEST_OFFSET));
-    }
-
-    @Test
-    public void testSingleDelay() throws Exception {
-        ExponentialBackoffMsgRetryManager manager = buildExponentialBackoffMsgRetryManager(100, 1d, 1000, Integer.MAX_VALUE);
-        manager.failed(TEST_OFFSET);
-        Time.advanceTime(5);
-        Long next = manager.nextFailedMessageToRetry();
-        assertNull("expect no message ready for retry yet", next);
-        assertFalse("message should not be ready for retry yet", manager.shouldReEmitMsg(TEST_OFFSET));
-
-        Time.advanceTime(100);
-        next = manager.nextFailedMessageToRetry();
-        assertEquals("expect test offset next available for retry", TEST_OFFSET, next);
-        assertTrue("message should be ready for retry", manager.shouldReEmitMsg(TEST_OFFSET));
-    }
-
-    @Test
-    public void testExponentialBackoff() throws Exception {
-        final long initial = 10;
-        final double mult = 2d;
-        ExponentialBackoffMsgRetryManager manager = buildExponentialBackoffMsgRetryManager(initial, mult, initial * 10, Integer.MAX_VALUE);
-
-        long expectedWaitTime = initial;
-        for (long i = 0L; i < 3L; ++i) {
-            manager.failed(TEST_OFFSET);
-
-            Time.advanceTime((expectedWaitTime + 1L) / 2L);
-            assertFalse("message should not be ready for retry yet", manager.shouldReEmitMsg(TEST_OFFSET));
-
-            Time.advanceTime((expectedWaitTime + 1L) / 2L);
-            Long next = manager.nextFailedMessageToRetry();
-            assertEquals("expect test offset next available for retry", TEST_OFFSET, next);
-            assertTrue("message should be ready for retry", manager.shouldReEmitMsg(TEST_OFFSET));
-
-            manager.retryStarted(TEST_OFFSET);
-            expectedWaitTime *= mult;
-        }
-    }
-
-    @Test
-    public void testRetryOrder() throws Exception {
-        final long initial = 10;
-        final double mult = 2d;
-        final long max = 20;
-        ExponentialBackoffMsgRetryManager manager = buildExponentialBackoffMsgRetryManager(initial, mult, max, Integer.MAX_VALUE);
-
-        manager.failed(TEST_OFFSET);
-        Time.advanceTime(initial);
-
-        manager.retryStarted(TEST_OFFSET);
-        manager.failed(TEST_OFFSET);
-        manager.failed(TEST_OFFSET2);
-
-        // although TEST_OFFSET failed first, it's retry delay time is longer b/c this is the second retry
-        // so TEST_OFFSET2 should come first
-
-        Time.advanceTime(initial * 2);
-        assertTrue("message " + TEST_OFFSET + "should be ready for retry", manager.shouldReEmitMsg(TEST_OFFSET));
-        assertTrue("message " + TEST_OFFSET2 + "should be ready for retry", manager.shouldReEmitMsg(TEST_OFFSET2));
-
-        Long next = manager.nextFailedMessageToRetry();
-        assertEquals("expect first message to retry is " + TEST_OFFSET2, TEST_OFFSET2, next);
-
-        Time.advanceTime(initial);
-
-        // haven't retried yet, so first should still be TEST_OFFSET2
-        next = manager.nextFailedMessageToRetry();
-        assertEquals("expect first message to retry is " + TEST_OFFSET2, TEST_OFFSET2, next);
-        manager.retryStarted(next);
-
-        // now it should be TEST_OFFSET
-        next = manager.nextFailedMessageToRetry();
-        assertEquals("expect message to retry is now " + TEST_OFFSET, TEST_OFFSET, next);
-        manager.retryStarted(next);
-
-        // now none left
-        next = manager.nextFailedMessageToRetry();
-        assertNull("expect no message to retry now", next);
-    }
-
-    @Test
-    public void testQueriesAfterRetriedAlready() throws Exception {
-        ExponentialBackoffMsgRetryManager manager = buildExponentialBackoffMsgRetryManager(0, 0d, 0, Integer.MAX_VALUE);
-        manager.failed(TEST_OFFSET);
-        Long next = manager.nextFailedMessageToRetry();
-        assertEquals("expect test offset next available for retry", TEST_OFFSET, next);
-        assertTrue("message should be ready for retry immediately", manager.shouldReEmitMsg(TEST_OFFSET));
-
-        manager.retryStarted(TEST_OFFSET);
-        next = manager.nextFailedMessageToRetry();
-        assertNull("expect no message ready after retried", next);
-        assertFalse("message should not be ready after retried", manager.shouldReEmitMsg(TEST_OFFSET));
-    }
-
-    @Test(expected = IllegalStateException.class)
-    public void testRetryWithoutFail() throws Exception {
-        ExponentialBackoffMsgRetryManager manager = buildExponentialBackoffMsgRetryManager(0, 0d, 0, Integer.MAX_VALUE);
-        manager.retryStarted(TEST_OFFSET);
-    }
-
-    @Test(expected = IllegalStateException.class)
-    public void testFailRetryRetry() throws Exception {
-        ExponentialBackoffMsgRetryManager manager = buildExponentialBackoffMsgRetryManager(0, 0d, 0, Integer.MAX_VALUE);
-        manager.failed(TEST_OFFSET);
-        try {
-            manager.retryStarted(TEST_OFFSET);
-        } catch (IllegalStateException ise) {
-            fail("IllegalStateException unexpected here: " + ise);
-        }
-
-        assertFalse("message should not be ready for retry", manager.shouldReEmitMsg(TEST_OFFSET));
-        manager.retryStarted(TEST_OFFSET);
-    }
-
-    @Test
-    public void testMaxBackoff() throws Exception {
-        final long initial = 100;
-        final double mult = 2d;
-        final long max = 2000;
-        ExponentialBackoffMsgRetryManager manager = buildExponentialBackoffMsgRetryManager(initial, mult, max, Integer.MAX_VALUE);
-
-        long expectedWaitTime = initial;
-        for (long i = 0L; i < 4L; ++i) {
-            manager.failed(TEST_OFFSET);
-
-            Time.advanceTime((expectedWaitTime + 1L) / 2L);
-            assertFalse("message should not be ready for retry yet", manager.shouldReEmitMsg(TEST_OFFSET));
-
-            Time.advanceTime((expectedWaitTime + 1L) / 2L);
-            Long next = manager.nextFailedMessageToRetry();
-            assertEquals("expect test offset next available for retry", TEST_OFFSET, next);
-            assertTrue("message should be ready for retry", manager.shouldReEmitMsg(TEST_OFFSET));
-
-            manager.retryStarted(TEST_OFFSET);
-            expectedWaitTime = Math.min((long) (expectedWaitTime * mult), max);
-        }
-    }
-
-    @Test
-    public void testFailThenAck() throws Exception {
-        ExponentialBackoffMsgRetryManager manager = buildExponentialBackoffMsgRetryManager(0, 0d, 0, Integer.MAX_VALUE);
-        manager.failed(TEST_OFFSET);
-        assertTrue("message should be ready for retry", manager.shouldReEmitMsg(TEST_OFFSET));
-
-        manager.acked(TEST_OFFSET);
-
-        Long next = manager.nextFailedMessageToRetry();
-        assertNull("expect no message ready after acked", next);
-        assertFalse("message should not be ready after acked", manager.shouldReEmitMsg(TEST_OFFSET));
-    }
-
-    @Test
-    public void testAckThenFail() throws Exception {
-        ExponentialBackoffMsgRetryManager manager = buildExponentialBackoffMsgRetryManager(0, 0d, 0, Integer.MAX_VALUE);
-        manager.acked(TEST_OFFSET);
-        assertFalse("message should not be ready after acked", manager.shouldReEmitMsg(TEST_OFFSET));
-
-        manager.failed(TEST_OFFSET);
-
-        Long next = manager.nextFailedMessageToRetry();
-        assertEquals("expect test offset next available for retry", TEST_OFFSET, next);
-        assertTrue("message should be ready for retry", manager.shouldReEmitMsg(TEST_OFFSET));
-    }
-
-    @Test
-    public void testClearInvalidMessages() throws Exception {
-        ExponentialBackoffMsgRetryManager manager = buildExponentialBackoffMsgRetryManager(0, 0d, 0, Integer.MAX_VALUE);
-        manager.failed(TEST_OFFSET);
-        manager.failed(TEST_OFFSET2);
-        manager.failed(TEST_OFFSET3);
-
-        assertTrue("message should be ready for retry", manager.shouldReEmitMsg(TEST_OFFSET));
-        assertTrue("message should be ready for retry", manager.shouldReEmitMsg(TEST_OFFSET2));
-        assertTrue("message should be ready for retry", manager.shouldReEmitMsg(TEST_OFFSET3));
-
-        manager.clearOffsetsBefore(TEST_NEW_OFFSET);
-
-        Long next = manager.nextFailedMessageToRetry();
-        assertEquals("expect test offset next available for retry", TEST_OFFSET3, next);
-
-        manager.acked(TEST_OFFSET3);
-        next = manager.nextFailedMessageToRetry();
-        assertNull("expect no message ready after acked", next);
-    }
-
-    @Test
-    public void testMaxRetry() throws Exception {
-        final long initial = 100;
-        final double mult = 2d;
-        final long max = 2000;
-        final int maxRetries = 2;
-        ExponentialBackoffMsgRetryManager manager = buildExponentialBackoffMsgRetryManager(initial, mult, max, maxRetries);
-        assertTrue(manager.retryFurther(TEST_OFFSET));
-        manager.failed(TEST_OFFSET);
-
-        assertTrue(manager.retryFurther(TEST_OFFSET));
-        manager.failed(TEST_OFFSET);
-
-        assertFalse(manager.retryFurther(TEST_OFFSET));
-    }
-
-    private ExponentialBackoffMsgRetryManager buildExponentialBackoffMsgRetryManager(long retryInitialDelayMs,
-                                                                                     double retryDelayMultiplier,
-                                                                                     long retryDelayMaxMs,
-                                                                                     int retryLimit) {
-        SpoutConfig spoutConfig = new SpoutConfig(null, null, null, null);
-        spoutConfig.retryInitialDelayMs = retryInitialDelayMs;
-        spoutConfig.retryDelayMultiplier = retryDelayMultiplier;
-        spoutConfig.retryDelayMaxMs = retryDelayMaxMs;
-        spoutConfig.retryLimit = retryLimit;
-        ExponentialBackoffMsgRetryManager exponentialBackoffMsgRetryManager = new ExponentialBackoffMsgRetryManager();
-        exponentialBackoffMsgRetryManager.prepare(spoutConfig, null);
-        return exponentialBackoffMsgRetryManager;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaErrorTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaErrorTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaErrorTest.java
deleted file mode 100644
index ad793ed..0000000
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaErrorTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.  The ASF licenses this file to you under the Apache License, Version
- * 2.0 (the "License"); you may not use this file except in compliance with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions
- * and limitations under the License.
- */
-
-package org.apache.storm.kafka;
-
-import org.junit.Test;
-
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertThat;
-
-/**
- * Date: 12/01/2014
- * Time: 18:09
- */
-public class KafkaErrorTest {
-
-    @Test
-    public void getError() {
-        assertThat(KafkaError.getError(0), is(equalTo(KafkaError.NO_ERROR)));
-    }
-
-    @Test
-    public void offsetMetaDataTooLarge() {
-        assertThat(KafkaError.getError(12), is(equalTo(KafkaError.OFFSET_METADATA_TOO_LARGE)));
-    }
-
-    @Test
-    public void unknownNegative() {
-        assertThat(KafkaError.getError(-1), is(equalTo(KafkaError.UNKNOWN)));
-    }
-
-    @Test
-    public void unknownPositive() {
-        assertThat(KafkaError.getError(75), is(equalTo(KafkaError.UNKNOWN)));
-    }
-
-    @Test
-    public void unknown() {
-        assertThat(KafkaError.getError(13), is(equalTo(KafkaError.UNKNOWN)));
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaTestBroker.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaTestBroker.java b/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaTestBroker.java
deleted file mode 100644
index f31386a..0000000
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaTestBroker.java
+++ /dev/null
@@ -1,177 +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;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Properties;
-import java.util.concurrent.TimeUnit;
-import kafka.admin.AdminUtils;
-import kafka.api.PartitionMetadata;
-import kafka.api.TopicMetadata;
-import kafka.common.ErrorMapping;
-import kafka.server.KafkaConfig;
-import kafka.server.KafkaServerStartable;
-import kafka.utils.ZKStringSerializer$;
-import org.I0Itec.zkclient.ZkClient;
-import org.apache.commons.io.FileUtils;
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.framework.imps.CuratorFrameworkState;
-import org.apache.curator.retry.ExponentialBackoffRetry;
-import org.apache.curator.test.InstanceSpec;
-import org.apache.curator.test.TestingServer;
-import scala.collection.JavaConversions;
-
-/**
- * Date: 11/01/2014
- * Time: 13:15
- */
-public class KafkaTestBroker {
-
-    // Bind services to the loopback address for environments where _localhost_ may resolve to an unreachable host
-    private static final String LOCALHOST = "127.0.0.1";
-
-    private int port;
-    private KafkaServerStartable kafka;
-    private TestingServer server;
-    private CuratorFramework zookeeper;
-    private File logDir;
-
-    public KafkaTestBroker() {
-        this(new Properties());
-    }
-
-    public KafkaTestBroker(Properties brokerProps) {
-        try {
-            InstanceSpec spec = new InstanceSpec(
-                null,
-                -1,
-                -1,
-                -1,
-                true,
-                -1,
-                -1,
-                -1,
-                null,
-                LOCALHOST
-            );
-            server = new TestingServer(spec, true);
-            String zookeeperConnectionString = server.getConnectString();
-            ExponentialBackoffRetry retryPolicy = new ExponentialBackoffRetry(1000, 3);
-            zookeeper = CuratorFrameworkFactory.newClient(zookeeperConnectionString, retryPolicy);
-            zookeeper.start();
-            port = InstanceSpec.getRandomPort();
-            logDir = new File(System.getProperty("java.io.tmpdir"), "kafka/logs/kafka-test-" + port);
-            KafkaConfig config = buildKafkaConfig(brokerProps, zookeeperConnectionString);
-            kafka = new KafkaServerStartable(config);
-            kafka.startup();
-        } catch (Exception ex) {
-            throw new RuntimeException("Could not start test broker", ex);
-        }
-    }
-
-    public void createTopic(String topicName, int numPartitions, Properties properties) {
-        ZkClient zkClient = new ZkClient(getZookeeperConnectionString());
-        zkClient.setZkSerializer(ZKStringSerializer$.MODULE$);
-
-        try {
-            AdminUtils.createTopic(zkClient, topicName, numPartitions, 1, properties);
-
-            ensureTopicCreated(zkClient, topicName);
-        } finally {
-            zkClient.close();
-        }
-    }
-
-
-    /**
-     * Wait for up to 30 seconds for the topic to be created and leader assignments for all partitions
-     */
-    private void ensureTopicCreated(ZkClient zkClient, String topicName) {
-        long maxWaitTime = TimeUnit.SECONDS.toNanos(30);
-        long waitTime = 0;
-        boolean partitionsHaveLeaders = false;
-
-        while (!partitionsHaveLeaders && waitTime < maxWaitTime) {
-            partitionsHaveLeaders = true;
-            TopicMetadata topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topicName, zkClient);
-            for (PartitionMetadata partitionMetadata : JavaConversions.seqAsJavaList(topicMetadata.partitionsMetadata())) {
-                if (partitionMetadata.leader().isEmpty() || partitionMetadata.errorCode() != ErrorMapping.NoError()) {
-                    partitionsHaveLeaders = false;
-                }
-            }
-
-            if (!partitionsHaveLeaders) {
-                long start = System.nanoTime();
-                try {
-                    Thread.sleep(100);
-                } catch (InterruptedException ex) {
-                    Thread.currentThread().interrupt();
-                    throw new RuntimeException("Interrupted while waiting for topic to be available");
-                }
-
-                waitTime += (System.nanoTime() - start);
-            }
-        }
-
-        if (!partitionsHaveLeaders) {
-            throw new RuntimeException("Could not create topic: " + topicName);
-        }
-    }
-
-    private kafka.server.KafkaConfig buildKafkaConfig(Properties brokerProps, String zookeeperConnectionString) {
-        Properties p = new Properties(brokerProps);
-        p.setProperty("zookeeper.connect", zookeeperConnectionString);
-        p.setProperty("broker.id", "0");
-        p.setProperty("port", "" + port);
-        p.setProperty("host.name", LOCALHOST);
-        p.setProperty("log.dirs", logDir.getAbsolutePath());
-        return new KafkaConfig(p);
-    }
-
-    public String getBrokerConnectionString() {
-        return LOCALHOST + ":" + port;
-    }
-
-    public String getZookeeperConnectionString() {
-        return server.getConnectString();
-    }
-
-    public int getZookeeperPort() {
-        return server.getPort();
-    }
-
-    public int getPort() {
-        return port;
-    }
-
-    public void shutdown() {
-        if (kafka != null) {
-            kafka.shutdown();
-            kafka.awaitShutdown();
-        }
-        //Ensure kafka server is eligible for garbage collection immediately
-        kafka = null;
-        if (zookeeper.getState().equals(CuratorFrameworkState.STARTED)) {
-            zookeeper.close();
-        }
-        try {
-            server.close();
-        } catch (IOException e) {
-            e.printStackTrace();
-        }
-        FileUtils.deleteQuietly(logDir);
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaUtilsTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaUtilsTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaUtilsTest.java
deleted file mode 100644
index 3c2c0d9..0000000
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaUtilsTest.java
+++ /dev/null
@@ -1,294 +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;
-
-import com.google.common.collect.ImmutableMap;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-import kafka.api.OffsetRequest;
-import kafka.javaapi.consumer.SimpleConsumer;
-import kafka.javaapi.message.ByteBufferMessageSet;
-import kafka.message.MessageAndOffset;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.storm.kafka.trident.GlobalPartitionInformation;
-import org.apache.storm.spout.SchemeAsMultiScheme;
-import org.apache.storm.utils.Utils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-
-public class KafkaUtilsTest {
-    private static final Logger LOG = LoggerFactory.getLogger(KafkaUtilsTest.class);
-    private String TEST_TOPIC = "testTopic";
-    private KafkaTestBroker broker;
-    private SimpleConsumer simpleConsumer;
-    private KafkaConfig config;
-    private BrokerHosts brokerHosts;
-
-    @Before
-    public void setup() {
-        broker = new KafkaTestBroker();
-        GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation(TEST_TOPIC);
-        globalPartitionInformation.addPartition(0, Broker.fromString(broker.getBrokerConnectionString()));
-        brokerHosts = new StaticHosts(globalPartitionInformation);
-        config = new KafkaConfig(brokerHosts, TEST_TOPIC);
-        simpleConsumer = new SimpleConsumer("localhost", broker.getPort(), 60000, 1024, "testClient");
-    }
-
-    @After
-    public void shutdown() {
-        simpleConsumer.close();
-        broker.shutdown();
-    }
-
-
-    @Test(expected = FailedFetchException.class)
-    public void topicDoesNotExist() throws Exception {
-        KafkaUtils
-            .fetchMessages(config, simpleConsumer, new Partition(Broker.fromString(broker.getBrokerConnectionString()), TEST_TOPIC, 0), 0);
-    }
-
-    @Test(expected = FailedFetchException.class)
-    public void brokerIsDown() throws Exception {
-        int port = broker.getPort();
-        broker.shutdown();
-        SimpleConsumer simpleConsumer = new SimpleConsumer("localhost", port, 100, 1024, "testClient");
-        try {
-            KafkaUtils
-                .fetchMessages(config, simpleConsumer, new Partition(Broker.fromString(broker.getBrokerConnectionString()), TEST_TOPIC, 0),
-                               OffsetRequest.LatestTime());
-        } finally {
-            simpleConsumer.close();
-        }
-    }
-
-    @Test
-    public void fetchMessage() throws Exception {
-        String value = "test";
-        createTopicAndSendMessage(value);
-        long offset = KafkaUtils.getOffset(simpleConsumer, config.topic, 0, OffsetRequest.LatestTime()) - 1;
-        ByteBufferMessageSet messageAndOffsets = KafkaUtils.fetchMessages(config, simpleConsumer,
-                                                                          new Partition(
-                                                                              Broker.fromString(broker.getBrokerConnectionString()),
-                                                                              TEST_TOPIC, 0), offset);
-        String message = new String(Utils.toByteArray(messageAndOffsets.iterator().next().message().payload()));
-        assertThat(message, is(equalTo(value)));
-    }
-
-    @Test(expected = FailedFetchException.class)
-    public void fetchMessagesWithInvalidOffsetAndDefaultHandlingDisabled() throws Exception {
-        config.useStartOffsetTimeIfOffsetOutOfRange = false;
-        KafkaUtils.fetchMessages(config, simpleConsumer,
-                                 new Partition(Broker.fromString(broker.getBrokerConnectionString()), TEST_TOPIC, 0), -99);
-    }
-
-    @Test(expected = TopicOffsetOutOfRangeException.class)
-    public void fetchMessagesWithInvalidOffsetAndDefaultHandlingEnabled() throws Exception {
-        config = new KafkaConfig(brokerHosts, "newTopic");
-        String value = "test";
-        createTopicAndSendMessage(value);
-        KafkaUtils.fetchMessages(config, simpleConsumer,
-                                 new Partition(Broker.fromString(broker.getBrokerConnectionString()), "newTopic", 0), -99);
-    }
-
-    @Test
-    public void getOffsetFromConfigAndDontForceFromStart() {
-        config.ignoreZkOffsets = false;
-        config.startOffsetTime = OffsetRequest.EarliestTime();
-        createTopicAndSendMessage();
-        long latestOffset = KafkaUtils.getOffset(simpleConsumer, config.topic, 0, OffsetRequest.EarliestTime());
-        long offsetFromConfig = KafkaUtils.getOffset(simpleConsumer, config.topic, 0, config);
-        assertThat(latestOffset, is(equalTo(offsetFromConfig)));
-    }
-
-    @Test
-    public void getOffsetFromConfigAndForceFromStart() {
-        config.ignoreZkOffsets = true;
-        config.startOffsetTime = OffsetRequest.EarliestTime();
-        createTopicAndSendMessage();
-        long earliestOffset = KafkaUtils.getOffset(simpleConsumer, config.topic, 0, OffsetRequest.EarliestTime());
-        long offsetFromConfig = KafkaUtils.getOffset(simpleConsumer, config.topic, 0, config);
-        assertThat(earliestOffset, is(equalTo(offsetFromConfig)));
-    }
-
-    @Test
-    public void generateTuplesWithoutKeyAndKeyValueScheme() {
-        config.scheme = new KeyValueSchemeAsMultiScheme(new StringKeyValueScheme());
-        runGetValueOnlyTuplesTest();
-    }
-
-    @Test
-    public void generateTuplesWithKeyAndKeyValueScheme() {
-        config.scheme = new KeyValueSchemeAsMultiScheme(new StringKeyValueScheme());
-        config.useStartOffsetTimeIfOffsetOutOfRange = false;
-        String value = "value";
-        String key = "key";
-        createTopicAndSendMessage(key, value);
-        ByteBufferMessageSet messageAndOffsets = getLastMessage();
-        for (MessageAndOffset msg : messageAndOffsets) {
-            Iterable<List<Object>> lists = KafkaUtils.generateTuples(config, msg.message(), config.topic);
-            assertEquals(ImmutableMap.of(key, value), lists.iterator().next().get(0));
-        }
-    }
-
-    @Test
-    public void generateTupelsWithValueScheme() {
-        config.scheme = new SchemeAsMultiScheme(new StringScheme());
-        runGetValueOnlyTuplesTest();
-    }
-
-    @Test
-    public void generateTuplesWithValueAndStringMultiSchemeWithTopic() {
-        config.scheme = new StringMultiSchemeWithTopic();
-        String value = "value";
-        createTopicAndSendMessage(value);
-        ByteBufferMessageSet messageAndOffsets = getLastMessage();
-        for (MessageAndOffset msg : messageAndOffsets) {
-            Iterable<List<Object>> lists = KafkaUtils.generateTuples(config, msg.message(), config.topic);
-            List<Object> list = lists.iterator().next();
-            assertEquals(value, list.get(0));
-            assertEquals(config.topic, list.get(1));
-        }
-    }
-
-    @Test
-    public void generateTuplesWithValueSchemeAndKeyValueMessage() {
-        config.scheme = new SchemeAsMultiScheme(new StringScheme());
-        String value = "value";
-        String key = "key";
-        createTopicAndSendMessage(key, value);
-        ByteBufferMessageSet messageAndOffsets = getLastMessage();
-        for (MessageAndOffset msg : messageAndOffsets) {
-            Iterable<List<Object>> lists = KafkaUtils.generateTuples(config, msg.message(), config.topic);
-            assertEquals(value, lists.iterator().next().get(0));
-        }
-    }
-
-    @Test
-    public void generateTuplesWithMessageAndMetadataScheme() {
-        String value = "value";
-        Partition mockPartition = Mockito.mock(Partition.class);
-        mockPartition.partition = 0;
-        long offset = 0L;
-
-        MessageMetadataSchemeAsMultiScheme scheme = new MessageMetadataSchemeAsMultiScheme(new StringMessageAndMetadataScheme());
-
-        createTopicAndSendMessage(null, value);
-        ByteBufferMessageSet messageAndOffsets = getLastMessage();
-        for (MessageAndOffset msg : messageAndOffsets) {
-            Iterable<List<Object>> lists = KafkaUtils.generateTuples(scheme, msg.message(), mockPartition, offset);
-            List<Object> values = lists.iterator().next();
-            assertEquals("Message is incorrect", value, values.get(0));
-            assertEquals("Partition is incorrect", mockPartition.partition, values.get(1));
-            assertEquals("Offset is incorrect", offset, values.get(2));
-        }
-    }
-
-    private ByteBufferMessageSet getLastMessage() {
-        long offsetOfLastMessage = KafkaUtils.getOffset(simpleConsumer, config.topic, 0, OffsetRequest.LatestTime()) - 1;
-        return KafkaUtils
-            .fetchMessages(config, simpleConsumer, new Partition(Broker.fromString(broker.getBrokerConnectionString()), TEST_TOPIC, 0),
-                           offsetOfLastMessage);
-    }
-
-    private void runGetValueOnlyTuplesTest() {
-        String value = "value";
-
-        createTopicAndSendMessage(null, value);
-        ByteBufferMessageSet messageAndOffsets = getLastMessage();
-        for (MessageAndOffset msg : messageAndOffsets) {
-            Iterable<List<Object>> lists = KafkaUtils.generateTuples(config, msg.message(), config.topic);
-            assertEquals(value, lists.iterator().next().get(0));
-        }
-    }
-
-    private void createTopicAndSendMessage() {
-        createTopicAndSendMessage(null, "someValue");
-    }
-
-    private void createTopicAndSendMessage(String value) {
-        createTopicAndSendMessage(null, value);
-    }
-
-    private void createTopicAndSendMessage(String key, String value) {
-        Properties p = new Properties();
-        p.put("acks", "1");
-        p.put("bootstrap.servers", broker.getBrokerConnectionString());
-        p.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-        p.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-        p.put("metadata.fetch.timeout.ms", 1000);
-        KafkaProducer<String, String> producer = new KafkaProducer<String, String>(p);
-        try {
-            producer.send(new ProducerRecord<String, String>(config.topic, key, value)).get();
-        } catch (Exception e) {
-            Assert.fail(e.getMessage());
-            LOG.error("Failed to do synchronous sending due to " + e, e);
-        } finally {
-            producer.close();
-        }
-    }
-
-    @Test
-    public void assignOnePartitionPerTask() {
-        runPartitionToTaskMappingTest(16, 1);
-    }
-
-    @Test
-    public void assignTwoPartitionsPerTask() {
-        runPartitionToTaskMappingTest(16, 2);
-    }
-
-    @Test
-    public void assignAllPartitionsToOneTask() {
-        runPartitionToTaskMappingTest(32, 32);
-    }
-
-    public void runPartitionToTaskMappingTest(int numPartitions, int partitionsPerTask) {
-        GlobalPartitionInformation globalPartitionInformation = TestUtils.buildPartitionInfo(numPartitions);
-        List<GlobalPartitionInformation> partitions = new ArrayList<GlobalPartitionInformation>();
-        partitions.add(globalPartitionInformation);
-        int numTasks = numPartitions / partitionsPerTask;
-        for (int i = 0; i < numTasks; i++) {
-            assertEquals(partitionsPerTask, KafkaUtils.calculatePartitionsForTask(partitions, numTasks, i, i).size());
-        }
-    }
-
-    @Test
-    public void moreTasksThanPartitions() {
-        GlobalPartitionInformation globalPartitionInformation = TestUtils.buildPartitionInfo(1);
-        List<GlobalPartitionInformation> partitions = new ArrayList<GlobalPartitionInformation>();
-        partitions.add(globalPartitionInformation);
-        int numTasks = 2;
-        assertEquals(1, KafkaUtils.calculatePartitionsForTask(partitions, numTasks, 0, 0).size());
-        assertEquals(0, KafkaUtils.calculatePartitionsForTask(partitions, numTasks, 1, 1).size());
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void assignInvalidTask() {
-        GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation(TEST_TOPIC);
-        List<GlobalPartitionInformation> partitions = new ArrayList<GlobalPartitionInformation>();
-        partitions.add(globalPartitionInformation);
-        KafkaUtils.calculatePartitionsForTask(partitions, 1, 1, 1);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/test/org/apache/storm/kafka/PartitionManagerTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/PartitionManagerTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/PartitionManagerTest.java
deleted file mode 100644
index a2824fb..0000000
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/PartitionManagerTest.java
+++ /dev/null
@@ -1,241 +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;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.TimeUnit;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.storm.Config;
-import org.apache.storm.kafka.KafkaSpout.EmitState;
-import org.apache.storm.kafka.PartitionManager.KafkaMessageId;
-import org.apache.storm.kafka.trident.ZkBrokerReader;
-import org.apache.storm.spout.ISpoutOutputCollector;
-import org.apache.storm.spout.SpoutOutputCollector;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-public class PartitionManagerTest {
-
-    private static final String TOPIC_NAME = "testTopic";
-
-    private KafkaTestBroker broker;
-    private TestingSpoutOutputCollector outputCollector;
-    private ZkState zkState;
-    private ZkCoordinator coordinator;
-    private KafkaProducer<String, String> producer;
-
-    @Before
-    public void setup() {
-        outputCollector = new TestingSpoutOutputCollector();
-
-        Properties brokerProps = new Properties();
-        brokerProps.setProperty("log.retention.check.interval.ms", "1000");
-
-        broker = new KafkaTestBroker(brokerProps);
-
-        // Configure Kafka to remove messages after 2 seconds
-        Properties topicProperties = new Properties();
-        topicProperties.put("delete.retention.ms", "2000");
-        topicProperties.put("retention.ms", "2000");
-
-        broker.createTopic(TOPIC_NAME, 1, topicProperties);
-
-        Map<String, Object> conf = new HashMap<>();
-        conf.put(Config.TRANSACTIONAL_ZOOKEEPER_PORT, broker.getZookeeperPort());
-        conf.put(Config.TRANSACTIONAL_ZOOKEEPER_SERVERS, Collections.singletonList("127.0.0.1"));
-        conf.put(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT, 20000);
-        conf.put(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT, 20000);
-        conf.put(Config.STORM_ZOOKEEPER_RETRY_TIMES, 3);
-        conf.put(Config.STORM_ZOOKEEPER_RETRY_INTERVAL, 30);
-        conf.put(Config.TOPOLOGY_NAME, "test");
-
-        zkState = new ZkState(conf);
-
-        ZkHosts zkHosts = new ZkHosts(broker.getZookeeperConnectionString());
-
-        SpoutConfig spoutConfig = new SpoutConfig(zkHosts, TOPIC_NAME, "/test", "id");
-
-        coordinator = new ZkCoordinator(
-            new DynamicPartitionConnections(spoutConfig, new ZkBrokerReader(conf, TOPIC_NAME, zkHosts)),
-            conf,
-            spoutConfig,
-            zkState,
-            0,
-            1,
-            1,
-            "topo"
-        );
-
-        Properties producerProps = new Properties();
-        producerProps.put("acks", "1");
-        producerProps.put("bootstrap.servers", broker.getBrokerConnectionString());
-        producerProps.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-        producerProps.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-        producerProps.put("metadata.fetch.timeout.ms", 1000);
-        producer = new KafkaProducer<>(producerProps);
-    }
-
-    @After
-    public void shutdown() {
-        producer.close();
-        broker.shutdown();
-    }
-
-    /**
-     * Test for STORM-2608
-     *
-     * - Send a few messages to topic
-     * - Emit those messages from the partition manager
-     * - Fail those tuples so that they are added to the failedMsgRetryManager
-     * - Commit partition info to Zookeeper
-     * - Wait for kafka to roll logs and remove those messages
-     * - Send a new message to the topic
-     * - On the next fetch request, a TopicOffsetOutOfRangeException is thrown and the new offset is after
-     *      the offset that is currently sitting in both the pending tree and the failedMsgRetryManager
-     * - Ack latest message to partition manager
-     * - Commit partition info to zookeeper
-     * - The committed offset should be the next offset _after_ the last one that was committed
-     *
-     */
-    @Test
-    public void test2608() throws Exception {
-        SpoutOutputCollector spoutOutputCollector = new SpoutOutputCollector(outputCollector);
-        List<PartitionManager> partitionManagers = coordinator.getMyManagedPartitions();
-        Assert.assertEquals(1, partitionManagers.size());
-
-        PartitionManager partitionManager = partitionManagers.get(0);
-
-        for (int i = 0; i < 5; i++) {
-            sendMessage("message-" + i);
-        }
-
-        waitForEmitState(partitionManager, spoutOutputCollector, EmitState.EMITTED_MORE_LEFT);
-        waitForEmitState(partitionManager, spoutOutputCollector, EmitState.EMITTED_MORE_LEFT);
-        waitForEmitState(partitionManager, spoutOutputCollector, EmitState.EMITTED_MORE_LEFT);
-        waitForEmitState(partitionManager, spoutOutputCollector, EmitState.EMITTED_MORE_LEFT);
-        waitForEmitState(partitionManager, spoutOutputCollector, EmitState.EMITTED_END);
-
-        partitionManager.commit();
-
-        Map<KafkaMessageId, List<Object>> emitted = outputCollector.getEmitted();
-
-        Assert.assertEquals(5, emitted.size());
-
-        for (KafkaMessageId messageId : emitted.keySet()) {
-            partitionManager.fail(messageId.offset);
-        }
-
-        // Kafka log roller task has an initial delay of 30 seconds so we need to wait for it
-        Thread.sleep(TimeUnit.SECONDS.toMillis(35));
-
-        outputCollector.clearEmittedMessages();
-
-        sendMessage("new message");
-
-        // First request will fail due to offset out of range
-        Assert.assertEquals(EmitState.NO_EMITTED, partitionManager.next(spoutOutputCollector));
-        waitForEmitState(partitionManager, spoutOutputCollector, EmitState.EMITTED_END);
-
-        emitted = outputCollector.getEmitted();
-
-        Assert.assertEquals(1, emitted.size());
-        KafkaMessageId messageId = emitted.keySet().iterator().next();
-
-        partitionManager.ack(messageId.offset);
-        partitionManager.commit();
-
-        Map<Object, Object> json = zkState.readJSON(partitionManager.committedPath());
-        Assert.assertNotNull(json);
-        long committedOffset = (long) json.get("offset");
-
-        Assert.assertEquals(messageId.offset + 1, committedOffset);
-    }
-
-    private void waitForEmitState(PartitionManager partitionManager, SpoutOutputCollector outputCollector, EmitState expectedState) {
-        int maxRetries = 5;
-        EmitState state = null;
-
-        for (int retryCount = 0; retryCount < maxRetries; retryCount++) {
-            state = partitionManager.next(outputCollector);
-
-            if (state == EmitState.NO_EMITTED) {
-                retryCount++;
-                try {
-                    Thread.sleep(TimeUnit.SECONDS.toMillis(1));
-                } catch (InterruptedException ex) {
-                    Thread.currentThread().interrupt();
-                    throw new RuntimeException("Interrupted while waiting for message");
-                }
-            } else {
-                break;
-            }
-        }
-
-        Assert.assertEquals(expectedState, state);
-    }
-
-    private void sendMessage(String value) {
-        try {
-            producer.send(new ProducerRecord<>(TOPIC_NAME, (String) null, value)).get();
-        } catch (Exception e) {
-            Assert.fail(e.getMessage());
-        }
-    }
-
-    private static class TestingSpoutOutputCollector implements ISpoutOutputCollector {
-
-        private final Map<KafkaMessageId, List<Object>> emitted = new HashMap<>();
-
-        Map<KafkaMessageId, List<Object>> getEmitted() {
-            return emitted;
-        }
-
-        void clearEmittedMessages() {
-            emitted.clear();
-        }
-
-        @Override
-        public List<Integer> emit(String streamId, List<Object> tuple, Object messageId) {
-            emitted.put((KafkaMessageId) messageId, tuple);
-            return Collections.emptyList();
-        }
-
-        @Override
-        public void reportError(Throwable error) {
-            throw new RuntimeException("Spout error", error);
-        }
-
-
-        @Override
-        public void emitDirect(int taskId, String streamId, List<Object> tuple, Object messageId) {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public long getPendingCount() {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public void flush() {
-        }
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/test/org/apache/storm/kafka/StringKeyValueSchemeTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/StringKeyValueSchemeTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/StringKeyValueSchemeTest.java
deleted file mode 100644
index 9c749fe..0000000
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/StringKeyValueSchemeTest.java
+++ /dev/null
@@ -1,56 +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;
-
-import com.google.common.collect.ImmutableMap;
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.util.Collections;
-import org.apache.storm.tuple.Fields;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class StringKeyValueSchemeTest {
-
-    private StringKeyValueScheme scheme = new StringKeyValueScheme();
-
-    private static ByteBuffer wrapString(String s) {
-        return ByteBuffer.wrap(s.getBytes(Charset.defaultCharset()));
-    }
-
-    @Test
-    public void testDeserialize() throws Exception {
-        assertEquals(Collections.singletonList("test"), scheme.deserialize(wrapString("test")));
-    }
-
-    @Test
-    public void testGetOutputFields() throws Exception {
-        Fields outputFields = scheme.getOutputFields();
-        assertTrue(outputFields.contains(StringScheme.STRING_SCHEME_KEY));
-        assertEquals(1, outputFields.size());
-    }
-
-    @Test
-    public void testDeserializeWithNullKeyAndValue() throws Exception {
-        assertEquals(Collections.singletonList("test"),
-                     scheme.deserializeKeyAndValue(null, wrapString("test")));
-    }
-
-    @Test
-    public void testDeserializeWithKeyAndValue() throws Exception {
-        assertEquals(Collections.singletonList(ImmutableMap.of("key", "test")),
-                     scheme.deserializeKeyAndValue(wrapString("key"), wrapString("test")));
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/test/org/apache/storm/kafka/TestStringScheme.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/TestStringScheme.java b/external/storm-kafka/src/test/org/apache/storm/kafka/TestStringScheme.java
deleted file mode 100644
index ac94e9d..0000000
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/TestStringScheme.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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;
-
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestStringScheme {
-    @Test
-    public void testDeserializeString() {
-        String s = "foo";
-        byte[] bytes = s.getBytes(StandardCharsets.UTF_8);
-        ByteBuffer direct = ByteBuffer.allocateDirect(bytes.length);
-        direct.put(bytes);
-        direct.flip();
-        String s1 = StringScheme.deserializeString(ByteBuffer.wrap(bytes));
-        String s2 = StringScheme.deserializeString(direct);
-        assertEquals(s, s1);
-        assertEquals(s, s2);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/test/org/apache/storm/kafka/TestUtils.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/TestUtils.java b/external/storm-kafka/src/test/org/apache/storm/kafka/TestUtils.java
deleted file mode 100644
index 921df3c..0000000
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/TestUtils.java
+++ /dev/null
@@ -1,97 +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;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-import kafka.api.OffsetRequest;
-import kafka.javaapi.consumer.SimpleConsumer;
-import kafka.javaapi.message.ByteBufferMessageSet;
-import kafka.message.Message;
-import kafka.message.MessageAndOffset;
-import org.apache.storm.kafka.trident.GlobalPartitionInformation;
-import org.apache.storm.utils.Utils;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestUtils {
-
-    public static final String TOPIC = "test";
-
-    public static GlobalPartitionInformation buildPartitionInfo(int numPartitions) {
-        return buildPartitionInfo(numPartitions, 9092);
-    }
-
-    public static List<GlobalPartitionInformation> buildPartitionInfoList(GlobalPartitionInformation partitionInformation) {
-        List<GlobalPartitionInformation> map = new ArrayList<GlobalPartitionInformation>();
-        map.add(partitionInformation);
-        return map;
-    }
-
-    public static GlobalPartitionInformation buildPartitionInfo(int numPartitions, int brokerPort) {
-        GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation(TOPIC);
-        for (int i = 0; i < numPartitions; i++) {
-            globalPartitionInformation.addPartition(i, Broker.fromString("broker-" + i + " :" + brokerPort));
-        }
-        return globalPartitionInformation;
-    }
-
-    public static SimpleConsumer getKafkaConsumer(KafkaTestBroker broker) {
-        BrokerHosts brokerHosts = getBrokerHosts(broker);
-        KafkaConfig kafkaConfig = new KafkaConfig(brokerHosts, TOPIC);
-        SimpleConsumer simpleConsumer = new SimpleConsumer("localhost", broker.getPort(), 60000, 1024, "testClient");
-        return simpleConsumer;
-    }
-
-    public static KafkaConfig getKafkaConfig(KafkaTestBroker broker) {
-        BrokerHosts brokerHosts = getBrokerHosts(broker);
-        KafkaConfig kafkaConfig = new KafkaConfig(brokerHosts, TOPIC);
-        return kafkaConfig;
-    }
-
-    private static BrokerHosts getBrokerHosts(KafkaTestBroker broker) {
-        GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation(TOPIC);
-        globalPartitionInformation.addPartition(0, Broker.fromString(broker.getBrokerConnectionString()));
-        return new StaticHosts(globalPartitionInformation);
-    }
-
-    public static Properties getProducerProperties(String brokerConnectionString) {
-        Properties props = new Properties();
-        props.put("bootstrap.servers", brokerConnectionString);
-        props.put("acks", "1");
-        props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-        props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-        return props;
-    }
-
-    public static boolean verifyMessage(String key, String message, KafkaTestBroker broker, SimpleConsumer simpleConsumer) {
-        long lastMessageOffset = KafkaUtils.getOffset(simpleConsumer, TestUtils.TOPIC, 0, OffsetRequest.LatestTime()) - 1;
-        ByteBufferMessageSet messageAndOffsets = KafkaUtils.fetchMessages(TestUtils.getKafkaConfig(broker), simpleConsumer,
-                                                                          new Partition(
-                                                                              Broker.fromString(broker.getBrokerConnectionString()),
-                                                                              TestUtils.TOPIC, 0), lastMessageOffset);
-        MessageAndOffset messageAndOffset = messageAndOffsets.iterator().next();
-        Message kafkaMessage = messageAndOffset.message();
-        ByteBuffer messageKeyBuffer = kafkaMessage.key();
-        String keyString = null;
-        String messageString = new String(Utils.toByteArray(kafkaMessage.payload()));
-        if (messageKeyBuffer != null) {
-            keyString = new String(Utils.toByteArray(messageKeyBuffer));
-        }
-        assertEquals(key, keyString);
-        assertEquals(message, messageString);
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/test/org/apache/storm/kafka/TridentKafkaTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/TridentKafkaTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/TridentKafkaTest.java
deleted file mode 100644
index 926b5fe..0000000
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/TridentKafkaTest.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.  The ASF licenses this file to you under the Apache License, Version
- * 2.0 (the "License"); you may not use this file except in compliance with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions
- * and limitations under the License.
- */
-
-package org.apache.storm.kafka;
-
-import java.util.ArrayList;
-import java.util.List;
-import kafka.javaapi.consumer.SimpleConsumer;
-import org.apache.storm.kafka.trident.TridentKafkaState;
-import org.apache.storm.kafka.trident.mapper.FieldNameBasedTupleToKafkaMapper;
-import org.apache.storm.kafka.trident.mapper.TridentTupleToKafkaMapper;
-import org.apache.storm.kafka.trident.selector.DefaultTopicSelector;
-import org.apache.storm.kafka.trident.selector.KafkaTopicSelector;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.apache.storm.trident.tuple.TridentTupleView;
-import org.apache.storm.tuple.Fields;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TridentKafkaTest {
-    private KafkaTestBroker broker;
-    private TridentKafkaState state;
-    private SimpleConsumer simpleConsumer;
-
-    @Before
-    public void setup() {
-        broker = new KafkaTestBroker();
-        simpleConsumer = TestUtils.getKafkaConsumer(broker);
-        TridentTupleToKafkaMapper<Object, Object> mapper = new FieldNameBasedTupleToKafkaMapper<Object, Object>("key", "message");
-        KafkaTopicSelector topicSelector = new DefaultTopicSelector(TestUtils.TOPIC);
-        state = new TridentKafkaState()
-            .withKafkaTopicSelector(topicSelector)
-            .withTridentTupleToKafkaMapper(mapper);
-        state.prepare(TestUtils.getProducerProperties(broker.getBrokerConnectionString()));
-    }
-
-    @Test
-    public void testKeyValue() {
-        String keyString = "key-123";
-        String valString = "message-123";
-        int batchSize = 10;
-
-        List<TridentTuple> tridentTuples = generateTupleBatch(keyString, valString, batchSize);
-
-        state.updateState(tridentTuples, null);
-
-        for (int i = 0; i < batchSize; i++) {
-            TestUtils.verifyMessage(keyString, valString, broker, simpleConsumer);
-        }
-    }
-
-    private List<TridentTuple> generateTupleBatch(String key, String message, int batchsize) {
-        List<TridentTuple> batch = new ArrayList<>();
-        for (int i = 0; i < batchsize; i++) {
-            batch.add(TridentTupleView.createFreshTuple(new Fields("key", "message"), key, message));
-        }
-        return batch;
-    }
-
-    @After
-    public void shutdown() {
-        simpleConsumer.close();
-        broker.shutdown();
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/test/org/apache/storm/kafka/ZkCoordinatorTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/ZkCoordinatorTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/ZkCoordinatorTest.java
deleted file mode 100644
index 93709cf..0000000
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/ZkCoordinatorTest.java
+++ /dev/null
@@ -1,191 +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;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import kafka.javaapi.consumer.SimpleConsumer;
-import org.apache.curator.test.TestingServer;
-import org.apache.storm.Config;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNotSame;
-import static org.junit.Assert.assertSame;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Mockito.when;
-
-public class ZkCoordinatorTest {
-
-
-    @Mock
-    private DynamicBrokersReader reader;
-
-    @Mock
-    private DynamicPartitionConnections dynamicPartitionConnections;
-
-    private KafkaTestBroker broker = new KafkaTestBroker();
-    private TestingServer server;
-    private Map<String, Object> topoConf = new HashMap();
-    private SpoutConfig spoutConfig;
-    private ZkState state;
-    private SimpleConsumer simpleConsumer;
-
-    @Before
-    public void setUp() throws Exception {
-        MockitoAnnotations.initMocks(this);
-        server = new TestingServer();
-        String connectionString = server.getConnectString();
-        ZkHosts hosts = new ZkHosts(connectionString);
-        hosts.refreshFreqSecs = 1;
-        spoutConfig = new SpoutConfig(hosts, "topic", "/test", "id");
-        Map<String, Object> conf = buildZookeeperConfig(server);
-        state = new ZkState(conf);
-        simpleConsumer = new SimpleConsumer("localhost", broker.getPort(), 60000, 1024, "testClient");
-        when(dynamicPartitionConnections.register(any(Broker.class), any(String.class), anyInt())).thenReturn(simpleConsumer);
-    }
-
-    private Map<String, Object> buildZookeeperConfig(TestingServer server) {
-        Map<String, Object> conf = new HashMap<>();
-        conf.put(Config.TRANSACTIONAL_ZOOKEEPER_PORT, server.getPort());
-        conf.put(Config.TRANSACTIONAL_ZOOKEEPER_SERVERS, Arrays.asList("localhost"));
-        conf.put(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT, 20000);
-        conf.put(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT, 20000);
-        conf.put(Config.STORM_ZOOKEEPER_RETRY_TIMES, 3);
-        conf.put(Config.STORM_ZOOKEEPER_RETRY_INTERVAL, 30);
-        return conf;
-    }
-
-    @After
-    public void shutdown() throws Exception {
-        simpleConsumer.close();
-        broker.shutdown();
-        server.close();
-    }
-
-    @Test
-    public void testOnePartitionPerTask() throws Exception {
-        int totalTasks = 64;
-        int partitionsPerTask = 1;
-        List<ZkCoordinator> coordinatorList = buildCoordinators(totalTasks / partitionsPerTask);
-        when(reader.getBrokerInfo()).thenReturn(TestUtils.buildPartitionInfoList(TestUtils.buildPartitionInfo(totalTasks)));
-        for (ZkCoordinator coordinator : coordinatorList) {
-            List<PartitionManager> myManagedPartitions = coordinator.getMyManagedPartitions();
-            assertEquals(partitionsPerTask, myManagedPartitions.size());
-            assertEquals(coordinator._taskIndex, myManagedPartitions.get(0).getPartition().partition);
-        }
-    }
-
-
-    @Test
-    public void testPartitionsChange() throws Exception {
-        final int totalTasks = 64;
-        int partitionsPerTask = 2;
-        List<ZkCoordinator> coordinatorList = buildCoordinators(totalTasks / partitionsPerTask);
-        when(reader.getBrokerInfo()).thenReturn(TestUtils.buildPartitionInfoList(TestUtils.buildPartitionInfo(totalTasks, 9092)));
-        List<List<PartitionManager>> partitionManagersBeforeRefresh = getPartitionManagers(coordinatorList);
-        waitForRefresh();
-        when(reader.getBrokerInfo()).thenReturn(TestUtils.buildPartitionInfoList(TestUtils.buildPartitionInfo(totalTasks, 9093)));
-        List<List<PartitionManager>> partitionManagersAfterRefresh = getPartitionManagers(coordinatorList);
-        assertEquals(partitionManagersBeforeRefresh.size(), partitionManagersAfterRefresh.size());
-        Iterator<List<PartitionManager>> iterator = partitionManagersAfterRefresh.iterator();
-        for (List<PartitionManager> partitionManagersBefore : partitionManagersBeforeRefresh) {
-            List<PartitionManager> partitionManagersAfter = iterator.next();
-            assertPartitionsAreDifferent(partitionManagersBefore, partitionManagersAfter, partitionsPerTask);
-        }
-    }
-
-    @Test
-    public void testPartitionManagerRecreate() throws Exception {
-        final int totalTasks = 2;
-        int partitionsPerTask = 2;
-        List<ZkCoordinator> coordinatorList = buildCoordinators(totalTasks / partitionsPerTask);
-        when(reader.getBrokerInfo()).thenReturn(TestUtils.buildPartitionInfoList(TestUtils.buildPartitionInfo(totalTasks, 9092)));
-        List<List<PartitionManager>> partitionManagersBeforeRefresh = getPartitionManagers(coordinatorList);
-        waitForRefresh();
-        when(reader.getBrokerInfo()).thenReturn(TestUtils.buildPartitionInfoList(TestUtils.buildPartitionInfo(totalTasks, 9093)));
-        List<List<PartitionManager>> partitionManagersAfterRefresh = getPartitionManagers(coordinatorList);
-        assertEquals(partitionManagersBeforeRefresh.size(), partitionManagersAfterRefresh.size());
-
-        HashMap<Integer, PartitionManager> managersAfterRefresh = new HashMap<Integer, PartitionManager>();
-        for (List<PartitionManager> partitionManagersAfter : partitionManagersAfterRefresh) {
-            for (PartitionManager manager : partitionManagersAfter) {
-                assertFalse("Multiple PartitionManagers for same partition",
-                            managersAfterRefresh.containsKey(manager.getPartition().partition));
-                managersAfterRefresh.put(manager.getPartition().partition, manager);
-            }
-        }
-
-        for (List<PartitionManager> partitionManagersBefore : partitionManagersBeforeRefresh) {
-            for (PartitionManager manager : partitionManagersBefore) {
-                assertStateIsTheSame(manager, managersAfterRefresh.get(manager.getPartition().partition));
-            }
-        }
-    }
-
-    private void assertStateIsTheSame(PartitionManager managerBefore, PartitionManager managerAfter) {
-        // check if state was actually moved from old PartitionManager
-        assertNotNull(managerBefore);
-        assertNotNull(managerAfter);
-        assertNotSame(managerBefore, managerAfter);
-        assertSame(managerBefore._waitingToEmit, managerAfter._waitingToEmit);
-        assertSame(managerBefore._emittedToOffset, managerAfter._emittedToOffset);
-        assertSame(managerBefore._committedTo, managerAfter._committedTo);
-    }
-
-    private void assertPartitionsAreDifferent(List<PartitionManager> partitionManagersBefore, List<PartitionManager> partitionManagersAfter,
-                                              int partitionsPerTask) {
-        assertEquals(partitionsPerTask, partitionManagersBefore.size());
-        assertEquals(partitionManagersBefore.size(), partitionManagersAfter.size());
-        for (int i = 0; i < partitionsPerTask; i++) {
-            assertNotEquals(partitionManagersBefore.get(i).getPartition(), partitionManagersAfter.get(i).getPartition());
-        }
-
-    }
-
-    private List<List<PartitionManager>> getPartitionManagers(List<ZkCoordinator> coordinatorList) {
-        List<List<PartitionManager>> partitions = new ArrayList<>();
-        for (ZkCoordinator coordinator : coordinatorList) {
-            partitions.add(coordinator.getMyManagedPartitions());
-        }
-        return partitions;
-    }
-
-    private void waitForRefresh() throws InterruptedException {
-        Thread.sleep(((ZkHosts) spoutConfig.hosts).refreshFreqSecs * 1000 + 1);
-    }
-
-    private List<ZkCoordinator> buildCoordinators(int totalTasks) {
-        List<ZkCoordinator> coordinatorList = new ArrayList<ZkCoordinator>();
-        for (int i = 0; i < totalTasks; i++) {
-            ZkCoordinator coordinator =
-                new ZkCoordinator(dynamicPartitionConnections, topoConf, spoutConfig, state, i, totalTasks, i, "test-id", reader);
-            coordinatorList.add(coordinator);
-        }
-        return coordinatorList;
-    }
-
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java
deleted file mode 100644
index 5c3053c..0000000
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/bolt/KafkaBoltTest.java
+++ /dev/null
@@ -1,355 +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.bolt;
-
-import com.google.common.collect.ImmutableList;
-import java.lang.reflect.Field;
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Properties;
-import java.util.concurrent.Future;
-import kafka.api.FetchRequest;
-import kafka.api.OffsetRequest;
-import kafka.javaapi.FetchResponse;
-import kafka.javaapi.OffsetResponse;
-import kafka.javaapi.consumer.SimpleConsumer;
-import kafka.javaapi.message.ByteBufferMessageSet;
-import kafka.message.Message;
-import kafka.message.MessageAndOffset;
-import org.apache.kafka.clients.producer.Callback;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.storm.Config;
-import org.apache.storm.Constants;
-import org.apache.storm.kafka.Broker;
-import org.apache.storm.kafka.BrokerHosts;
-import org.apache.storm.kafka.KafkaConfig;
-import org.apache.storm.kafka.KafkaTestBroker;
-import org.apache.storm.kafka.KafkaUtils;
-import org.apache.storm.kafka.Partition;
-import org.apache.storm.kafka.StaticHosts;
-import org.apache.storm.kafka.trident.GlobalPartitionInformation;
-import org.apache.storm.task.GeneralTopologyContext;
-import org.apache.storm.task.IOutputCollector;
-import org.apache.storm.task.OutputCollector;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Tuple;
-import org.apache.storm.tuple.TupleImpl;
-import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.TupleUtils;
-import org.apache.storm.utils.Utils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.anyInt;
-import static org.mockito.Mockito.anyString;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-public class KafkaBoltTest {
-
-    private static final String TEST_TOPIC = "test-topic";
-    private KafkaTestBroker broker;
-    private KafkaBolt bolt;
-    private Config config = new Config();
-    private KafkaConfig kafkaConfig;
-    private SimpleConsumer simpleConsumer;
-
-    @Mock
-    private IOutputCollector collector;
-
-    private static ByteBufferMessageSet mockSingleMessage(byte[] key, byte[] message) {
-        ByteBufferMessageSet sets = mock(ByteBufferMessageSet.class);
-        MessageAndOffset msg = mock(MessageAndOffset.class);
-        final List<MessageAndOffset> msgs = ImmutableList.of(msg);
-        doReturn(msgs.iterator()).when(sets).iterator();
-        Message kafkaMessage = mock(Message.class);
-        doReturn(ByteBuffer.wrap(key)).when(kafkaMessage).key();
-        doReturn(ByteBuffer.wrap(message)).when(kafkaMessage).payload();
-        doReturn(kafkaMessage).when(msg).message();
-        return sets;
-    }
-
-    private static SimpleConsumer mockSimpleConsumer(ByteBufferMessageSet mockMsg) {
-        SimpleConsumer simpleConsumer = mock(SimpleConsumer.class);
-        FetchResponse resp = mock(FetchResponse.class);
-        doReturn(resp).when(simpleConsumer).fetch(any(FetchRequest.class));
-        OffsetResponse mockOffsetResponse = mock(OffsetResponse.class);
-        doReturn(new long[]{}).when(mockOffsetResponse).offsets(anyString(), anyInt());
-        doReturn(mockOffsetResponse).when(simpleConsumer).getOffsetsBefore(any(kafka.javaapi.OffsetRequest.class));
-        doReturn(mockMsg).when(resp).messageSet(anyString(), anyInt());
-        return simpleConsumer;
-    }
-
-    @Before
-    public void initMocks() {
-        MockitoAnnotations.initMocks(this);
-        broker = new KafkaTestBroker();
-        setupKafkaConsumer();
-        config.put(KafkaBolt.TOPIC, TEST_TOPIC);
-        bolt = generateStringSerializerBolt();
-    }
-
-    @After
-    public void shutdown() {
-        simpleConsumer.close();
-        broker.shutdown();
-        bolt.cleanup();
-    }
-
-    private void setupKafkaConsumer() {
-        GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation(TEST_TOPIC);
-        globalPartitionInformation.addPartition(0, Broker.fromString(broker.getBrokerConnectionString()));
-        BrokerHosts brokerHosts = new StaticHosts(globalPartitionInformation);
-        kafkaConfig = new KafkaConfig(brokerHosts, TEST_TOPIC);
-        simpleConsumer = new SimpleConsumer("localhost", broker.getPort(), 60000, 1024, "testClient");
-    }
-
-    @Test
-    public void shouldNotAcknowledgeTickTuples() throws Exception {
-        // Given
-        Tuple tickTuple = mockTickTuple();
-
-        // When
-        bolt.execute(tickTuple);
-
-        // Then
-        verify(collector, never()).ack(tickTuple);
-    }
-
-    @Test
-    public void executeWithKey() throws Exception {
-        String message = "value-123";
-        String key = "key-123";
-        Tuple tuple = generateTestTuple(key, message);
-        bolt.execute(tuple);
-        verify(collector).ack(tuple);
-        verifyMessage(key, message);
-    }
-
-    /* test synchronous sending */
-    @Test
-    public void executeWithByteArrayKeyAndMessageSync() throws Exception {
-        boolean async = false;
-        boolean fireAndForget = false;
-        bolt = generateDefaultSerializerBolt(async, fireAndForget, null);
-        String keyString = "test-key";
-        String messageString = "test-message";
-        byte[] key = keyString.getBytes();
-        byte[] message = messageString.getBytes();
-        Tuple tuple = generateTestTuple(key, message);
-        bolt.execute(tuple);
-        verify(collector).ack(tuple);
-        verifyMessage(keyString, messageString);
-    }
-
-    /* test asynchronous sending (default) */
-    @Test
-    public void executeWithByteArrayKeyAndMessageAsync() throws Exception {
-        boolean async = true;
-        boolean fireAndForget = false;
-        String keyString = "test-key";
-        String messageString = "test-message";
-        byte[] key = keyString.getBytes();
-        byte[] message = messageString.getBytes();
-        final Tuple tuple = generateTestTuple(key, message);
-
-        final ByteBufferMessageSet mockMsg = mockSingleMessage(key, message);
-        simpleConsumer.close();
-        simpleConsumer = mockSimpleConsumer(mockMsg);
-        KafkaProducer<?, ?> producer = mock(KafkaProducer.class);
-        when(producer.send(any(ProducerRecord.class), any(Callback.class))).thenAnswer(new Answer<Future>() {
-            @Override
-            public Future answer(InvocationOnMock invocationOnMock) throws Throwable {
-                Callback cb = (Callback) invocationOnMock.getArguments()[1];
-                cb.onCompletion(null, null);
-                return mock(Future.class);
-            }
-        });
-        bolt = generateDefaultSerializerBolt(async, fireAndForget, producer);
-        bolt.execute(tuple);
-        verify(collector).ack(tuple);
-        verifyMessage(keyString, messageString);
-    }
-
-    /* test with fireAndForget option enabled */
-    @Test
-    public void executeWithByteArrayKeyAndMessageFire() throws Exception {
-        boolean async = true;
-        boolean fireAndForget = true;
-        bolt = generateDefaultSerializerBolt(async, fireAndForget, null);
-        String keyString = "test-key";
-        String messageString = "test-message";
-        byte[] key = keyString.getBytes();
-        byte[] message = messageString.getBytes();
-        Tuple tuple = generateTestTuple(key, message);
-        final ByteBufferMessageSet mockMsg = mockSingleMessage(key, message);
-        simpleConsumer.close();
-        simpleConsumer = mockSimpleConsumer(mockMsg);
-        KafkaProducer<?, ?> producer = mock(KafkaProducer.class);
-        // do not invoke the callback of send() in order to test whether the bolt handle the fireAndForget option
-        // properly.
-        doReturn(mock(Future.class)).when(producer).send(any(ProducerRecord.class), any(Callback.class));
-        bolt.execute(tuple);
-        verify(collector).ack(tuple);
-        verifyMessage(keyString, messageString);
-    }
-
-    /* test bolt specified properties */
-    @Test
-    public void executeWithBoltSpecifiedProperties() {
-        boolean async = false;
-        boolean fireAndForget = false;
-        bolt = defaultSerializerBoltWithSpecifiedProperties(async, fireAndForget);
-        String keyString = "test-key";
-        String messageString = "test-message";
-        byte[] key = keyString.getBytes();
-        byte[] message = messageString.getBytes();
-        Tuple tuple = generateTestTuple(key, message);
-        bolt.execute(tuple);
-        verify(collector).ack(tuple);
-        verifyMessage(keyString, messageString);
-    }
-
-    private KafkaBolt generateStringSerializerBolt() {
-        Properties props = new Properties();
-        props.put("acks", "1");
-        props.put("bootstrap.servers", broker.getBrokerConnectionString());
-        props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-        props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-        props.put("metadata.fetch.timeout.ms", 1000);
-        KafkaBolt bolt = new KafkaBolt().withProducerProperties(props);
-        bolt.prepare(config, null, new OutputCollector(collector));
-        bolt.setAsync(false);
-        return bolt;
-    }
-
-    private KafkaBolt generateDefaultSerializerBolt(boolean async, boolean fireAndForget,
-                                                    KafkaProducer<?, ?> mockProducer) throws Exception {
-        Properties props = new Properties();
-        props.put("acks", "1");
-        props.put("bootstrap.servers", broker.getBrokerConnectionString());
-        props.put("key.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer");
-        props.put("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer");
-        props.put("metadata.fetch.timeout.ms", 1000);
-        props.put("linger.ms", 0);
-        KafkaBolt bolt = new KafkaBolt().withProducerProperties(props);
-        bolt.prepare(config, null, new OutputCollector(collector));
-        bolt.setAsync(async);
-        bolt.setFireAndForget(fireAndForget);
-        if (mockProducer != null) {
-            Field producerField = bolt.getClass().getDeclaredField("producer");
-            producerField.setAccessible(true);
-            producerField.set(bolt, mockProducer);
-        }
-        return bolt;
-    }
-
-    private KafkaBolt defaultSerializerBoltWithSpecifiedProperties(boolean async, boolean fireAndForget) {
-        Properties props = new Properties();
-        props.put("acks", "1");
-        props.put("bootstrap.servers", broker.getBrokerConnectionString());
-        props.put("key.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer");
-        props.put("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer");
-        props.put("metadata.fetch.timeout.ms", 1000);
-        props.put("linger.ms", 0);
-        KafkaBolt bolt = new KafkaBolt().withProducerProperties(props);
-        bolt.prepare(config, null, new OutputCollector(collector));
-        bolt.setAsync(async);
-        bolt.setFireAndForget(fireAndForget);
-        return bolt;
-    }
-
-    @Test
-    public void executeWithoutKey() throws Exception {
-        String message = "value-234";
-        Tuple tuple = generateTestTuple(message);
-        bolt.execute(tuple);
-        verify(collector).ack(tuple);
-        verifyMessage(null, message);
-    }
-
-    @Test
-    public void executeWithBrokerDown() throws Exception {
-        broker.shutdown();
-        String message = "value-234";
-        Tuple tuple = generateTestTuple(message);
-        bolt.execute(tuple);
-        verify(collector).fail(tuple);
-    }
-
-    private boolean verifyMessage(String key, String message) {
-        long lastMessageOffset = KafkaUtils.getOffset(simpleConsumer, kafkaConfig.topic, 0, OffsetRequest.LatestTime()) - 1;
-        ByteBufferMessageSet messageAndOffsets = KafkaUtils.fetchMessages(kafkaConfig, simpleConsumer,
-                                                                          new Partition(
-                                                                              Broker.fromString(broker.getBrokerConnectionString()),
-                                                                              kafkaConfig.topic, 0), lastMessageOffset);
-        MessageAndOffset messageAndOffset = messageAndOffsets.iterator().next();
-        Message kafkaMessage = messageAndOffset.message();
-        ByteBuffer messageKeyBuffer = kafkaMessage.key();
-        String keyString = null;
-        String messageString = new String(Utils.toByteArray(kafkaMessage.payload()));
-        if (messageKeyBuffer != null) {
-            keyString = new String(Utils.toByteArray(messageKeyBuffer));
-        }
-        assertEquals(key, keyString);
-        assertEquals(message, messageString);
-        return true;
-    }
-
-    private Tuple generateTestTuple(Object key, Object message) {
-        TopologyBuilder builder = new TopologyBuilder();
-        GeneralTopologyContext topologyContext =
-            new GeneralTopologyContext(builder.createTopology(), new Config(), new HashMap<>(), new HashMap<>(), new HashMap<>(), "") {
-                @Override
-                public Fields getComponentOutputFields(String componentId, String streamId) {
-                    return new Fields("key", "message");
-                }
-            };
-        return new TupleImpl(topologyContext, new Values(key, message), topologyContext.getComponentId(1), 1, "");
-    }
-
-    private Tuple generateTestTuple(Object message) {
-        TopologyBuilder builder = new TopologyBuilder();
-        GeneralTopologyContext topologyContext =
-            new GeneralTopologyContext(builder.createTopology(), new Config(), new HashMap<>(), new HashMap<>(), new HashMap<>(), "") {
-                @Override
-                public Fields getComponentOutputFields(String componentId, String streamId) {
-                    return new Fields("message");
-                }
-            };
-        return new TupleImpl(topologyContext, new Values(message), topologyContext.getComponentId(1), 1, "");
-    }
-
-    private Tuple mockTickTuple() {
-        Tuple tuple = mock(Tuple.class);
-        when(tuple.getSourceComponent()).thenReturn(Constants.SYSTEM_COMPONENT_ID);
-        when(tuple.getSourceStreamId()).thenReturn(Constants.SYSTEM_TICK_STREAM_ID);
-        // Sanity check
-        assertTrue(TupleUtils.isTick(tuple));
-        return tuple;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/flux/flux-core/pom.xml
----------------------------------------------------------------------
diff --git a/flux/flux-core/pom.xml b/flux/flux-core/pom.xml
index d29e619..f457562 100644
--- a/flux/flux-core/pom.xml
+++ b/flux/flux-core/pom.xml
@@ -43,6 +43,11 @@
             <scope>test</scope>
         </dependency>
         <dependency>
+            <groupId>org.apache.kafka</groupId>
+            <artifactId>kafka-clients</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
             <groupId>org.apache.storm</groupId>
             <artifactId>storm-hdfs</artifactId>
             <version>${project.version}</version>

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/flux/flux-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flux/flux-examples/pom.xml b/flux/flux-examples/pom.xml
index dca5a83..c084709 100644
--- a/flux/flux-examples/pom.xml
+++ b/flux/flux-examples/pom.xml
@@ -97,6 +97,10 @@
             <artifactId>storm-kafka-client</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.kafka</groupId>
+            <artifactId>kafka-clients</artifactId>
+        </dependency>
     </dependencies>
 
     <build>

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/flux/pom.xml
----------------------------------------------------------------------
diff --git a/flux/pom.xml b/flux/pom.xml
index 834cfa7..2e41147 100644
--- a/flux/pom.xml
+++ b/flux/pom.xml
@@ -56,10 +56,5 @@
             <artifactId>commons-cli</artifactId>
             <version>1.2</version>
         </dependency>
-        <dependency>
-            <groupId>org.apache.kafka</groupId>
-            <artifactId>${storm.kafka.artifact.id}</artifactId>
-            <scope>test</scope>
-        </dependency>
     </dependencies>
 </project>


[4/7] storm git commit: STORM-2953: Remove storm-kafka

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java
deleted file mode 100644
index 38958b2..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java
+++ /dev/null
@@ -1,288 +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;
-
-import com.google.common.base.Preconditions;
-import java.io.IOException;
-import java.net.ConnectException;
-import java.net.SocketTimeoutException;
-import java.nio.ByteBuffer;
-import java.nio.channels.UnresolvedAddressException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import kafka.api.FetchRequest;
-import kafka.api.FetchRequestBuilder;
-import kafka.api.PartitionOffsetRequestInfo;
-import kafka.common.TopicAndPartition;
-import kafka.javaapi.FetchResponse;
-import kafka.javaapi.OffsetRequest;
-import kafka.javaapi.consumer.SimpleConsumer;
-import kafka.javaapi.message.ByteBufferMessageSet;
-import kafka.message.Message;
-import org.apache.storm.kafka.trident.GlobalPartitionInformation;
-import org.apache.storm.kafka.trident.IBrokerReader;
-import org.apache.storm.kafka.trident.StaticBrokerReader;
-import org.apache.storm.kafka.trident.ZkBrokerReader;
-import org.apache.storm.metric.api.IMetric;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-public class KafkaUtils {
-
-    private static final Logger LOG = LoggerFactory.getLogger(KafkaUtils.class);
-    private static final int NO_OFFSET = -5;
-
-    //suppress default constructor for noninstantiablility
-    private KafkaUtils() {
-        throw new AssertionError();
-    }
-
-    public static IBrokerReader makeBrokerReader(Map<String, Object> topoConf, KafkaConfig conf) {
-        if (conf.hosts instanceof StaticHosts) {
-            return new StaticBrokerReader(conf.topic, ((StaticHosts) conf.hosts).getPartitionInformation());
-        } else {
-            return new ZkBrokerReader(topoConf, conf.topic, (ZkHosts) conf.hosts);
-        }
-    }
-
-
-    public static long getOffset(SimpleConsumer consumer, String topic, int partition, KafkaConfig config) {
-        long startOffsetTime = config.startOffsetTime;
-        return getOffset(consumer, topic, partition, startOffsetTime);
-    }
-
-    public static long getOffset(SimpleConsumer consumer, String topic, int partition, long startOffsetTime) {
-        TopicAndPartition topicAndPartition = new TopicAndPartition(topic, partition);
-        Map<TopicAndPartition, PartitionOffsetRequestInfo> requestInfo = new HashMap<TopicAndPartition, PartitionOffsetRequestInfo>();
-        requestInfo.put(topicAndPartition, new PartitionOffsetRequestInfo(startOffsetTime, 1));
-        OffsetRequest request = new OffsetRequest(
-            requestInfo, kafka.api.OffsetRequest.CurrentVersion(), consumer.clientId());
-
-        long[] offsets = consumer.getOffsetsBefore(request).offsets(topic, partition);
-        if (offsets.length > 0) {
-            return offsets[0];
-        } else {
-            return NO_OFFSET;
-        }
-    }
-
-    public static ByteBufferMessageSet fetchMessages(KafkaConfig config, SimpleConsumer consumer, Partition partition, long offset)
-        throws TopicOffsetOutOfRangeException, FailedFetchException, RuntimeException {
-        ByteBufferMessageSet msgs = null;
-        String topic = partition.topic;
-        int partitionId = partition.partition;
-        FetchRequestBuilder builder = new FetchRequestBuilder();
-        FetchRequest fetchRequest = builder.addFetch(topic, partitionId, offset, config.fetchSizeBytes).
-            clientId(config.clientId).maxWait(config.fetchMaxWait).minBytes(config.minFetchByte).build();
-        FetchResponse fetchResponse;
-        try {
-            fetchResponse = consumer.fetch(fetchRequest);
-        } catch (Exception e) {
-            if (e instanceof ConnectException ||
-                e instanceof SocketTimeoutException ||
-                e instanceof IOException ||
-                e instanceof UnresolvedAddressException
-                ) {
-                LOG.warn("Network error when fetching messages:", e);
-                throw new FailedFetchException(e);
-            } else {
-                throw new RuntimeException(e);
-            }
-        }
-        if (fetchResponse.hasError()) {
-            KafkaError error = KafkaError.getError(fetchResponse.errorCode(topic, partitionId));
-            if (error.equals(KafkaError.OFFSET_OUT_OF_RANGE) && config.useStartOffsetTimeIfOffsetOutOfRange) {
-                String msg = partition + " Got fetch request with offset out of range: [" + offset + "]";
-                LOG.warn(msg);
-                throw new TopicOffsetOutOfRangeException(msg);
-            } else {
-                String message = "Error fetching data from [" + partition + "] for topic [" + topic + "]: [" + error + "]";
-                LOG.error(message);
-                throw new FailedFetchException(message);
-            }
-        } else {
-            msgs = fetchResponse.messageSet(topic, partitionId);
-        }
-        LOG.debug("Messages fetched. [config = {}], [consumer = {}], [partition = {}], [offset = {}], [msgs = {}]", config, consumer,
-                  partition, offset, msgs);
-        return msgs;
-    }
-
-    public static Iterable<List<Object>> generateTuples(KafkaConfig kafkaConfig, Message msg, String topic) {
-        Iterable<List<Object>> tups;
-        ByteBuffer payload = msg.payload();
-        if (payload == null) {
-            return null;
-        }
-        ByteBuffer key = msg.key();
-        if (key != null && kafkaConfig.scheme instanceof KeyValueSchemeAsMultiScheme) {
-            tups = ((KeyValueSchemeAsMultiScheme) kafkaConfig.scheme).deserializeKeyAndValue(key, payload);
-        } else {
-            if (kafkaConfig.scheme instanceof StringMultiSchemeWithTopic) {
-                tups = ((StringMultiSchemeWithTopic) kafkaConfig.scheme).deserializeWithTopic(topic, payload);
-            } else {
-                tups = kafkaConfig.scheme.deserialize(payload);
-            }
-        }
-        return tups;
-    }
-
-    public static Iterable<List<Object>> generateTuples(MessageMetadataSchemeAsMultiScheme scheme, Message msg, Partition partition,
-                                                        long offset) {
-        ByteBuffer payload = msg.payload();
-        if (payload == null) {
-            return null;
-        }
-        return scheme.deserializeMessageWithMetadata(payload, partition, offset);
-    }
-
-    public static List<Partition> calculatePartitionsForTask(List<GlobalPartitionInformation> partitons,
-                                                             int totalTasks, int taskIndex, int taskId) {
-        Preconditions.checkArgument(taskIndex < totalTasks, "task index must be less that total tasks");
-        List<Partition> taskPartitions = new ArrayList<Partition>();
-        List<Partition> partitions = new ArrayList<Partition>();
-        for (GlobalPartitionInformation partitionInformation : partitons) {
-            partitions.addAll(partitionInformation.getOrderedPartitions());
-        }
-        int numPartitions = partitions.size();
-        if (numPartitions < totalTasks) {
-            LOG.warn("there are more tasks than partitions (tasks: " + totalTasks + "; partitions: " + numPartitions +
-                     "), some tasks will be idle");
-        }
-        for (int i = taskIndex; i < numPartitions; i += totalTasks) {
-            Partition taskPartition = partitions.get(i);
-            taskPartitions.add(taskPartition);
-        }
-        logPartitionMapping(totalTasks, taskIndex, taskPartitions, taskId);
-        return taskPartitions;
-    }
-
-    private static void logPartitionMapping(int totalTasks, int taskIndex, List<Partition> taskPartitions, int taskId) {
-        String taskPrefix = taskPrefix(taskIndex, totalTasks, taskId);
-        if (taskPartitions.isEmpty()) {
-            LOG.warn(taskPrefix + " no partitions assigned");
-        } else {
-            LOG.info(taskPrefix + " assigned " + taskPartitions);
-        }
-    }
-
-    public static String taskPrefix(int taskIndex, int totalTasks, int taskId) {
-        return "Task [" + (taskIndex + 1) + "/" + totalTasks + "], Task-ID: " + taskId;
-    }
-
-    public static class KafkaOffsetMetric implements IMetric {
-        Map<Partition, PartitionManager.OffsetData> _partitionToOffset = new HashMap<Partition, PartitionManager.OffsetData>();
-        Set<Partition> _partitions;
-        DynamicPartitionConnections _connections;
-
-        public KafkaOffsetMetric(DynamicPartitionConnections connections) {
-            _connections = connections;
-        }
-
-        public void setOffsetData(Partition partition, PartitionManager.OffsetData offsetData) {
-            _partitionToOffset.put(partition, offsetData);
-        }
-
-        @Override
-        public Object getValueAndReset() {
-            try {
-                HashMap<String, Long> ret = new HashMap<>();
-                if (_partitions != null && _partitions.size() == _partitionToOffset.size()) {
-                    Map<String, TopicMetrics> topicMetricsMap = new TreeMap<String, TopicMetrics>();
-                    for (Map.Entry<Partition, PartitionManager.OffsetData> e : _partitionToOffset.entrySet()) {
-                        Partition partition = e.getKey();
-                        SimpleConsumer consumer = _connections.getConnection(partition);
-                        if (consumer == null) {
-                            LOG.warn("partitionToOffset contains partition not found in _connections. Stale partition data?");
-                            return null;
-                        }
-                        long latestTimeOffset =
-                            getOffset(consumer, partition.topic, partition.partition, kafka.api.OffsetRequest.LatestTime());
-                        long earliestTimeOffset =
-                            getOffset(consumer, partition.topic, partition.partition, kafka.api.OffsetRequest.EarliestTime());
-                        if (latestTimeOffset == KafkaUtils.NO_OFFSET) {
-                            LOG.warn("No data found in Kafka Partition " + partition.getId());
-                            return null;
-                        }
-                        long latestEmittedOffset = e.getValue().latestEmittedOffset;
-                        long latestCompletedOffset = e.getValue().latestCompletedOffset;
-                        long spoutLag = latestTimeOffset - latestCompletedOffset;
-                        String topic = partition.topic;
-                        String metricPath = partition.getId();
-                        //Handle the case where Partition Path Id does not contain topic name Partition.getId() == "partition_" + partition
-                        if (!metricPath.startsWith(topic + "/")) {
-                            metricPath = topic + "/" + metricPath;
-                        }
-                        ret.put(metricPath + "/" + "spoutLag", spoutLag);
-                        ret.put(metricPath + "/" + "earliestTimeOffset", earliestTimeOffset);
-                        ret.put(metricPath + "/" + "latestTimeOffset", latestTimeOffset);
-                        ret.put(metricPath + "/" + "latestEmittedOffset", latestEmittedOffset);
-                        ret.put(metricPath + "/" + "latestCompletedOffset", latestCompletedOffset);
-
-                        if (!topicMetricsMap.containsKey(partition.topic)) {
-                            topicMetricsMap.put(partition.topic, new TopicMetrics());
-                        }
-
-                        TopicMetrics topicMetrics = topicMetricsMap.get(partition.topic);
-                        topicMetrics.totalSpoutLag += spoutLag;
-                        topicMetrics.totalEarliestTimeOffset += earliestTimeOffset;
-                        topicMetrics.totalLatestTimeOffset += latestTimeOffset;
-                        topicMetrics.totalLatestEmittedOffset += latestEmittedOffset;
-                        topicMetrics.totalLatestCompletedOffset += latestCompletedOffset;
-                    }
-
-                    for (Map.Entry<String, TopicMetrics> e : topicMetricsMap.entrySet()) {
-                        String topic = e.getKey();
-                        TopicMetrics topicMetrics = e.getValue();
-                        ret.put(topic + "/" + "totalSpoutLag", topicMetrics.totalSpoutLag);
-                        ret.put(topic + "/" + "totalEarliestTimeOffset", topicMetrics.totalEarliestTimeOffset);
-                        ret.put(topic + "/" + "totalLatestTimeOffset", topicMetrics.totalLatestTimeOffset);
-                        ret.put(topic + "/" + "totalLatestEmittedOffset", topicMetrics.totalLatestEmittedOffset);
-                        ret.put(topic + "/" + "totalLatestCompletedOffset", topicMetrics.totalLatestCompletedOffset);
-                    }
-
-                    return ret;
-                } else {
-                    LOG.info("Metrics Tick: Not enough data to calculate spout lag.");
-                }
-            } catch (Throwable t) {
-                LOG.warn("Metrics Tick: Exception when computing kafkaOffset metric.", t);
-            }
-            return null;
-        }
-
-        public void refreshPartitions(Set<Partition> partitions) {
-            _partitions = partitions;
-            Iterator<Partition> it = _partitionToOffset.keySet().iterator();
-            while (it.hasNext()) {
-                if (!partitions.contains(it.next())) {
-                    it.remove();
-                }
-            }
-        }
-
-        private class TopicMetrics {
-            long totalSpoutLag = 0;
-            long totalEarliestTimeOffset = 0;
-            long totalLatestTimeOffset = 0;
-            long totalLatestEmittedOffset = 0;
-            long totalLatestCompletedOffset = 0;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/KeyValueScheme.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KeyValueScheme.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KeyValueScheme.java
deleted file mode 100644
index 6bb1dc5..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KeyValueScheme.java
+++ /dev/null
@@ -1,21 +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;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-import org.apache.storm.spout.Scheme;
-
-public interface KeyValueScheme extends Scheme {
-    List<Object> deserializeKeyAndValue(ByteBuffer key, ByteBuffer value);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/KeyValueSchemeAsMultiScheme.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KeyValueSchemeAsMultiScheme.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KeyValueSchemeAsMultiScheme.java
deleted file mode 100644
index 00983cc..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KeyValueSchemeAsMultiScheme.java
+++ /dev/null
@@ -1,35 +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;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.storm.spout.SchemeAsMultiScheme;
-
-public class KeyValueSchemeAsMultiScheme extends SchemeAsMultiScheme {
-
-    public KeyValueSchemeAsMultiScheme(KeyValueScheme scheme) {
-        super(scheme);
-    }
-
-    public Iterable<List<Object>> deserializeKeyAndValue(final ByteBuffer key, final ByteBuffer value) {
-        List<Object> o = ((KeyValueScheme) scheme).deserializeKeyAndValue(key, value);
-        if (o == null) {
-            return null;
-        } else {
-            return Arrays.asList(o);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/MessageMetadataScheme.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/MessageMetadataScheme.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/MessageMetadataScheme.java
deleted file mode 100644
index f77f419..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/MessageMetadataScheme.java
+++ /dev/null
@@ -1,21 +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;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-import org.apache.storm.spout.Scheme;
-
-public interface MessageMetadataScheme extends Scheme {
-    List<Object> deserializeMessageWithMetadata(ByteBuffer message, Partition partition, long offset);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/MessageMetadataSchemeAsMultiScheme.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/MessageMetadataSchemeAsMultiScheme.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/MessageMetadataSchemeAsMultiScheme.java
deleted file mode 100644
index f52a772..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/MessageMetadataSchemeAsMultiScheme.java
+++ /dev/null
@@ -1,35 +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;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.storm.spout.SchemeAsMultiScheme;
-
-public class MessageMetadataSchemeAsMultiScheme extends SchemeAsMultiScheme {
-    private static final long serialVersionUID = -7172403703813625116L;
-
-    public MessageMetadataSchemeAsMultiScheme(MessageMetadataScheme scheme) {
-        super(scheme);
-    }
-
-    public Iterable<List<Object>> deserializeMessageWithMetadata(ByteBuffer message, Partition partition, long offset) {
-        List<Object> o = ((MessageMetadataScheme) scheme).deserializeMessageWithMetadata(message, partition, offset);
-        if (o == null) {
-            return null;
-        } else {
-            return Arrays.asList(o);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/Partition.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/Partition.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/Partition.java
deleted file mode 100644
index 9edf28b..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/Partition.java
+++ /dev/null
@@ -1,85 +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;
-
-import com.google.common.base.Objects;
-import java.io.Serializable;
-import org.apache.storm.trident.spout.ISpoutPartition;
-
-
-public class Partition implements ISpoutPartition, Serializable {
-
-    public Broker host;
-    public int partition;
-    public String topic;
-
-    //Flag to keep the Partition Path Id backward compatible with Old implementation of Partition.getId() == "partition_" + partition
-    private Boolean bUseTopicNameForPartitionPathId;
-
-    // for kryo compatibility
-    private Partition() {
-
-    }
-
-    public Partition(Broker host, String topic, int partition) {
-        this.topic = topic;
-        this.host = host;
-        this.partition = partition;
-        this.bUseTopicNameForPartitionPathId = false;
-    }
-
-    public Partition(Broker host, String topic, int partition, Boolean bUseTopicNameForPartitionPathId) {
-        this.topic = topic;
-        this.host = host;
-        this.partition = partition;
-        this.bUseTopicNameForPartitionPathId = bUseTopicNameForPartitionPathId;
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hashCode(host, topic, partition);
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj) {
-            return true;
-        }
-        if (obj == null || getClass() != obj.getClass()) {
-            return false;
-        }
-        final Partition other = (Partition) obj;
-        return Objects.equal(this.host, other.host) && Objects.equal(this.topic, other.topic) &&
-               Objects.equal(this.partition, other.partition);
-    }
-
-    @Override
-    public String toString() {
-        return "Partition{" +
-               "host=" + host +
-               ", topic=" + topic +
-               ", partition=" + partition +
-               '}';
-    }
-
-    @Override
-    public String getId() {
-        if (bUseTopicNameForPartitionPathId) {
-            return topic + "/partition_" + partition;
-        } else {
-            //Keep the Partition Id backward compatible with Old implementation of Partition.getId() == "partition_" + partition
-            return "partition_" + partition;
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionCoordinator.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionCoordinator.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionCoordinator.java
deleted file mode 100644
index 4dba709..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionCoordinator.java
+++ /dev/null
@@ -1,23 +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;
-
-import java.util.List;
-
-public interface PartitionCoordinator {
-    List<PartitionManager> getMyManagedPartitions();
-
-    PartitionManager getManager(Partition partition);
-
-    void refresh();
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java
deleted file mode 100644
index 5805c21..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java
+++ /dev/null
@@ -1,405 +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;
-
-import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableMap;
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import kafka.javaapi.consumer.SimpleConsumer;
-import kafka.javaapi.message.ByteBufferMessageSet;
-import kafka.message.MessageAndOffset;
-import org.apache.storm.Config;
-import org.apache.storm.kafka.KafkaSpout.EmitState;
-import org.apache.storm.kafka.trident.MaxMetric;
-import org.apache.storm.metric.api.CombinedMetric;
-import org.apache.storm.metric.api.CountMetric;
-import org.apache.storm.metric.api.MeanReducer;
-import org.apache.storm.metric.api.ReducedMetric;
-import org.apache.storm.spout.SpoutOutputCollector;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class PartitionManager {
-    private static final Logger LOG = LoggerFactory.getLogger(PartitionManager.class);
-
-    private final CombinedMetric _fetchAPILatencyMax;
-    private final ReducedMetric _fetchAPILatencyMean;
-    private final CountMetric _fetchAPICallCount;
-    private final CountMetric _fetchAPIMessageCount;
-    // Count of messages which could not be emitted or retried because they were deleted from kafka
-    private final CountMetric _lostMessageCount;
-    // Count of messages which were not retried because failedMsgRetryManager didn't consider offset eligible for
-    // retry
-    private final CountMetric _messageIneligibleForRetryCount;
-    private final FailedMsgRetryManager _failedMsgRetryManager;
-    Long _emittedToOffset;
-    // retryRecords key = Kafka offset, value = retry info for the given message
-    Long _committedTo;
-    LinkedList<MessageAndOffset> _waitingToEmit = new LinkedList<MessageAndOffset>();
-    Partition _partition;
-    SpoutConfig _spoutConfig;
-    String _topologyInstanceId;
-    SimpleConsumer _consumer;
-    DynamicPartitionConnections _connections;
-    ZkState _state;
-    Map _topoConf;
-    long numberFailed, numberAcked;
-    // _pending key = Kafka offset, value = time at which the message was first submitted to the topology
-    private SortedMap<Long, Long> _pending = new TreeMap<Long, Long>();
-
-    public PartitionManager(
-        DynamicPartitionConnections connections,
-        String topologyInstanceId,
-        ZkState state,
-        Map<String, Object> topoConf,
-        SpoutConfig spoutConfig,
-        Partition id) {
-        this(connections, topologyInstanceId, state, topoConf, spoutConfig, id, null);
-    }
-
-    /**
-     * @param previousManager previous partition manager if manager for partition is being recreated
-     */
-    public PartitionManager(
-        DynamicPartitionConnections connections,
-        String topologyInstanceId,
-        ZkState state,
-        Map<String, Object> topoConf,
-        SpoutConfig spoutConfig,
-        Partition id,
-        PartitionManager previousManager) {
-        _partition = id;
-        _connections = connections;
-        _spoutConfig = spoutConfig;
-        _topologyInstanceId = topologyInstanceId;
-        _consumer = connections.register(id.host, id.topic, id.partition);
-        _state = state;
-        _topoConf = topoConf;
-        numberAcked = numberFailed = 0;
-
-        if (previousManager != null) {
-            _failedMsgRetryManager = previousManager._failedMsgRetryManager;
-            _committedTo = previousManager._committedTo;
-            _emittedToOffset = previousManager._emittedToOffset;
-            _waitingToEmit = previousManager._waitingToEmit;
-            _pending = previousManager._pending;
-            LOG.info("Recreating PartitionManager based on previous manager, _waitingToEmit size: {}, _pending size: {}",
-                     _waitingToEmit.size(),
-                     _pending.size());
-        } else {
-            try {
-                _failedMsgRetryManager = (FailedMsgRetryManager) Class.forName(spoutConfig.failedMsgRetryManagerClass).newInstance();
-                _failedMsgRetryManager.prepare(spoutConfig, _topoConf);
-            } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) {
-                throw new IllegalArgumentException(String.format("Failed to create an instance of <%s> from: <%s>",
-                                                                 FailedMsgRetryManager.class,
-                                                                 spoutConfig.failedMsgRetryManagerClass), e);
-            }
-
-            String jsonTopologyId = null;
-            Long jsonOffset = null;
-            String path = committedPath();
-            try {
-                Map<Object, Object> json = _state.readJSON(path);
-                LOG.info("Read partition information from: " + path + "  --> " + json);
-                if (json != null) {
-                    jsonTopologyId = (String) ((Map<Object, Object>) json.get("topology")).get("id");
-                    jsonOffset = (Long) json.get("offset");
-                }
-            } catch (Throwable e) {
-                LOG.warn("Error reading and/or parsing at ZkNode: " + path, e);
-            }
-
-            String topic = _partition.topic;
-            Long currentOffset = KafkaUtils.getOffset(_consumer, topic, id.partition, spoutConfig);
-
-            if (jsonTopologyId == null || jsonOffset == null) { // failed to parse JSON?
-                _committedTo = currentOffset;
-                LOG.info("No partition information found, using configuration to determine offset");
-            } else if (!topologyInstanceId.equals(jsonTopologyId) && spoutConfig.ignoreZkOffsets) {
-                _committedTo = KafkaUtils.getOffset(_consumer, topic, id.partition, spoutConfig.startOffsetTime);
-                LOG.info("Topology change detected and ignore zookeeper offsets set to true, using configuration to determine offset");
-            } else {
-                _committedTo = jsonOffset;
-                LOG.info("Read last commit offset from zookeeper: " + _committedTo + "; old topology_id: " + jsonTopologyId +
-                         " - new topology_id: " + topologyInstanceId);
-            }
-
-            if (currentOffset - _committedTo > spoutConfig.maxOffsetBehind || _committedTo <= 0) {
-                LOG.info("Last commit offset from zookeeper: " + _committedTo);
-                Long lastCommittedOffset = _committedTo;
-                _committedTo = currentOffset;
-                LOG.info("Commit offset " + lastCommittedOffset + " is more than " +
-                         spoutConfig.maxOffsetBehind + " behind latest offset " + currentOffset + ", resetting to startOffsetTime=" +
-                         spoutConfig.startOffsetTime);
-            }
-
-            LOG.info("Starting Kafka " + _consumer.host() + " " + id + " from offset " + _committedTo);
-            _emittedToOffset = _committedTo;
-        }
-
-        _fetchAPILatencyMax = new CombinedMetric(new MaxMetric());
-        _fetchAPILatencyMean = new ReducedMetric(new MeanReducer());
-        _fetchAPICallCount = new CountMetric();
-        _fetchAPIMessageCount = new CountMetric();
-        _lostMessageCount = new CountMetric();
-        _messageIneligibleForRetryCount = new CountMetric();
-    }
-
-    public Map<String, Object> getMetricsDataMap() {
-        String metricPrefix = _partition.getId();
-
-        Map<String, Object> ret = new HashMap<>();
-        ret.put(metricPrefix + "/fetchAPILatencyMax", _fetchAPILatencyMax.getValueAndReset());
-        ret.put(metricPrefix + "/fetchAPILatencyMean", _fetchAPILatencyMean.getValueAndReset());
-        ret.put(metricPrefix + "/fetchAPICallCount", _fetchAPICallCount.getValueAndReset());
-        ret.put(metricPrefix + "/fetchAPIMessageCount", _fetchAPIMessageCount.getValueAndReset());
-        ret.put(metricPrefix + "/lostMessageCount", _lostMessageCount.getValueAndReset());
-        ret.put(metricPrefix + "/messageIneligibleForRetryCount", _messageIneligibleForRetryCount.getValueAndReset());
-        return ret;
-    }
-
-    //returns false if it's reached the end of current batch
-    public EmitState next(SpoutOutputCollector collector) {
-        if (_waitingToEmit.isEmpty()) {
-            fill();
-        }
-        while (true) {
-            MessageAndOffset toEmit = _waitingToEmit.pollFirst();
-            if (toEmit == null) {
-                return EmitState.NO_EMITTED;
-            }
-
-            Iterable<List<Object>> tups;
-            if (_spoutConfig.scheme instanceof MessageMetadataSchemeAsMultiScheme) {
-                tups = KafkaUtils.generateTuples((MessageMetadataSchemeAsMultiScheme) _spoutConfig.scheme, toEmit.message(), _partition,
-                                                 toEmit.offset());
-            } else {
-                tups = KafkaUtils.generateTuples(_spoutConfig, toEmit.message(), _partition.topic);
-            }
-
-            if ((tups != null) && tups.iterator().hasNext()) {
-                if (!Strings.isNullOrEmpty(_spoutConfig.outputStreamId)) {
-                    for (List<Object> tup : tups) {
-                        collector.emit(_spoutConfig.outputStreamId, tup, new KafkaMessageId(_partition, toEmit.offset()));
-                    }
-                } else {
-                    for (List<Object> tup : tups) {
-                        collector.emit(tup, new KafkaMessageId(_partition, toEmit.offset()));
-                    }
-                }
-                break;
-            } else {
-                ack(toEmit.offset());
-            }
-        }
-        if (!_waitingToEmit.isEmpty()) {
-            return EmitState.EMITTED_MORE_LEFT;
-        } else {
-            return EmitState.EMITTED_END;
-        }
-    }
-
-
-    private void fill() {
-        long start = System.currentTimeMillis();
-        Long offset;
-
-        // Are there failed tuples? If so, fetch those first.
-        offset = this._failedMsgRetryManager.nextFailedMessageToRetry();
-        final boolean processingNewTuples = (offset == null);
-        if (processingNewTuples) {
-            offset = _emittedToOffset;
-        }
-
-        ByteBufferMessageSet msgs = null;
-        try {
-            msgs = KafkaUtils.fetchMessages(_spoutConfig, _consumer, _partition, offset);
-        } catch (TopicOffsetOutOfRangeException e) {
-            offset = KafkaUtils.getOffset(_consumer, _partition.topic, _partition.partition, kafka.api.OffsetRequest.EarliestTime());
-            // fetch failed, so don't update the fetch metrics
-
-            //fix bug [STORM-643] : remove outdated failed offsets
-            if (!processingNewTuples) {
-                // For the case of EarliestTime it would be better to discard
-                // all the failed offsets, that are earlier than actual EarliestTime
-                // offset, since they are anyway not there.
-                // These calls to broker API will be then saved.
-                Set<Long> omitted = this._failedMsgRetryManager.clearOffsetsBefore(offset);
-
-                // Omitted messages have not been acked and may be lost
-                if (null != omitted) {
-                    _lostMessageCount.incrBy(omitted.size());
-                }
-
-                _pending.headMap(offset).clear();
-
-                LOG.warn("Removing the failed offsets for {} that are out of range: {}", _partition, omitted);
-            }
-
-            if (offset > _emittedToOffset) {
-                _lostMessageCount.incrBy(offset - _emittedToOffset);
-                _emittedToOffset = offset;
-                LOG.warn("{} Using new offset: {}", _partition, _emittedToOffset);
-            }
-
-            return;
-        }
-        long millis = System.currentTimeMillis() - start;
-        _fetchAPILatencyMax.update(millis);
-        _fetchAPILatencyMean.update(millis);
-        _fetchAPICallCount.incr();
-        if (msgs != null) {
-            int numMessages = 0;
-
-            for (MessageAndOffset msg : msgs) {
-                final Long cur_offset = msg.offset();
-                if (cur_offset < offset) {
-                    // Skip any old offsets.
-                    continue;
-                }
-                if (processingNewTuples || this._failedMsgRetryManager.shouldReEmitMsg(cur_offset)) {
-                    numMessages += 1;
-                    if (!_pending.containsKey(cur_offset)) {
-                        _pending.put(cur_offset, System.currentTimeMillis());
-                    }
-                    _waitingToEmit.add(msg);
-                    _emittedToOffset = Math.max(msg.nextOffset(), _emittedToOffset);
-                    if (_failedMsgRetryManager.shouldReEmitMsg(cur_offset)) {
-                        this._failedMsgRetryManager.retryStarted(cur_offset);
-                    }
-                }
-            }
-            _fetchAPIMessageCount.incrBy(numMessages);
-        }
-    }
-
-    public void ack(Long offset) {
-        if (!_pending.isEmpty() && _pending.firstKey() < offset - _spoutConfig.maxOffsetBehind) {
-            // Too many things pending!
-            _pending.headMap(offset - _spoutConfig.maxOffsetBehind).clear();
-        }
-        _pending.remove(offset);
-        this._failedMsgRetryManager.acked(offset);
-        numberAcked++;
-    }
-
-    public void fail(Long offset) {
-        if (offset < _emittedToOffset - _spoutConfig.maxOffsetBehind) {
-            LOG.info(
-                "Skipping failed tuple at offset={}" +
-                " because it's more than maxOffsetBehind={}" +
-                " behind _emittedToOffset={} for {}",
-                offset,
-                _spoutConfig.maxOffsetBehind,
-                _emittedToOffset,
-                _partition
-            );
-        } else {
-            LOG.debug("Failing at offset={} with _pending.size()={} pending and _emittedToOffset={} for {}", offset, _pending.size(),
-                      _emittedToOffset, _partition);
-            numberFailed++;
-            if (numberAcked == 0 && numberFailed > _spoutConfig.maxOffsetBehind) {
-                throw new RuntimeException("Too many tuple failures");
-            }
-
-            // Offset may not be considered for retry by failedMsgRetryManager
-            if (this._failedMsgRetryManager.retryFurther(offset)) {
-                this._failedMsgRetryManager.failed(offset);
-            } else {
-                // state for the offset should be cleaned up
-                LOG.warn("Will not retry failed kafka offset {} further", offset);
-                _messageIneligibleForRetryCount.incr();
-                this._failedMsgRetryManager.cleanOffsetAfterRetries(_partition, offset);
-                _pending.remove(offset);
-                this._failedMsgRetryManager.acked(offset);
-            }
-        }
-    }
-
-    public void commit() {
-        long lastCompletedOffset = lastCompletedOffset();
-        if (_committedTo != lastCompletedOffset) {
-            LOG.debug("Writing last completed offset ({}) to ZK for {} for topology: {}", lastCompletedOffset, _partition,
-                      _topologyInstanceId);
-            Map<Object, Object> data = (Map<Object, Object>) ImmutableMap.builder()
-                                                                         .put("topology", ImmutableMap.of("id", _topologyInstanceId,
-                                                                                                          "name", _topoConf
-                                                                                                              .get(Config.TOPOLOGY_NAME)))
-                                                                         .put("offset", lastCompletedOffset)
-                                                                         .put("partition", _partition.partition)
-                                                                         .put("broker", ImmutableMap.of("host", _partition.host.host,
-                                                                                                        "port", _partition.host.port))
-                                                                         .put("topic", _partition.topic).build();
-            _state.writeJSON(committedPath(), data);
-
-            _committedTo = lastCompletedOffset;
-            LOG.debug("Wrote last completed offset ({}) to ZK for {} for topology: {}", lastCompletedOffset, _partition,
-                      _topologyInstanceId);
-        } else {
-            LOG.debug("No new offset for {} for topology: {}", _partition, _topologyInstanceId);
-        }
-    }
-
-    protected String committedPath() {
-        return _spoutConfig.zkRoot + "/" + _spoutConfig.id + "/" + _partition.getId();
-    }
-
-    public long lastCompletedOffset() {
-        if (_pending.isEmpty()) {
-            return _emittedToOffset;
-        } else {
-            return _pending.firstKey();
-        }
-    }
-
-    public OffsetData getOffsetData() {
-        return new OffsetData(_emittedToOffset, lastCompletedOffset());
-    }
-
-    public Partition getPartition() {
-        return _partition;
-    }
-
-    public void close() {
-        commit();
-        _connections.unregister(_partition.host, _partition.topic, _partition.partition);
-    }
-
-    static class KafkaMessageId implements Serializable {
-        public Partition partition;
-        public long offset;
-
-        public KafkaMessageId(Partition partition, long offset) {
-            this.partition = partition;
-            this.offset = offset;
-        }
-    }
-
-    public static class OffsetData {
-        public long latestEmittedOffset;
-        public long latestCompletedOffset;
-
-        public OffsetData(long latestEmittedOffset, long latestCompletedOffset) {
-            this.latestEmittedOffset = latestEmittedOffset;
-            this.latestCompletedOffset = latestCompletedOffset;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/SpoutConfig.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/SpoutConfig.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/SpoutConfig.java
deleted file mode 100644
index 74a4a3b..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/SpoutConfig.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.  The ASF licenses this file to you under the Apache License, Version
- * 2.0 (the "License"); you may not use this file except in compliance with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions
- * and limitations under the License.
- */
-
-package org.apache.storm.kafka;
-
-import java.io.Serializable;
-import java.util.List;
-
-
-public class SpoutConfig extends KafkaConfig implements Serializable {
-    private static final long serialVersionUID = -1247769246497567352L;
-    public List<String> zkServers = null;
-    public Integer zkPort = null;
-    public String zkRoot = null;
-    public String id = null;
-
-    public String outputStreamId;
-
-    // setting for how often to save the current kafka offset to ZooKeeper
-    public long stateUpdateIntervalMs = 2000;
-
-    // Retry strategy for failed messages
-    public String failedMsgRetryManagerClass = ExponentialBackoffMsgRetryManager.class.getName();
-
-    // Exponential back-off retry settings.  These are used by ExponentialBackoffMsgRetryManager for retrying messages after a bolt
-    // calls OutputCollector.fail(). These come into effect only if ExponentialBackoffMsgRetryManager is being used.
-    public long retryInitialDelayMs = 0;
-    public double retryDelayMultiplier = 1.0;
-    public long retryDelayMaxMs = 60 * 1000;
-    public int retryLimit = -1;
-
-    /**
-     * Create a SpoutConfig without setting client.id, which can make the source application ambiguous when tracing Kafka calls.
-     */
-    public SpoutConfig(BrokerHosts hosts, String topic, String zkRoot, String id) {
-        super(hosts, topic);
-        this.zkRoot = zkRoot;
-        this.id = id;
-    }
-
-    /**
-     * Create a SpoutConfig with a client.id value.
-     */
-    public SpoutConfig(BrokerHosts hosts, String topic, String clientId, String zkRoot, String id) {
-        super(hosts, topic, clientId);
-        this.zkRoot = zkRoot;
-        this.id = id;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticCoordinator.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticCoordinator.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticCoordinator.java
deleted file mode 100644
index 8d12ee1..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticCoordinator.java
+++ /dev/null
@@ -1,50 +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;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.storm.kafka.trident.GlobalPartitionInformation;
-
-
-public class StaticCoordinator implements PartitionCoordinator {
-    Map<Partition, PartitionManager> _managers = new HashMap<Partition, PartitionManager>();
-    List<PartitionManager> _allManagers = new ArrayList<>();
-
-    public StaticCoordinator(DynamicPartitionConnections connections, Map<String, Object> topoConf, SpoutConfig config, ZkState state,
-                             int taskIndex, int totalTasks, int taskId, String topologyInstanceId) {
-        StaticHosts hosts = (StaticHosts) config.hosts;
-        List<GlobalPartitionInformation> partitions = new ArrayList<GlobalPartitionInformation>();
-        partitions.add(hosts.getPartitionInformation());
-        List<Partition> myPartitions = KafkaUtils.calculatePartitionsForTask(partitions, totalTasks, taskIndex, taskId);
-        for (Partition myPartition : myPartitions) {
-            _managers.put(myPartition, new PartitionManager(connections, topologyInstanceId, state, topoConf, config, myPartition));
-        }
-        _allManagers = new ArrayList<>(_managers.values());
-    }
-
-    @Override
-    public List<PartitionManager> getMyManagedPartitions() {
-        return _allManagers;
-    }
-
-    public PartitionManager getManager(Partition partition) {
-        return _managers.get(partition);
-    }
-
-    @Override
-    public void refresh() { return; }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticHosts.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticHosts.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticHosts.java
deleted file mode 100644
index 1f8f903..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticHosts.java
+++ /dev/null
@@ -1,33 +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;
-
-import org.apache.storm.kafka.trident.GlobalPartitionInformation;
-
-/**
- * Date: 11/05/2013
- * Time: 14:43
- */
-public class StaticHosts implements BrokerHosts {
-
-
-    private GlobalPartitionInformation partitionInformation;
-
-    public StaticHosts(GlobalPartitionInformation partitionInformation) {
-        this.partitionInformation = partitionInformation;
-    }
-
-    public GlobalPartitionInformation getPartitionInformation() {
-        return partitionInformation;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticPartitionConnections.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticPartitionConnections.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticPartitionConnections.java
deleted file mode 100644
index 2c9d4f2..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StaticPartitionConnections.java
+++ /dev/null
@@ -1,46 +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;
-
-import java.util.HashMap;
-import java.util.Map;
-import kafka.javaapi.consumer.SimpleConsumer;
-
-public class StaticPartitionConnections {
-    Map<Integer, SimpleConsumer> _kafka = new HashMap<Integer, SimpleConsumer>();
-    KafkaConfig _config;
-    StaticHosts hosts;
-
-    public StaticPartitionConnections(KafkaConfig conf) {
-        _config = conf;
-        if (!(conf.hosts instanceof StaticHosts)) {
-            throw new RuntimeException("Must configure with static hosts");
-        }
-        this.hosts = (StaticHosts) conf.hosts;
-    }
-
-    public SimpleConsumer getConsumer(int partition) {
-        if (!_kafka.containsKey(partition)) {
-            Broker hp = hosts.getPartitionInformation().getBrokerFor(partition);
-            _kafka.put(partition, new SimpleConsumer(hp.host, hp.port, _config.socketTimeoutMs, _config.bufferSizeBytes, _config.clientId));
-
-        }
-        return _kafka.get(partition);
-    }
-
-    public void close() {
-        for (SimpleConsumer consumer : _kafka.values()) {
-            consumer.close();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringKeyValueScheme.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringKeyValueScheme.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringKeyValueScheme.java
deleted file mode 100644
index 3d62961..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringKeyValueScheme.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.  The ASF licenses this file to you under the Apache License, Version
- * 2.0 (the "License"); you may not use this file except in compliance with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions
- * and limitations under the License.
- */
-
-package org.apache.storm.kafka;
-
-import com.google.common.collect.ImmutableMap;
-import java.nio.ByteBuffer;
-import java.util.List;
-import org.apache.storm.tuple.Values;
-
-public class StringKeyValueScheme extends StringScheme implements KeyValueScheme {
-
-    @Override
-    public List<Object> deserializeKeyAndValue(ByteBuffer key, ByteBuffer value) {
-        if (key == null) {
-            return deserialize(value);
-        }
-        String keyString = StringScheme.deserializeString(key);
-        String valueString = StringScheme.deserializeString(value);
-        return new Values(ImmutableMap.of(keyString, valueString));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringMessageAndMetadataScheme.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringMessageAndMetadataScheme.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringMessageAndMetadataScheme.java
deleted file mode 100644
index ab6e500..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringMessageAndMetadataScheme.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.  The ASF licenses this file to you under the Apache License, Version
- * 2.0 (the "License"); you may not use this file except in compliance with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions
- * and limitations under the License.
- */
-
-package org.apache.storm.kafka;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
-
-public class StringMessageAndMetadataScheme extends StringScheme implements MessageMetadataScheme {
-    public static final String STRING_SCHEME_PARTITION_KEY = "partition";
-    public static final String STRING_SCHEME_OFFSET = "offset";
-    private static final long serialVersionUID = -5441841920447947374L;
-
-    @Override
-    public List<Object> deserializeMessageWithMetadata(ByteBuffer message, Partition partition, long offset) {
-        String stringMessage = StringScheme.deserializeString(message);
-        return new Values(stringMessage, partition.partition, offset);
-    }
-
-    @Override
-    public Fields getOutputFields() {
-        return new Fields(STRING_SCHEME_KEY, STRING_SCHEME_PARTITION_KEY, STRING_SCHEME_OFFSET);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringMultiSchemeWithTopic.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringMultiSchemeWithTopic.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringMultiSchemeWithTopic.java
deleted file mode 100644
index 061b30a..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringMultiSchemeWithTopic.java
+++ /dev/null
@@ -1,41 +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;
-
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.List;
-import org.apache.storm.spout.MultiScheme;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
-
-public class StringMultiSchemeWithTopic
-    implements MultiScheme {
-    public static final String STRING_SCHEME_KEY = "str";
-
-    public static final String TOPIC_KEY = "topic";
-
-    @Override
-    public Iterable<List<Object>> deserialize(ByteBuffer bytes) {
-        throw new UnsupportedOperationException();
-    }
-
-    public Iterable<List<Object>> deserializeWithTopic(String topic, ByteBuffer bytes) {
-        List<Object> items = new Values(StringScheme.deserializeString(bytes), topic);
-        return Collections.singletonList(items);
-    }
-
-    public Fields getOutputFields() {
-        return new Fields(STRING_SCHEME_KEY, TOPIC_KEY);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringScheme.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringScheme.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringScheme.java
deleted file mode 100644
index bcbc058..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringScheme.java
+++ /dev/null
@@ -1,44 +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;
-
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.nio.charset.StandardCharsets;
-import java.util.List;
-import org.apache.storm.spout.Scheme;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
-
-public class StringScheme implements Scheme {
-    public static final String STRING_SCHEME_KEY = "str";
-    private static final Charset UTF8_CHARSET = StandardCharsets.UTF_8;
-
-    public static String deserializeString(ByteBuffer string) {
-        if (string.hasArray()) {
-            int base = string.arrayOffset();
-            return new String(string.array(), base + string.position(), string.remaining(), UTF8_CHARSET);
-        } else {
-            return new String(Utils.toByteArray(string), UTF8_CHARSET);
-        }
-    }
-
-    public List<Object> deserialize(ByteBuffer bytes) {
-        return new Values(deserializeString(bytes));
-    }
-
-    public Fields getOutputFields() {
-        return new Fields(STRING_SCHEME_KEY);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/TopicOffsetOutOfRangeException.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/TopicOffsetOutOfRangeException.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/TopicOffsetOutOfRangeException.java
deleted file mode 100644
index 613a62e..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/TopicOffsetOutOfRangeException.java
+++ /dev/null
@@ -1,26 +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;
-
-public class TopicOffsetOutOfRangeException extends RuntimeException {
-
-    public TopicOffsetOutOfRangeException(String message) {
-        super(message);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkCoordinator.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkCoordinator.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkCoordinator.java
deleted file mode 100644
index bc9ebd5..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkCoordinator.java
+++ /dev/null
@@ -1,127 +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;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.storm.kafka.trident.GlobalPartitionInformation;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.storm.kafka.KafkaUtils.taskPrefix;
-
-public class ZkCoordinator implements PartitionCoordinator {
-    private static final Logger LOG = LoggerFactory.getLogger(ZkCoordinator.class);
-
-    SpoutConfig _spoutConfig;
-    int _taskIndex;
-    int _totalTasks;
-    int _taskId;
-    String _topologyInstanceId;
-    Map<Partition, PartitionManager> _managers = new HashMap();
-    List<PartitionManager> _cachedList = new ArrayList<PartitionManager>();
-    Long _lastRefreshTime = null;
-    int _refreshFreqMs;
-    DynamicPartitionConnections _connections;
-    DynamicBrokersReader _reader;
-    ZkState _state;
-    Map _topoConf;
-
-    public ZkCoordinator(DynamicPartitionConnections connections, Map<String, Object> topoConf, SpoutConfig spoutConfig, ZkState state,
-                         int taskIndex, int totalTasks, int taskId, String topologyInstanceId) {
-        this(connections, topoConf, spoutConfig, state, taskIndex, totalTasks, taskId, topologyInstanceId,
-             buildReader(topoConf, spoutConfig));
-    }
-
-    public ZkCoordinator(DynamicPartitionConnections connections, Map<String, Object> topoConf, SpoutConfig spoutConfig, ZkState state,
-                         int taskIndex, int totalTasks, int taskId, String topologyInstanceId, DynamicBrokersReader reader) {
-        _spoutConfig = spoutConfig;
-        _connections = connections;
-        _taskIndex = taskIndex;
-        _totalTasks = totalTasks;
-        _taskId = taskId;
-        _topologyInstanceId = topologyInstanceId;
-        _topoConf = topoConf;
-        _state = state;
-        ZkHosts brokerConf = (ZkHosts) spoutConfig.hosts;
-        _refreshFreqMs = brokerConf.refreshFreqSecs * 1000;
-        _reader = reader;
-    }
-
-    private static DynamicBrokersReader buildReader(Map<String, Object> topoConf, SpoutConfig spoutConfig) {
-        ZkHosts hosts = (ZkHosts) spoutConfig.hosts;
-        return new DynamicBrokersReader(topoConf, hosts.brokerZkStr, hosts.brokerZkPath, spoutConfig.topic);
-    }
-
-    @Override
-    public List<PartitionManager> getMyManagedPartitions() {
-        if (_lastRefreshTime == null || (System.currentTimeMillis() - _lastRefreshTime) > _refreshFreqMs) {
-            refresh();
-            _lastRefreshTime = System.currentTimeMillis();
-        }
-        return _cachedList;
-    }
-
-    @Override
-    public void refresh() {
-        try {
-            LOG.info(taskPrefix(_taskIndex, _totalTasks, _taskId) + " Refreshing partition manager connections");
-            List<GlobalPartitionInformation> brokerInfo = _reader.getBrokerInfo();
-            List<Partition> mine = KafkaUtils.calculatePartitionsForTask(brokerInfo, _totalTasks, _taskIndex, _taskId);
-
-            Set<Partition> curr = _managers.keySet();
-            Set<Partition> newPartitions = new HashSet<Partition>(mine);
-            newPartitions.removeAll(curr);
-
-            Set<Partition> deletedPartitions = new HashSet<Partition>(curr);
-            deletedPartitions.removeAll(mine);
-
-            LOG.info(taskPrefix(_taskIndex, _totalTasks, _taskId) + " Deleted partition managers: " + deletedPartitions.toString());
-
-            Map<Integer, PartitionManager> deletedManagers = new HashMap<>();
-            for (Partition id : deletedPartitions) {
-                deletedManagers.put(id.partition, _managers.remove(id));
-            }
-            for (PartitionManager manager : deletedManagers.values()) {
-                if (manager != null) manager.close();
-            }
-            LOG.info(taskPrefix(_taskIndex, _totalTasks, _taskId) + " New partition managers: " + newPartitions.toString());
-
-            for (Partition id : newPartitions) {
-                PartitionManager man = new PartitionManager(
-                    _connections,
-                    _topologyInstanceId,
-                    _state,
-                    _topoConf,
-                    _spoutConfig,
-                    id,
-                    deletedManagers.get(id.partition));
-                _managers.put(id, man);
-            }
-
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-        _cachedList = new ArrayList<PartitionManager>(_managers.values());
-        LOG.info(taskPrefix(_taskIndex, _totalTasks, _taskId) + " Finished refreshing");
-    }
-
-    @Override
-    public PartitionManager getManager(Partition partition) {
-        return _managers.get(partition);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkHosts.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkHosts.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkHosts.java
deleted file mode 100644
index 9c6b29d..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkHosts.java
+++ /dev/null
@@ -1,31 +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;
-
-
-public class ZkHosts implements BrokerHosts {
-    private static final String DEFAULT_ZK_PATH = "/brokers";
-
-    public String brokerZkStr = null;
-    public String brokerZkPath = null; // e.g., /kafka/brokers
-    public int refreshFreqSecs = 60;
-
-    public ZkHosts(String brokerZkStr, String brokerZkPath) {
-        this.brokerZkStr = brokerZkStr;
-        this.brokerZkPath = brokerZkPath;
-    }
-
-    public ZkHosts(String brokerZkStr) {
-        this(brokerZkStr, DEFAULT_ZK_PATH);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkState.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkState.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkState.java
deleted file mode 100644
index 3d27173..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkState.java
+++ /dev/null
@@ -1,112 +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;
-
-import java.nio.charset.Charset;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.retry.RetryNTimes;
-import org.apache.storm.Config;
-import org.apache.storm.utils.ObjectReader;
-import org.apache.zookeeper.CreateMode;
-import org.json.simple.JSONValue;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class ZkState {
-    private static final Logger LOG = LoggerFactory.getLogger(ZkState.class);
-    CuratorFramework _curator;
-
-    public ZkState(Map<String, Object> stateConf) {
-        stateConf = new HashMap<>(stateConf);
-
-        try {
-            _curator = newCurator(stateConf);
-            _curator.start();
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private CuratorFramework newCurator(final Map<String, Object> stateConf)
-        throws Exception {
-        Integer port = (Integer) stateConf.get(Config.TRANSACTIONAL_ZOOKEEPER_PORT);
-        String serverPorts = "";
-        for (String server : (List<String>) stateConf.get(Config.TRANSACTIONAL_ZOOKEEPER_SERVERS)) {
-            serverPorts = serverPorts + server + ":" + port + ",";
-        }
-        return CuratorFrameworkFactory.newClient(serverPorts,
-                                                 ObjectReader.getInt(stateConf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT)),
-                                                 ObjectReader.getInt(stateConf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT)),
-                                                 new RetryNTimes(ObjectReader.getInt(stateConf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES)),
-                                                                 ObjectReader
-                                                                     .getInt(stateConf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL))));
-    }
-
-    public CuratorFramework getCurator() {
-        assert _curator != null;
-        return _curator;
-    }
-
-    public void writeJSON(String path, Map<Object, Object> data) {
-        LOG.debug("Writing {} the data {}", path, data.toString());
-        writeBytes(path, JSONValue.toJSONString(data).getBytes(Charset.forName("UTF-8")));
-    }
-
-    public void writeBytes(String path, byte[] bytes) {
-        try {
-            if (_curator.checkExists().forPath(path) == null) {
-                _curator.create()
-                        .creatingParentsIfNeeded()
-                        .withMode(CreateMode.PERSISTENT)
-                        .forPath(path, bytes);
-            } else {
-                _curator.setData().forPath(path, bytes);
-            }
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public Map<Object, Object> readJSON(String path) {
-        try {
-            byte[] b = readBytes(path);
-            if (b == null) {
-                return null;
-            }
-            return (Map<Object, Object>) JSONValue.parseWithException(new String(b, "UTF-8"));
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public byte[] readBytes(String path) {
-        try {
-            if (_curator.checkExists().forPath(path) != null) {
-                return _curator.getData().forPath(path);
-            } else {
-                return null;
-            }
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public void close() {
-        _curator.close();
-        _curator = null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/KafkaBolt.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/KafkaBolt.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/KafkaBolt.java
deleted file mode 100644
index e0b94f3..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/KafkaBolt.java
+++ /dev/null
@@ -1,172 +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.bolt;
-
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import org.apache.kafka.clients.producer.Callback;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.storm.kafka.bolt.mapper.FieldNameBasedTupleToKafkaMapper;
-import org.apache.storm.kafka.bolt.mapper.TupleToKafkaMapper;
-import org.apache.storm.kafka.bolt.selector.DefaultTopicSelector;
-import org.apache.storm.kafka.bolt.selector.KafkaTopicSelector;
-import org.apache.storm.task.OutputCollector;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseTickTupleAwareRichBolt;
-import org.apache.storm.tuple.Tuple;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * Bolt implementation that can send Tuple data to Kafka
- * <p/>
- * It expects the producer configuration and topic in storm config under
- * <p/>
- * 'kafka.broker.properties' and 'topic'
- * <p/>
- * respectively.
- * <p/>
- * This bolt uses 0.8.2 Kafka Producer API.
- * <p/>
- * It works for sending tuples to older Kafka version (0.8.1).
- * @deprecated Please use the KafkaBolt in storm-kafka-client
- */
-@Deprecated
-public class KafkaBolt<K, V> extends BaseTickTupleAwareRichBolt {
-
-    public static final String TOPIC = "topic";
-    private static final Logger LOG = LoggerFactory.getLogger(KafkaBolt.class);
-    private KafkaProducer<K, V> producer;
-    private OutputCollector collector;
-    private TupleToKafkaMapper<K, V> mapper;
-    private KafkaTopicSelector topicSelector;
-    private Properties boltSpecfiedProperties = new Properties();
-    /**
-     * With default setting for fireAndForget and async, the callback is called when the sending succeeds.
-     * By setting fireAndForget true, the send will not wait at all for kafka to ack.
-     * "acks" setting in 0.8.2 Producer API config doesn't matter if fireAndForget is set.
-     * By setting async false, synchronous sending is used. 
-     */
-    private boolean fireAndForget = false;
-    private boolean async = true;
-
-    public KafkaBolt() {}
-
-    public KafkaBolt<K, V> withTupleToKafkaMapper(TupleToKafkaMapper<K, V> mapper) {
-        this.mapper = mapper;
-        return this;
-    }
-
-    public KafkaBolt<K, V> withTopicSelector(KafkaTopicSelector selector) {
-        this.topicSelector = selector;
-        return this;
-    }
-
-    public KafkaBolt<K, V> withProducerProperties(Properties producerProperties) {
-        this.boltSpecfiedProperties = producerProperties;
-        return this;
-    }
-
-    @Override
-    public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) {
-        //for backward compatibility.
-        if (mapper == null) {
-            this.mapper = new FieldNameBasedTupleToKafkaMapper<K, V>();
-        }
-
-        //for backward compatibility.
-        if (topicSelector == null) {
-            if (topoConf.containsKey(TOPIC)) {
-                this.topicSelector = new DefaultTopicSelector((String) topoConf.get(TOPIC));
-            } else {
-                throw new IllegalArgumentException("topic should be specified in bolt's configuration");
-            }
-        }
-
-        producer = new KafkaProducer<>(boltSpecfiedProperties);
-        this.collector = collector;
-    }
-
-    @Override
-    protected void process(final Tuple input) {
-        K key = null;
-        V message = null;
-        String topic = null;
-        try {
-            key = mapper.getKeyFromTuple(input);
-            message = mapper.getMessageFromTuple(input);
-            topic = topicSelector.getTopic(input);
-            if (topic != null) {
-                Callback callback = null;
-
-                if (!fireAndForget && async) {
-                    callback = new Callback() {
-                        @Override
-                        public void onCompletion(RecordMetadata ignored, Exception e) {
-                            synchronized (collector) {
-                                if (e != null) {
-                                    collector.reportError(e);
-                                    collector.fail(input);
-                                } else {
-                                    collector.ack(input);
-                                }
-                            }
-                        }
-                    };
-                }
-                Future<RecordMetadata> result = producer.send(new ProducerRecord<K, V>(topic, key, message), callback);
-                if (!async) {
-                    try {
-                        result.get();
-                        collector.ack(input);
-                    } catch (ExecutionException err) {
-                        collector.reportError(err);
-                        collector.fail(input);
-                    }
-                } else if (fireAndForget) {
-                    collector.ack(input);
-                }
-            } else {
-                LOG.warn("skipping key = " + key + ", topic selector returned null.");
-                collector.ack(input);
-            }
-        } catch (Exception ex) {
-            collector.reportError(ex);
-            collector.fail(input);
-        }
-    }
-
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-
-    }
-
-    @Override
-    public void cleanup() {
-        producer.close();
-    }
-
-    public void setFireAndForget(boolean fireAndForget) {
-        this.fireAndForget = fireAndForget;
-    }
-
-    public void setAsync(boolean async) {
-        this.async = async;
-    }
-}


[3/7] storm git commit: STORM-2953: Remove storm-kafka

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/mapper/FieldNameBasedTupleToKafkaMapper.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/mapper/FieldNameBasedTupleToKafkaMapper.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/mapper/FieldNameBasedTupleToKafkaMapper.java
deleted file mode 100644
index 7276ef6..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/mapper/FieldNameBasedTupleToKafkaMapper.java
+++ /dev/null
@@ -1,43 +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.bolt.mapper;
-
-import org.apache.storm.tuple.Tuple;
-
-public class FieldNameBasedTupleToKafkaMapper<K, V> implements TupleToKafkaMapper<K, V> {
-
-    public static final String BOLT_KEY = "key";
-    public static final String BOLT_MESSAGE = "message";
-    public String boltKeyField;
-    public String boltMessageField;
-
-    public FieldNameBasedTupleToKafkaMapper() {
-        this(BOLT_KEY, BOLT_MESSAGE);
-    }
-
-    public FieldNameBasedTupleToKafkaMapper(String boltKeyField, String boltMessageField) {
-        this.boltKeyField = boltKeyField;
-        this.boltMessageField = boltMessageField;
-    }
-
-    @Override
-    public K getKeyFromTuple(Tuple tuple) {
-        //for backward compatibility, we return null when key is not present.
-        return tuple.contains(boltKeyField) ? (K) tuple.getValueByField(boltKeyField) : null;
-    }
-
-    @Override
-    public V getMessageFromTuple(Tuple tuple) {
-        return (V) tuple.getValueByField(boltMessageField);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/mapper/TupleToKafkaMapper.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/mapper/TupleToKafkaMapper.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/mapper/TupleToKafkaMapper.java
deleted file mode 100644
index 7012e6b..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/mapper/TupleToKafkaMapper.java
+++ /dev/null
@@ -1,27 +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.bolt.mapper;
-
-import java.io.Serializable;
-import org.apache.storm.tuple.Tuple;
-
-/**
- * as the really verbose name suggests this interface mapps a storm tuple to kafka key and message.
- * @param <K> type of key.
- * @param <V> type of value.
- */
-public interface TupleToKafkaMapper<K, V> extends Serializable {
-    K getKeyFromTuple(Tuple tuple);
-
-    V getMessageFromTuple(Tuple tuple);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/DefaultTopicSelector.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/DefaultTopicSelector.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/DefaultTopicSelector.java
deleted file mode 100644
index d1784b0..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/DefaultTopicSelector.java
+++ /dev/null
@@ -1,29 +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.bolt.selector;
-
-import org.apache.storm.tuple.Tuple;
-
-public class DefaultTopicSelector implements KafkaTopicSelector {
-
-    private final String topicName;
-
-    public DefaultTopicSelector(final String topicName) {
-        this.topicName = topicName;
-    }
-
-    @Override
-    public String getTopic(Tuple tuple) {
-        return topicName;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java
deleted file mode 100644
index 50c5c1f..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldIndexTopicSelector.java
+++ /dev/null
@@ -1,43 +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.bolt.selector;
-
-import org.apache.storm.tuple.Tuple;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Uses field index to select topic name from tuple .
- */
-public class FieldIndexTopicSelector implements KafkaTopicSelector {
-
-    private static final Logger LOG = LoggerFactory.getLogger(FieldIndexTopicSelector.class);
-
-    private final int fieldIndex;
-    private final String defaultTopicName;
-
-    public FieldIndexTopicSelector(int fieldIndex, String defaultTopicName) {
-        this.fieldIndex = fieldIndex;
-        this.defaultTopicName = defaultTopicName;
-    }
-
-    @Override
-    public String getTopic(Tuple tuple) {
-        if (fieldIndex < tuple.size()) {
-            return tuple.getString(fieldIndex);
-        } else {
-            LOG.warn("Field Index " + fieldIndex + " Out of bound . Using default topic " + defaultTopicName);
-            return defaultTopicName;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java
deleted file mode 100644
index d3c304a..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/FieldNameTopicSelector.java
+++ /dev/null
@@ -1,44 +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.bolt.selector;
-
-import org.apache.storm.tuple.Tuple;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Uses field name to select topic name from tuple .
- */
-public class FieldNameTopicSelector implements KafkaTopicSelector {
-
-    private static final Logger LOG = LoggerFactory.getLogger(FieldNameTopicSelector.class);
-
-    private final String fieldName;
-    private final String defaultTopicName;
-
-
-    public FieldNameTopicSelector(String fieldName, String defaultTopicName) {
-        this.fieldName = fieldName;
-        this.defaultTopicName = defaultTopicName;
-    }
-
-    @Override
-    public String getTopic(Tuple tuple) {
-        if (tuple.contains(fieldName)) {
-            return tuple.getStringByField(fieldName);
-        } else {
-            LOG.warn("Field " + fieldName + " Not Found . Using default topic " + defaultTopicName);
-            return defaultTopicName;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/KafkaTopicSelector.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/KafkaTopicSelector.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/KafkaTopicSelector.java
deleted file mode 100644
index 4045df7..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/bolt/selector/KafkaTopicSelector.java
+++ /dev/null
@@ -1,20 +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.bolt.selector;
-
-import java.io.Serializable;
-import org.apache.storm.tuple.Tuple;
-
-public interface KafkaTopicSelector extends Serializable {
-    String getTopic(Tuple tuple);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/Coordinator.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/Coordinator.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/Coordinator.java
deleted file mode 100644
index 0e8dba1..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/Coordinator.java
+++ /dev/null
@@ -1,46 +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.trident;
-
-import java.util.List;
-import java.util.Map;
-import org.apache.storm.kafka.KafkaUtils;
-import org.apache.storm.trident.spout.IOpaquePartitionedTridentSpout;
-import org.apache.storm.trident.spout.IPartitionedTridentSpout;
-
-class Coordinator implements IPartitionedTridentSpout.Coordinator<List<GlobalPartitionInformation>>,
-                             IOpaquePartitionedTridentSpout.Coordinator<List<GlobalPartitionInformation>> {
-
-    private IBrokerReader reader;
-    private TridentKafkaConfig config;
-
-    public Coordinator(Map<String, Object> conf, TridentKafkaConfig tridentKafkaConfig) {
-        config = tridentKafkaConfig;
-        reader = KafkaUtils.makeBrokerReader(conf, config);
-    }
-
-    @Override
-    public void close() {
-        config.coordinator.close();
-    }
-
-    @Override
-    public boolean isReady(long txid) {
-        return config.coordinator.isReady(txid);
-    }
-
-    @Override
-    public List<GlobalPartitionInformation> getPartitionsForBatch() {
-        return reader.getAllBrokers();
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/DefaultCoordinator.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/DefaultCoordinator.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/DefaultCoordinator.java
deleted file mode 100644
index 575e235..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/DefaultCoordinator.java
+++ /dev/null
@@ -1,26 +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.trident;
-
-public class DefaultCoordinator implements IBatchCoordinator {
-
-    @Override
-    public boolean isReady(long txid) {
-        return true;
-    }
-
-    @Override
-    public void close() {
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/GlobalPartitionInformation.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/GlobalPartitionInformation.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/GlobalPartitionInformation.java
deleted file mode 100644
index b26dc7f..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/GlobalPartitionInformation.java
+++ /dev/null
@@ -1,113 +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.trident;
-
-import com.google.common.base.Objects;
-import java.io.Serializable;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-import org.apache.storm.kafka.Broker;
-import org.apache.storm.kafka.Partition;
-
-
-public class GlobalPartitionInformation implements Iterable<Partition>, Serializable {
-
-    public String topic;
-    private Map<Integer, Broker> partitionMap;
-    //Flag to keep the Partition Path Id backward compatible with Old implementation of Partition.getId() == "partition_" + partition
-    private Boolean bUseTopicNameForPartitionPathId;
-
-    public GlobalPartitionInformation(String topic, Boolean bUseTopicNameForPartitionPathId) {
-        this.topic = topic;
-        this.partitionMap = new TreeMap<Integer, Broker>();
-        this.bUseTopicNameForPartitionPathId = bUseTopicNameForPartitionPathId;
-    }
-
-    public GlobalPartitionInformation(String topic) {
-        this.topic = topic;
-        this.partitionMap = new TreeMap<Integer, Broker>();
-        this.bUseTopicNameForPartitionPathId = false;
-    }
-
-    public void addPartition(int partitionId, Broker broker) {
-        partitionMap.put(partitionId, broker);
-    }
-
-    public Boolean getbUseTopicNameForPartitionPathId() {
-        return bUseTopicNameForPartitionPathId;
-    }
-
-    @Override
-    public String toString() {
-        return "GlobalPartitionInformation{" +
-               "topic=" + topic +
-               ", partitionMap=" + partitionMap +
-               '}';
-    }
-
-    public Broker getBrokerFor(Integer partitionId) {
-        return partitionMap.get(partitionId);
-    }
-
-    public List<Partition> getOrderedPartitions() {
-        List<Partition> partitions = new LinkedList<Partition>();
-        for (Map.Entry<Integer, Broker> partition : partitionMap.entrySet()) {
-            partitions.add(new Partition(partition.getValue(), this.topic, partition.getKey(), this.bUseTopicNameForPartitionPathId));
-        }
-        return partitions;
-    }
-
-    @Override
-    public Iterator<Partition> iterator() {
-        final Iterator<Map.Entry<Integer, Broker>> iterator = partitionMap.entrySet().iterator();
-        final String topic = this.topic;
-        final Boolean bUseTopicNameForPartitionPathId = this.bUseTopicNameForPartitionPathId;
-        return new Iterator<Partition>() {
-            @Override
-            public boolean hasNext() {
-                return iterator.hasNext();
-            }
-
-            @Override
-            public Partition next() {
-                Map.Entry<Integer, Broker> next = iterator.next();
-                return new Partition(next.getValue(), topic, next.getKey(), bUseTopicNameForPartitionPathId);
-            }
-
-            @Override
-            public void remove() {
-                iterator.remove();
-            }
-        };
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hashCode(partitionMap);
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj) {
-            return true;
-        }
-        if (obj == null || getClass() != obj.getClass()) {
-            return false;
-        }
-        final GlobalPartitionInformation other = (GlobalPartitionInformation) obj;
-        return Objects.equal(this.partitionMap, other.partitionMap);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/IBatchCoordinator.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/IBatchCoordinator.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/IBatchCoordinator.java
deleted file mode 100644
index 4c6c404..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/IBatchCoordinator.java
+++ /dev/null
@@ -1,21 +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.trident;
-
-import java.io.Serializable;
-
-public interface IBatchCoordinator extends Serializable {
-    boolean isReady(long txid);
-
-    void close();
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/IBrokerReader.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/IBrokerReader.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/IBrokerReader.java
deleted file mode 100644
index c5cf8b2..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/IBrokerReader.java
+++ /dev/null
@@ -1,24 +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.trident;
-
-import java.util.List;
-
-public interface IBrokerReader {
-
-    GlobalPartitionInformation getBrokerForTopic(String topic);
-
-    List<GlobalPartitionInformation> getAllBrokers();
-
-    void close();
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/MaxMetric.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/MaxMetric.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/MaxMetric.java
deleted file mode 100644
index 14324ed..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/MaxMetric.java
+++ /dev/null
@@ -1,35 +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.trident;
-
-
-import org.apache.storm.metric.api.ICombiner;
-
-public class MaxMetric implements ICombiner<Long> {
-    @Override
-    public Long identity() {
-        return null;
-    }
-
-    @Override
-    public Long combine(Long l1, Long l2) {
-        if (l1 == null) {
-            return l2;
-        }
-        if (l2 == null) {
-            return l1;
-        }
-        return Math.max(l1, l2);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/OpaqueTridentKafkaSpout.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/OpaqueTridentKafkaSpout.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/OpaqueTridentKafkaSpout.java
deleted file mode 100644
index c98be42..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/OpaqueTridentKafkaSpout.java
+++ /dev/null
@@ -1,62 +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.trident;
-
-import java.util.List;
-import java.util.Map;
-import org.apache.storm.kafka.Partition;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.trident.spout.IOpaquePartitionedTridentSpout;
-import org.apache.storm.tuple.Fields;
-
-
-public class OpaqueTridentKafkaSpout
-    implements IOpaquePartitionedTridentSpout<
-    List<GlobalPartitionInformation>,
-    Partition,
-    Map<String, Object>> {
-
-
-    TridentKafkaConfig _config;
-
-    public OpaqueTridentKafkaSpout(TridentKafkaConfig config) {
-        _config = config;
-    }
-
-    @Override
-    public Emitter<List<GlobalPartitionInformation>,
-        Partition,
-        Map<String, Object>> getEmitter(Map<String, Object> conf,
-                                        TopologyContext context) {
-        return new TridentKafkaEmitter(conf, context, _config, context
-            .getStormId()).asOpaqueEmitter();
-    }
-
-    @Override
-    public IOpaquePartitionedTridentSpout.Coordinator getCoordinator(
-        Map<String, Object> conf,
-        TopologyContext tc) {
-        return new org.apache.storm.kafka.trident.Coordinator(conf, _config);
-    }
-
-    @Override
-    public Fields getOutputFields() {
-        return _config.scheme.getOutputFields();
-    }
-
-    @Override
-    public Map<String, Object> getComponentConfiguration() {
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/StaticBrokerReader.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/StaticBrokerReader.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/StaticBrokerReader.java
deleted file mode 100644
index 3c5cc09..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/StaticBrokerReader.java
+++ /dev/null
@@ -1,44 +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.trident;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
-public class StaticBrokerReader implements IBrokerReader {
-
-    private Map<String, GlobalPartitionInformation> brokers = new TreeMap<String, GlobalPartitionInformation>();
-
-    public StaticBrokerReader(String topic, GlobalPartitionInformation partitionInformation) {
-        this.brokers.put(topic, partitionInformation);
-    }
-
-    @Override
-    public GlobalPartitionInformation getBrokerForTopic(String topic) {
-        if (brokers.containsKey(topic)) return brokers.get(topic);
-        return null;
-    }
-
-    @Override
-    public List<GlobalPartitionInformation> getAllBrokers() {
-        List<GlobalPartitionInformation> list = new ArrayList<GlobalPartitionInformation>();
-        list.addAll(brokers.values());
-        return list;
-    }
-
-    @Override
-    public void close() {
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TransactionalTridentKafkaSpout.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TransactionalTridentKafkaSpout.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TransactionalTridentKafkaSpout.java
deleted file mode 100644
index 7b1d4dd..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TransactionalTridentKafkaSpout.java
+++ /dev/null
@@ -1,50 +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.trident;
-
-import java.util.Map;
-import org.apache.storm.kafka.Partition;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.trident.spout.IPartitionedTridentSpout;
-import org.apache.storm.tuple.Fields;
-
-
-public class TransactionalTridentKafkaSpout implements IPartitionedTridentSpout<GlobalPartitionInformation, Partition, Map> {
-
-    TridentKafkaConfig _config;
-
-    public TransactionalTridentKafkaSpout(TridentKafkaConfig config) {
-        _config = config;
-    }
-
-
-    @Override
-    public IPartitionedTridentSpout.Coordinator getCoordinator(Map<String, Object> conf, TopologyContext context) {
-        return new org.apache.storm.kafka.trident.Coordinator(conf, _config);
-    }
-
-    @Override
-    public IPartitionedTridentSpout.Emitter getEmitter(Map<String, Object> conf, TopologyContext context) {
-        return new TridentKafkaEmitter(conf, context, _config, context.getStormId()).asTransactionalEmitter();
-    }
-
-    @Override
-    public Fields getOutputFields() {
-        return _config.scheme.getOutputFields();
-    }
-
-    @Override
-    public Map<String, Object> getComponentConfiguration() {
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaConfig.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaConfig.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaConfig.java
deleted file mode 100644
index 3dac221..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaConfig.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.  The ASF licenses this file to you under the Apache License, Version
- * 2.0 (the "License"); you may not use this file except in compliance with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions
- * and limitations under the License.
- */
-
-package org.apache.storm.kafka.trident;
-
-import org.apache.storm.kafka.BrokerHosts;
-import org.apache.storm.kafka.KafkaConfig;
-
-
-public class TridentKafkaConfig extends KafkaConfig {
-
-
-    public final IBatchCoordinator coordinator = new DefaultCoordinator();
-
-    public TridentKafkaConfig(BrokerHosts hosts, String topic) {
-        super(hosts, topic);
-    }
-
-    public TridentKafkaConfig(BrokerHosts hosts, String topic, String clientId) {
-        super(hosts, topic, clientId);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaEmitter.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaEmitter.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaEmitter.java
deleted file mode 100644
index cb00579..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaEmitter.java
+++ /dev/null
@@ -1,306 +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.trident;
-
-import com.google.common.collect.ImmutableMap;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import kafka.javaapi.consumer.SimpleConsumer;
-import kafka.javaapi.message.ByteBufferMessageSet;
-import kafka.message.Message;
-import kafka.message.MessageAndOffset;
-import org.apache.storm.Config;
-import org.apache.storm.kafka.DynamicPartitionConnections;
-import org.apache.storm.kafka.FailedFetchException;
-import org.apache.storm.kafka.KafkaUtils;
-import org.apache.storm.kafka.MessageMetadataSchemeAsMultiScheme;
-import org.apache.storm.kafka.Partition;
-import org.apache.storm.kafka.PartitionManager;
-import org.apache.storm.kafka.TopicOffsetOutOfRangeException;
-import org.apache.storm.metric.api.CombinedMetric;
-import org.apache.storm.metric.api.MeanReducer;
-import org.apache.storm.metric.api.ReducedMetric;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.trident.operation.TridentCollector;
-import org.apache.storm.trident.spout.IOpaquePartitionedTridentSpout;
-import org.apache.storm.trident.spout.IPartitionedTridentSpout;
-import org.apache.storm.trident.topology.TransactionAttempt;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TridentKafkaEmitter {
-
-    private static final Logger LOG = LoggerFactory.getLogger(TridentKafkaEmitter.class);
-
-    private DynamicPartitionConnections _connections;
-    private String _topologyName;
-    private KafkaUtils.KafkaOffsetMetric _kafkaOffsetMetric;
-    private ReducedMetric _kafkaMeanFetchLatencyMetric;
-    private CombinedMetric _kafkaMaxFetchLatencyMetric;
-    private TridentKafkaConfig _config;
-    private String _topologyInstanceId;
-
-    public TridentKafkaEmitter(Map<String, Object> conf, TopologyContext context, TridentKafkaConfig config, String topologyInstanceId) {
-        _config = config;
-        _topologyInstanceId = topologyInstanceId;
-        _connections = new DynamicPartitionConnections(_config, KafkaUtils.makeBrokerReader(conf, _config));
-        _topologyName = (String) conf.get(Config.TOPOLOGY_NAME);
-        _kafkaOffsetMetric = new KafkaUtils.KafkaOffsetMetric(_connections);
-        context.registerMetric("kafkaOffset", _kafkaOffsetMetric, _config.metricsTimeBucketSizeInSecs);
-        _kafkaMeanFetchLatencyMetric = context.registerMetric("kafkaFetchAvg", new MeanReducer(), _config.metricsTimeBucketSizeInSecs);
-        _kafkaMaxFetchLatencyMetric = context.registerMetric("kafkaFetchMax", new MaxMetric(), _config.metricsTimeBucketSizeInSecs);
-    }
-
-
-    private Map<String, Object> failFastEmitNewPartitionBatch(
-        final TransactionAttempt attempt,
-        TridentCollector collector,
-        Partition partition,
-        Map<String, Object> lastMeta) {
-        SimpleConsumer consumer = _connections.register(partition);
-        Map<String, Object> ret = doEmitNewPartitionBatch(consumer, partition, collector, lastMeta, attempt);
-        Long offset = (Long) ret.get("offset");
-        Long endOffset = (Long) ret.get("nextOffset");
-        _kafkaOffsetMetric.setOffsetData(partition, new PartitionManager.OffsetData(endOffset, offset));
-        return ret;
-    }
-
-    private Map<String, Object> emitNewPartitionBatch(TransactionAttempt attempt, TridentCollector collector, Partition partition,
-                                                      Map<String, Object> lastMeta) {
-        try {
-            return failFastEmitNewPartitionBatch(attempt, collector, partition, lastMeta);
-        } catch (FailedFetchException e) {
-            LOG.warn("Failed to fetch from partition " + partition);
-            if (lastMeta == null) {
-                return null;
-            } else {
-                Map<String, Object> ret = new HashMap<>();
-                ret.put("offset", lastMeta.get("nextOffset"));
-                ret.put("nextOffset", lastMeta.get("nextOffset"));
-                ret.put("partition", partition.partition);
-                ret.put("broker", ImmutableMap.of("host", partition.host.host, "port", partition.host.port));
-                ret.put("topic", partition.topic);
-                ret.put("topology", ImmutableMap.of("name", _topologyName, "id", _topologyInstanceId));
-                return ret;
-            }
-        }
-    }
-
-    private Map<String, Object> doEmitNewPartitionBatch(SimpleConsumer consumer,
-                                                        Partition partition,
-                                                        TridentCollector collector,
-                                                        Map<String, Object> lastMeta,
-                                                        TransactionAttempt attempt) {
-        LOG.debug("Emitting new partition batch - [transaction = {}], [lastMeta = {}]", attempt, lastMeta);
-        long offset;
-        if (lastMeta != null) {
-            String lastInstanceId = null;
-            Map<String, Object> lastTopoMeta = (Map<String, Object>)
-                lastMeta.get("topology");
-            if (lastTopoMeta != null) {
-                lastInstanceId = (String) lastTopoMeta.get("id");
-            }
-            if (_config.ignoreZkOffsets && !_topologyInstanceId.equals(lastInstanceId)) {
-                offset = KafkaUtils.getOffset(consumer, partition.topic, partition.partition, _config.startOffsetTime);
-            } else {
-                offset = (Long) lastMeta.get("nextOffset");
-            }
-        } else {
-            offset = KafkaUtils.getOffset(consumer, partition.topic, partition.partition, _config);
-        }
-        LOG.debug("[transaction = {}], [OFFSET = {}]", attempt, offset);
-
-        ByteBufferMessageSet msgs = null;
-        try {
-            msgs = fetchMessages(consumer, partition, offset);
-        } catch (TopicOffsetOutOfRangeException e) {
-            long newOffset = KafkaUtils.getOffset(consumer, partition.topic, partition.partition, kafka.api.OffsetRequest.EarliestTime());
-            LOG.warn("OffsetOutOfRange: Updating offset from offset = " + offset + " to offset = " + newOffset);
-            offset = newOffset;
-            msgs = KafkaUtils.fetchMessages(_config, consumer, partition, offset);
-        }
-
-        long endoffset = offset;
-        for (MessageAndOffset msg : msgs) {
-            emit(collector, msg.message(), partition, msg.offset(), attempt);
-            endoffset = msg.nextOffset();
-        }
-        Map<String, Object> newMeta = new HashMap<>();
-        newMeta.put("offset", offset);
-        newMeta.put("nextOffset", endoffset);
-        newMeta.put("instanceId", _topologyInstanceId);
-        newMeta.put("partition", partition.partition);
-        newMeta.put("broker", ImmutableMap.of("host", partition.host.host, "port", partition.host.port));
-        newMeta.put("topic", partition.topic);
-        newMeta.put("topology", ImmutableMap.of("name", _topologyName, "id", _topologyInstanceId));
-        LOG.debug("[transaction = {}], [newMeta = {}]", attempt, newMeta);
-        return newMeta;
-    }
-
-    private ByteBufferMessageSet fetchMessages(SimpleConsumer consumer, Partition partition, long offset) {
-        long start = System.currentTimeMillis();
-        ByteBufferMessageSet msgs = null;
-        msgs = KafkaUtils.fetchMessages(_config, consumer, partition, offset);
-        long millis = System.currentTimeMillis() - start;
-        _kafkaMeanFetchLatencyMetric.update(millis);
-        _kafkaMaxFetchLatencyMetric.update(millis);
-        return msgs;
-    }
-
-    /**
-     * re-emit the batch described by the meta data provided
-     */
-    private void reEmitPartitionBatch(TransactionAttempt attempt, TridentCollector collector, Partition partition,
-                                      Map<String, Object> meta) {
-        LOG.info("re-emitting batch, attempt " + attempt);
-        String instanceId = (String) meta.get("instanceId");
-        if (!_config.ignoreZkOffsets || instanceId.equals(_topologyInstanceId)) {
-            SimpleConsumer consumer = _connections.register(partition);
-            long offset = (Long) meta.get("offset");
-            long nextOffset = (Long) meta.get("nextOffset");
-            ByteBufferMessageSet msgs = null;
-            msgs = fetchMessages(consumer, partition, offset);
-
-            if (msgs != null) {
-                for (MessageAndOffset msg : msgs) {
-                    if (offset == nextOffset) {
-                        break;
-                    }
-                    if (offset > nextOffset) {
-                        throw new RuntimeException("Error when re-emitting batch. overshot the end offset");
-                    }
-                    emit(collector, msg.message(), partition, msg.offset(), attempt);
-                    offset = msg.nextOffset();
-                }
-            }
-        }
-    }
-
-    private void emit(TridentCollector collector, Message msg, Partition partition, long offset, TransactionAttempt attempt) {
-        Iterable<List<Object>> values;
-        if (_config.scheme instanceof MessageMetadataSchemeAsMultiScheme) {
-            values = KafkaUtils.generateTuples((MessageMetadataSchemeAsMultiScheme) _config.scheme, msg, partition, offset);
-        } else {
-            values = KafkaUtils.generateTuples(_config, msg, partition.topic);
-        }
-
-        if (values != null) {
-            for (List<Object> value : values) {
-                LOG.debug("Emitting: [Transaction: {}], [Data: {}]", attempt, value);
-                collector.emit(value);
-            }
-        } else {
-            LOG.debug("NOT Emitting NULL data. [Transaction: {}]", attempt);
-        }
-    }
-
-    private void clear() {
-        _connections.clear();
-    }
-
-    private List<Partition> orderPartitions(List<GlobalPartitionInformation> partitions) {
-        List<Partition> part = new ArrayList<Partition>();
-        for (GlobalPartitionInformation globalPartitionInformation : partitions)
-            part.addAll(globalPartitionInformation.getOrderedPartitions());
-        return part;
-    }
-
-    private void refresh(List<Partition> list) {
-        _connections.clear();
-        _kafkaOffsetMetric.refreshPartitions(new HashSet<Partition>(list));
-    }
-
-
-    public IOpaquePartitionedTridentSpout.Emitter<List<GlobalPartitionInformation>, Partition, Map<String, Object>> asOpaqueEmitter() {
-
-        return new IOpaquePartitionedTridentSpout.Emitter<List<GlobalPartitionInformation>, Partition, Map<String, Object>>() {
-
-            /**
-             * Emit a batch of tuples for a partition/transaction.
-             *
-             * Return the metadata describing this batch that will be used as lastPartitionMeta
-             * for defining the parameters of the next batch.
-             */
-            @Override
-            public Map<String, Object> emitPartitionBatch(TransactionAttempt transactionAttempt, TridentCollector tridentCollector,
-                                                          Partition partition, Map<String, Object> map) {
-                return emitNewPartitionBatch(transactionAttempt, tridentCollector, partition, map);
-            }
-
-            @Override
-            public void refreshPartitions(List<Partition> partitions) {
-                refresh(partitions);
-            }
-
-            @Override
-            public List<Partition> getOrderedPartitions(List<GlobalPartitionInformation> partitionInformation) {
-                return orderPartitions(partitionInformation);
-            }
-
-            @Override
-            public void close() {
-                clear();
-            }
-        };
-    }
-
-    public IPartitionedTridentSpout.Emitter asTransactionalEmitter() {
-        return new IPartitionedTridentSpout.Emitter<List<GlobalPartitionInformation>, Partition, Map<String, Object>>() {
-
-            /**
-             * Emit a batch of tuples for a partition/transaction that's never been emitted before.
-             * Return the metadata that can be used to reconstruct this partition/batch in the future.
-             */
-            @Override
-            public Map<String, Object> emitPartitionBatchNew(TransactionAttempt transactionAttempt, TridentCollector tridentCollector,
-                                                             Partition partition, Map<String, Object> map) {
-                return failFastEmitNewPartitionBatch(transactionAttempt, tridentCollector, partition, map);
-            }
-
-            /**
-             * Emit a batch of tuples for a partition/transaction that has been emitted before, using
-             * the metadata created when it was first emitted.
-             */
-            @Override
-            public void emitPartitionBatch(TransactionAttempt transactionAttempt, TridentCollector tridentCollector, Partition partition,
-                                           Map<String, Object> map) {
-                reEmitPartitionBatch(transactionAttempt, tridentCollector, partition, map);
-            }
-
-            /**
-             * This method is called when this task is responsible for a new set of partitions. Should be used
-             * to manage things like connections to brokers.
-             */
-            @Override
-            public void refreshPartitions(List<Partition> partitions) {
-                refresh(partitions);
-            }
-
-            @Override
-            public List<Partition> getOrderedPartitions(List<GlobalPartitionInformation> partitionInformation) {
-                return orderPartitions(partitionInformation);
-            }
-
-            @Override
-            public void close() {
-                clear();
-            }
-        };
-
-    }
-
-
-}

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

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaStateFactory.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaStateFactory.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaStateFactory.java
deleted file mode 100644
index 5b66fd8..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaStateFactory.java
+++ /dev/null
@@ -1,57 +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.trident;
-
-import java.util.Map;
-import java.util.Properties;
-import org.apache.storm.kafka.trident.mapper.TridentTupleToKafkaMapper;
-import org.apache.storm.kafka.trident.selector.KafkaTopicSelector;
-import org.apache.storm.task.IMetricsContext;
-import org.apache.storm.trident.state.State;
-import org.apache.storm.trident.state.StateFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TridentKafkaStateFactory implements StateFactory {
-
-    private static final Logger LOG = LoggerFactory.getLogger(TridentKafkaStateFactory.class);
-
-    private TridentTupleToKafkaMapper mapper;
-    private KafkaTopicSelector topicSelector;
-    private Properties producerProperties = new Properties();
-
-    public TridentKafkaStateFactory withTridentTupleToKafkaMapper(TridentTupleToKafkaMapper mapper) {
-        this.mapper = mapper;
-        return this;
-    }
-
-    public TridentKafkaStateFactory withKafkaTopicSelector(KafkaTopicSelector selector) {
-        this.topicSelector = selector;
-        return this;
-    }
-
-    public TridentKafkaStateFactory withProducerProperties(Properties props) {
-        this.producerProperties = props;
-        return this;
-    }
-
-    @Override
-    public State makeState(Map<String, Object> conf, IMetricsContext metrics, int partitionIndex, int numPartitions) {
-        LOG.info("makeState(partitonIndex={}, numpartitions={}", partitionIndex, numPartitions);
-        TridentKafkaState state = new TridentKafkaState()
-            .withKafkaTopicSelector(this.topicSelector)
-            .withTridentTupleToKafkaMapper(this.mapper);
-        state.prepare(producerProperties);
-        return state;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaUpdater.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaUpdater.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaUpdater.java
deleted file mode 100644
index 1100b66..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaUpdater.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.  The ASF licenses this file to you under the Apache License, Version
- * 2.0 (the "License"); you may not use this file except in compliance with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions
- * and limitations under the License.
- */
-
-package org.apache.storm.kafka.trident;
-
-import java.util.List;
-import org.apache.storm.trident.operation.TridentCollector;
-import org.apache.storm.trident.state.BaseStateUpdater;
-import org.apache.storm.trident.tuple.TridentTuple;
-
-public class TridentKafkaUpdater extends BaseStateUpdater<TridentKafkaState> {
-    @Override
-    public void updateState(TridentKafkaState state, List<TridentTuple> tuples, TridentCollector collector) {
-        state.updateState(tuples, collector);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/ZkBrokerReader.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/ZkBrokerReader.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/ZkBrokerReader.java
deleted file mode 100644
index d40256e..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/ZkBrokerReader.java
+++ /dev/null
@@ -1,79 +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.trident;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import org.apache.storm.kafka.DynamicBrokersReader;
-import org.apache.storm.kafka.ZkHosts;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-public class ZkBrokerReader implements IBrokerReader {
-
-    private static final Logger LOG = LoggerFactory.getLogger(ZkBrokerReader.class);
-
-    List<GlobalPartitionInformation> cachedBrokers = new ArrayList<GlobalPartitionInformation>();
-    DynamicBrokersReader reader;
-    long lastRefreshTimeMs;
-
-
-    long refreshMillis;
-
-    public ZkBrokerReader(Map<String, Object> conf, String topic, ZkHosts hosts) {
-        try {
-            reader = new DynamicBrokersReader(conf, hosts.brokerZkStr, hosts.brokerZkPath, topic);
-            cachedBrokers = reader.getBrokerInfo();
-            lastRefreshTimeMs = System.currentTimeMillis();
-            refreshMillis = hosts.refreshFreqSecs * 1000L;
-        } catch (java.net.SocketTimeoutException e) {
-            LOG.warn("Failed to update brokers", e);
-        }
-
-    }
-
-    private void refresh() {
-        long currTime = System.currentTimeMillis();
-        if (currTime > lastRefreshTimeMs + refreshMillis) {
-            try {
-                LOG.info("brokers need refreshing because " + refreshMillis + "ms have expired");
-                cachedBrokers = reader.getBrokerInfo();
-                lastRefreshTimeMs = currTime;
-            } catch (java.net.SocketTimeoutException e) {
-                LOG.warn("Failed to update brokers", e);
-            }
-        }
-    }
-
-    @Override
-    public GlobalPartitionInformation getBrokerForTopic(String topic) {
-        refresh();
-        for (GlobalPartitionInformation partitionInformation : cachedBrokers) {
-            if (partitionInformation.topic.equals(topic)) return partitionInformation;
-        }
-        return null;
-    }
-
-    @Override
-    public List<GlobalPartitionInformation> getAllBrokers() {
-        refresh();
-        return cachedBrokers;
-    }
-
-    @Override
-    public void close() {
-        reader.close();
-    }
-}

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

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/mapper/TridentTupleToKafkaMapper.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/mapper/TridentTupleToKafkaMapper.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/mapper/TridentTupleToKafkaMapper.java
deleted file mode 100644
index 4a522d6..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/mapper/TridentTupleToKafkaMapper.java
+++ /dev/null
@@ -1,22 +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.trident.mapper;
-
-import java.io.Serializable;
-import org.apache.storm.trident.tuple.TridentTuple;
-
-public interface TridentTupleToKafkaMapper<K, V> extends Serializable {
-    K getKeyFromTuple(TridentTuple tuple);
-
-    V getMessageFromTuple(TridentTuple tuple);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/selector/DefaultTopicSelector.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/selector/DefaultTopicSelector.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/selector/DefaultTopicSelector.java
deleted file mode 100644
index 93b5566..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/selector/DefaultTopicSelector.java
+++ /dev/null
@@ -1,29 +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.trident.selector;
-
-import org.apache.storm.trident.tuple.TridentTuple;
-
-public class DefaultTopicSelector implements KafkaTopicSelector {
-
-    private final String topicName;
-
-    public DefaultTopicSelector(final String topicName) {
-        this.topicName = topicName;
-    }
-
-    @Override
-    public String getTopic(TridentTuple tuple) {
-        return topicName;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/selector/KafkaTopicSelector.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/selector/KafkaTopicSelector.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/selector/KafkaTopicSelector.java
deleted file mode 100644
index 6de3921..0000000
--- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/selector/KafkaTopicSelector.java
+++ /dev/null
@@ -1,20 +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.trident.selector;
-
-import java.io.Serializable;
-import org.apache.storm.trident.tuple.TridentTuple;
-
-public interface KafkaTopicSelector extends Serializable {
-    String getTopic(TridentTuple tuple);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e58ac3e0/external/storm-kafka/src/test/org/apache/storm/kafka/DynamicBrokersReaderTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/DynamicBrokersReaderTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/DynamicBrokersReaderTest.java
deleted file mode 100644
index a6bb61c..0000000
--- a/external/storm-kafka/src/test/org/apache/storm/kafka/DynamicBrokersReaderTest.java
+++ /dev/null
@@ -1,245 +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;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.retry.ExponentialBackoffRetry;
-import org.apache.curator.test.TestingServer;
-import org.apache.curator.utils.ZKPaths;
-import org.apache.storm.Config;
-import org.apache.storm.kafka.trident.GlobalPartitionInformation;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
-/**
- * Date: 16/05/2013
- * Time: 20:35
- */
-public class DynamicBrokersReaderTest {
-    private DynamicBrokersReader dynamicBrokersReader, wildCardBrokerReader;
-    private String masterPath = "/brokers";
-    private String topic = "testing1";
-    private String secondTopic = "testing2";
-    private String thirdTopic = "testing3";
-
-    private CuratorFramework zookeeper;
-    private TestingServer server;
-
-    @Before
-    public void setUp() throws Exception {
-        server = new TestingServer();
-        String connectionString = server.getConnectString();
-        Map<String, Object> conf = new HashMap<>();
-        conf.put(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT, 1000);
-        conf.put(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT, 1000);
-        conf.put(Config.STORM_ZOOKEEPER_RETRY_TIMES, 4);
-        conf.put(Config.STORM_ZOOKEEPER_RETRY_INTERVAL, 5);
-
-        ExponentialBackoffRetry retryPolicy = new ExponentialBackoffRetry(1000, 3);
-        zookeeper = CuratorFrameworkFactory.newClient(connectionString, retryPolicy);
-        dynamicBrokersReader = new DynamicBrokersReader(conf, connectionString, masterPath, topic);
-
-        Map<String, Object> conf2 = new HashMap<>();
-        conf2.putAll(conf);
-        conf2.put("kafka.topic.wildcard.match", true);
-
-        wildCardBrokerReader = new DynamicBrokersReader(conf2, connectionString, masterPath, "^test.*$");
-        zookeeper.start();
-    }
-
-    @After
-    public void tearDown() throws Exception {
-        dynamicBrokersReader.close();
-        zookeeper.close();
-        server.close();
-    }
-
-    private void addPartition(int id, String host, int port, String topic) throws Exception {
-        writePartitionId(id, topic);
-        writeLeader(id, 0, topic);
-        writeLeaderDetails(0, host, port);
-    }
-
-    private void addPartition(int id, int leader, String host, int port, String topic) throws Exception {
-        writePartitionId(id, topic);
-        writeLeader(id, leader, topic);
-        writeLeaderDetails(leader, host, port);
-    }
-
-    private void writePartitionId(int id, String topic) throws Exception {
-        String path = dynamicBrokersReader.partitionPath(topic);
-        writeDataToPath(path, ("" + id));
-    }
-
-    private void writeDataToPath(String path, String data) throws Exception {
-        ZKPaths.mkdirs(zookeeper.getZookeeperClient().getZooKeeper(), path);
-        zookeeper.setData().forPath(path, data.getBytes());
-    }
-
-    private void writeLeader(int id, int leaderId, String topic) throws Exception {
-        String path = dynamicBrokersReader.partitionPath(topic) + "/" + id + "/state";
-        String value = " { \"controller_epoch\":4, \"isr\":[ 1, 0 ], \"leader\":" + leaderId + ", \"leader_epoch\":1, \"version\":1 }";
-        writeDataToPath(path, value);
-    }
-
-    private void writeLeaderDetails(int leaderId, String host, int port) throws Exception {
-        String path = dynamicBrokersReader.brokerPath() + "/" + leaderId;
-        String value = "{ \"host\":\"" + host + "\", \"jmx_port\":9999, \"port\":" + port + ", \"version\":1 }";
-        writeDataToPath(path, value);
-    }
-
-
-    private GlobalPartitionInformation getByTopic(List<GlobalPartitionInformation> partitions, String topic) {
-        for (GlobalPartitionInformation partitionInformation : partitions) {
-            if (partitionInformation.topic.equals(topic)) return partitionInformation;
-        }
-        return null;
-    }
-
-    @Test
-    public void testGetBrokerInfo() throws Exception {
-        String host = "localhost";
-        int port = 9092;
-        int partition = 0;
-        addPartition(partition, host, port, topic);
-        List<GlobalPartitionInformation> partitions = dynamicBrokersReader.getBrokerInfo();
-
-        GlobalPartitionInformation brokerInfo = getByTopic(partitions, topic);
-        assertNotNull(brokerInfo);
-        assertEquals(1, brokerInfo.getOrderedPartitions().size());
-        assertEquals(port, brokerInfo.getBrokerFor(partition).port);
-        assertEquals(host, brokerInfo.getBrokerFor(partition).host);
-    }
-
-    @Test
-    public void testGetBrokerInfoWildcardMatch() throws Exception {
-        String host = "localhost";
-        int port = 9092;
-        int partition = 0;
-        addPartition(partition, host, port, topic);
-        addPartition(partition, host, port, secondTopic);
-
-        List<GlobalPartitionInformation> partitions = wildCardBrokerReader.getBrokerInfo();
-
-        GlobalPartitionInformation brokerInfo = getByTopic(partitions, topic);
-        assertNotNull(brokerInfo);
-        assertEquals(1, brokerInfo.getOrderedPartitions().size());
-        assertEquals(port, brokerInfo.getBrokerFor(partition).port);
-        assertEquals(host, brokerInfo.getBrokerFor(partition).host);
-
-        brokerInfo = getByTopic(partitions, secondTopic);
-        assertNotNull(brokerInfo);
-        assertEquals(1, brokerInfo.getOrderedPartitions().size());
-        assertEquals(port, brokerInfo.getBrokerFor(partition).port);
-        assertEquals(host, brokerInfo.getBrokerFor(partition).host);
-
-        addPartition(partition, host, port, thirdTopic);
-        //Discover newly added topic
-        partitions = wildCardBrokerReader.getBrokerInfo();
-        assertNotNull(getByTopic(partitions, topic));
-        assertNotNull(getByTopic(partitions, secondTopic));
-        assertNotNull(getByTopic(partitions, secondTopic));
-    }
-
-
-    @Test
-    public void testMultiplePartitionsOnDifferentHosts() throws Exception {
-        String host = "localhost";
-        int port = 9092;
-        int secondPort = 9093;
-        int partition = 0;
-        int secondPartition = partition + 1;
-        addPartition(partition, 0, host, port, topic);
-        addPartition(secondPartition, 1, host, secondPort, topic);
-
-        List<GlobalPartitionInformation> partitions = dynamicBrokersReader.getBrokerInfo();
-
-        GlobalPartitionInformation brokerInfo = getByTopic(partitions, topic);
-        assertNotNull(brokerInfo);
-        assertEquals(2, brokerInfo.getOrderedPartitions().size());
-
-        assertEquals(port, brokerInfo.getBrokerFor(partition).port);
-        assertEquals(host, brokerInfo.getBrokerFor(partition).host);
-
-        assertEquals(secondPort, brokerInfo.getBrokerFor(secondPartition).port);
-        assertEquals(host, brokerInfo.getBrokerFor(secondPartition).host);
-    }
-
-
-    @Test
-    public void testMultiplePartitionsOnSameHost() throws Exception {
-        String host = "localhost";
-        int port = 9092;
-        int partition = 0;
-        int secondPartition = partition + 1;
-        addPartition(partition, 0, host, port, topic);
-        addPartition(secondPartition, 0, host, port, topic);
-
-        List<GlobalPartitionInformation> partitions = dynamicBrokersReader.getBrokerInfo();
-
-        GlobalPartitionInformation brokerInfo = getByTopic(partitions, topic);
-        assertNotNull(brokerInfo);
-        assertEquals(2, brokerInfo.getOrderedPartitions().size());
-
-        assertEquals(port, brokerInfo.getBrokerFor(partition).port);
-        assertEquals(host, brokerInfo.getBrokerFor(partition).host);
-
-        assertEquals(port, brokerInfo.getBrokerFor(secondPartition).port);
-        assertEquals(host, brokerInfo.getBrokerFor(secondPartition).host);
-    }
-
-    @Test
-    public void testSwitchHostForPartition() throws Exception {
-        String host = "localhost";
-        int port = 9092;
-        int partition = 0;
-        addPartition(partition, host, port, topic);
-        List<GlobalPartitionInformation> partitions = dynamicBrokersReader.getBrokerInfo();
-
-        GlobalPartitionInformation brokerInfo = getByTopic(partitions, topic);
-        assertNotNull(brokerInfo);
-        assertEquals(port, brokerInfo.getBrokerFor(partition).port);
-        assertEquals(host, brokerInfo.getBrokerFor(partition).host);
-
-        String newHost = host + "switch";
-        int newPort = port + 1;
-        addPartition(partition, newHost, newPort, topic);
-        partitions = dynamicBrokersReader.getBrokerInfo();
-
-        brokerInfo = getByTopic(partitions, topic);
-        assertNotNull(brokerInfo);
-        assertEquals(newPort, brokerInfo.getBrokerFor(partition).port);
-        assertEquals(newHost, brokerInfo.getBrokerFor(partition).host);
-    }
-
-    @Test(expected = NullPointerException.class)
-    public void testErrorLogsWhenConfigIsMissing() throws Exception {
-        String connectionString = server.getConnectString();
-        Map<String, Object> conf = new HashMap<>();
-        conf.put(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT, 1000);
-        //        conf.put(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT, 1000);
-        conf.put(Config.STORM_ZOOKEEPER_RETRY_TIMES, 4);
-        conf.put(Config.STORM_ZOOKEEPER_RETRY_INTERVAL, 5);
-
-        DynamicBrokersReader dynamicBrokersReader1 = new DynamicBrokersReader(conf, connectionString, masterPath, topic);
-
-    }
-}