giraph-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Avery Ching <ach...@apache.org>
Subject Re: Benchmark runs but tests fail
Date Mon, 28 Nov 2011 06:25:21 GMT
Inlining the response.  Sorry for the delay, been out a lot of today.

Avery

On 11/26/11 9:32 PM, Oana Theogarajan wrote:
>     Hi Avery,
> thanks for the quick response.
> About the unittests:
>
> I was indeed specifying the wrong host:port
>  1) the LocalJobRunner test (mvn test) works
>  2) The test against the actual Hadoop instance (mvn test 
> -Dprop.mapred.job.tracker=hdfs://ip-10-202-59-170.ec2.internal:50002) 
> fails - they do execute, they assign maps etc, but the tests failed. 
> The output is attached in the Testlogs.txt file. I am also attaching 
> the job logs in case there is more info there that might be helpful to 
> you.
>
> About the PageRankBenchmark - I run the following command:
> hadoop jar giraph-0.70-jar-with-dependencies.jar 
> org.apache.giraph.benchmark.PageRankBenchmark -e 1 -s 10 -v -V 1000000 
> -w 8
>
> It works fine - attached is the master task log for the successful 
> case (mastersuccess.txt).
> Then I test it by killing a tasktracker (I make sure it's not the one 
> that runs the master-zookeeper task, I also make sure I'm passed 
> superstep 2 so I can have a valid checkpoint). I am attaching the 
> master task log as masterFailed.txt
>
> Looks like the master is trying to start again from the last 
> checkpoint, but it's waiting to have 8 running map tasks, which 
> doesn't happen after I killed 2 of them.
> ("This occurs if you do not have enough map tasks available 
> simultaneously on your Hadoop instance to fulfill the number of 
> requested workers.")
>  I was thinking the master would start more maps if it finds that some 
> died. It looks like the master kills himself ? 

The way that Giraph works is by waiting until some minimum number of 
workers are available.  If that minimum is not met or some percent of 
the workers do not respond in time, them master will die and the job 
will fail.  So if you only have 8 map slots on the whole Hadoop instance 
and you permanently remove some, but the job is waiting for 8 maps to 
simultaneously be running, the job will fail.  Since everything is in 
memory, the user is expected to choose a reasonable minimum and maximum 
number of workers that make sense for their application.

> and then a bunch of other maps get started trying to recover- in the 
> end 32 tasks get launched (4 attempts for each one I'm assuming - 4 is 
> the default map.max.attempts., I didn't change it). The number of 
> simultaneously running maps is always less than 8 though - not sure 
> why, but the job would need to have 8 running simultaneously in order 
> to recover. Does it have anything to do with the fact that the number 
> of workers is fixed - from the source code looks like the 
> PageRankBenchmark effectively sets the minWorker and maxWorker to the 
> number specified at the command line? I'm just making un-educated 
> guesses at this point.
>
You are right =), good guess.  There is one unit test that checks 
whether the automatic checkpoint restart works (only when run against a 
real Hadoop instance).  See 
src/test/java/org/apache/giraph/TestAutoCheckpoint.java.  It fails a 
worker and then recovers from a previous checkpoint.

> Hopefully the logs give you some useful info. Let me know if you have 
> any questions about them or you need more info. I'm hoping it's 
> something relevant rather than something stupid I might be doing....
>

The master log appears to confirm what you suspected:

2011-11-27 03:34:54,073 INFO org.apache.giraph.graph.BspServiceMaster: 
setJobState: 
{"_stateKey":"START_SUPERSTEP","_applicationAttemptKey":1,"_superstepKey":2} 
on superstep 2
2011-11-27 03:35:34,932 INFO org.apache.giraph.graph.BspServiceMaster: 
checkWorkers: Only found 6 responses of 8 needed to start superstep 2.  
Sleeping for 30000 msecs and used 0 of 10 attempts.
2011-11-27 03:36:04,941 INFO org.apache.giraph.graph.BspServiceMaster: 
checkWorkers: Only found 6 responses of 8 needed to start superstep 2.  
Sleeping for 30000 msecs and used 1 of 10 attempts.
2011-11-27 03:36:34,951 INFO org.apache.giraph.graph.BspServiceMaster: 
checkWorkers: Only found 6 responses of 8 needed to start superstep 2.  
Sleeping for 30000 msecs and used 2 of 10 attempts.
2011-11-27 03:37:04,962 INFO org.apache.giraph.graph.BspServiceMaster: 
checkWorkers: Only found 6 responses of 8 needed to start superstep 2.  
Sleeping for 30000 msecs and used 3 of 10 attempts.
2011-11-27 03:37:34,971 INFO org.apache.giraph.graph.BspServiceMaster: 
checkWorkers: Only found 6 responses of 8 needed to start superstep 2.  
Sleeping for 30000 msecs and used 4 of 10 attempts.
2011-11-27 03:38:04,982 INFO org.apache.giraph.graph.BspServiceMaster: 
checkWorkers: Only found 6 responses of 8 needed to start superstep 2.  
Sleeping for 30000 msecs and used 5 of 10 attempts.
2011-11-27 03:38:34,991 INFO org.apache.giraph.graph.BspServiceMaster: 
checkWorkers: Only found 6 responses of 8 needed to start superstep 2.  
Sleeping for 30000 msecs and used 6 of 10 attempts.
2011-11-27 03:39:05,002 INFO org.apache.giraph.graph.BspServiceMaster: 
checkWorkers: Only found 6 responses of 8 needed to start superstep 2.  
Sleeping for 30000 msecs and used 7 of 10 attempts.
2011-11-27 03:39:35,011 INFO org.apache.giraph.graph.BspServiceMaster: 
checkWorkers: Only found 6 responses of 8 needed to start superstep 2.  
Sleeping for 30000 msecs and used 8 of 10 attempts.
2011-11-27 03:40:05,051 INFO org.apache.giraph.graph.BspServiceMaster: 
checkWorkers: Only found 6 responses of 8 needed to start superstep 2.  
Sleeping for 30000 msecs and used 9 of 10 attempts.
2011-11-27 03:40:05,052 ERROR org.apache.giraph.graph.BspServiceMaster: 
checkWorkers: Did not receive enough processes in time (only 6 of 8 
required).  This occurs if you do not have enough map tasks available 
simultaneously on your Hadoop instance to fulfill the number of 
requested workers.

> Thanks again,
>     Oana
>
>
> On 11/26/11 2:41 PM, Avery Ching wrote:
>> Hi Oana,
>>
>> Thanks for your questions.  The fault tolerance should work if there 
>> is a viable checkpoint and there is a master and ZooKeeper process 
>> available to coordinate the application.  The only reason I believe 
>> that the fault tolerance won't work is if the number of task failures 
>> is exceeded (Hadoop configurable variable - map.max.attempts).  Can 
>> you show me the log of the master task?  It would be really helpful.
>>
>> As far as the unittests failing, do you actually have a Hadoop 
>> instance running at localhost:50030?  The unittests can be run two 
>> different ways:
>>
>> - Against an actual Hadoop instance (i.e. mvn test 
>> -Dprop.mapred.job.tracker=<jobtracker hostname>:<jobtracker port>)
>>
>> - Using something called LocalJobRunner that simulates a Hadoop 
>> instance with a single map task at a time (i.e mvn test).
>>
>> Hope that helps, let me know if you have other questions.
>>
>> Avery
>>
>> On 11/26/11 3:09 PM, Oana Theogarajan wrote:
>>> Hi,
>>> I've been testing Giraph on a hadoop custer set up on Amazon EC2 and 
>>> I encounter some issues. I can successfully run the 
>>> PageRankBenchmark, however if I am trying to test the fault 
>>> tolerance by killing a tasktracker the job eventually dies after 
>>> trying repeatedly. I have checkpoints enabled (the default every 2 
>>> supersteps - and I can see them written in the checkpointing directory)
>>> I then tried to run the unit tests using
>>> mvn test -Dprop.mapred.job.tracker=localhost:50030
>>> and a lot of them fail. The output is quoted below. The surefire 
>>> logs show the following error. I am pretty new to both hadoop and 
>>> Giraph and I can't tell what could cause this error. I am puzzled 
>>> since can run Giraph PageRankBenchmark jobs but the tests fail.
>>>
>>> Thanks in advance for your help figuring this out.
>>> Best,
>>>     Oana
>>>
>>> Tests run: 9, Failures: 0, Errors: 7, Skipped: 0, Time elapsed: 0.5 
>>> sec <<< FAILURE!
>>> testBspFail(org.apache.giraph.TestBspBasic)  Time elapsed: 0.054 sec 
>>> <<< ERROR!
>>> java.io.IOException: Call to localhost/127.0.0.1:50030 failed on 
>>> local exception: java.io.EOFException
>>>     at org.apache.hadoop.ipc.Client.wrapException(Client.java:1065)
>>>     at org.apache.hadoop.ipc.Client.call(Client.java:1033)
>>>     at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:224)
>>>     at org.apache.hadoop.mapred.$Proxy2.getProtocolVersion(Unknown 
>>> Source)
>>>     at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:364)
>>>     at 
>>> org.apache.hadoop.mapred.JobClient.createRPCProxy(JobClient.java:460)
>>>     at org.apache.hadoop.mapred.JobClient.init(JobClient.java:454)
>>>     at org.apache.hadoop.mapred.JobClient.<init>(JobClient.java:437)
>>>     at org.apache.hadoop.mapreduce.Job$1.run(Job.java:477)
>>>     at java.security.AccessController.doPrivileged(Native Method)
>>>     at javax.security.auth.Subject.doAs(Subject.java:416)
>>>     at 
>>> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1059)
>>>     at org.apache.hadoop.mapreduce.Job.connect(Job.java:475)
>>>     at org.apache.hadoop.mapreduce.Job.submit(Job.java:464)
>>>     at org.apache.hadoop.mapreduce.Job.waitForCompletion(Job.java:494)
>>>     at org.apache.giraph.graph.GiraphJob.run(GiraphJob.java:524)
>>>     at 
>>> org.apache.giraph.TestBspBasic.testBspFail(TestBspBasic.java:180)
>>> Caused by: java.io.EOFException
>>>     at java.io.DataInputStream.readInt(DataInputStream.java:392)
>>>     at 
>>> org.apache.hadoop.ipc.Client$Connection.receiveResponse(Client.java:774) 
>>>
>>>     at org.apache.hadoop.ipc.Client$Connection.run(Client.java:712)
>>>
>>>
>>> -------------------------------------------------------
>>>  T E S T S
>>> -------------------------------------------------------
>>> Running org.apache.giraph.TestManualCheckpoint
>>> Setting tasks to 3 for testBspCheckpoint since JobTracker exists...
>>> setup: Sending job to job tracker localhost:50030 with jar path 
>>> target/giraph-0.70-jar-with-dependencies.jar for testBspCheckpoint
>>> Tests run: 1, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 
>>> 0.664 sec <<< FAILURE!
>>> Running org.apache.giraph.TestAutoCheckpoint
>>> Setting tasks to 3 for testSingleFault since JobTracker exists...
>>> setup: Sending job to job tracker localhost:50030 with jar path 
>>> target/giraph-0.70-jar-with-dependencies.jar for testSingleFault
>>> Tests run: 1, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 
>>> 0.069 sec <<< FAILURE!
>>> Running org.apache.giraph.TestBspBasic
>>> Setting tasks to 3 for testInstantiateVertex since JobTracker exists...
>>> testInstantiateVertex: 
>>> java.class.path=/home/ubuntu/giraph/trunk/target/test-classes:/home/ubuntu/giraph/trunk/target/classes:/home/ubuntu/.m2/repository/junit/junit/3.8.1/junit-3.8.1.jar:/home/ubuntu/.m2/repository/org/apache/hadoop/hadoop-core/0.20.203.0/hadoop-core-0.20.203.0.jar:/home/ubuntu/.m2/repository/xmlenc/xmlenc/0.52/xmlenc-0.52.jar:/home/ubuntu/.m2/repository/commons-httpclient/commons-httpclient/3.0.1/commons-httpclient-3.0.1.jar:/home/ubuntu/.m2/repository/commons-logging/commons-logging/1.0.3/commons-logging-1.0.3.jar:/home/ubuntu/.m2/repository/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/home/ubuntu/.m2/repository/org/apache/commons/commons-math/2.1/commons-math-2.1.jar:/home/ubuntu/.m2/repository/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/home/ubuntu/.m2/repository/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1.jar:/home/ubuntu/.m2/repository/commons-lang/commons-lang/2.4/commons-lang-2.4.jar:/home/ubuntu/.m2/repository/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/home/ubuntu/.m2/repository/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/home/ubuntu/.m2/repository/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar:/home/ubuntu/.m2/repository/commons-net/commons-net/1.4.1/commons-net-1.4.1.jar:/home/ubuntu/.m2/repository/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/home/ubuntu/.m2/repository/org/mortbay/jetty/servlet-api/2.5-20081211/servlet-api-2.5-20081211.jar:/home/ubuntu/.m2/repository/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/home/ubuntu/.m2/repository/tomcat/jasper-runtime/5.5.12/jasper-runtime-5.5.12.jar:/home/ubuntu/.m2/repository/tomcat/jasper-compiler/5.5.12/jasper-compiler-5.5.12.jar:/home/ubuntu/.m2/repository/org/mortbay/jetty/jsp-api-2.1/6.1.14/jsp-api-2.1-6.1.14.jar:/home/ubuntu/.m2/repository/org/mortbay/jetty/servlet-api-2.5/6.1.14/servlet-api-2.5-6.1.14.jar:/home/ubuntu/.m2/repository/org/mortbay/jetty/jsp-2.1/6.1.14/jsp-2.1-6.1.14.jar:/home/ubuntu/.m2/repository/ant/ant/1.6.5/ant-1.6.5.jar:/home/ubuntu/.m2/repository/commons-el/commons-el/1.0/commons-el-1.0.jar:/home/ubuntu/.m2/repository/net/java/dev/jets3t/jets3t/0.7.1/jets3t-0.7.1.jar:/home/ubuntu/.m2/repository/net/sf/kosmosfs/kfs/0.3/kfs-0.3.jar:/home/ubuntu/.m2/repository/hsqldb/hsqldb/1.8.0.10/hsqldb-1.8.0.10.jar:/home/ubuntu/.m2/repository/oro/oro/2.0.8/oro-2.0.8.jar:/home/ubuntu/.m2/repository/org/eclipse/jdt/core/3.1.1/core-3.1.1.jar:/home/ubuntu/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.8.0/jackson-core-asl-1.8.0.jar:/home/ubuntu/.m2/repository/org/apache/mahout/mahout-collections/1.0/mahout-collections-1.0.jar:/home/ubuntu/.m2/repository/com/google/guava/guava/r09/guava-r09.jar:/home/ubuntu/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.8.0/jackson-mapper-asl-1.8.0.jar:/home/ubuntu/.m2/repository/org/apache/zookeeper/zookeeper/3.3.3/zookeeper-3.3.3.jar:/home/ubuntu/.m2/repository/log4j/log4j/1.2.15/log4j-1.2.15.jar:/home/ubuntu/.m2/repository/javax/mail/mail/1.4/mail-1.4.jar:/home/ubuntu/.m2/repository/javax/activation/activation/1.1/activation-1.1.jar:/home/ubuntu/.m2/repository/jline/jline/0.9.94/jline-0.9.94.jar:/home/ubuntu/.m2/repository/org/apache/commons/commons-io/1.3.2/commons-io-1.3.2.jar:/home/ubuntu/.m2/repository/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/home/ubuntu/.m2/repository/net/iharder/base64/2.3.8/base64-2.3.8.jar:/home/ubuntu/.m2/repository/org/json/json/20090211/json-20090211.jar:/home/ubuntu/.m2/repository/org/mockito/mockito-all/1.8.5/mockito-all-1.8.5.jar:
>>> testInstantiateVertex: Got vertex 
>>> Vertex(id=null,value=null,#edges=0), 
>>> graphStateorg.apache.giraph.graph.GraphState@877ef83
>>> testInstantiateVertex: Example output split =
>>> Setting tasks to 3 for testLocalJobRunnerConfig since JobTracker 
>>> exists...
>>> testLocalJobRunnerConfig: Skipping for non-local
>>> Setting tasks to 3 for testBspFail since JobTracker exists...
>>> setup: Sending job to job tracker localhost:50030 with jar path 
>>> target/giraph-0.70-jar-with-dependencies.jar for testBspFail
>>> Setting tasks to 3 for testBspSuperStep since JobTracker exists...
>>> setup: Sending job to job tracker localhost:50030 with jar path 
>>> target/giraph-0.70-jar-with-dependencies.jar for testBspSuperStep
>>> Setting tasks to 3 for testBspMsg since JobTracker exists...
>>> setup: Sending job to job tracker localhost:50030 with jar path 
>>> target/giraph-0.70-jar-with-dependencies.jar for testBspMsg
>>> Setting tasks to 3 for testEmptyVertexInputFormat since JobTracker 
>>> exists...
>>> setup: Sending job to job tracker localhost:50030 with jar path 
>>> target/giraph-0.70-jar-with-dependencies.jar for 
>>> testEmptyVertexInputFormat
>>> Setting tasks to 3 for testBspCombiner since JobTracker exists...
>>> setup: Sending job to job tracker localhost:50030 with jar path 
>>> target/giraph-0.70-jar-with-dependencies.jar for testBspCombiner
>>> Setting tasks to 3 for testBspPageRank since JobTracker exists...
>>> setup: Sending job to job tracker localhost:50030 with jar path 
>>> target/giraph-0.70-jar-with-dependencies.jar for testBspPageRank
>>> Setting tasks to 3 for testBspShortestPaths since JobTracker exists...
>>> setup: Sending job to job tracker localhost:50030 with jar path 
>>> target/giraph-0.70-jar-with-dependencies.jar for testBspShortestPaths
>>> Tests run: 9, Failures: 0, Errors: 7, Skipped: 0, Time elapsed: 
>>> 0.501 sec <<< FAILURE!
>>> Running 
>>> org.apache.giraph.lib.TestTextDoubleDoubleAdjacencyListVertexInputFormat 
>>>
>>> Tests run: 5, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 
>>> 0.321 sec
>>> Running org.apache.giraph.TestGraphPartitioner
>>> Setting tasks to 3 for testPartitioners since JobTracker exists...
>>> setup: Sending job to job tracker localhost:50030 with jar path 
>>> target/giraph-0.70-jar-with-dependencies.jar for testPartitioners
>>> Tests run: 1, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 
>>> 0.025 sec <<< FAILURE!
>>> Running org.apache.giraph.TestVertexTypes
>>> 11/11/26 21:29:55 WARN graph.GraphMapper: Input format vertex index 
>>> type is not known
>>> 11/11/26 21:29:55 WARN graph.GraphMapper: Input format vertex value 
>>> type is not known
>>> 11/11/26 21:29:55 WARN graph.GraphMapper: Input format edge value 
>>> type is not known
>>> 11/11/26 21:29:55 WARN graph.GraphMapper: Output format vertex index 
>>> type is not known
>>> 11/11/26 21:29:55 WARN graph.GraphMapper: Output format vertex value 
>>> type is not known
>>> 11/11/26 21:29:55 WARN graph.GraphMapper: Output format edge value 
>>> type is not known
>>> Tests run: 6, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 
>>> 0.062 sec
>>> Running 
>>> org.apache.giraph.lib.TestLongDoubleDoubleAdjacencyListVertexInputFormat 
>>>
>>> Tests run: 4, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 
>>> 0.043 sec
>>> Running org.apache.giraph.TestZooKeeperExt
>>> testCreateExt: No prop.zookeeper.list set, skipping test
>>> testDeleteExt: No prop.zookeeper.list set, skipping test
>>> testGetChildrenExt: No prop.zookeeper.list set, skipping test
>>> Tests run: 3, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 
>>> 0.004 sec
>>> Running org.apache.giraph.lib.TestAdjacencyListTextVertexOutputFormat
>>> Tests run: 3, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 
>>> 0.082 sec
>>> Running org.apache.giraph.TestNotEnoughMapTasks
>>> Setting tasks to 3 for testNotEnoughMapTasks since JobTracker exists...
>>> setup: Sending job to job tracker localhost:50030 with jar path 
>>> target/giraph-0.70-jar-with-dependencies.jar for testNotEnoughMapTasks
>>> Tests run: 1, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 
>>> 0.028 sec <<< FAILURE!
>>> Running org.apache.giraph.TestMutateGraphVertex
>>> Setting tasks to 3 for testMutateGraph since JobTracker exists...
>>> setup: Sending job to job tracker localhost:50030 with jar path 
>>> target/giraph-0.70-jar-with-dependencies.jar for testMutateGraph
>>> Tests run: 1, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 
>>> 0.031 sec <<< FAILURE!
>>> Running org.apache.giraph.TestJsonBase64Format
>>> Setting tasks to 3 for testContinue since JobTracker exists...
>>> setup: Sending job to job tracker localhost:50030 with jar path 
>>> target/giraph-0.70-jar-with-dependencies.jar for testContinue
>>> Tests run: 1, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 
>>> 0.016 sec <<< FAILURE!
>>> Running org.apache.giraph.TestPredicateLock
>>> testWaitMsecs:
>>> Tests run: 3, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 
>>> 0.576 sec
>>>
>>> Results :
>>>
>>> Tests in error:
>>>   testBspCheckpoint(org.apache.giraph.TestManualCheckpoint)
>>>   testSingleFault(org.apache.giraph.TestAutoCheckpoint)
>>>   testBspFail(org.apache.giraph.TestBspBasic)
>>>   testBspSuperStep(org.apache.giraph.TestBspBasic)
>>>   testBspMsg(org.apache.giraph.TestBspBasic)
>>>   testEmptyVertexInputFormat(org.apache.giraph.TestBspBasic)
>>>   testBspCombiner(org.apache.giraph.TestBspBasic)
>>>   testBspPageRank(org.apache.giraph.TestBspBasic)
>>>   testBspShortestPaths(org.apache.giraph.TestBspBasic)
>>>   testPartitioners(org.apache.giraph.TestGraphPartitioner)
>>>   testNotEnoughMapTasks(org.apache.giraph.TestNotEnoughMapTasks)
>>>   testMutateGraph(org.apache.giraph.TestMutateGraphVertex)
>>>   testContinue(org.apache.giraph.TestJsonBase64Format)
>>>
>>> Tests run: 39, Failures: 0, Errors: 13, Skipped: 0
>>>
>>> [INFO] 
>>> ------------------------------------------------------------------------ 
>>>
>>> [INFO] BUILD FAILURE
>>> [INFO] 
>>> ------------------------------------------------------------------------ 
>>>
>>> [INFO] Total time: 11.267s
>>> [INFO] Finished at: Sat Nov 26 21:29:55 UTC 2011
>>> [INFO] Final Memory: 11M/324M
>>> [INFO] 
>>> ------------------------------------------------------------------------ 
>>>
>>> [ERROR] Failed to execute goal 
>>> org.apache.maven.plugins:maven-surefire-plugin:2.6:test 
>>> (default-test) on project giraph: There are test failures.
>>> [ERROR]
>>>
>>
>>



Mime
View raw message