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 2016/08/23 00:14:32 UTC

storm git commit: fix code in KafkaSpoutRetryExponentialBackoff in storm-kafka-client

Repository: storm
Updated Branches:
  refs/heads/1.x-branch dc20e9ce0 -> 85504a5af


fix code in KafkaSpoutRetryExponentialBackoff in storm-kafka-client

* Closes #1629
* also fix document to use simpler method


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

Branch: refs/heads/1.x-branch
Commit: 85504a5af78accd99e427018ab0a6def780b696d
Parents: dc20e9c
Author: leedohyun <le...@gmail.com>
Authored: Wed Aug 17 15:51:51 2016 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Tue Aug 23 09:13:50 2016 +0900

----------------------------------------------------------------------
 external/storm-kafka-client/README.md                          | 2 +-
 .../storm/kafka/spout/KafkaSpoutRetryExponentialBackoff.java   | 2 +-
 .../kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java    | 6 +-----
 3 files changed, 3 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/85504a5a/external/storm-kafka-client/README.md
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/README.md b/external/storm-kafka-client/README.md
index 652d6e1..515f9ac 100644
--- a/external/storm-kafka-client/README.md
+++ b/external/storm-kafka-client/README.md
@@ -35,7 +35,7 @@ kafkaConsumerProps.put(KafkaSpoutConfig.Consumer.GROUP_ID,"kafkaSpoutTestGroup")
 kafkaConsumerProps.put(KafkaSpoutConfig.Consumer.KEY_DESERIALIZER,"org.apache.kafka.common.serialization.StringDeserializer");
 kafkaConsumerProps.put(KafkaSpoutConfig.Consumer.VALUE_DESERIALIZER,"org.apache.kafka.common.serialization.StringDeserializer");
 
-KafkaSpoutRetryService retryService = new KafkaSpoutRetryExponentialBackoff(new KafkaSpoutRetryExponentialBackoff.TimeInterval(500, TimeUnit.MICROSECONDS),
+KafkaSpoutRetryService retryService = new KafkaSpoutRetryExponentialBackoff(KafkaSpoutRetryExponentialBackoff.TimeInterval.microSeconds(500),
         KafkaSpoutRetryExponentialBackoff.TimeInterval.milliSeconds(2), Integer.MAX_VALUE, KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(10));
 ```
 

http://git-wip-us.apache.org/repos/asf/storm/blob/85504a5a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutRetryExponentialBackoff.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutRetryExponentialBackoff.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutRetryExponentialBackoff.java
index 6fe997c..f59367d 100644
--- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutRetryExponentialBackoff.java
+++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutRetryExponentialBackoff.java
@@ -117,7 +117,7 @@ public class KafkaSpoutRetryExponentialBackoff implements KafkaSpoutRetryService
         }
 
         public static TimeInterval microSeconds(long length) {
-            return new TimeInterval(length, TimeUnit.MILLISECONDS);
+            return new TimeInterval(length, TimeUnit.MICROSECONDS);
         }
 
         public long lengthNanos() {

http://git-wip-us.apache.org/repos/asf/storm/blob/85504a5a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java
index 952c5d3..5a78137 100644
--- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java
+++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java
@@ -104,14 +104,10 @@ public class KafkaSpoutTopologyMainNamedTopics {
     }
 
     protected KafkaSpoutRetryService getRetryService() {
-            return new KafkaSpoutRetryExponentialBackoff(getTimeInterval(500, TimeUnit.MICROSECONDS),
+            return new KafkaSpoutRetryExponentialBackoff(TimeInterval.microSeconds(500),
                     TimeInterval.milliSeconds(2), Integer.MAX_VALUE, TimeInterval.seconds(10));
     }
 
-    protected TimeInterval getTimeInterval(long delay, TimeUnit timeUnit) {
-        return new TimeInterval(delay, timeUnit);
-    }
-
     protected Map<String,Object> getKafkaConsumerProps() {
         Map<String, Object> props = new HashMap<>();
 //        props.put(KafkaSpoutConfig.Consumer.ENABLE_AUTO_COMMIT, "true");