> On Nov. 9, 2012, 1:59 a.m., Alessandro Presta wrote:
> > http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/BinaryCombiner.java,
> >  line 47
> > <https://reviews.apache.org/r/7975/diff/1/?file=187405#file187405line47>
> >
> >     Do we have evidence that this in-place version is much better than the 
> > obvious M combine(M a, M b)? I understand the idea (minimizing object 
> > instantiation and garbage-collection), just want to make sure we make 
> > informed choices when going with a less-intuitive interface.
> >     
> >     I'm also suggesting that the vertex index is not needed. I see the 
> > parallel with the key in a MapReduce Reducer, but I'm not sure how it would 
> > help here to know just the vertex id but not the value.
> 
> Maja Kabiljo wrote:
>     I'll run some experiments, there are two steps which we are skipping by 
> having this interface: object creation, and putting object back to the map.
>     
>     Went with vertexId because it was in the original version. Does anyone 
> have a reason why we might need it there?

I tried one benchmark from above:
  6 compute threads, superstep time 28s->15s
and with M combine(M, M) interface it took 21s per superstep.


> On Nov. 9, 2012, 1:59 a.m., Alessandro Presta wrote:
> > http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/BinaryCombiner.java,
> >  line 56
> > <https://reviews.apache.org/r/7975/diff/1/?file=187405#file187405line56>
> >
> >     We could get rid of this method by using the following logic when 
> > processing messages:
> >     - if the current message is null, just set it to the incoming one
> >     - otherwise, combine them
> 
> Maja Kabiljo wrote:
>     The issue here is that we can't always modify the message which message 
> store receives, because of local requests and sendMessageToAllEdges which 
> keeps just one copy of message.
> 
> Alessandro Presta wrote:
>     Got it. We could make a copy of the first message then.

Do we have a way to do this? The only thing which I can think of is to 
serialize the message, and then create a new one and deserialize, but that's 
silly :-)


- Maja


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/7975/#review13268
-----------------------------------------------------------


On Nov. 9, 2012, 10:34 p.m., Maja Kabiljo wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/7975/
> -----------------------------------------------------------
> 
> (Updated Nov. 9, 2012, 10:34 p.m.)
> 
> 
> Review request for giraph.
> 
> 
> Description
> -------
> 
> Current combiner interface is very general, but also doesn't provide the best 
> performance. All the combiners we currently have are binary combiners, i.e. 
> they can combine two messages into one. Having a lists around this simple 
> concept makes it slower and requires more object creations.
> Adding BinaryCombiner, and a specialized message store which will be used 
> with it. This message store has only one message per vertex instead of having 
> a collection per vertex.
> 
> 
> This addresses bug GIRAPH-414.
>     https://issues.apache.org/jira/browse/GIRAPH-414
> 
> 
> Diffs
> -----
> 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/GiraphConfiguration.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/GiraphRunner.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/ImmutableClassesGiraphConfiguration.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/comm/messages/CollectionOfMessagesPerVertexStore.java
>  PRE-CREATION 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/comm/messages/DiskBackedMessageStore.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java
>  PRE-CREATION 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/comm/messages/SimpleMessageStore.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/examples/DoubleSumBinaryCombiner.java
>  PRE-CREATION 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/examples/DoubleSumCombiner.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/examples/MinimumDoubleCombiner.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/examples/MinimumIntCombiner.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/examples/SimpleSumCombiner.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/BinaryCombiner.java
>  PRE-CREATION 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/BspUtils.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/Combiner.java
>  PRE-CREATION 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/GiraphTypeValidator.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/VertexCombiner.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/test/java/org/apache/giraph/TestVertexTypes.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/test/java/org/apache/giraph/comm/ConnectionTest.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/test/java/org/apache/giraph/comm/RequestFailureTest.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/test/java/org/apache/giraph/comm/RequestTest.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/test/java/org/apache/giraph/comm/SaslConnectionTest.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/test/java/org/apache/giraph/comm/TestMessageStores.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/test/java/org/apache/giraph/examples/MinimumIntCombinerTest.java
>  1406748 
>   
> http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/test/java/org/apache/giraph/utils/MockUtils.java
>  1406748 
> 
> Diff: https://reviews.apache.org/r/7975/diff/
> 
> 
> Testing
> -------
> 
> mvn verify
> 
> PageRankBenchmark
> 20m vertices, 100 edges per vertex, 20 workers
> 1 compute thread, superstep time 55s->45s
> 6 compute threads, superstep time 28s->15s
> 12 compute threads, 1 netty server thread, superstep time 185s->112s
> 
> Our internal application
> Similar speedup as Page Rank
> 
> 
> Thanks,
> 
> Maja Kabiljo
> 
>

Reply via email to