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);
-
- }
-}