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 2015/07/23 01:39:44 UTC

[1/7] storm git commit: Correctly hash byte array tuple values

Repository: storm
Updated Branches:
  refs/heads/master 6a21b6a4e -> 9b2fd72e9


Correctly hash byte array tuple values


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

Branch: refs/heads/master
Commit: b132520adb00def6d03ef020b1cb5973eb3b3519
Parents: 98cbb34
Author: Derek Dagit <de...@yahoo-inc.com>
Authored: Fri Jul 17 18:45:39 2015 -0500
Committer: Derek Dagit <de...@yahoo-inc.com>
Committed: Fri Jul 17 18:45:39 2015 -0500

----------------------------------------------------------------------
 .../src/clj/backtype/storm/daemon/task.clj      |  1 -
 storm-core/src/clj/backtype/storm/tuple.clj     |  7 +-
 .../storm/grouping/PartialKeyGrouping.java      |  7 +-
 .../storm/testing/TestWordBytesCounter.java     | 27 ++++++++
 .../backtype/storm/testing/TestWordCounter.java |  6 +-
 .../backtype/storm/testing/WordBytesCount.java  | 35 ++++++++++
 .../test/clj/backtype/storm/grouping_test.clj   | 67 ++++++++++++--------
 7 files changed, 118 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b132520a/storm-core/src/clj/backtype/storm/daemon/task.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/task.clj b/storm-core/src/clj/backtype/storm/daemon/task.clj
index 3bc1d05..9cf2b85 100644
--- a/storm-core/src/clj/backtype/storm/daemon/task.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/task.clj
@@ -27,7 +27,6 @@
   (:import [backtype.storm.spout ShellSpout])
   (:import [java.util Collection List ArrayList])
   (:require [backtype.storm
-             [tuple :as tuple]
              [thrift :as thrift]
              [stats :as stats]])
   (:require [backtype.storm.daemon.builtin-metrics :as builtin-metrics]))

http://git-wip-us.apache.org/repos/asf/storm/blob/b132520a/storm-core/src/clj/backtype/storm/tuple.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/tuple.clj b/storm-core/src/clj/backtype/storm/tuple.clj
index a88b73b..790a823 100644
--- a/storm-core/src/clj/backtype/storm/tuple.clj
+++ b/storm-core/src/clj/backtype/storm/tuple.clj
@@ -15,8 +15,11 @@
 ;; limitations under the License.
 
 (ns backtype.storm.tuple
-  (:import [java.util List]))
+  (:import [java.util Arrays List]))
 
 (defn list-hash-code
   [^List alist]
-  (.hashCode alist))
+  (if (nil? alist)
+    1
+    (let [^"[Ljava.lang.Object;" array (.toArray alist)] ;; Object[]
+      (Arrays/deepHashCode array))))

http://git-wip-us.apache.org/repos/asf/storm/blob/b132520a/storm-core/src/jvm/backtype/storm/grouping/PartialKeyGrouping.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/grouping/PartialKeyGrouping.java b/storm-core/src/jvm/backtype/storm/grouping/PartialKeyGrouping.java
index d1f534b..456a8ef 100644
--- a/storm-core/src/jvm/backtype/storm/grouping/PartialKeyGrouping.java
+++ b/storm-core/src/jvm/backtype/storm/grouping/PartialKeyGrouping.java
@@ -20,6 +20,7 @@ package backtype.storm.grouping;
 import java.io.Serializable;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 import backtype.storm.generated.GlobalStreamId;
@@ -65,7 +66,11 @@ public class PartialKeyGrouping implements CustomStreamGrouping, Serializable {
                 List<Object> selectedFields = outFields.select(fields, values);
                 ByteBuffer out = ByteBuffer.allocate(selectedFields.size() * 4);
                 for (Object o: selectedFields) {
-                    out.putInt(o.hashCode());
+                    if (o instanceof Object[]) {
+                        out.putInt(Arrays.deepHashCode((Object[])o));
+                    } else {
+                        out.putInt(o.hashCode());
+                    }
                 }
                 raw = out.array();
             } else {

http://git-wip-us.apache.org/repos/asf/storm/blob/b132520a/storm-core/src/jvm/backtype/storm/testing/TestWordBytesCounter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/testing/TestWordBytesCounter.java b/storm-core/src/jvm/backtype/storm/testing/TestWordBytesCounter.java
new file mode 100644
index 0000000..e8a09a7
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/testing/TestWordBytesCounter.java
@@ -0,0 +1,27 @@
+/**
+ * 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 backtype.storm.testing;
+
+import backtype.storm.tuple.Tuple;
+
+public class TestWordBytesCounter extends TestWordCounter {
+    @Override
+    protected String getTupleValue(Tuple t, int idx) {
+      return new String(t.getBinary(idx));
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/b132520a/storm-core/src/jvm/backtype/storm/testing/TestWordCounter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/testing/TestWordCounter.java b/storm-core/src/jvm/backtype/storm/testing/TestWordCounter.java
index 551b054..08b1397 100644
--- a/storm-core/src/jvm/backtype/storm/testing/TestWordCounter.java
+++ b/storm-core/src/jvm/backtype/storm/testing/TestWordCounter.java
@@ -38,9 +38,13 @@ public class TestWordCounter extends BaseBasicBolt {
     public void prepare(Map stormConf, TopologyContext context) {
         _counts = new HashMap<String, Integer>();
     }
+
+    protected String getTupleValue(Tuple t, int idx) {
+      return (String) t.getValues().get(idx);
+    }
     
     public void execute(Tuple input, BasicOutputCollector collector) {
-        String word = (String) input.getValues().get(0);
+        String word = getTupleValue(input, 0);
         int count = 0;
         if(_counts.containsKey(word)) {
             count = _counts.get(word);

http://git-wip-us.apache.org/repos/asf/storm/blob/b132520a/storm-core/src/jvm/backtype/storm/testing/WordBytesCount.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/testing/WordBytesCount.java b/storm-core/src/jvm/backtype/storm/testing/WordBytesCount.java
new file mode 100644
index 0000000..31fdca6
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/testing/WordBytesCount.java
@@ -0,0 +1,35 @@
+/**
+ * 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 backtype.storm.testing;
+
+import backtype.storm.topology.BasicOutputCollector;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.base.BaseBasicBolt;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Values;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class WordBytesCount extends WordCount {
+  @Override
+  protected String getWordFromTuple(Tuple tuple, int idx) {
+    return new String(tuple.getBinary(idx));
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/b132520a/storm-core/test/clj/backtype/storm/grouping_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/grouping_test.clj b/storm-core/test/clj/backtype/storm/grouping_test.clj
index fc13b0c..9402842 100644
--- a/storm-core/test/clj/backtype/storm/grouping_test.clj
+++ b/storm-core/test/clj/backtype/storm/grouping_test.clj
@@ -15,40 +15,53 @@
 ;; limitations under the License.
 (ns backtype.storm.grouping-test
   (:use [clojure test])
-  (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter NGrouping]
+  (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter TestWordBytesCounter NGrouping]
            [backtype.storm.generated JavaObject JavaObjectArg])
   (:use [backtype.storm testing clojure])
   (:use [backtype.storm.daemon common])
   (:require [backtype.storm [thrift :as thrift]]))
 
-(deftest test-shuffle
+ (deftest test-shuffle
+   (with-simulated-time-local-cluster [cluster :supervisors 4]
+     (let [topology (thrift/mk-topology
+                     {"1" (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 4)}
+                     {"2" (thrift/mk-bolt-spec {"1" :shuffle} (TestGlobalCount.)
+                                             :parallelism-hint 6)
+                      })
+           results (complete-topology cluster
+                                      topology
+                                      ;; important for test that
+                                      ;; #tuples = multiple of 4 and 6
+                                      :mock-sources {"1" (->> [["a"] ["b"]]
+                                                              (repeat 12)
+                                                              (apply concat))})]
+       (is (ms= (apply concat (repeat 6 [[1] [2] [3] [4]]))
+                (read-tuples results "2")))
+       )))
+
+(deftest test-field
   (with-simulated-time-local-cluster [cluster :supervisors 4]
-    (let [topology (thrift/mk-topology
-                    {"1" (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 4)}
-                    {"2" (thrift/mk-bolt-spec {"1" :shuffle} (TestGlobalCount.)
-                                            :parallelism-hint 6)
+    (let [spout-phint 4
+          bolt-phint 6
+          topology (thrift/mk-topology
+                    {"1" (thrift/mk-spout-spec (TestWordSpout. true)
+                                               :parallelism-hint spout-phint)}
+                    {"2" (thrift/mk-bolt-spec {"1" ["word"]}
+                                              (TestWordBytesCounter.)
+                                              :parallelism-hint bolt-phint)
                      })
-          results (complete-topology cluster
-                                     topology
-                                     ;; important for test that
-                                     ;; #tuples = multiple of 4 and 6
-                                     :mock-sources {"1" [["a"] ["b"]
-                                                         ["a"] ["b"]
-                                                         ["a"] ["b"]
-                                                         ["a"] ["b"]
-                                                         ["a"] ["b"]
-                                                         ["a"] ["b"]
-                                                         ["a"] ["b"]
-                                                         ["a"] ["b"]
-                                                         ["a"] ["b"]
-                                                         ["a"] ["b"]
-                                                         ["a"] ["b"]
-                                                         ["a"] ["b"]
-                                                       ]}
-                                     )]
-      (is (ms= (apply concat (repeat 6 [[1] [2] [3] [4]]))
-               (read-tuples results "2")))
-      )))
+          results (complete-topology
+                    cluster
+                    topology
+                    :mock-sources {"1" (->> [[(.getBytes "a")]
+                                             [(.getBytes "b")]]
+                                            (repeat (* spout-phint bolt-phint))
+                                            (apply concat))})]
+      (is (ms= (apply concat
+                      (for [value '("a" "b")
+                            sum (range 1 (inc (* spout-phint bolt-phint)))]
+                        [[value sum]]))
+               (read-tuples results "2"))))))
 
 (defbolt id-bolt ["val"] [tuple collector]
   (emit-bolt! collector (.getValues tuple))


[4/7] storm git commit: handle Lists and primitive array types

Posted by ka...@apache.org.
handle Lists and primitive array types


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

Branch: refs/heads/master
Commit: 639f6a78cb1fc462590d9fe9c0fd7bbbd959505c
Parents: 70cdda2
Author: Derek Dagit <de...@yahoo-inc.com>
Authored: Wed Jul 22 12:17:39 2015 -0500
Committer: Derek Dagit <de...@yahoo-inc.com>
Committed: Wed Jul 22 12:17:39 2015 -0500

----------------------------------------------------------------------
 .../storm/grouping/PartialKeyGrouping.java      | 24 ++++++++++++++++++--
 1 file changed, 22 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/639f6a78/storm-core/src/jvm/backtype/storm/grouping/PartialKeyGrouping.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/grouping/PartialKeyGrouping.java b/storm-core/src/jvm/backtype/storm/grouping/PartialKeyGrouping.java
index 456a8ef..43ad5a0 100644
--- a/storm-core/src/jvm/backtype/storm/grouping/PartialKeyGrouping.java
+++ b/storm-core/src/jvm/backtype/storm/grouping/PartialKeyGrouping.java
@@ -66,10 +66,30 @@ public class PartialKeyGrouping implements CustomStreamGrouping, Serializable {
                 List<Object> selectedFields = outFields.select(fields, values);
                 ByteBuffer out = ByteBuffer.allocate(selectedFields.size() * 4);
                 for (Object o: selectedFields) {
-                    if (o instanceof Object[]) {
+                    if (o instanceof List) {
+                        out.putInt(Arrays.deepHashCode(((List)o).toArray()));
+                    } else if (o instanceof Object[]) {
                         out.putInt(Arrays.deepHashCode((Object[])o));
-                    } else {
+                    } else if (o instanceof byte[]) {
+                        out.putInt(Arrays.hashCode((byte[]) o));
+                    } else if (o instanceof short[]) {
+                        out.putInt(Arrays.hashCode((short[]) o));
+                    } else if (o instanceof int[]) {
+                        out.putInt(Arrays.hashCode((int[]) o));
+                    } else if (o instanceof long[]) {
+                        out.putInt(Arrays.hashCode((long[]) o));
+                    } else if (o instanceof char[]) {
+                        out.putInt(Arrays.hashCode((char[]) o));
+                    } else if (o instanceof float[]) {
+                        out.putInt(Arrays.hashCode((float[]) o));
+                    } else if (o instanceof double[]) {
+                        out.putInt(Arrays.hashCode((double[]) o));
+                    } else if (o instanceof boolean[]) {
+                        out.putInt(Arrays.hashCode((boolean[]) o));
+                    } else if (o != null) {
                         out.putInt(o.hashCode());
+                    } else {
+                      out.putInt(0);
                     }
                 }
                 raw = out.array();


[2/7] storm git commit: remove unnecessary file

Posted by ka...@apache.org.
remove unnecessary file


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

Branch: refs/heads/master
Commit: 1b581ef1cd2c6a8c9f07f2097816c0312ae003ba
Parents: b132520
Author: Derek Dagit <de...@yahoo-inc.com>
Authored: Fri Jul 17 18:52:20 2015 -0500
Committer: Derek Dagit <de...@yahoo-inc.com>
Committed: Fri Jul 17 18:52:20 2015 -0500

----------------------------------------------------------------------
 .../backtype/storm/testing/WordBytesCount.java  | 35 --------------------
 1 file changed, 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/1b581ef1/storm-core/src/jvm/backtype/storm/testing/WordBytesCount.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/testing/WordBytesCount.java b/storm-core/src/jvm/backtype/storm/testing/WordBytesCount.java
deleted file mode 100644
index 31fdca6..0000000
--- a/storm-core/src/jvm/backtype/storm/testing/WordBytesCount.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 backtype.storm.testing;
-
-import backtype.storm.topology.BasicOutputCollector;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.base.BaseBasicBolt;
-import backtype.storm.tuple.Fields;
-import backtype.storm.tuple.Tuple;
-import backtype.storm.tuple.Values;
-
-import java.util.HashMap;
-import java.util.Map;
-
-public class WordBytesCount extends WordCount {
-  @Override
-  protected String getWordFromTuple(Tuple tuple, int idx) {
-    return new String(tuple.getBinary(idx));
-  }
-}


[7/7] storm git commit: add STORM-139 to CHANGELOG.md

Posted by ka...@apache.org.
add STORM-139 to CHANGELOG.md


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

Branch: refs/heads/master
Commit: 9b2fd72e90b962175e8610cf072f1aeb8164c527
Parents: 7a75884
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Thu Jul 23 08:39:24 2015 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Thu Jul 23 08:39:24 2015 +0900

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


http://git-wip-us.apache.org/repos/asf/storm/blob/9b2fd72e/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index e188165..6da5d07 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -20,6 +20,7 @@
  * STORM-942: Add FluxParser method parseInputStream() to eliminate disk usage
  * STORM-67: Provide API for spouts to know how many pending messages there are
  * STORM-918: Storm CLI could validate arguments/print usage
+ * STORM-139: hashCode does not work for byte[]
 
 ## 0.10.0-beta2
  * STORM-843: [storm-redis] Add Javadoc to storm-redis


[5/7] storm git commit: fix indent

Posted by ka...@apache.org.
fix indent


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

Branch: refs/heads/master
Commit: e4cd21a6628c376da411b5cc4076226789e75baa
Parents: 639f6a7
Author: Derek Dagit <de...@yahoo-inc.com>
Authored: Wed Jul 22 12:17:59 2015 -0500
Committer: Derek Dagit <de...@yahoo-inc.com>
Committed: Wed Jul 22 12:17:59 2015 -0500

----------------------------------------------------------------------
 .../test/clj/backtype/storm/grouping_test.clj   | 34 ++++++++++----------
 1 file changed, 17 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/e4cd21a6/storm-core/test/clj/backtype/storm/grouping_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/grouping_test.clj b/storm-core/test/clj/backtype/storm/grouping_test.clj
index 9402842..2bfa066 100644
--- a/storm-core/test/clj/backtype/storm/grouping_test.clj
+++ b/storm-core/test/clj/backtype/storm/grouping_test.clj
@@ -21,23 +21,23 @@
   (:use [backtype.storm.daemon common])
   (:require [backtype.storm [thrift :as thrift]]))
 
- (deftest test-shuffle
-   (with-simulated-time-local-cluster [cluster :supervisors 4]
-     (let [topology (thrift/mk-topology
-                     {"1" (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 4)}
-                     {"2" (thrift/mk-bolt-spec {"1" :shuffle} (TestGlobalCount.)
-                                             :parallelism-hint 6)
-                      })
-           results (complete-topology cluster
-                                      topology
-                                      ;; important for test that
-                                      ;; #tuples = multiple of 4 and 6
-                                      :mock-sources {"1" (->> [["a"] ["b"]]
-                                                              (repeat 12)
-                                                              (apply concat))})]
-       (is (ms= (apply concat (repeat 6 [[1] [2] [3] [4]]))
-                (read-tuples results "2")))
-       )))
+(deftest test-shuffle
+  (with-simulated-time-local-cluster [cluster :supervisors 4]
+    (let [topology (thrift/mk-topology
+                    {"1" (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 4)}
+                    {"2" (thrift/mk-bolt-spec {"1" :shuffle} (TestGlobalCount.)
+                                            :parallelism-hint 6)
+                     })
+          results (complete-topology cluster
+                                     topology
+                                     ;; important for test that
+                                     ;; #tuples = multiple of 4 and 6
+                                     :mock-sources {"1" (->> [["a"] ["b"]]
+                                                             (repeat 12)
+                                                             (apply concat))})]
+      (is (ms= (apply concat (repeat 6 [[1] [2] [3] [4]]))
+               (read-tuples results "2")))
+      )))
 
 (deftest test-field
   (with-simulated-time-local-cluster [cluster :supervisors 4]


[3/7] storm git commit: remove unneeded type hint

Posted by ka...@apache.org.
remove unneeded type hint


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

Branch: refs/heads/master
Commit: 70cdda253e2f744a4f106400da5e4646b2230b1b
Parents: 1b581ef
Author: Derek Dagit <de...@yahoo-inc.com>
Authored: Wed Jul 22 12:17:14 2015 -0500
Committer: Derek Dagit <de...@yahoo-inc.com>
Committed: Wed Jul 22 12:17:14 2015 -0500

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/tuple.clj | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/70cdda25/storm-core/src/clj/backtype/storm/tuple.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/tuple.clj b/storm-core/src/clj/backtype/storm/tuple.clj
index 790a823..f45d249 100644
--- a/storm-core/src/clj/backtype/storm/tuple.clj
+++ b/storm-core/src/clj/backtype/storm/tuple.clj
@@ -21,5 +21,4 @@
   [^List alist]
   (if (nil? alist)
     1
-    (let [^"[Ljava.lang.Object;" array (.toArray alist)] ;; Object[]
-      (Arrays/deepHashCode array))))
+    (Arrays/deepHashCode (.toArray alist))))


[6/7] storm git commit: Merge branch 'storm-138-byte-array-hashcode' of https://github.com/d2r/storm

Posted by ka...@apache.org.
Merge branch 'storm-138-byte-array-hashcode' of https://github.com/d2r/storm


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

Branch: refs/heads/master
Commit: 7a758840d525311435549111d658499a84405090
Parents: 6a21b6a e4cd21a
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Thu Jul 23 07:33:54 2015 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Thu Jul 23 07:33:54 2015 +0900

----------------------------------------------------------------------
 .../src/clj/backtype/storm/daemon/task.clj      |  1 -
 storm-core/src/clj/backtype/storm/tuple.clj     |  6 ++-
 .../storm/grouping/PartialKeyGrouping.java      | 27 +++++++++++-
 .../storm/testing/TestWordBytesCounter.java     | 27 ++++++++++++
 .../backtype/storm/testing/TestWordCounter.java |  6 ++-
 .../test/clj/backtype/storm/grouping_test.clj   | 43 +++++++++++++-------
 6 files changed, 90 insertions(+), 20 deletions(-)
----------------------------------------------------------------------