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

2012-04-13 Thread Paolo Castagna
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

2012-04-13 Thread Paolo Castagna
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:

  IterableString 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

2012-04-13 Thread Paolo Castagna
Paolo Castagna wrote:
 This is a better way:
 
   IterableString 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

2012-04-13 Thread Avery Ching

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

Re: java.lang.RuntimeException [...] msgMap did not exist [...]

2012-04-13 Thread Avery Ching

Hi Etienne,

Thanks for your questions.  Giraph uses map tasks to run its master and 
workers.  Can you provide the task output logs?  It looks like your 
workers failed to report status for some reason and we need to find out 
why.  The datanode logs can't help us here.


Avery

On 4/13/12 3:35 AM, Etienne Dumoulin wrote:

Hi Guys,

I tried out giraph yesterday and I have an issue to run the shortest 
path example.


I am working on a toy heterogeneous cluster of 3 datanodes and 1 
namenode, jobtracker, with hadoop 0.20.203.0.
One of the datanode is a small server quad-core 16 GB ram, the others 
are small PC 1 core 1GB ram, same OS: ubuntu-server 10.04.


I run on a first issue with the 0.1 version, the same described here: 
https://issues.apache.org/jira/browse/GIRAPH-114.

Before I found the patch I tried different configurations:
It works on a standalone environment, with the namenode and the 
server, with the namenode and the two small PC.
It does not work either with the entire cluster, or with one small PC 
and the server as datanode.


Then I downloaded today the svn version, no luck, it has the same 
behaviour than the 0.1 version (go till 100% then go back to 0%) but 
not the same info logs.
Bellow the svn version console log, nantes is the name of the big 
datanode, rennes the namenode/jobtracker:


hadoop@rennes:~/test$ hadoop jar 
~/project/giraph/trunk_2012_04_13/target/giraph-0.2-SNAPSHOT-jar-with-dependencies.jar 
org.apache.giraph.examples.SimpleShortestPathsVertex 
shortestPathsInputGraph shortestPathsOutputGraph 0 3
12/04/13 10:05:58 INFO mapred.JobClient: Running job: 
job_201204121836_0003

12/04/13 10:05:59 INFO mapred.JobClient:  map 0% reduce 0%
12/04/13 10:06:18 INFO mapred.JobClient:  map 25% reduce 0%
12/04/13 10:08:55 INFO mapred.JobClient:  map 100% reduce 0%
12/04/13 10:21:28 INFO mapred.JobClient:  map 75% reduce 0%
12/04/13 10:21:33 INFO mapred.JobClient: Task Id : 
attempt_201204121836_0003_m_02_0, Status : FAILED
Task attempt_201204121836_0003_m_02_0 failed to report status for 
600 seconds. Killing!
12/04/13 10:23:57 INFO mapred.JobClient: Task Id : 
attempt_201204121836_0003_m_01_0, Status : FAILED
java.lang.RuntimeException: sendMessage: msgMap did not exist for 
nantes:30002 for vertex 2
at 
org.apache.giraph.comm.BasicRPCCommunications.sendMessageReq(BasicRPCCommunications.java:993)
at 
org.apache.giraph.graph.BasicVertex.sendMsg(BasicVertex.java:168)
at 
org.apache.giraph.examples.SimpleShortestPathsVertex.compute(SimpleShortestPathsVertex.java:104)

at org.apache.giraph.graph.GraphMapper.map(GraphMapper.java:593)
at org.apache.giraph.graph.GraphMapper.run(GraphMapper.java:648)
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.Child$4.run(Child.java:259)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:396)
at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1059)

at org.apache.hadoop.mapred.Child.main(Child.java:253)

Task attempt_201204121836_0003_m_01_0 failed to report status for 
601 seconds. Killing!

12/04/13 10:23:58 INFO mapred.JobClient:  map 50% reduce 0%
12/04/13 10:24:01 INFO mapred.JobClient:  map 25% reduce 0%
12/04/13 10:24:06 INFO mapred.JobClient: Task Id : 
attempt_201204121836_0003_m_03_0, Status : FAILED
Task attempt_201204121836_0003_m_03_0 failed to report status for 
602 seconds. Killing!


I attached the hadoop logs for rennes namenode and jobtraker and for 
nantes the big datanode.


Is someone already got this error/found a fix?

Thanks for your time,

Étienne