----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/7975/#review13375 -----------------------------------------------------------
This is almost ready to go. Looks great, although I will need to get rid of all your collection methods in the future (don't worry about it now) to support byte [] based messaging. http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java <https://reviews.apache.org/r/7975/#comment28657> Given the way we are using this, should we rename getNeutralMessage() to createInitialMessage()? http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java <https://reviews.apache.org/r/7975/#comment28664> Perhaps a bit more info would be nice "Using OneMessagePervertexStore since combiner enabled" http://svn.apache.org/repos/asf/giraph/trunk/giraph/src/main/java/org/apache/giraph/graph/Combiner.java <https://reviews.apache.org/r/7975/#comment28651> I should have mentioned this earlier but getNeutralMessage() sounds a bit strange. - Avery Ching On Nov. 12, 2012, 6:39 p.m., Maja Kabiljo wrote: > > ----------------------------------------------------------- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache.org/r/7975/ > ----------------------------------------------------------- > > (Updated Nov. 12, 2012, 6:39 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/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/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 > >