You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@giraph.apache.org by Paolo Castagna <ca...@googlemail.com> on 2012/04/10 21:17:02 UTC

A simple use case: shortest paths on a FOAF (i.e. Friend of a Friend) graph

Hi,
I am still learning Giraph, so, please, be patient with me and forgive my
trivial questions.

As a simple initial use case, I want to compute the shortest paths from a single
source in a social graph in RDF format using the FOAF [1] vocabulary.
This example also will hopefully inform GIRAPH-170 [2] and related issues, such
as: GIRAPH-141 [3].

Here is an example in Turtle [4] format of a tiny graph using FOAF:
----
@prefix : <http://example.org/> .
@prefix foaf:   <http://xmlns.com/foaf/0.1/> .

:alice
    a           foaf:Person ;
    foaf:name   "Alice" ;
    foaf:mbox   <ma...@example.org> ;
    foaf:knows  :bob ;
    foaf:knows  :charlie ;
    foaf:knows  :snoopy ;
    .

:bob
    foaf:name   "Bob" ;
    foaf:knows  :charlie ;
    .

:charlie
    foaf:name   "Charlie" ;
    foaf:knows  :alice ;
    .
----
This is nice, human friendly (RDF without angle brackets!), but not easily
splittable to be processed with MapReduce (or Giraph).

Here is the same graph in N-Triples [5] format:
----
<http://example.org/alice> <http://www.w3.org/1999/02/22-rdf-syntax-ns#type>
<http://xmlns.com/foaf/0.1/Person> .
<http://example.org/alice> <http://xmlns.com/foaf/0.1/name> "Alice" .
<http://example.org/alice> <http://xmlns.com/foaf/0.1/mbox>
<ma...@example.org> .
<http://example.org/alice> <http://xmlns.com/foaf/0.1/knows>
<http://example.org/bob> .
<http://example.org/alice> <http://xmlns.com/foaf/0.1/knows>
<http://example.org/charlie> .
<http://example.org/alice> <http://xmlns.com/foaf/0.1/knows>
<http://example.org/snoopy> .
<http://example.org/charlie> <http://xmlns.com/foaf/0.1/name> "Charlie" .
<http://example.org/charlie> <http://xmlns.com/foaf/0.1/knows>
<http://example.org/alice> .
<http://example.org/bob> <http://xmlns.com/foaf/0.1/name> "Bob" .
<http://example.org/bob> <http://xmlns.com/foaf/0.1/knows>
<http://example.org/charlie> .
----
This is more verbose and ugly, but splittable.

The graph I am interested in is the graph represented by foaf:knows
relationships/links between people (please, note --knows--> relationship here
has a direction, this isn't symmetric as in centralized social networking
websites such as Facebook or LinkedIn. Alice can claim to know Bob, without Bob
knowing it and/or it might even be a false claim):

alice --knows--> bob
alice --knows--> charlie
alice --knows--> snoopy
bob --knows--> charlie
charlie --knows--> alice

As a first step, I wrote a MapReduce job [6] to transform the RDF graph above in
a sort of adjacency list using Turtle syntax, here is the output (three lines):
----
<http://example.org/alice> <http://xmlns.com/foaf/0.1/mbox>
<ma...@example.org>; <http://xmlns.com/foaf/0.1/name> "Alice";
<http://www.w3.org/1999/02/22-rdf-syntax-ns#type>
<http://xmlns.com/foaf/0.1/Person>; <http://xmlns.com/foaf/0.1/knows>
<http://example.org/charlie>, <http://example.org/bob>,
<http://example.org/snoopy>; . <http://example.org/charlie>
<http://xmlns.com/foaf/0.1/knows> <http://example.org/alice>.

<http://example.org/bob> <http://xmlns.com/foaf/0.1/name> "Bob";
<http://xmlns.com/foaf/0.1/knows> <http://example.org/charlie>; .
<http://example.org/alice> <http://xmlns.com/foaf/0.1/knows>
<http://example.org/bob>.

<http://example.org/charlie> <http://xmlns.com/foaf/0.1/name> "Charlie";
<http://xmlns.com/foaf/0.1/knows> <http://example.org/alice>; .
<http://example.org/bob> <http://xmlns.com/foaf/0.1/knows>
<http://example.org/charlie>. <http://example.org/alice>
<http://xmlns.com/foaf/0.1/knows> <http://example.org/charlie>.
----
This is legal Turtle, but it is also splittable. Each line has all the RDF
statements (i.e. egdes) for a person (there are also incoming edges).

I wrote a TurtleVertexReader [7] which extends TextVertexReader<NodeWritable,
Text, NodeWritable, Text> and a TurtleVertexInputFormat [8] which extends
TextVertexInputFormat<NodeWritable, Text, NodeWritable, Text>.
I wrote (copying from the example SimpleShortestPathsVertex) a
FoafShortestPathsVertex [9] which extends EdgeListVertex<NodeWritable,
IntWritable, NodeWritable, IntWritable> and I am running it locally using these
arguments: -Dgiraph.maxWorkers=1 -Dgiraph.SplitMasterWorker=false
-DoverwriteOutput=true src/test/resources/data3.ttl target/foaf
http://example.org/alice 1

TurtleVertexReader, TurtleVertexInputFormat and FoafShortestPathsVertex are
still work in progress and I am sure there are plenty of stupid errors.
However, I do not understand why when I run FoafShortestPathsVertex with the
DEBUG level, I see debug statements from FoafShortestPathsVertex:
19:34:44 DEBUG FoafShortestPathsVertex   :: main({-Dgiraph.maxWorkers=1,
-Dgiraph.SplitMasterWorker=false, -DoverwriteOutput=true,
src/test/resources/data3.ttl, target/foaf, http://example.org/alice, 1})
19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() --> null
19:34:44 DEBUG FoafShortestPathsVertex   :: setConf(Configuration:
core-default.xml, core-site.xml)
19:34:44 DEBUG FoafShortestPathsVertex   :: run({src/test/resources/data3.ttl,
target/foaf, http://example.org/alice, 1})
19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() --> Configuration:
core-default.xml, core-site.xml
19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() --> Configuration:
core-default.xml, core-site.xml, mapred-default.xml, mapred-site.xml,
giraph-site.xml

But, I do not see anything else, no log statement from TurtleVertexReader or
TurtleVertexInputFormat. Why? What am I doing wrong?
Is it because I am running it locally?

Thanks,
Paolo

 [1] http://en.wikipedia.org/wiki/FOAF_%28software%29
 [2] https://issues.apache.org/jira/browse/GIRAPH-170
 [3] https://issues.apache.org/jira/browse/GIRAPH-141
 [4] http://en.wikipedia.org/wiki/Turtle_%28syntax%29
 [5] http://en.wikipedia.org/wiki/N-Triples
 [6]
https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/mapreduce/Rdf2AdjacencyListDriver.java
 [7]
https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/TurtleVertexReader.java
 [8]
https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/TurtleVertexInputFormat.java
 [9]
https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/FoafShortestPathsVertex.java

Re: A simple use case: shortest paths on a FOAF (i.e. Friend of a Friend) graph

Posted by Paolo Castagna <ca...@googlemail.com>.
Hi Avery,
first of all, apologies for my delay (I was on holiday) and many thanks for your
help. Further comments inline.

Avery Ching wrote:
> I think we should have some kind of a helper script (similar to
> bin/giraph) to running simple tests in LocalJobRunner.

That would be good, new developer might not have an Hadoop cluster at hand or
may want to debug the stuff they write and test it on their laptop before run it
on a real cluster.

> One thing to remember is that if you rerun it, you'll have to remove the
> _bsp directories that are created, otherwise it will think it has
> already been completed.

This are the program arguments I used to run PageRankBenchmark locally directly
from Eclipse:

  -libjars target/giraph-0.2-SNAPSHOT-jar-with-dependencies.jar
-Dgiraph.SplitMasterWorker=false -Dlocal.test.mode=true  -c 1 -e 2 -s 2 -V 10 -w 1

As you suggested, each time I need to delete the _bsp directories (not ideal,
but necessary).

The -libjars parameter is necessary, otherwise you get a NullPointerException:

12/04/23 15:56:54 WARN mapred.LocalJobRunner: job_local_0001
java.lang.NullPointerException
	at org.apache.giraph.graph.GraphMapper.setup(GraphMapper.java:398)
	at org.apache.giraph.graph.GraphMapper.run(GraphMapper.java:646)
	at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:763)
	at org.apache.hadoop.mapred.MapTask.run(MapTask.java:369)
	at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:210)

I hope this help others to get started and run/debug their Giraph jobs with Eclipse.

Thanks again,
Paolo

Re: A simple use case: shortest paths on a FOAF (i.e. Friend of a Friend) graph

Posted by Avery Ching <ac...@apache.org>.
Hi Paulo,

Can you try something for me?  I was able to get the PageRankBenchmark 
to work running in local mode just fine on my side.

I think we should have some kind of a helper script (similar to 
bin/giraph) to running simple tests in LocalJobRunner.

I believe that for LocalJobRunner to run, we need to do 
-Dgiraph.SplitMasterWorker=false -Dlocal.test.mode=true.  In the case of 
PageRankBenchmark, I also have to set the workers to 1 (LocalJobRunner 
can only run one task at a time).

So I get the class path that bin/giraph was using to run (just added a 
echo $CLASSPATH at the end) and then inserted the 
giraph-0.2-SNAPSHOT-jar-with-dependencies.jar in front of it (this is 
necessary for the ZooKeeper jar inclusion).  Then I just ran a normal 
java command and the output below.

One thing to remember is that if you rerun it, you'll have to remove the 
_bsp directories that are created, otherwise it will think it has 
already been completed.

Hope that helps,

Avery

  java -cp 
target/giraph-0.2-SNAPSHOT-jar-with-dependencies.jar:/Users/aching/git/git_svn_giraph_trunk/conf:/Users/aching/.m2/repository/ant/ant/1.6.5/ant-1.6.5.jar:/Users/aching/.m2/repository/com/google/guava/guava/r09/guava-r09.jar:/Users/aching/.m2/repository/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/Users/aching/.m2/repository/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar:/Users/aching/.m2/repository/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/Users/aching/.m2/repository/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/Users/aching/.m2/repository/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1.jar:/Users/aching/.m2/repository/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/Users/aching/.m2/repository/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/Users/aching/.m2/repository/commons-el/commons-el/1.0/commons-el-1.0.jar:/Users/aching/.m2/repository/commons-httpclient/commons-httpclient/3.0.1/commons-httpclient-3.0.1.jar:/Users/aching/.m2/repository/commons-lang/commons-lang/2.4/commons-lang-2.4.jar:/Users/aching/.m2/repository/commons-logging/commons-logging/1.0.3/commons-logging-1.0.3.jar:/Users/aching/.m2/repository/commons-net/commons-net/1.4.1/commons-net-1.4.1.jar:/Users/aching/.m2/repository/hsqldb/hsqldb/1.8.0.10/hsqldb-1.8.0.10.jar:/Users/aching/.m2/repository/javax/activation/activation/1.1/activation-1.1.jar:/Users/aching/.m2/repository/javax/mail/mail/1.4/mail-1.4.jar:/Users/aching/.m2/repository/jline/jline/0.9.94/jline-0.9.94.jar:/Users/aching/.m2/repository/junit/junit/3.8.1/junit-3.8.1.jar:/Users/aching/.m2/repository/log4j/log4j/1.2.15/log4j-1.2.15.jar:/Users/aching/.m2/repository/net/iharder/base64/2.3.8/base64-2.3.8.jar:/Users/aching/.m2/repository/net/java/dev/jets3t/jets3t/0.7.1/jets3t-0.7.1.jar:/Users/aching/.m2/repository/net/sf/kosmosfs/kfs/0.3/kfs-0.3.jar:/Users/aching/.m2/repository/org/apache/commons/commons-io/1.3.2/commons-io-1.3.2.jar:/Users/aching/.m2/repository/org/apache/commons/commons-math/2.1/commons-math-2.1.jar:/Users/aching/.m2/repository/org/apache/hadoop/hadoop-core/0.20.203.0/hadoop-core-0.20.203.0.jar:/Users/aching/.m2/repository/org/apache/mahout/mahout-collections/1.0/mahout-collections-1.0.jar:/Users/aching/.m2/repository/org/apache/zookeeper/zookeeper/3.3.3/zookeeper-3.3.3.jar:/Users/aching/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.8.0/jackson-core-asl-1.8.0.jar:/Users/aching/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.8.0/jackson-mapper-asl-1.8.0.jar:/Users/aching/.m2/repository/org/eclipse/jdt/core/3.1.1/core-3.1.1.jar:/Users/aching/.m2/repository/org/json/json/20090211/json-20090211.jar:/Users/aching/.m2/repository/org/mockito/mockito-all/1.8.5/mockito-all-1.8.5.jar:/Users/aching/.m2/repository/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/Users/aching/.m2/repository/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/Users/aching/.m2/repository/org/mortbay/jetty/jsp-2.1/6.1.14/jsp-2.1-6.1.14.jar:/Users/aching/.m2/repository/org/mortbay/jetty/jsp-api-2.1/6.1.14/jsp-api-2.1-6.1.14.jar:/Users/aching/.m2/repository/org/mortbay/jetty/servlet-api/2.5-20081211/servlet-api-2.5-20081211.jar:/Users/aching/.m2/repository/org/mortbay/jetty/servlet-api-2.5/6.1.14/servlet-api-2.5-6.1.14.jar:/Users/aching/.m2/repository/oro/oro/2.0.8/oro-2.0.8.jar:/Users/aching/.m2/repository/tomcat/jasper-compiler/5.5.12/jasper-compiler-5.5.12.jar:/Users/aching/.m2/repository/tomcat/jasper-runtime/5.5.12/jasper-runtime-5.5.12.jar:/Users/aching/.m2/repository/xmlenc/xmlenc/0.52/xmlenc-0.52.jar 
org.apache.giraph.benchmark.PageRankBenchmark 
-Dgiraph.SplitMasterWorker=false -Dlocal.test.mode=true  -c 1 -e 2 -s 2 
-V 10 -w 1

2012-04-13 09:30:27.261 java[45785:1903] Unable to load realm mapping 
info from SCDynamicStore
12/04/13 09:30:27 INFO benchmark.PageRankBenchmark: Using class 
org.apache.giraph.benchmark.PageRankBenchmark
12/04/13 09:30:27 WARN bsp.BspOutputFormat: checkOutputSpecs: 
ImmutableOutputCommiter will not check anything
12/04/13 09:30:27 WARN bsp.BspOutputFormat: getOutputCommitter: 
Returning ImmutableOutputCommiter (does nothing).
12/04/13 09:30:27 INFO graph.GraphMapper: Distributed cache is empty. 
Assuming fatjar.
12/04/13 09:30:27 INFO graph.GraphMapper: setup: classpath @ 
/tmp/hadoop-aching/mapred/staging/aching-221077155/.staging/job_local_0001/job.jar
12/04/13 09:30:27 INFO zk.ZooKeeperManager: createCandidateStamp: Made 
the directory _bsp/_defaultZkManagerDir/job_local_0001
12/04/13 09:30:27 INFO zk.ZooKeeperManager: createCandidateStamp: 
Creating my filestamp 
_bsp/_defaultZkManagerDir/job_local_0001/_task/192.168.1.100 0
12/04/13 09:30:27 INFO zk.ZooKeeperManager: getZooKeeperServerList: Got 
[192.168.1.100] 1 hosts from 1 candidates when 1 required (polling 
period is 3000) on attempt 0
12/04/13 09:30:27 INFO zk.ZooKeeperManager: createZooKeeperServerList: 
Creating the final ZooKeeper file 
'_bsp/_defaultZkManagerDir/job_local_0001/zkServerList_192.168.1.100 0 '
12/04/13 09:30:27 INFO zk.ZooKeeperManager: getZooKeeperServerList: For 
task 0, got file 'zkServerList_192.168.1.100 0 ' (polling period is 3000)
12/04/13 09:30:27 INFO zk.ZooKeeperManager: getZooKeeperServerList: 
Found [192.168.1.100, 0] 2 hosts in filename 'zkServerList_192.168.1.100 0 '
12/04/13 09:30:27 INFO zk.ZooKeeperManager: onlineZooKeeperServers: 
Trying to delete old directory 
/Users/aching/git/git_svn_giraph_trunk/_bspZooKeeper
12/04/13 09:30:27 INFO zk.ZooKeeperManager: generateZooKeeperConfigFile: 
Creating file 
/Users/aching/git/git_svn_giraph_trunk/_bspZooKeeper/zoo.cfg in 
/Users/aching/git/git_svn_giraph_trunk/_bspZooKeeper with base port 22181
12/04/13 09:30:27 INFO zk.ZooKeeperManager: generateZooKeeperConfigFile: 
Make directory of _bspZooKeeper = true
12/04/13 09:30:27 INFO zk.ZooKeeperManager: generateZooKeeperConfigFile: 
Delete of zoo.cfg = false
12/04/13 09:30:27 INFO zk.ZooKeeperManager: onlineZooKeeperServers: 
Attempting to start ZooKeeper server with command 
[/System/Library/Java/JavaVirtualMachines/1.6.0.jdk/Contents/Home/bin/java, 
-Xmx512m, -XX:ParallelGCThreads=4, -XX:+UseConcMarkSweepGC, 
-XX:CMSInitiatingOccupancyFraction=70, -XX:MaxGCPauseMillis=100, -cp, 
/tmp/hadoop-aching/mapred/staging/aching-221077155/.staging/job_local_0001/job.jar, 
org.apache.zookeeper.server.quorum.QuorumPeerMain, 
/Users/aching/git/git_svn_giraph_trunk/_bspZooKeeper/zoo.cfg] in 
directory /Users/aching/git/git_svn_giraph_trunk/_bspZooKeeper
12/04/13 09:30:27 INFO zk.ZooKeeperManager: onlineZooKeeperServers: 
Connect attempt 0 of 10 max trying to connect to 192.168.1.100:22181 
with poll msecs = 3000
12/04/13 09:30:27 WARN zk.ZooKeeperManager: onlineZooKeeperServers: Got 
ConnectException
java.net.ConnectException: Connection refused
     at java.net.PlainSocketImpl.socketConnect(Native Method)
     at java.net.PlainSocketImpl.doConnect(PlainSocketImpl.java:351)
     at java.net.PlainSocketImpl.connectToAddress(PlainSocketImpl.java:213)
     at java.net.PlainSocketImpl.connect(PlainSocketImpl.java:200)
     at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:432)
     at java.net.Socket.connect(Socket.java:529)
     at 
org.apache.giraph.zk.ZooKeeperManager.onlineZooKeeperServers(ZooKeeperManager.java:661)
     at org.apache.giraph.graph.GraphMapper.setup(GraphMapper.java:425)
     at org.apache.giraph.graph.GraphMapper.run(GraphMapper.java:646)
     at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:763)
     at org.apache.hadoop.mapred.MapTask.run(MapTask.java:369)
     at 
org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:210)
12/04/13 09:30:30 INFO zk.ZooKeeperManager: onlineZooKeeperServers: 
Connect attempt 1 of 10 max trying to connect to 192.168.1.100:22181 
with poll msecs = 3000
12/04/13 09:30:30 INFO zk.ZooKeeperManager: onlineZooKeeperServers: 
Connected to /192.168.1.100:22181!
12/04/13 09:30:30 INFO zk.ZooKeeperManager: onlineZooKeeperServers: 
Creating my filestamp 
_bsp/_defaultZkManagerDir/job_local_0001/_zkServer/192.168.1.100 0
12/04/13 09:30:30 INFO graph.GraphMapper: setup: Starting up 
BspServiceMaster (master thread)...
12/04/13 09:30:30 INFO graph.BspService: BspService: Connecting to 
ZooKeeper with job job_local_0001, 0 on 192.168.1.100:22181
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Client 
environment:zookeeper.version=3.3.3-1073969, built on 02/23/2011 22:27 GMT
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Client 
environment:host.name=192.168.1.100
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Client 
environment:java.version=1.6.0_31
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Client 
environment:java.vendor=Apple Inc.
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Client 
environment:java.home=/System/Library/Java/JavaVirtualMachines/1.6.0.jdk/Contents/Home
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Client 
environment:java.class.path=target/giraph-0.2-SNAPSHOT-jar-with-dependencies.jar:/Users/aching/git/git_svn_giraph_trunk/conf:/Users/aching/.m2/repository/ant/ant/1.6.5/ant-1.6.5.jar:/Users/aching/.m2/repository/com/google/guava/guava/r09/guava-r09.jar:/Users/aching/.m2/repository/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/Users/aching/.m2/repository/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar:/Users/aching/.m2/repository/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/Users/aching/.m2/repository/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/Users/aching/.m2/repository/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1.jar:/Users/aching/.m2/repository/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/Users/aching/.m2/repository/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/Users/aching/.m2/repository/commons-el/commons-el/1.0/commons-el-1.0.jar:/Users/aching/.m2/repository/commons-httpclient/commons-httpclient/3.0.1/commons-httpclient-3.0.1.jar:/Users/aching/.m2/repository/commons-lang/commons-lang/2.4/commons-lang-2.4.jar:/Users/aching/.m2/repository/commons-logging/commons-logging/1.0.3/commons-logging-1.0.3.jar:/Users/aching/.m2/repository/commons-net/commons-net/1.4.1/commons-net-1.4.1.jar:/Users/aching/.m2/repository/hsqldb/hsqldb/1.8.0.10/hsqldb-1.8.0.10.jar:/Users/aching/.m2/repository/javax/activation/activation/1.1/activation-1.1.jar:/Users/aching/.m2/repository/javax/mail/mail/1.4/mail-1.4.jar:/Users/aching/.m2/repository/jline/jline/0.9.94/jline-0.9.94.jar:/Users/aching/.m2/repository/junit/junit/3.8.1/junit-3.8.1.jar:/Users/aching/.m2/repository/log4j/log4j/1.2.15/log4j-1.2.15.jar:/Users/aching/.m2/repository/net/iharder/base64/2.3.8/base64-2.3.8.jar:/Users/aching/.m2/repository/net/java/dev/jets3t/jets3t/0.7.1/jets3t-0.7.1.jar:/Users/aching/.m2/repository/net/sf/kosmosfs/kfs/0.3/kfs-0.3.jar:/Users/aching/.m2/repository/org/apache/commons/commons-io/1.3.2/commons-io-1.3.2.jar:/Users/aching/.m2/repository/org/apache/commons/commons-math/2.1/commons-math-2.1.jar:/Users/aching/.m2/repository/org/apache/hadoop/hadoop-core/0.20.203.0/hadoop-core-0.20.203.0.jar:/Users/aching/.m2/repository/org/apache/mahout/mahout-collections/1.0/mahout-collections-1.0.jar:/Users/aching/.m2/repository/org/apache/zookeeper/zookeeper/3.3.3/zookeeper-3.3.3.jar:/Users/aching/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.8.0/jackson-core-asl-1.8.0.jar:/Users/aching/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.8.0/jackson-mapper-asl-1.8.0.jar:/Users/aching/.m2/repository/org/eclipse/jdt/core/3.1.1/core-3.1.1.jar:/Users/aching/.m2/repository/org/json/json/20090211/json-20090211.jar:/Users/aching/.m2/repository/org/mockito/mockito-all/1.8.5/mockito-all-1.8.5.jar:/Users/aching/.m2/repository/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/Users/aching/.m2/repository/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/Users/aching/.m2/repository/org/mortbay/jetty/jsp-2.1/6.1.14/jsp-2.1-6.1.14.jar:/Users/aching/.m2/repository/org/mortbay/jetty/jsp-api-2.1/6.1.14/jsp-api-2.1-6.1.14.jar:/Users/aching/.m2/repository/org/mortbay/jetty/servlet-api/2.5-20081211/servlet-api-2.5-20081211.jar:/Users/aching/.m2/repository/org/mortbay/jetty/servlet-api-2.5/6.1.14/servlet-api-2.5-6.1.14.jar:/Users/aching/.m2/repository/oro/oro/2.0.8/oro-2.0.8.jar:/Users/aching/.m2/repository/tomcat/jasper-compiler/5.5.12/jasper-compiler-5.5.12.jar:/Users/aching/.m2/repository/tomcat/jasper-runtime/5.5.12/jasper-runtime-5.5.12.jar:/Users/aching/.m2/repository/xmlenc/xmlenc/0.52/xmlenc-0.52.jar
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Client 
environment:java.library.path=.:/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Client 
environment:java.io.tmpdir=/var/folders/ks/2sm0ygc947x00lmprdkbkhqhb4b6h3/T/
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Client 
environment:java.compiler=<NA>
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Client 
environment:os.name=Mac OS X
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Client 
environment:os.arch=x86_64
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Client 
environment:os.version=10.7.3
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Client 
environment:user.name=aching
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Client 
environment:user.home=/Users/aching
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Client 
environment:user.dir=/Users/aching/git/git_svn_giraph_trunk
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Initiating client 
connection, connectString=192.168.1.100:22181 sessionTimeout=60000 
watcher=org.apache.giraph.graph.BspServiceMaster@5e2b323e
12/04/13 09:30:30 INFO zookeeper.ClientCnxn: Opening socket connection 
to server /192.168.1.100:22181
12/04/13 09:30:30 INFO zookeeper.ClientCnxn: Socket connection 
established to 192.168.1.100/192.168.1.100:22181, initiating session
12/04/13 09:30:30 INFO zookeeper.ClientCnxn: Session establishment 
complete on server 192.168.1.100/192.168.1.100:22181, sessionid = 
0x136ac8aa05e0000, negotiated timeout = 300000
12/04/13 09:30:30 INFO graph.BspService: process: Asynchronous 
connection complete.
12/04/13 09:30:30 INFO graph.GraphMapper: setup: Starting up 
BspServiceWorker...
12/04/13 09:30:30 INFO graph.BspService: BspService: Connecting to 
ZooKeeper with job job_local_0001, 0 on 192.168.1.100:22181
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Initiating client 
connection, connectString=192.168.1.100:22181 sessionTimeout=60000 
watcher=org.apache.giraph.graph.BspServiceWorker@2c31f2a7
12/04/13 09:30:30 INFO zookeeper.ClientCnxn: Opening socket connection 
to server /192.168.1.100:22181
12/04/13 09:30:30 INFO zookeeper.ClientCnxn: Socket connection 
established to 192.168.1.100/192.168.1.100:22181, initiating session
12/04/13 09:30:30 INFO zookeeper.ClientCnxn: Session establishment 
complete on server 192.168.1.100/192.168.1.100:22181, sessionid = 
0x136ac8aa05e0001, negotiated timeout = 300000
12/04/13 09:30:30 INFO graph.BspService: process: Asynchronous 
connection complete.
12/04/13 09:30:30 INFO ipc.Server: Starting SocketReader
12/04/13 09:30:30 INFO ipc.Server: IPC Server Responder: starting
12/04/13 09:30:30 INFO ipc.Server: IPC Server listener on 30000: starting
12/04/13 09:30:30 INFO comm.BasicRPCCommunications: 
BasicRPCCommunications: Started RPC communication server: 
achingmbp15.local/192.168.1.100:30000 with 1 handlers and 1 flush 
threads on bind attempt 0
12/04/13 09:30:30 INFO ipc.Server: IPC Server handler 0 on 30000: starting
12/04/13 09:30:30 INFO graph.GraphMapper: setup: Registering health of 
this worker...
12/04/13 09:30:30 INFO graph.BspService: getJobState: Job state already 
exists (/_hadoopBsp/job_local_0001/_masterJobState)
12/04/13 09:30:30 INFO graph.BspServiceMaster: becomeMaster: First child 
is 
'/_hadoopBsp/job_local_0001/_masterElectionDir/achingmbp15.local_00000000000' 
and my bid is 
'/_hadoopBsp/job_local_0001/_masterElectionDir/achingmbp15.local_00000000000'
12/04/13 09:30:30 INFO graph.BspService: getApplicationAttempt: Node 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir already exists!
12/04/13 09:30:30 INFO graph.BspServiceMaster: becomeMaster: I am now 
the master!
12/04/13 09:30:30 INFO graph.BspService: process: 
applicationAttemptChanged signaled
12/04/13 09:30:30 INFO graph.BspService: process: 
applicationAttemptChanged signaled
12/04/13 09:30:30 INFO graph.BspService: getApplicationAttempt: Node 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir already exists!
12/04/13 09:30:30 WARN graph.BspService: process: Unknown and 
unprocessed event 
(path=/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir, type=NodeChildrenChanged, 
state=SyncConnected)
12/04/13 09:30:30 WARN graph.BspService: process: Unknown and 
unprocessed event 
(path=/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir, type=NodeChildrenChanged, 
state=SyncConnected)
12/04/13 09:30:30 INFO graph.BspServiceWorker: registerHealth: Created 
my health node for attempt=0, superstep=-1 with 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/-1/_workerHealthyDir/achingmbp15.local_0 
and workerInfo= Worker(hostname=achingmbp15.local, MRpartition=0, 
port=30000)
12/04/13 09:30:30 INFO graph.BspServiceMaster: generateInputSplits: Got 
1 input splits for 1 workers
12/04/13 09:30:30 INFO partition.HashMasterPartitioner: 
createInitialPartitionOwners: Creating 1, default would have been 1 
partitions.
12/04/13 09:30:30 INFO graph.BspService: process: 
partitionAssignmentsReadyChanged (partitions are assigned)
12/04/13 09:30:30 INFO graph.BspServiceWorker: startSuperstep: Ready for 
computation on superstep -1 since worker selection and vertex range 
assignments are done in 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/-1/_partitionAssignments
12/04/13 09:30:30 INFO graph.BspServiceWorker: reserveInputSplit: 
Reserved input split path /_hadoopBsp/job_local_0001/_inputSplitDir/0, 
overall roughly 0.0% input splits finished
12/04/13 09:30:30 INFO graph.BspServiceWorker: getInputSplitForVertices: 
Reserved /_hadoopBsp/job_local_0001/_inputSplitDir/0 from ZooKeeper and 
got input split ''org.apache.giraph.bsp.BspInputSplit, index=0, num=1'
12/04/13 09:30:30 INFO graph.BspServiceWorker: 
loadVerticesFromInputSplit: Finished loading 
/_hadoopBsp/job_local_0001/_inputSplitDir/0 (v=10, e=20)
12/04/13 09:30:30 INFO graph.BspServiceWorker: reserveInputSplit: 
reservedPath = null, 1 of 1 InputSplits are finished.
12/04/13 09:30:30 INFO graph.BspServiceWorker: setup: Finally loaded a 
total of (v=10, e=20)
12/04/13 09:30:30 INFO graph.BspServiceMaster: barrierOnWorkerList: 1 
out of 1 workers finished on superstep -1 on path 
/_hadoopBsp/job_local_0001/_inputSplitDoneDir
12/04/13 09:30:30 INFO graph.BspService: process: 
inputSplitsAllDoneChanged (all vertices sent from input splits)
12/04/13 09:30:30 INFO graph.BspServiceWorker: moveVerticesToWorker: 
Adding 10 vertices for partition id 0
12/04/13 09:30:30 INFO comm.BasicRPCCommunications: flush: starting for 
superstep -1 totalMem = 81.0625M, maxMem = 123.9375M, freeMem = 73.0065M
12/04/13 09:30:30 INFO graph.BspServiceMaster: barrierOnWorkerList: 0 
out of 1 workers finished on superstep -1 on path 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/-1/_workerFinishedDir
12/04/13 09:30:30 INFO comm.BasicRPCCommunications: flush: ended for 
superstep -1 totalMem = 81.0625M, maxMem = 123.9375M, freeMem = 73.00642M
12/04/13 09:30:30 INFO graph.BspServiceWorker: finishSuperstep: 
Superstep -1 totalMem = 81.0625M, maxMem = 123.9375M, freeMem = 73.00642M
12/04/13 09:30:30 INFO graph.BspServiceMaster: aggregateWorkerStats: 
Aggregation found (vtx=10,finVtx=0,edges=20,msgCount=0) on superstep = -1
12/04/13 09:30:30 INFO graph.BspService: process: superstepFinished signaled
12/04/13 09:30:30 INFO graph.MasterThread: masterThread: Coordination of 
superstep -1 took 0.036 seconds ended with state THIS_SUPERSTEP_DONE and 
is now on superstep 0
12/04/13 09:30:30 INFO graph.BspServiceWorker: finishSuperstep: 
Completed superstep -1 with global stats 
(vtx=10,finVtx=0,edges=20,msgCount=0)
12/04/13 09:30:30 INFO comm.BasicRPCCommunications: prepareSuperstep: 
Superstep 0 totalMem = 81.0625M, maxMem = 123.9375M, freeMem = 72.63331M
12/04/13 09:30:30 INFO graph.BspServiceWorker: registerHealth: Created 
my health node for attempt=0, superstep=0 with 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/0/_workerHealthyDir/achingmbp15.local_0 
and workerInfo= Worker(hostname=achingmbp15.local, MRpartition=0, 
port=30000)
12/04/13 09:30:30 INFO partition.PartitionBalancer: 
balancePartitionsAcrossWorkers: Using algorithm static
12/04/13 09:30:30 INFO partition.PartitionUtils: analyzePartitionStats: 
Vertices - Mean: 10, Min: Worker(hostname=achingmbp15.local, 
MRpartition=0, port=30000) - 10, Max: Worker(hostname=achingmbp15.local, 
MRpartition=0, port=30000) - 10
12/04/13 09:30:30 INFO partition.PartitionUtils: analyzePartitionStats: 
Edges - Mean: 20, Min: Worker(hostname=achingmbp15.local, MRpartition=0, 
port=30000) - 20, Max: Worker(hostname=achingmbp15.local, MRpartition=0, 
port=30000) - 20
12/04/13 09:30:30 INFO graph.BspService: process: 
partitionAssignmentsReadyChanged (partitions are assigned)
12/04/13 09:30:30 INFO graph.BspServiceWorker: startSuperstep: Ready for 
computation on superstep 0 since worker selection and vertex range 
assignments are done in 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/0/_partitionAssignments
12/04/13 09:30:30 INFO graph.GraphMapper: map: Chosen to run ZooKeeper...
12/04/13 09:30:30 INFO graph.BspServiceWorker: sendWorkerPartitions: 
Done sending all my partitions.
12/04/13 09:30:30 INFO graph.BspServiceWorker: exchangeVertexPartitions: 
Done with exchange.
12/04/13 09:30:30 INFO graph.BspServiceWorker: storeCheckpoint: Finished 
metadata 
(_bsp/_checkpoints/job_local_0001/0.achingmbp15.local_0.metadata) and 
vertices (_bsp/_checkpoints/job_local_0001/0.achingmbp15.local_0.vertices).
12/04/13 09:30:30 INFO comm.BasicRPCCommunications: flush: starting for 
superstep 0 totalMem = 81.0625M, maxMem = 123.9375M, freeMem = 71.916M
12/04/13 09:30:30 INFO comm.BasicRPCCommunications: flush: ended for 
superstep 0 totalMem = 81.0625M, maxMem = 123.9375M, freeMem = 71.749374M
12/04/13 09:30:30 INFO graph.BspServiceWorker: finishSuperstep: 
Superstep 0 totalMem = 81.0625M, maxMem = 123.9375M, freeMem = 71.749374M
12/04/13 09:30:30 INFO graph.BspServiceMaster: barrierOnWorkerList: 1 
out of 1 workers finished on superstep 0 on path 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/0/_workerFinishedDir
12/04/13 09:30:30 INFO graph.BspServiceMaster: aggregateWorkerStats: 
Aggregation found (vtx=10,finVtx=0,edges=20,msgCount=20) on superstep = 0
12/04/13 09:30:30 INFO graph.BspService: process: superstepFinished signaled
12/04/13 09:30:30 INFO graph.BspServiceWorker: finishSuperstep: 
Completed superstep 0 with global stats 
(vtx=10,finVtx=0,edges=20,msgCount=20)
12/04/13 09:30:30 INFO comm.BasicRPCCommunications: prepareSuperstep: 
Superstep 1 totalMem = 81.0625M, maxMem = 123.9375M, freeMem = 71.749374M
12/04/13 09:30:30 INFO graph.MasterThread: masterThread: Coordination of 
superstep 0 took 0.039 seconds ended with state THIS_SUPERSTEP_DONE and 
is now on superstep 1
12/04/13 09:30:30 INFO graph.BspServiceWorker: registerHealth: Created 
my health node for attempt=0, superstep=1 with 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/1/_workerHealthyDir/achingmbp15.local_0 
and workerInfo= Worker(hostname=achingmbp15.local, MRpartition=0, 
port=30000)
12/04/13 09:30:30 INFO partition.PartitionBalancer: 
balancePartitionsAcrossWorkers: Using algorithm static
12/04/13 09:30:30 INFO partition.PartitionUtils: analyzePartitionStats: 
Vertices - Mean: 10, Min: Worker(hostname=achingmbp15.local, 
MRpartition=0, port=30000) - 10, Max: Worker(hostname=achingmbp15.local, 
MRpartition=0, port=30000) - 10
12/04/13 09:30:30 INFO partition.PartitionUtils: analyzePartitionStats: 
Edges - Mean: 20, Min: Worker(hostname=achingmbp15.local, MRpartition=0, 
port=30000) - 20, Max: Worker(hostname=achingmbp15.local, MRpartition=0, 
port=30000) - 20
12/04/13 09:30:30 INFO graph.BspService: process: 
partitionAssignmentsReadyChanged (partitions are assigned)
12/04/13 09:30:30 INFO graph.BspServiceMaster: barrierOnWorkerList: 0 
out of 1 workers finished on superstep 1 on path 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/1/_workerFinishedDir
12/04/13 09:30:30 INFO graph.BspServiceWorker: startSuperstep: Ready for 
computation on superstep 1 since worker selection and vertex range 
assignments are done in 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/1/_partitionAssignments
12/04/13 09:30:30 INFO graph.BspServiceWorker: getAggregatorValues: no 
aggregators in 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/0/_mergedAggregatorDir 
on superstep 1
12/04/13 09:30:30 INFO graph.GraphMapper: map: Chosen to run ZooKeeper...
12/04/13 09:30:30 INFO graph.BspServiceWorker: sendWorkerPartitions: 
Done sending all my partitions.
12/04/13 09:30:30 INFO graph.BspServiceWorker: exchangeVertexPartitions: 
Done with exchange.
12/04/13 09:30:30 INFO comm.BasicRPCCommunications: flush: starting for 
superstep 1 totalMem = 81.0625M, maxMem = 123.9375M, freeMem = 71.399864M
12/04/13 09:30:30 INFO comm.BasicRPCCommunications: flush: ended for 
superstep 1 totalMem = 81.0625M, maxMem = 123.9375M, freeMem = 71.399864M
12/04/13 09:30:30 INFO graph.BspServiceWorker: finishSuperstep: 
Superstep 1 totalMem = 81.0625M, maxMem = 123.9375M, freeMem = 71.399864M
12/04/13 09:30:30 INFO graph.BspServiceMaster: aggregateWorkerStats: 
Aggregation found (vtx=10,finVtx=0,edges=20,msgCount=20) on superstep = 1
12/04/13 09:30:30 INFO graph.BspService: process: superstepFinished signaled
12/04/13 09:30:30 INFO graph.BspServiceMaster: coordinateSuperstep: 
Cleaning up old Superstep 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/0
12/04/13 09:30:30 INFO graph.BspServiceWorker: finishSuperstep: 
Completed superstep 1 with global stats 
(vtx=10,finVtx=0,edges=20,msgCount=20)
12/04/13 09:30:30 INFO comm.BasicRPCCommunications: prepareSuperstep: 
Superstep 2 totalMem = 81.0625M, maxMem = 123.9375M, freeMem = 71.2334M
12/04/13 09:30:30 WARN graph.BspService: process: Unknown and 
unprocessed event 
(path=/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/0/_partitionAssignments, 
type=NodeDeleted, state=SyncConnected)
12/04/13 09:30:30 INFO graph.BspServiceWorker: registerHealth: Created 
my health node for attempt=0, superstep=2 with 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/2/_workerHealthyDir/achingmbp15.local_0 
and workerInfo= Worker(hostname=achingmbp15.local, MRpartition=0, 
port=30000)
12/04/13 09:30:30 INFO graph.BspServiceWorker: processEvent : 
partitionExchangeChildrenChanged (at least one worker is done sending 
partitions)
12/04/13 09:30:30 WARN graph.BspService: process: Unknown and 
unprocessed event 
(path=/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/0/_superstepFinished, 
type=NodeDeleted, state=SyncConnected)
12/04/13 09:30:30 INFO graph.MasterThread: masterThread: Coordination of 
superstep 1 took 0.039 seconds ended with state THIS_SUPERSTEP_DONE and 
is now on superstep 2
12/04/13 09:30:30 INFO partition.PartitionBalancer: 
balancePartitionsAcrossWorkers: Using algorithm static
12/04/13 09:30:30 INFO partition.PartitionUtils: analyzePartitionStats: 
Vertices - Mean: 10, Min: Worker(hostname=achingmbp15.local, 
MRpartition=0, port=30000) - 10, Max: Worker(hostname=achingmbp15.local, 
MRpartition=0, port=30000) - 10
12/04/13 09:30:30 INFO partition.PartitionUtils: analyzePartitionStats: 
Edges - Mean: 20, Min: Worker(hostname=achingmbp15.local, MRpartition=0, 
port=30000) - 20, Max: Worker(hostname=achingmbp15.local, MRpartition=0, 
port=30000) - 20
12/04/13 09:30:30 INFO graph.BspService: process: 
partitionAssignmentsReadyChanged (partitions are assigned)
12/04/13 09:30:30 INFO graph.BspServiceWorker: startSuperstep: Ready for 
computation on superstep 2 since worker selection and vertex range 
assignments are done in 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/2/_partitionAssignments
12/04/13 09:30:30 INFO graph.BspServiceWorker: getAggregatorValues: no 
aggregators in 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/1/_mergedAggregatorDir 
on superstep 2
12/04/13 09:30:30 INFO graph.GraphMapper: map: Chosen to run ZooKeeper...
12/04/13 09:30:30 INFO graph.BspServiceWorker: sendWorkerPartitions: 
Done sending all my partitions.
12/04/13 09:30:30 INFO graph.BspServiceWorker: exchangeVertexPartitions: 
Done with exchange.
12/04/13 09:30:30 INFO graph.BspServiceMaster: barrierOnWorkerList: 0 
out of 1 workers finished on superstep 2 on path 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/2/_workerFinishedDir
12/04/13 09:30:30 INFO graph.BspServiceWorker: storeCheckpoint: Finished 
metadata 
(_bsp/_checkpoints/job_local_0001/2.achingmbp15.local_0.metadata) and 
vertices (_bsp/_checkpoints/job_local_0001/2.achingmbp15.local_0.vertices).
12/04/13 09:30:30 INFO comm.BasicRPCCommunications: flush: starting for 
superstep 2 totalMem = 81.0625M, maxMem = 123.9375M, freeMem = 70.699936M
12/04/13 09:30:30 INFO comm.BasicRPCCommunications: flush: ended for 
superstep 2 totalMem = 81.0625M, maxMem = 123.9375M, freeMem = 70.699936M
12/04/13 09:30:30 INFO graph.BspServiceWorker: finishSuperstep: 
Superstep 2 totalMem = 81.0625M, maxMem = 123.9375M, freeMem = 70.699936M
12/04/13 09:30:30 INFO graph.BspServiceMaster: aggregateWorkerStats: 
Aggregation found (vtx=10,finVtx=10,edges=20,msgCount=0) on superstep = 2
12/04/13 09:30:30 INFO graph.BspService: process: superstepFinished signaled
12/04/13 09:30:30 INFO graph.BspServiceWorker: finishSuperstep: 
Completed superstep 2 with global stats 
(vtx=10,finVtx=10,edges=20,msgCount=0)
12/04/13 09:30:30 INFO graph.GraphMapper: map: BSP application done 
(global vertices marked done)
12/04/13 09:30:30 INFO graph.GraphMapper: cleanup: Starting for ALL
12/04/13 09:30:30 WARN graph.BspServiceWorker: saveVertices: 
giraph.vertexOutputFormatClass not specified -- there will be no saved 
output
12/04/13 09:30:30 INFO graph.BspServiceMaster: coordinateSuperstep: 
Cleaning up old Superstep 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/1
12/04/13 09:30:30 WARN graph.BspService: process: Unknown and 
unprocessed event 
(path=/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/1/_partitionAssignments, 
type=NodeDeleted, state=SyncConnected)
12/04/13 09:30:30 INFO graph.BspServiceWorker: cleanup: Notifying master 
its okay to cleanup with /_hadoopBsp/job_local_0001/_cleanedUpDir/0_worker
12/04/13 09:30:30 INFO zookeeper.ZooKeeper: Session: 0x136ac8aa05e0001 
closed
12/04/13 09:30:30 INFO zookeeper.ClientCnxn: EventThread shut down
12/04/13 09:30:30 INFO comm.BasicRPCCommunications: close: shutting down 
RPC server
12/04/13 09:30:30 INFO ipc.Server: Stopping server on 30000
12/04/13 09:30:30 WARN graph.BspServiceMaster: coordinateBarrier: 
Already cleaned up 
/_hadoopBsp/job_local_0001/_applicationAttemptsDir/0/_superstepDir/1
12/04/13 09:30:30 INFO ipc.Server: IPC Server handler 0 on 30000: exiting
12/04/13 09:30:30 INFO ipc.Server: Stopping IPC Server listener on 30000
12/04/13 09:30:30 INFO graph.MasterThread: masterThread: Coordination of 
superstep 2 took 0.027 seconds ended with state ALL_SUPERSTEPS_DONE and 
is now on superstep 3
12/04/13 09:30:30 INFO graph.BspServiceMaster: setJobState: 
{"_stateKey":"FINISHED","_applicationAttemptKey":-1,"_superstepKey":-1} 
on superstep 3
12/04/13 09:30:30 INFO metrics.RpcInstrumentation: shut down
12/04/13 09:30:30 INFO ipc.Server: Stopping IPC Server Responder
12/04/13 09:30:30 INFO graph.BspServiceMaster: cleanup: Notifying master 
its okay to cleanup with /_hadoopBsp/job_local_0001/_cleanedUpDir/0_master
12/04/13 09:30:30 INFO graph.BspServiceMaster: cleanUpZooKeeper: Node 
/_hadoopBsp/job_local_0001/_cleanedUpDir already exists, no need to create.
12/04/13 09:30:30 INFO graph.BspServiceMaster: cleanUpZooKeeper: Got 2 
of 2 desired children from /_hadoopBsp/job_local_0001/_cleanedUpDir
12/04/13 09:30:30 INFO graph.BspServiceMaster: cleanupZooKeeper: 
Removing the following path and all children - /_hadoopBsp/job_local_0001
12/04/13 09:30:30 INFO graph.BspService: process: 
masterElectionChildrenChanged signaled
12/04/13 09:30:31 INFO graph.BspService: process: 
cleanedUpChildrenChanged signaled
12/04/13 09:30:31 INFO graph.BspServiceMaster: cleanup: Removed HDFS 
checkpoint directory (_bsp/_checkpoints//job_local_0001) with return = 
true since this job succeeded
12/04/13 09:30:31 INFO zookeeper.ZooKeeper: Session: 0x136ac8aa05e0000 
closed
12/04/13 09:30:31 INFO zookeeper.ClientCnxn: EventThread shut down
12/04/13 09:30:31 INFO graph.MasterThread: setup: Took 0.065 seconds.
12/04/13 09:30:31 INFO graph.MasterThread: vertex input superstep: Took 
0.036 seconds.
12/04/13 09:30:31 INFO graph.MasterThread: superstep 0: Took 0.039 seconds.
12/04/13 09:30:31 INFO graph.MasterThread: superstep 1: Took 0.039 seconds.
12/04/13 09:30:31 INFO graph.MasterThread: superstep 2: Took 0.027 seconds.
12/04/13 09:30:31 INFO graph.MasterThread: shutdown: Took 0.059 seconds.
12/04/13 09:30:31 INFO graph.MasterThread: total: Took 0.266 seconds.
12/04/13 09:30:31 INFO zk.ZooKeeperManager: createZooKeeperClosedStamp: 
Creating my filestamp 
_bsp/_defaultZkManagerDir/job_local_0001/_task/0.COMPUTATION_DONE
12/04/13 09:30:31 INFO zk.ZooKeeperManager: waitUntilAllTasksDone: Got 1 
and 1 desired (polling period is 3000) on attempt 0
12/04/13 09:30:31 INFO zk.ZooKeeperManager: offlineZooKeeperServers: 
waitFor returned 143 and deleted directory 
/Users/aching/git/git_svn_giraph_trunk/_bspZooKeeper
12/04/13 09:30:31 INFO mapred.Task: Task:attempt_local_0001_m_000000_0 
is done. And is in the process of commiting
12/04/13 09:30:33 INFO mapred.LocalJobRunner: finishSuperstep: (all 
workers done) ALL - Attempt=0, Superstep=3
12/04/13 09:30:33 INFO mapred.LocalJobRunner: finishSuperstep: (all 
workers done) ALL - Attempt=0, Superstep=3
12/04/13 09:30:33 INFO mapred.Task: Task 'attempt_local_0001_m_000000_0' 
done.
12/04/13 09:30:33 WARN mapred.FileOutputCommitter: Output path is null 
in cleanup


On 4/13/12 4:51 AM, Paolo Castagna wrote:
> Paolo Castagna wrote:
>> This is a better way:
>>
>>    Iterable<String>  results = InternalVertexRunner.run(
>>      SimpleShortestPathsVertex.class,
>>      SimpleShortestPathsVertex.SimpleShortestPathsVertexInputFormat.class,
>>      SimpleShortestPathsVertex.SimpleShortestPathsVertexOutputFormat.class,
>>      params, graph);
>>
>> ... which starts a local ZooKeeper properly.
>>
>> However, I still have a question: when I run it in a unit test everything is
>> fine. When I run it on a Java main method, it hangs towards the end.
> I am using Hadoop 1.0.1, Pig 0.9.2, ZooKeeper 3.4.3 and Giraph from trunk:
>
> [INFO] +- org.apache.hadoop:hadoop-core:jar:1.0.1:compile
> ...
> [INFO] +- org.apache.pig:pig:jar:0.9.2:compile
> ...
> [INFO] +- org.apache.hbase:hbase:jar:0.92.1:compile
> ...
> [INFO] +- org.apache.zookeeper:zookeeper:jar:3.4.3:compile
> ...
> [INFO] +- org.apache.giraph:giraph:jar:0.2-SNAPSHOT:compile
>
>
> Paolo



Re: A simple use case: shortest paths on a FOAF (i.e. Friend of a Friend) graph

Posted by Paolo Castagna <ca...@googlemail.com>.
Paolo Castagna wrote:
> This is a better way:
> 
>   Iterable<String> results = InternalVertexRunner.run(
>     SimpleShortestPathsVertex.class,
>     SimpleShortestPathsVertex.SimpleShortestPathsVertexInputFormat.class,
>     SimpleShortestPathsVertex.SimpleShortestPathsVertexOutputFormat.class,
>     params, graph);
> 
> ... which starts a local ZooKeeper properly.
> 
> However, I still have a question: when I run it in a unit test everything is
> fine. When I run it on a Java main method, it hangs towards the end.

I am using Hadoop 1.0.1, Pig 0.9.2, ZooKeeper 3.4.3 and Giraph from trunk:

[INFO] +- org.apache.hadoop:hadoop-core:jar:1.0.1:compile
...
[INFO] +- org.apache.pig:pig:jar:0.9.2:compile
...
[INFO] +- org.apache.hbase:hbase:jar:0.92.1:compile
...
[INFO] +- org.apache.zookeeper:zookeeper:jar:3.4.3:compile
...
[INFO] +- org.apache.giraph:giraph:jar:0.2-SNAPSHOT:compile


Paolo

Re: A simple use case: shortest paths on a FOAF (i.e. Friend of a Friend) graph

Posted by Paolo Castagna <ca...@googlemail.com>.
Paolo Castagna wrote:
> Avery Ching wrote:
>> It shouldn't be, your code looks very similar to the unittests (i.e.
>> TestManualCheckpoint.java).  So, you're trying to run your test with the
>> local hadoop (similar to the unittests)?  Or are you using an actual
>> hadoop setup?
> 
> Hi Avery,
> here is a few more details on what I am trying to do, in order to run my Giraph
> jobs on a local Hadoop running (for testing and debugging stuff locally):
> 
>   GiraphJob job = new GiraphJob("shortest paths");
>   Configuration conf = job.getConfiguration();
>   conf.setBoolean(GiraphJob.SPLIT_MASTER_WORKER, false);
>   conf.setBoolean(GiraphJob.LOCAL_TEST_MODE, true);
>   // conf.set(GiraphJob.ZOOKEEPER_JAR,
> "file://target/dependency/zookeeper-3.3.3.jar");
>   job.setWorkerConfiguration(1, 1, 100.0f);
>   job.setVertexClass(SimpleShortestPathsVertex.class);
>   job.setVertexInputFormatClass(SimpleShortestPathsVertexInputFormat.class);
>   job.setVertexOutputFormatClass(SimpleShortestPathsVertexOutputFormat.class);
>   FileInputFormat.addInputPath(job.getInternalJob(), new
> Path("src/main/resources/giraph1.txt"));
>   Path outputPath = new Path("target/giraph1");
>   FileSystem hdfs = FileSystem.get(conf);
>   hdfs.delete(outputPath, true);
>   FileOutputFormat.setOutputPath(job.getInternalJob(), outputPath);
>   job.run(true);
> 
> Am I doing something wrong/stupid here?
> Am I missing something important? (probably! but I do not see what I am missing)

This is a better way:

  Iterable<String> results = InternalVertexRunner.run(
    SimpleShortestPathsVertex.class,
    SimpleShortestPathsVertex.SimpleShortestPathsVertexInputFormat.class,
    SimpleShortestPathsVertex.SimpleShortestPathsVertexOutputFormat.class,
    params, graph);

... which starts a local ZooKeeper properly.

However, I still have a question: when I run it in a unit test everything is
fine. When I run it on a Java main method, it hangs towards the end.

Paolo

Re: A simple use case: shortest paths on a FOAF (i.e. Friend of a Friend) graph

Posted by Paolo Castagna <ca...@googlemail.com>.
Avery Ching wrote:
> It shouldn't be, your code looks very similar to the unittests (i.e.
> TestManualCheckpoint.java).  So, you're trying to run your test with the
> local hadoop (similar to the unittests)?  Or are you using an actual
> hadoop setup?

Hi Avery,
here is a few more details on what I am trying to do, in order to run my Giraph
jobs on a local Hadoop running (for testing and debugging stuff locally):

  GiraphJob job = new GiraphJob("shortest paths");
  Configuration conf = job.getConfiguration();
  conf.setBoolean(GiraphJob.SPLIT_MASTER_WORKER, false);
  conf.setBoolean(GiraphJob.LOCAL_TEST_MODE, true);
  // conf.set(GiraphJob.ZOOKEEPER_JAR,
"file://target/dependency/zookeeper-3.3.3.jar");
  job.setWorkerConfiguration(1, 1, 100.0f);
  job.setVertexClass(SimpleShortestPathsVertex.class);
  job.setVertexInputFormatClass(SimpleShortestPathsVertexInputFormat.class);
  job.setVertexOutputFormatClass(SimpleShortestPathsVertexOutputFormat.class);
  FileInputFormat.addInputPath(job.getInternalJob(), new
Path("src/main/resources/giraph1.txt"));
  Path outputPath = new Path("target/giraph1");
  FileSystem hdfs = FileSystem.get(conf);
  hdfs.delete(outputPath, true);
  FileOutputFormat.setOutputPath(job.getInternalJob(), outputPath);
  job.run(true);

Am I doing something wrong/stupid here?
Am I missing something important? (probably! but I do not see what I am missing)

This is what I think happens...

In GraphMapper something goes wrong during setup(context), probably because
GiraphJob.ZOOKEEPER_JAR is not set(?) and an exception different from
IOException is thrown and I do not see any useful error message:

    try {
      setup(context);
      while (context.nextKeyValue()) {
        map(context.getCurrentKey(),
            context.getCurrentValue(),
            context);
      }
      cleanup(context);
    } catch (IOException e) {
      if (mapFunctions == MapFunctions.WORKER_ONLY) {
        serviceWorker.failureCleanup();
      }
      throw new IllegalStateException(
          "run: Caught an unrecoverable exception " + e.getMessage(), e);
    }

My question is: is it possible to run a Giraph job as I am trying to do above
(for testing only) or developers need to have an Hadoop cluster either remote
or locally and ZooKeeper running (either remote or locally)?

Thanks,
Paolo

Re: A simple use case: shortest paths on a FOAF (i.e. Friend of a Friend) graph

Posted by Paolo Castagna <ca...@googlemail.com>.
Avery Ching wrote:
> It shouldn't be, your code looks very similar to the unittests (i.e.
> TestManualCheckpoint.java).  So, you're trying to run your test with the
> local hadoop (similar to the unittests)?  Or are you using an actual
> hadoop setup?

Hi Avery,
while I am learning and writing the first examples, I am trying to run with
a local hadoop (similar to the unit tests). This way, I can easily run and
debug the code from the IDE.

Tomorrow, I'll look at the unit tests again trying to see if I can spot what
I am doing wrong.

Thanks,
Paolo

> 
> Avery
> 
> On 4/10/12 11:41 PM, Paolo Castagna wrote:
>> I am using hadoop-core-1.0.1.jar ... could that be a problem?
>>
>> Paolo
>>
>> Paolo Castagna wrote:
>>> Hi Avery,
>>> nope, no luck.
>>>
>>> I have changed all my log.debug(...) into log.info(...). Same behavior.
>>>
>>> I have a log4j.properties [1] file in my classpath and it has:
>>> log4j.logger.org.apache.jena.grande=DEBUG
>>> log4j.logger.org.apache.jena.grande.giraph=DEBUG
>>> I also tried to change that to:
>>> log4j.logger.org.apache.jena.grande=INFO
>>> log4j.logger.org.apache.jena.grande.giraph=INFO
>>> No luck.
>>>
>>> My Giraph job has:
>>> GiraphJob job = new GiraphJob(getConf(), getClass().getName());
>>> job.setVertexClass(getClass());
>>> job.setVertexInputFormatClass(TurtleVertexInputFormat.class);
>>> job.setVertexOutputFormatClass(TurtleVertexOutputFormat.class);
>>>
>>> But, if I run in debug with a breakpoint in the
>>> TurtleVertexInputFormat.class
>>> constructor, it is never instanciated. How can it be?
>>>
>>> So perhaps the problem is not the logging, it is the fact that
>>> my GiraphJob is not using TurtleVertexInputFormat.class and
>>> TurtleVertexOutputFormat.class, but I don't see what I am doing
>>> wrong. :-/
>>>
>>> Thanks,
>>> Paolo
>>>
>>>   [1]
>>> https://github.com/castagna/jena-grande/blob/master/src/test/resources/log4j.properties
>>>
>>>
>>> Avery Ching wrote:
>>>> I think the issue might be that Hadoop only logs INFO and above
>>>> messages
>>>> by default.  Can you retry with INFO level logging?
>>>>
>>>> Avery
>>>>
>>>> On 4/10/12 12:17 PM, Paolo Castagna wrote:
>>>>> Hi,
>>>>> I am still learning Giraph, so, please, be patient with me and
>>>>> forgive my
>>>>> trivial questions.
>>>>>
>>>>> As a simple initial use case, I want to compute the shortest paths
>>>>> from a single
>>>>> source in a social graph in RDF format using the FOAF [1] vocabulary.
>>>>> This example also will hopefully inform GIRAPH-170 [2] and related
>>>>> issues, such
>>>>> as: GIRAPH-141 [3].
>>>>>
>>>>> Here is an example in Turtle [4] format of a tiny graph using FOAF:
>>>>> ----
>>>>> @prefix :<http://example.org/>   .
>>>>> @prefix foaf:<http://xmlns.com/foaf/0.1/>   .
>>>>>
>>>>> :alice
>>>>>       a           foaf:Person ;
>>>>>       foaf:name   "Alice" ;
>>>>>       foaf:mbox<ma...@example.org>   ;
>>>>>       foaf:knows  :bob ;
>>>>>       foaf:knows  :charlie ;
>>>>>       foaf:knows  :snoopy ;
>>>>>       .
>>>>>
>>>>> :bob
>>>>>       foaf:name   "Bob" ;
>>>>>       foaf:knows  :charlie ;
>>>>>       .
>>>>>
>>>>> :charlie
>>>>>       foaf:name   "Charlie" ;
>>>>>       foaf:knows  :alice ;
>>>>>       .
>>>>> ----
>>>>> This is nice, human friendly (RDF without angle brackets!), but not
>>>>> easily
>>>>> splittable to be processed with MapReduce (or Giraph).
>>>>>
>>>>> Here is the same graph in N-Triples [5] format:
>>>>> ----
>>>>> <http://example.org/alice>
>>>>> <http://www.w3.org/1999/02/22-rdf-syntax-ns#type>
>>>>> <http://xmlns.com/foaf/0.1/Person>   .
>>>>> <http://example.org/alice>   <http://xmlns.com/foaf/0.1/name>  
>>>>> "Alice" .
>>>>> <http://example.org/alice>   <http://xmlns.com/foaf/0.1/mbox>
>>>>> <ma...@example.org>   .
>>>>> <http://example.org/alice>   <http://xmlns.com/foaf/0.1/knows>
>>>>> <http://example.org/bob>   .
>>>>> <http://example.org/alice>   <http://xmlns.com/foaf/0.1/knows>
>>>>> <http://example.org/charlie>   .
>>>>> <http://example.org/alice>   <http://xmlns.com/foaf/0.1/knows>
>>>>> <http://example.org/snoopy>   .
>>>>> <http://example.org/charlie>   <http://xmlns.com/foaf/0.1/name>
>>>>> "Charlie" .
>>>>> <http://example.org/charlie>   <http://xmlns.com/foaf/0.1/knows>
>>>>> <http://example.org/alice>   .
>>>>> <http://example.org/bob>   <http://xmlns.com/foaf/0.1/name>   "Bob" .
>>>>> <http://example.org/bob>   <http://xmlns.com/foaf/0.1/knows>
>>>>> <http://example.org/charlie>   .
>>>>> ----
>>>>> This is more verbose and ugly, but splittable.
>>>>>
>>>>> The graph I am interested in is the graph represented by foaf:knows
>>>>> relationships/links between people (please, note --knows-->
>>>>> relationship here
>>>>> has a direction, this isn't symmetric as in centralized social
>>>>> networking
>>>>> websites such as Facebook or LinkedIn. Alice can claim to know Bob,
>>>>> without Bob
>>>>> knowing it and/or it might even be a false claim):
>>>>>
>>>>> alice --knows-->   bob
>>>>> alice --knows-->   charlie
>>>>> alice --knows-->   snoopy
>>>>> bob --knows-->   charlie
>>>>> charlie --knows-->   alice
>>>>>
>>>>> As a first step, I wrote a MapReduce job [6] to transform the RDF
>>>>> graph above in
>>>>> a sort of adjacency list using Turtle syntax, here is the output
>>>>> (three lines):
>>>>> ----
>>>>> <http://example.org/alice>   <http://xmlns.com/foaf/0.1/mbox>
>>>>> <ma...@example.org>;<http://xmlns.com/foaf/0.1/name>   "Alice";
>>>>> <http://www.w3.org/1999/02/22-rdf-syntax-ns#type>
>>>>> <http://xmlns.com/foaf/0.1/Person>;<http://xmlns.com/foaf/0.1/knows>
>>>>> <http://example.org/charlie>,<http://example.org/bob>,
>>>>> <http://example.org/snoopy>; .<http://example.org/charlie>
>>>>> <http://xmlns.com/foaf/0.1/knows>   <http://example.org/alice>.
>>>>>
>>>>> <http://example.org/bob>   <http://xmlns.com/foaf/0.1/name>   "Bob";
>>>>> <http://xmlns.com/foaf/0.1/knows>   <http://example.org/charlie>; .
>>>>> <http://example.org/alice>   <http://xmlns.com/foaf/0.1/knows>
>>>>> <http://example.org/bob>.
>>>>>
>>>>> <http://example.org/charlie>   <http://xmlns.com/foaf/0.1/name>
>>>>> "Charlie";
>>>>> <http://xmlns.com/foaf/0.1/knows>   <http://example.org/alice>; .
>>>>> <http://example.org/bob>   <http://xmlns.com/foaf/0.1/knows>
>>>>> <http://example.org/charlie>.<http://example.org/alice>
>>>>> <http://xmlns.com/foaf/0.1/knows>   <http://example.org/charlie>.
>>>>> ----
>>>>> This is legal Turtle, but it is also splittable. Each line has all the
>>>>> RDF
>>>>> statements (i.e. egdes) for a person (there are also incoming edges).
>>>>>
>>>>> I wrote a TurtleVertexReader [7] which extends
>>>>> TextVertexReader<NodeWritable,
>>>>> Text, NodeWritable, Text>   and a TurtleVertexInputFormat [8] which
>>>>> extends
>>>>> TextVertexInputFormat<NodeWritable, Text, NodeWritable, Text>.
>>>>> I wrote (copying from the example SimpleShortestPathsVertex) a
>>>>> FoafShortestPathsVertex [9] which extends EdgeListVertex<NodeWritable,
>>>>> IntWritable, NodeWritable, IntWritable>   and I am running it locally
>>>>> using these
>>>>> arguments: -Dgiraph.maxWorkers=1 -Dgiraph.SplitMasterWorker=false
>>>>> -DoverwriteOutput=true src/test/resources/data3.ttl target/foaf
>>>>> http://example.org/alice 1
>>>>>
>>>>> TurtleVertexReader, TurtleVertexInputFormat and
>>>>> FoafShortestPathsVertex are
>>>>> still work in progress and I am sure there are plenty of stupid
>>>>> errors.
>>>>> However, I do not understand why when I run FoafShortestPathsVertex
>>>>> with the
>>>>> DEBUG level, I see debug statements from FoafShortestPathsVertex:
>>>>> 19:34:44 DEBUG FoafShortestPathsVertex   ::
>>>>> main({-Dgiraph.maxWorkers=1,
>>>>> -Dgiraph.SplitMasterWorker=false, -DoverwriteOutput=true,
>>>>> src/test/resources/data3.ttl, target/foaf,
>>>>> http://example.org/alice, 1})
>>>>> 19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() -->   null
>>>>> 19:34:44 DEBUG FoafShortestPathsVertex   :: setConf(Configuration:
>>>>> core-default.xml, core-site.xml)
>>>>> 19:34:44 DEBUG FoafShortestPathsVertex   ::
>>>>> run({src/test/resources/data3.ttl,
>>>>> target/foaf, http://example.org/alice, 1})
>>>>> 19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() -->  
>>>>> Configuration:
>>>>> core-default.xml, core-site.xml
>>>>> 19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() -->  
>>>>> Configuration:
>>>>> core-default.xml, core-site.xml, mapred-default.xml, mapred-site.xml,
>>>>> giraph-site.xml
>>>>>
>>>>> But, I do not see anything else, no log statement from
>>>>> TurtleVertexReader or
>>>>> TurtleVertexInputFormat. Why? What am I doing wrong?
>>>>> Is it because I am running it locally?
>>>>>
>>>>> Thanks,
>>>>> Paolo
>>>>>
>>>>>    [1] http://en.wikipedia.org/wiki/FOAF_%28software%29
>>>>>    [2] https://issues.apache.org/jira/browse/GIRAPH-170
>>>>>    [3] https://issues.apache.org/jira/browse/GIRAPH-141
>>>>>    [4] http://en.wikipedia.org/wiki/Turtle_%28syntax%29
>>>>>    [5] http://en.wikipedia.org/wiki/N-Triples
>>>>>    [6]
>>>>> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/mapreduce/Rdf2AdjacencyListDriver.java
>>>>>
>>>>>
>>>>>    [7]
>>>>> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/TurtleVertexReader.java
>>>>>
>>>>>
>>>>>    [8]
>>>>> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/TurtleVertexInputFormat.java
>>>>>
>>>>>
>>>>>    [9]
>>>>> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/FoafShortestPathsVertex.java
>>>>>
>>>>>
> 

Re: A simple use case: shortest paths on a FOAF (i.e. Friend of a Friend) graph

Posted by Avery Ching <ac...@apache.org>.
It shouldn't be, your code looks very similar to the unittests (i.e. 
TestManualCheckpoint.java).  So, you're trying to run your test with the 
local hadoop (similar to the unittests)?  Or are you using an actual 
hadoop setup?

Avery

On 4/10/12 11:41 PM, Paolo Castagna wrote:
> I am using hadoop-core-1.0.1.jar ... could that be a problem?
>
> Paolo
>
> Paolo Castagna wrote:
>> Hi Avery,
>> nope, no luck.
>>
>> I have changed all my log.debug(...) into log.info(...). Same behavior.
>>
>> I have a log4j.properties [1] file in my classpath and it has:
>> log4j.logger.org.apache.jena.grande=DEBUG
>> log4j.logger.org.apache.jena.grande.giraph=DEBUG
>> I also tried to change that to:
>> log4j.logger.org.apache.jena.grande=INFO
>> log4j.logger.org.apache.jena.grande.giraph=INFO
>> No luck.
>>
>> My Giraph job has:
>> GiraphJob job = new GiraphJob(getConf(), getClass().getName());
>> job.setVertexClass(getClass());
>> job.setVertexInputFormatClass(TurtleVertexInputFormat.class);
>> job.setVertexOutputFormatClass(TurtleVertexOutputFormat.class);
>>
>> But, if I run in debug with a breakpoint in the TurtleVertexInputFormat.class
>> constructor, it is never instanciated. How can it be?
>>
>> So perhaps the problem is not the logging, it is the fact that
>> my GiraphJob is not using TurtleVertexInputFormat.class and
>> TurtleVertexOutputFormat.class, but I don't see what I am doing
>> wrong. :-/
>>
>> Thanks,
>> Paolo
>>
>>   [1]
>> https://github.com/castagna/jena-grande/blob/master/src/test/resources/log4j.properties
>>
>> Avery Ching wrote:
>>> I think the issue might be that Hadoop only logs INFO and above messages
>>> by default.  Can you retry with INFO level logging?
>>>
>>> Avery
>>>
>>> On 4/10/12 12:17 PM, Paolo Castagna wrote:
>>>> Hi,
>>>> I am still learning Giraph, so, please, be patient with me and forgive my
>>>> trivial questions.
>>>>
>>>> As a simple initial use case, I want to compute the shortest paths
>>>> from a single
>>>> source in a social graph in RDF format using the FOAF [1] vocabulary.
>>>> This example also will hopefully inform GIRAPH-170 [2] and related
>>>> issues, such
>>>> as: GIRAPH-141 [3].
>>>>
>>>> Here is an example in Turtle [4] format of a tiny graph using FOAF:
>>>> ----
>>>> @prefix :<http://example.org/>   .
>>>> @prefix foaf:<http://xmlns.com/foaf/0.1/>   .
>>>>
>>>> :alice
>>>>       a           foaf:Person ;
>>>>       foaf:name   "Alice" ;
>>>>       foaf:mbox<ma...@example.org>   ;
>>>>       foaf:knows  :bob ;
>>>>       foaf:knows  :charlie ;
>>>>       foaf:knows  :snoopy ;
>>>>       .
>>>>
>>>> :bob
>>>>       foaf:name   "Bob" ;
>>>>       foaf:knows  :charlie ;
>>>>       .
>>>>
>>>> :charlie
>>>>       foaf:name   "Charlie" ;
>>>>       foaf:knows  :alice ;
>>>>       .
>>>> ----
>>>> This is nice, human friendly (RDF without angle brackets!), but not
>>>> easily
>>>> splittable to be processed with MapReduce (or Giraph).
>>>>
>>>> Here is the same graph in N-Triples [5] format:
>>>> ----
>>>> <http://example.org/alice>
>>>> <http://www.w3.org/1999/02/22-rdf-syntax-ns#type>
>>>> <http://xmlns.com/foaf/0.1/Person>   .
>>>> <http://example.org/alice>   <http://xmlns.com/foaf/0.1/name>   "Alice" .
>>>> <http://example.org/alice>   <http://xmlns.com/foaf/0.1/mbox>
>>>> <ma...@example.org>   .
>>>> <http://example.org/alice>   <http://xmlns.com/foaf/0.1/knows>
>>>> <http://example.org/bob>   .
>>>> <http://example.org/alice>   <http://xmlns.com/foaf/0.1/knows>
>>>> <http://example.org/charlie>   .
>>>> <http://example.org/alice>   <http://xmlns.com/foaf/0.1/knows>
>>>> <http://example.org/snoopy>   .
>>>> <http://example.org/charlie>   <http://xmlns.com/foaf/0.1/name>
>>>> "Charlie" .
>>>> <http://example.org/charlie>   <http://xmlns.com/foaf/0.1/knows>
>>>> <http://example.org/alice>   .
>>>> <http://example.org/bob>   <http://xmlns.com/foaf/0.1/name>   "Bob" .
>>>> <http://example.org/bob>   <http://xmlns.com/foaf/0.1/knows>
>>>> <http://example.org/charlie>   .
>>>> ----
>>>> This is more verbose and ugly, but splittable.
>>>>
>>>> The graph I am interested in is the graph represented by foaf:knows
>>>> relationships/links between people (please, note --knows-->
>>>> relationship here
>>>> has a direction, this isn't symmetric as in centralized social networking
>>>> websites such as Facebook or LinkedIn. Alice can claim to know Bob,
>>>> without Bob
>>>> knowing it and/or it might even be a false claim):
>>>>
>>>> alice --knows-->   bob
>>>> alice --knows-->   charlie
>>>> alice --knows-->   snoopy
>>>> bob --knows-->   charlie
>>>> charlie --knows-->   alice
>>>>
>>>> As a first step, I wrote a MapReduce job [6] to transform the RDF
>>>> graph above in
>>>> a sort of adjacency list using Turtle syntax, here is the output
>>>> (three lines):
>>>> ----
>>>> <http://example.org/alice>   <http://xmlns.com/foaf/0.1/mbox>
>>>> <ma...@example.org>;<http://xmlns.com/foaf/0.1/name>   "Alice";
>>>> <http://www.w3.org/1999/02/22-rdf-syntax-ns#type>
>>>> <http://xmlns.com/foaf/0.1/Person>;<http://xmlns.com/foaf/0.1/knows>
>>>> <http://example.org/charlie>,<http://example.org/bob>,
>>>> <http://example.org/snoopy>; .<http://example.org/charlie>
>>>> <http://xmlns.com/foaf/0.1/knows>   <http://example.org/alice>.
>>>>
>>>> <http://example.org/bob>   <http://xmlns.com/foaf/0.1/name>   "Bob";
>>>> <http://xmlns.com/foaf/0.1/knows>   <http://example.org/charlie>; .
>>>> <http://example.org/alice>   <http://xmlns.com/foaf/0.1/knows>
>>>> <http://example.org/bob>.
>>>>
>>>> <http://example.org/charlie>   <http://xmlns.com/foaf/0.1/name>
>>>> "Charlie";
>>>> <http://xmlns.com/foaf/0.1/knows>   <http://example.org/alice>; .
>>>> <http://example.org/bob>   <http://xmlns.com/foaf/0.1/knows>
>>>> <http://example.org/charlie>.<http://example.org/alice>
>>>> <http://xmlns.com/foaf/0.1/knows>   <http://example.org/charlie>.
>>>> ----
>>>> This is legal Turtle, but it is also splittable. Each line has all the
>>>> RDF
>>>> statements (i.e. egdes) for a person (there are also incoming edges).
>>>>
>>>> I wrote a TurtleVertexReader [7] which extends
>>>> TextVertexReader<NodeWritable,
>>>> Text, NodeWritable, Text>   and a TurtleVertexInputFormat [8] which
>>>> extends
>>>> TextVertexInputFormat<NodeWritable, Text, NodeWritable, Text>.
>>>> I wrote (copying from the example SimpleShortestPathsVertex) a
>>>> FoafShortestPathsVertex [9] which extends EdgeListVertex<NodeWritable,
>>>> IntWritable, NodeWritable, IntWritable>   and I am running it locally
>>>> using these
>>>> arguments: -Dgiraph.maxWorkers=1 -Dgiraph.SplitMasterWorker=false
>>>> -DoverwriteOutput=true src/test/resources/data3.ttl target/foaf
>>>> http://example.org/alice 1
>>>>
>>>> TurtleVertexReader, TurtleVertexInputFormat and
>>>> FoafShortestPathsVertex are
>>>> still work in progress and I am sure there are plenty of stupid errors.
>>>> However, I do not understand why when I run FoafShortestPathsVertex
>>>> with the
>>>> DEBUG level, I see debug statements from FoafShortestPathsVertex:
>>>> 19:34:44 DEBUG FoafShortestPathsVertex   :: main({-Dgiraph.maxWorkers=1,
>>>> -Dgiraph.SplitMasterWorker=false, -DoverwriteOutput=true,
>>>> src/test/resources/data3.ttl, target/foaf, http://example.org/alice, 1})
>>>> 19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() -->   null
>>>> 19:34:44 DEBUG FoafShortestPathsVertex   :: setConf(Configuration:
>>>> core-default.xml, core-site.xml)
>>>> 19:34:44 DEBUG FoafShortestPathsVertex   ::
>>>> run({src/test/resources/data3.ttl,
>>>> target/foaf, http://example.org/alice, 1})
>>>> 19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() -->   Configuration:
>>>> core-default.xml, core-site.xml
>>>> 19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() -->   Configuration:
>>>> core-default.xml, core-site.xml, mapred-default.xml, mapred-site.xml,
>>>> giraph-site.xml
>>>>
>>>> But, I do not see anything else, no log statement from
>>>> TurtleVertexReader or
>>>> TurtleVertexInputFormat. Why? What am I doing wrong?
>>>> Is it because I am running it locally?
>>>>
>>>> Thanks,
>>>> Paolo
>>>>
>>>>    [1] http://en.wikipedia.org/wiki/FOAF_%28software%29
>>>>    [2] https://issues.apache.org/jira/browse/GIRAPH-170
>>>>    [3] https://issues.apache.org/jira/browse/GIRAPH-141
>>>>    [4] http://en.wikipedia.org/wiki/Turtle_%28syntax%29
>>>>    [5] http://en.wikipedia.org/wiki/N-Triples
>>>>    [6]
>>>> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/mapreduce/Rdf2AdjacencyListDriver.java
>>>>
>>>>    [7]
>>>> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/TurtleVertexReader.java
>>>>
>>>>    [8]
>>>> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/TurtleVertexInputFormat.java
>>>>
>>>>    [9]
>>>> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/FoafShortestPathsVertex.java
>>>>


Re: A simple use case: shortest paths on a FOAF (i.e. Friend of a Friend) graph

Posted by Paolo Castagna <ca...@googlemail.com>.
I am using hadoop-core-1.0.1.jar ... could that be a problem?

Paolo

Paolo Castagna wrote:
> Hi Avery,
> nope, no luck.
> 
> I have changed all my log.debug(...) into log.info(...). Same behavior.
> 
> I have a log4j.properties [1] file in my classpath and it has:
> log4j.logger.org.apache.jena.grande=DEBUG
> log4j.logger.org.apache.jena.grande.giraph=DEBUG
> I also tried to change that to:
> log4j.logger.org.apache.jena.grande=INFO
> log4j.logger.org.apache.jena.grande.giraph=INFO
> No luck.
> 
> My Giraph job has:
> GiraphJob job = new GiraphJob(getConf(), getClass().getName());
> job.setVertexClass(getClass());
> job.setVertexInputFormatClass(TurtleVertexInputFormat.class);
> job.setVertexOutputFormatClass(TurtleVertexOutputFormat.class);
> 
> But, if I run in debug with a breakpoint in the TurtleVertexInputFormat.class
> constructor, it is never instanciated. How can it be?
> 
> So perhaps the problem is not the logging, it is the fact that
> my GiraphJob is not using TurtleVertexInputFormat.class and
> TurtleVertexOutputFormat.class, but I don't see what I am doing
> wrong. :-/
> 
> Thanks,
> Paolo
> 
>  [1]
> https://github.com/castagna/jena-grande/blob/master/src/test/resources/log4j.properties
> 
> Avery Ching wrote:
>> I think the issue might be that Hadoop only logs INFO and above messages
>> by default.  Can you retry with INFO level logging?
>>
>> Avery
>>
>> On 4/10/12 12:17 PM, Paolo Castagna wrote:
>>> Hi,
>>> I am still learning Giraph, so, please, be patient with me and forgive my
>>> trivial questions.
>>>
>>> As a simple initial use case, I want to compute the shortest paths
>>> from a single
>>> source in a social graph in RDF format using the FOAF [1] vocabulary.
>>> This example also will hopefully inform GIRAPH-170 [2] and related
>>> issues, such
>>> as: GIRAPH-141 [3].
>>>
>>> Here is an example in Turtle [4] format of a tiny graph using FOAF:
>>> ----
>>> @prefix :<http://example.org/>  .
>>> @prefix foaf:<http://xmlns.com/foaf/0.1/>  .
>>>
>>> :alice
>>>      a           foaf:Person ;
>>>      foaf:name   "Alice" ;
>>>      foaf:mbox<ma...@example.org>  ;
>>>      foaf:knows  :bob ;
>>>      foaf:knows  :charlie ;
>>>      foaf:knows  :snoopy ;
>>>      .
>>>
>>> :bob
>>>      foaf:name   "Bob" ;
>>>      foaf:knows  :charlie ;
>>>      .
>>>
>>> :charlie
>>>      foaf:name   "Charlie" ;
>>>      foaf:knows  :alice ;
>>>      .
>>> ----
>>> This is nice, human friendly (RDF without angle brackets!), but not
>>> easily
>>> splittable to be processed with MapReduce (or Giraph).
>>>
>>> Here is the same graph in N-Triples [5] format:
>>> ----
>>> <http://example.org/alice> 
>>> <http://www.w3.org/1999/02/22-rdf-syntax-ns#type>
>>> <http://xmlns.com/foaf/0.1/Person>  .
>>> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/name>  "Alice" .
>>> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/mbox>
>>> <ma...@example.org>  .
>>> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/knows>
>>> <http://example.org/bob>  .
>>> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/knows>
>>> <http://example.org/charlie>  .
>>> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/knows>
>>> <http://example.org/snoopy>  .
>>> <http://example.org/charlie>  <http://xmlns.com/foaf/0.1/name> 
>>> "Charlie" .
>>> <http://example.org/charlie>  <http://xmlns.com/foaf/0.1/knows>
>>> <http://example.org/alice>  .
>>> <http://example.org/bob>  <http://xmlns.com/foaf/0.1/name>  "Bob" .
>>> <http://example.org/bob>  <http://xmlns.com/foaf/0.1/knows>
>>> <http://example.org/charlie>  .
>>> ----
>>> This is more verbose and ugly, but splittable.
>>>
>>> The graph I am interested in is the graph represented by foaf:knows
>>> relationships/links between people (please, note --knows--> 
>>> relationship here
>>> has a direction, this isn't symmetric as in centralized social networking
>>> websites such as Facebook or LinkedIn. Alice can claim to know Bob,
>>> without Bob
>>> knowing it and/or it might even be a false claim):
>>>
>>> alice --knows-->  bob
>>> alice --knows-->  charlie
>>> alice --knows-->  snoopy
>>> bob --knows-->  charlie
>>> charlie --knows-->  alice
>>>
>>> As a first step, I wrote a MapReduce job [6] to transform the RDF
>>> graph above in
>>> a sort of adjacency list using Turtle syntax, here is the output
>>> (three lines):
>>> ----
>>> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/mbox>
>>> <ma...@example.org>;<http://xmlns.com/foaf/0.1/name>  "Alice";
>>> <http://www.w3.org/1999/02/22-rdf-syntax-ns#type>
>>> <http://xmlns.com/foaf/0.1/Person>;<http://xmlns.com/foaf/0.1/knows>
>>> <http://example.org/charlie>,<http://example.org/bob>,
>>> <http://example.org/snoopy>; .<http://example.org/charlie>
>>> <http://xmlns.com/foaf/0.1/knows>  <http://example.org/alice>.
>>>
>>> <http://example.org/bob>  <http://xmlns.com/foaf/0.1/name>  "Bob";
>>> <http://xmlns.com/foaf/0.1/knows>  <http://example.org/charlie>; .
>>> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/knows>
>>> <http://example.org/bob>.
>>>
>>> <http://example.org/charlie>  <http://xmlns.com/foaf/0.1/name> 
>>> "Charlie";
>>> <http://xmlns.com/foaf/0.1/knows>  <http://example.org/alice>; .
>>> <http://example.org/bob>  <http://xmlns.com/foaf/0.1/knows>
>>> <http://example.org/charlie>.<http://example.org/alice>
>>> <http://xmlns.com/foaf/0.1/knows>  <http://example.org/charlie>.
>>> ----
>>> This is legal Turtle, but it is also splittable. Each line has all the
>>> RDF
>>> statements (i.e. egdes) for a person (there are also incoming edges).
>>>
>>> I wrote a TurtleVertexReader [7] which extends
>>> TextVertexReader<NodeWritable,
>>> Text, NodeWritable, Text>  and a TurtleVertexInputFormat [8] which
>>> extends
>>> TextVertexInputFormat<NodeWritable, Text, NodeWritable, Text>.
>>> I wrote (copying from the example SimpleShortestPathsVertex) a
>>> FoafShortestPathsVertex [9] which extends EdgeListVertex<NodeWritable,
>>> IntWritable, NodeWritable, IntWritable>  and I am running it locally
>>> using these
>>> arguments: -Dgiraph.maxWorkers=1 -Dgiraph.SplitMasterWorker=false
>>> -DoverwriteOutput=true src/test/resources/data3.ttl target/foaf
>>> http://example.org/alice 1
>>>
>>> TurtleVertexReader, TurtleVertexInputFormat and
>>> FoafShortestPathsVertex are
>>> still work in progress and I am sure there are plenty of stupid errors.
>>> However, I do not understand why when I run FoafShortestPathsVertex
>>> with the
>>> DEBUG level, I see debug statements from FoafShortestPathsVertex:
>>> 19:34:44 DEBUG FoafShortestPathsVertex   :: main({-Dgiraph.maxWorkers=1,
>>> -Dgiraph.SplitMasterWorker=false, -DoverwriteOutput=true,
>>> src/test/resources/data3.ttl, target/foaf, http://example.org/alice, 1})
>>> 19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() -->  null
>>> 19:34:44 DEBUG FoafShortestPathsVertex   :: setConf(Configuration:
>>> core-default.xml, core-site.xml)
>>> 19:34:44 DEBUG FoafShortestPathsVertex   ::
>>> run({src/test/resources/data3.ttl,
>>> target/foaf, http://example.org/alice, 1})
>>> 19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() -->  Configuration:
>>> core-default.xml, core-site.xml
>>> 19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() -->  Configuration:
>>> core-default.xml, core-site.xml, mapred-default.xml, mapred-site.xml,
>>> giraph-site.xml
>>>
>>> But, I do not see anything else, no log statement from
>>> TurtleVertexReader or
>>> TurtleVertexInputFormat. Why? What am I doing wrong?
>>> Is it because I am running it locally?
>>>
>>> Thanks,
>>> Paolo
>>>
>>>   [1] http://en.wikipedia.org/wiki/FOAF_%28software%29
>>>   [2] https://issues.apache.org/jira/browse/GIRAPH-170
>>>   [3] https://issues.apache.org/jira/browse/GIRAPH-141
>>>   [4] http://en.wikipedia.org/wiki/Turtle_%28syntax%29
>>>   [5] http://en.wikipedia.org/wiki/N-Triples
>>>   [6]
>>> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/mapreduce/Rdf2AdjacencyListDriver.java
>>>
>>>   [7]
>>> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/TurtleVertexReader.java
>>>
>>>   [8]
>>> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/TurtleVertexInputFormat.java
>>>
>>>   [9]
>>> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/FoafShortestPathsVertex.java
>>>
> 


Re: A simple use case: shortest paths on a FOAF (i.e. Friend of a Friend) graph

Posted by Paolo Castagna <ca...@googlemail.com>.
Hi Avery,
nope, no luck.

I have changed all my log.debug(...) into log.info(...). Same behavior.

I have a log4j.properties [1] file in my classpath and it has:
log4j.logger.org.apache.jena.grande=DEBUG
log4j.logger.org.apache.jena.grande.giraph=DEBUG
I also tried to change that to:
log4j.logger.org.apache.jena.grande=INFO
log4j.logger.org.apache.jena.grande.giraph=INFO
No luck.

My Giraph job has:
GiraphJob job = new GiraphJob(getConf(), getClass().getName());
job.setVertexClass(getClass());
job.setVertexInputFormatClass(TurtleVertexInputFormat.class);
job.setVertexOutputFormatClass(TurtleVertexOutputFormat.class);

But, if I run in debug with a breakpoint in the TurtleVertexInputFormat.class
constructor, it is never instanciated. How can it be?

So perhaps the problem is not the logging, it is the fact that
my GiraphJob is not using TurtleVertexInputFormat.class and
TurtleVertexOutputFormat.class, but I don't see what I am doing
wrong. :-/

Thanks,
Paolo

 [1]
https://github.com/castagna/jena-grande/blob/master/src/test/resources/log4j.properties

Avery Ching wrote:
> I think the issue might be that Hadoop only logs INFO and above messages
> by default.  Can you retry with INFO level logging?
> 
> Avery
> 
> On 4/10/12 12:17 PM, Paolo Castagna wrote:
>> Hi,
>> I am still learning Giraph, so, please, be patient with me and forgive my
>> trivial questions.
>>
>> As a simple initial use case, I want to compute the shortest paths
>> from a single
>> source in a social graph in RDF format using the FOAF [1] vocabulary.
>> This example also will hopefully inform GIRAPH-170 [2] and related
>> issues, such
>> as: GIRAPH-141 [3].
>>
>> Here is an example in Turtle [4] format of a tiny graph using FOAF:
>> ----
>> @prefix :<http://example.org/>  .
>> @prefix foaf:<http://xmlns.com/foaf/0.1/>  .
>>
>> :alice
>>      a           foaf:Person ;
>>      foaf:name   "Alice" ;
>>      foaf:mbox<ma...@example.org>  ;
>>      foaf:knows  :bob ;
>>      foaf:knows  :charlie ;
>>      foaf:knows  :snoopy ;
>>      .
>>
>> :bob
>>      foaf:name   "Bob" ;
>>      foaf:knows  :charlie ;
>>      .
>>
>> :charlie
>>      foaf:name   "Charlie" ;
>>      foaf:knows  :alice ;
>>      .
>> ----
>> This is nice, human friendly (RDF without angle brackets!), but not
>> easily
>> splittable to be processed with MapReduce (or Giraph).
>>
>> Here is the same graph in N-Triples [5] format:
>> ----
>> <http://example.org/alice> 
>> <http://www.w3.org/1999/02/22-rdf-syntax-ns#type>
>> <http://xmlns.com/foaf/0.1/Person>  .
>> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/name>  "Alice" .
>> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/mbox>
>> <ma...@example.org>  .
>> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/knows>
>> <http://example.org/bob>  .
>> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/knows>
>> <http://example.org/charlie>  .
>> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/knows>
>> <http://example.org/snoopy>  .
>> <http://example.org/charlie>  <http://xmlns.com/foaf/0.1/name> 
>> "Charlie" .
>> <http://example.org/charlie>  <http://xmlns.com/foaf/0.1/knows>
>> <http://example.org/alice>  .
>> <http://example.org/bob>  <http://xmlns.com/foaf/0.1/name>  "Bob" .
>> <http://example.org/bob>  <http://xmlns.com/foaf/0.1/knows>
>> <http://example.org/charlie>  .
>> ----
>> This is more verbose and ugly, but splittable.
>>
>> The graph I am interested in is the graph represented by foaf:knows
>> relationships/links between people (please, note --knows--> 
>> relationship here
>> has a direction, this isn't symmetric as in centralized social networking
>> websites such as Facebook or LinkedIn. Alice can claim to know Bob,
>> without Bob
>> knowing it and/or it might even be a false claim):
>>
>> alice --knows-->  bob
>> alice --knows-->  charlie
>> alice --knows-->  snoopy
>> bob --knows-->  charlie
>> charlie --knows-->  alice
>>
>> As a first step, I wrote a MapReduce job [6] to transform the RDF
>> graph above in
>> a sort of adjacency list using Turtle syntax, here is the output
>> (three lines):
>> ----
>> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/mbox>
>> <ma...@example.org>;<http://xmlns.com/foaf/0.1/name>  "Alice";
>> <http://www.w3.org/1999/02/22-rdf-syntax-ns#type>
>> <http://xmlns.com/foaf/0.1/Person>;<http://xmlns.com/foaf/0.1/knows>
>> <http://example.org/charlie>,<http://example.org/bob>,
>> <http://example.org/snoopy>; .<http://example.org/charlie>
>> <http://xmlns.com/foaf/0.1/knows>  <http://example.org/alice>.
>>
>> <http://example.org/bob>  <http://xmlns.com/foaf/0.1/name>  "Bob";
>> <http://xmlns.com/foaf/0.1/knows>  <http://example.org/charlie>; .
>> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/knows>
>> <http://example.org/bob>.
>>
>> <http://example.org/charlie>  <http://xmlns.com/foaf/0.1/name> 
>> "Charlie";
>> <http://xmlns.com/foaf/0.1/knows>  <http://example.org/alice>; .
>> <http://example.org/bob>  <http://xmlns.com/foaf/0.1/knows>
>> <http://example.org/charlie>.<http://example.org/alice>
>> <http://xmlns.com/foaf/0.1/knows>  <http://example.org/charlie>.
>> ----
>> This is legal Turtle, but it is also splittable. Each line has all the
>> RDF
>> statements (i.e. egdes) for a person (there are also incoming edges).
>>
>> I wrote a TurtleVertexReader [7] which extends
>> TextVertexReader<NodeWritable,
>> Text, NodeWritable, Text>  and a TurtleVertexInputFormat [8] which
>> extends
>> TextVertexInputFormat<NodeWritable, Text, NodeWritable, Text>.
>> I wrote (copying from the example SimpleShortestPathsVertex) a
>> FoafShortestPathsVertex [9] which extends EdgeListVertex<NodeWritable,
>> IntWritable, NodeWritable, IntWritable>  and I am running it locally
>> using these
>> arguments: -Dgiraph.maxWorkers=1 -Dgiraph.SplitMasterWorker=false
>> -DoverwriteOutput=true src/test/resources/data3.ttl target/foaf
>> http://example.org/alice 1
>>
>> TurtleVertexReader, TurtleVertexInputFormat and
>> FoafShortestPathsVertex are
>> still work in progress and I am sure there are plenty of stupid errors.
>> However, I do not understand why when I run FoafShortestPathsVertex
>> with the
>> DEBUG level, I see debug statements from FoafShortestPathsVertex:
>> 19:34:44 DEBUG FoafShortestPathsVertex   :: main({-Dgiraph.maxWorkers=1,
>> -Dgiraph.SplitMasterWorker=false, -DoverwriteOutput=true,
>> src/test/resources/data3.ttl, target/foaf, http://example.org/alice, 1})
>> 19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() -->  null
>> 19:34:44 DEBUG FoafShortestPathsVertex   :: setConf(Configuration:
>> core-default.xml, core-site.xml)
>> 19:34:44 DEBUG FoafShortestPathsVertex   ::
>> run({src/test/resources/data3.ttl,
>> target/foaf, http://example.org/alice, 1})
>> 19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() -->  Configuration:
>> core-default.xml, core-site.xml
>> 19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() -->  Configuration:
>> core-default.xml, core-site.xml, mapred-default.xml, mapred-site.xml,
>> giraph-site.xml
>>
>> But, I do not see anything else, no log statement from
>> TurtleVertexReader or
>> TurtleVertexInputFormat. Why? What am I doing wrong?
>> Is it because I am running it locally?
>>
>> Thanks,
>> Paolo
>>
>>   [1] http://en.wikipedia.org/wiki/FOAF_%28software%29
>>   [2] https://issues.apache.org/jira/browse/GIRAPH-170
>>   [3] https://issues.apache.org/jira/browse/GIRAPH-141
>>   [4] http://en.wikipedia.org/wiki/Turtle_%28syntax%29
>>   [5] http://en.wikipedia.org/wiki/N-Triples
>>   [6]
>> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/mapreduce/Rdf2AdjacencyListDriver.java
>>
>>   [7]
>> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/TurtleVertexReader.java
>>
>>   [8]
>> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/TurtleVertexInputFormat.java
>>
>>   [9]
>> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/FoafShortestPathsVertex.java
>>
> 


Re: A simple use case: shortest paths on a FOAF (i.e. Friend of a Friend) graph

Posted by Avery Ching <ac...@apache.org>.
I think the issue might be that Hadoop only logs INFO and above messages 
by default.  Can you retry with INFO level logging?

Avery

On 4/10/12 12:17 PM, Paolo Castagna wrote:
> Hi,
> I am still learning Giraph, so, please, be patient with me and forgive my
> trivial questions.
>
> As a simple initial use case, I want to compute the shortest paths from a single
> source in a social graph in RDF format using the FOAF [1] vocabulary.
> This example also will hopefully inform GIRAPH-170 [2] and related issues, such
> as: GIRAPH-141 [3].
>
> Here is an example in Turtle [4] format of a tiny graph using FOAF:
> ----
> @prefix :<http://example.org/>  .
> @prefix foaf:<http://xmlns.com/foaf/0.1/>  .
>
> :alice
>      a           foaf:Person ;
>      foaf:name   "Alice" ;
>      foaf:mbox<ma...@example.org>  ;
>      foaf:knows  :bob ;
>      foaf:knows  :charlie ;
>      foaf:knows  :snoopy ;
>      .
>
> :bob
>      foaf:name   "Bob" ;
>      foaf:knows  :charlie ;
>      .
>
> :charlie
>      foaf:name   "Charlie" ;
>      foaf:knows  :alice ;
>      .
> ----
> This is nice, human friendly (RDF without angle brackets!), but not easily
> splittable to be processed with MapReduce (or Giraph).
>
> Here is the same graph in N-Triples [5] format:
> ----
> <http://example.org/alice>  <http://www.w3.org/1999/02/22-rdf-syntax-ns#type>
> <http://xmlns.com/foaf/0.1/Person>  .
> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/name>  "Alice" .
> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/mbox>
> <ma...@example.org>  .
> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/knows>
> <http://example.org/bob>  .
> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/knows>
> <http://example.org/charlie>  .
> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/knows>
> <http://example.org/snoopy>  .
> <http://example.org/charlie>  <http://xmlns.com/foaf/0.1/name>  "Charlie" .
> <http://example.org/charlie>  <http://xmlns.com/foaf/0.1/knows>
> <http://example.org/alice>  .
> <http://example.org/bob>  <http://xmlns.com/foaf/0.1/name>  "Bob" .
> <http://example.org/bob>  <http://xmlns.com/foaf/0.1/knows>
> <http://example.org/charlie>  .
> ----
> This is more verbose and ugly, but splittable.
>
> The graph I am interested in is the graph represented by foaf:knows
> relationships/links between people (please, note --knows-->  relationship here
> has a direction, this isn't symmetric as in centralized social networking
> websites such as Facebook or LinkedIn. Alice can claim to know Bob, without Bob
> knowing it and/or it might even be a false claim):
>
> alice --knows-->  bob
> alice --knows-->  charlie
> alice --knows-->  snoopy
> bob --knows-->  charlie
> charlie --knows-->  alice
>
> As a first step, I wrote a MapReduce job [6] to transform the RDF graph above in
> a sort of adjacency list using Turtle syntax, here is the output (three lines):
> ----
> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/mbox>
> <ma...@example.org>;<http://xmlns.com/foaf/0.1/name>  "Alice";
> <http://www.w3.org/1999/02/22-rdf-syntax-ns#type>
> <http://xmlns.com/foaf/0.1/Person>;<http://xmlns.com/foaf/0.1/knows>
> <http://example.org/charlie>,<http://example.org/bob>,
> <http://example.org/snoopy>; .<http://example.org/charlie>
> <http://xmlns.com/foaf/0.1/knows>  <http://example.org/alice>.
>
> <http://example.org/bob>  <http://xmlns.com/foaf/0.1/name>  "Bob";
> <http://xmlns.com/foaf/0.1/knows>  <http://example.org/charlie>; .
> <http://example.org/alice>  <http://xmlns.com/foaf/0.1/knows>
> <http://example.org/bob>.
>
> <http://example.org/charlie>  <http://xmlns.com/foaf/0.1/name>  "Charlie";
> <http://xmlns.com/foaf/0.1/knows>  <http://example.org/alice>; .
> <http://example.org/bob>  <http://xmlns.com/foaf/0.1/knows>
> <http://example.org/charlie>.<http://example.org/alice>
> <http://xmlns.com/foaf/0.1/knows>  <http://example.org/charlie>.
> ----
> This is legal Turtle, but it is also splittable. Each line has all the RDF
> statements (i.e. egdes) for a person (there are also incoming edges).
>
> I wrote a TurtleVertexReader [7] which extends TextVertexReader<NodeWritable,
> Text, NodeWritable, Text>  and a TurtleVertexInputFormat [8] which extends
> TextVertexInputFormat<NodeWritable, Text, NodeWritable, Text>.
> I wrote (copying from the example SimpleShortestPathsVertex) a
> FoafShortestPathsVertex [9] which extends EdgeListVertex<NodeWritable,
> IntWritable, NodeWritable, IntWritable>  and I am running it locally using these
> arguments: -Dgiraph.maxWorkers=1 -Dgiraph.SplitMasterWorker=false
> -DoverwriteOutput=true src/test/resources/data3.ttl target/foaf
> http://example.org/alice 1
>
> TurtleVertexReader, TurtleVertexInputFormat and FoafShortestPathsVertex are
> still work in progress and I am sure there are plenty of stupid errors.
> However, I do not understand why when I run FoafShortestPathsVertex with the
> DEBUG level, I see debug statements from FoafShortestPathsVertex:
> 19:34:44 DEBUG FoafShortestPathsVertex   :: main({-Dgiraph.maxWorkers=1,
> -Dgiraph.SplitMasterWorker=false, -DoverwriteOutput=true,
> src/test/resources/data3.ttl, target/foaf, http://example.org/alice, 1})
> 19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() -->  null
> 19:34:44 DEBUG FoafShortestPathsVertex   :: setConf(Configuration:
> core-default.xml, core-site.xml)
> 19:34:44 DEBUG FoafShortestPathsVertex   :: run({src/test/resources/data3.ttl,
> target/foaf, http://example.org/alice, 1})
> 19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() -->  Configuration:
> core-default.xml, core-site.xml
> 19:34:44 DEBUG FoafShortestPathsVertex   :: getConf() -->  Configuration:
> core-default.xml, core-site.xml, mapred-default.xml, mapred-site.xml,
> giraph-site.xml
>
> But, I do not see anything else, no log statement from TurtleVertexReader or
> TurtleVertexInputFormat. Why? What am I doing wrong?
> Is it because I am running it locally?
>
> Thanks,
> Paolo
>
>   [1] http://en.wikipedia.org/wiki/FOAF_%28software%29
>   [2] https://issues.apache.org/jira/browse/GIRAPH-170
>   [3] https://issues.apache.org/jira/browse/GIRAPH-141
>   [4] http://en.wikipedia.org/wiki/Turtle_%28syntax%29
>   [5] http://en.wikipedia.org/wiki/N-Triples
>   [6]
> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/mapreduce/Rdf2AdjacencyListDriver.java
>   [7]
> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/TurtleVertexReader.java
>   [8]
> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/TurtleVertexInputFormat.java
>   [9]
> https://github.com/castagna/jena-grande/blob/a650758a56cfe0680320445434e6d6adf2d7e544/src/main/java/org/apache/jena/grande/giraph/FoafShortestPathsVertex.java