THis is interesting. We were mitigating it by avoid power of 2. Thanks Florian.
On Tue, Oct 6, 2015 at 5:12 AM, Florian Hussonnois <[email protected]> wrote: > Hi Kashyap, > > You could improve your tuples distribution by implementing a > CustomStreamGrouping. > I have tried yours example with Murmur3 algorithm and the result looks > better. > > Arrays.deepHashCode : [-35, -35, -35, -3, -3, -3, -3, 29, 29, 29, 29, 41, > 51, 61, 61, 61, 61, 61] > Murmur3 : [-61, -58, -57, -48, -37, -31, -15, -7, -4, 3, 6, 12, 20, 27, > 45, 49, 56, 57] > > You can find my implementation here : > https://github.com/fhussonnois/storm-cassandra/blob/master/src/main/java/com/github/fhuss/storm/cassandra/Murmur3StreamGrouping.java > > Hope this help. > > 2015-10-01 11:08 GMT+02:00 Matthias J. Sax <[email protected]>: > >> The hash code will only be computed on the fields specified as grouping >> attributes >> >> Thus, Values(str2,str3) will be used. >> >> The code is basically, Tuple.selectFields(groupingFiels).hashValue() >> >> -Matthias >> >> On 09/30/2015 04:05 PM, Kashyap Mhaisekar wrote: >> > Thanks Matthias. My question was this - >> > If am emitting out str1,str2,str3 but field grouped on str2,str3 only >> > then will the hash be on Values(str1,str2,str3) or on Values(str2,str3) >> > alone?. In my case str1,str2 are changing but I see the values go to >> > same bolt instance. Can we debug what is the hash generated? >> > >> > Thanks you! >> > >> > Kashyap >> > >> > On Sep 30, 2015 5:14 AM, "Matthias J. Sax" <[email protected] >> > <mailto:[email protected]>> wrote: >> > >> > Yes. That's right. >> > >> > "Values" extends ArrayList and does not overwrite .hashCode(). >> > >> > -Matthias >> > >> > On 09/30/2015 11:21 AM, Kashyap Mhaisekar wrote: >> > > Is the computation right for hash? >> ArrayList(str1,str2...).hashcode() >> > > where str1,str2 etc are fields being grouped? >> > > >> > > Thanks >> > > Kashyap >> > > >> > > On Sep 29, 2015 18:04, "Kashyap Mhaisekar" <[email protected] >> > <mailto:[email protected]> >> > > <mailto:[email protected] <mailto:[email protected]>>> wrote: >> > > >> > > Thanks guys. From what I understand, partial key grouping is >> used >> > > when you know your grouping will create imbalance. In my case, >> > most >> > > of my field groups to one bolt thereby causing it to be a >> > > bottleneck. Since I emit string, I guess the hash is on >> > > ArrayList(str1,str2...).hashcode(). This hashcode is coming >> > out same >> > > for different string combinations... >> > > >> > > Thanks >> > > Kashyap >> > > >> > > On Sep 29, 2015 17:51, "Matthias J. Sax" <[email protected] >> > <mailto:[email protected]> >> > > <mailto:[email protected] <mailto:[email protected]>>> wrote: >> > > >> > > If you can use "partial key grouping" depends on your use >> > case. >> > > Think >> > > careful before you apply it... >> > > >> > > Maybe you want to read the research paper about it. It >> clearly >> > > describes >> > > when you can use it and when not: >> > > >> > >> https://melmeric.files.wordpress.com/2014/11/the-power-of-both-choices-practical-load-balancing-for-distributed-stream-processing-engines.pdf >> > > >> > > >> > > -Matthias >> > > >> > > On 09/30/2015 12:18 AM, Ken Danniswara wrote: >> > > > Hi, >> > > > >> > > > From what I read, the default FieldGrouping did not >> balance >> > > the load as >> > > > like ShuffleGrouping do. In this case, there is a >> > discussion about >> > > > custom Grouping implementation called partial key >> grouping >> > > where it have >> > > > better balancing problem. Maybe it >> > > > helps. https://github.com/gdfm/partial-key-grouping >> > > > >> > > > On Wed, Sep 30, 2015 at 12:11 AM, Kashyap Mhaisekar >> > > <[email protected] <mailto:[email protected]> >> > <mailto:[email protected] <mailto:[email protected]>> >> > > > <mailto:[email protected] <mailto:[email protected] >> > >> > <mailto:[email protected] <mailto:[email protected]>>>> wrote: >> > > > >> > > > Thanks Derek. I use strings and I still end up with >> > some bolts >> > > > having the maximum requests :( >> > > > >> > > > On Tue, Sep 29, 2015 at 5:03 PM, Derek Dagit >> > > <[email protected] <mailto:[email protected]> >> > <mailto:[email protected] <mailto:[email protected]>> >> > > > <mailto:[email protected] >> > <mailto:[email protected]> >> > > <mailto:[email protected] >> > <mailto:[email protected]>>>> wrote: >> > > > >> > > > The code that hashes the field values is here: >> > > > >> > > > >> > > >> > >> https://github.com/apache/storm/blob/9d911ec1b4f7b5aabe646a5d2cd31591fe4df1b0/storm-core/src/clj/backtype/storm/tuple.clj#L24 >> > > > >> > > > >> > > > You can write a little java program, something >> like: >> > > > >> > > > public static void main(String[] args) { >> > > > ArrayList<String> myList = new >> > ArrayList<String>(); >> > > > myList.add("first field value"); >> > > > myList.add("second field value"); >> > > > >> > > > int hash = >> > Arrays.deephashCode(myList.toArray()); // >> > > as in >> > > > tuple.clj >> > > > >> > > > >> > > > System.out.println("hash is "+hash); >> > > > int numTasks = 32; >> > > > >> > > > System.out.println("task index is " + hash % >> > numTasks); >> > > > >> > > > } >> > > > >> > > > >> > > > There are certain types of values that may not >> hash >> > > > consistently. If you are using String values, >> > then it >> > > should be >> > > > fine. Other types may or may not, depending on >> > how the >> > > class >> > > > implements hashCode(). >> > > > >> > > > >> > > > -- >> > > > Derek >> > > > >> > > > >> > > > ________________________________ >> > > > From: Kashyap Mhaisekar <[email protected] >> > <mailto:[email protected]> >> > > <mailto:[email protected] <mailto:[email protected]>> >> > > > <mailto:[email protected] >> > <mailto:[email protected]> <mailto:[email protected] >> > <mailto:[email protected]>>>> >> > > > To: [email protected] >> > <mailto:[email protected]> >> > > <mailto:[email protected] >> > <mailto:[email protected]>> <mailto:[email protected] >> > <mailto:[email protected]> >> > > <mailto:[email protected] <mailto: >> [email protected]>>> >> > > > Sent: Tuesday, September 29, 2015 4:28 PM >> > > > Subject: Field Group Hash Computation >> > > > >> > > > >> > > > >> > > > Hi, >> > > > I have a field grouping based on 2 fields. I >> have 32 >> > > consumers >> > > > for the tuple and I see most of the times, out >> of 64 >> > > bolts, the >> > > > field group is always on 8 of them. Of the 8, 2 >> have >> > > more than >> > > > 60% of the data. The data for the field >> grouping can >> > > have 20 >> > > > different combinations. >> > > > >> > > > Do you know what is the way to compute the Hash >> > of the >> > > fields >> > > > used for computing? One of the groups mails >> indicate >> > > that the >> > > > approach is - >> > > > >> > > > It calls "hashCode" on the list of selected >> > values and >> > > mods it >> > > > by the >> > > > number of consumer tasks. You can play around >> with >> > > that function >> > > > to see if >> > > > something about your data is causing something >> > > degenerative to >> > > > happen and >> > > > cause skew >> > > > >> > > > I saw the clojure code but not sure how to >> > understand >> > > this. >> > > > >> > > > Thanks >> > > > Kashyap >> > > > >> > > > >> > > > >> > > >> > >> >> > > > -- > Florian HUSSONNOIS >
