Return-Path: X-Original-To: apmail-giraph-dev-archive@www.apache.org Delivered-To: apmail-giraph-dev-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 30468105AF for ; Sat, 19 Apr 2014 18:28:20 +0000 (UTC) Received: (qmail 71891 invoked by uid 500); 19 Apr 2014 18:28:18 -0000 Delivered-To: apmail-giraph-dev-archive@giraph.apache.org Received: (qmail 71770 invoked by uid 500); 19 Apr 2014 18:28:18 -0000 Mailing-List: contact dev-help@giraph.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@giraph.apache.org Delivered-To: mailing list dev@giraph.apache.org Received: (qmail 71667 invoked by uid 500); 19 Apr 2014 18:28:16 -0000 Delivered-To: apmail-incubator-giraph-dev@incubator.apache.org Received: (qmail 71581 invoked by uid 99); 19 Apr 2014 18:28:15 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 19 Apr 2014 18:28:15 +0000 Date: Sat, 19 Apr 2014 18:28:15 +0000 (UTC) From: "Eli Reisman (JIRA)" To: giraph-dev@incubator.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (GIRAPH-811) Infinite ZooKeeper CleanUp MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/GIRAPH-811?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13974922#comment-13974922 ] Eli Reisman commented on GIRAPH-811: ------------------------------------ This is a good solution, are you certain this issue is fixed, I think since the transition to Hadoop 2.2.0 YARN support I have seen some more recent reports of this problem? This (rather than the >= solution) seems like the right fix if we still need one. [~aching] ^ if people are still reporting cleanup problems due to the "extra" master task in Giraph-on-YARN runs, I'd take a look at this patch or a variation of it. > Infinite ZooKeeper CleanUp > -------------------------- > > Key: GIRAPH-811 > URL: https://issues.apache.org/jira/browse/GIRAPH-811 > Project: Giraph > Issue Type: Bug > Components: bsp, zookeeper > Affects Versions: 1.1.0 > Reporter: Alexandre Fonseca > Labels: yarn > Attachments: GIRAPH-811.patch > > > While executing the SimpleShortestPaths example with Giraph 1.1.0-SNAPSHOT compiled for Hadoop Yarn 2.2.0, I've noticed that the application would never stop even after recognizing that all supersteps had completed and the output had been written to the output directory. > Looking at the logs, I found that the BspServiceMaster is stuck at the while loop at the end of cleanrUpZooKeeper() (BspServiceMaster.java:1729): > {code}2013-12-08 03:51:21,698 INFO [org.apache.giraph.master.MasterThread] master.MasterThread (MasterThread.java:run(121)) - masterThread: Coordination of superstep 3 took 0.433 seconds ended with state ALL_SUPERSTEPS_DONE and is now on superstep 4 > 2013-12-08 03:51:21,699 INFO [org.apache.giraph.master.MasterThread] master.BspServiceMaster (BspServiceMaster.java:setJobState(261)) - setJobState: {"_stateKey":"FINISHED","_applicationAttemptKey":-1,"_superstepKey":-1} on superstep 4 > 2013-12-08 03:51:21,753 INFO [org.apache.giraph.master.MasterThread] master.BspServiceMaster (BspServiceMaster.java:cleanup(1836)) - cleanup: Notifying master its okay to cleanup with /_hadoopBsp/giraph_yarn_application_1386468390622_0005/_cleanedUpDir/0_master > 2013-12-08 03:51:21,790 INFO [org.apache.giraph.master.MasterThread] master.BspServiceMaster (BspServiceMaster.java:cleanUpZooKeeper(1711)) - cleanUpZooKeeper: Node /_hadoopBsp/giraph_yarn_application_1386468390622_0005/_cleanedUpDir already exists, no need to create. > 2013-12-08 03:51:21,792 INFO [org.apache.giraph.master.MasterThread] bsp.BspInputFormat (BspInputFormat.java:getMaxTasks(64)) - getMaxTasks: Max workers = 1, split master/worker = true, is YARN-only job = true, total max tasks = 1 > 2013-12-08 03:51:21,792 INFO [org.apache.giraph.master.MasterThread] master.BspServiceMaster (BspServiceMaster.java:cleanUpZooKeeper(1735)) - cleanUpZooKeeper: Got 2 of 1 desired children from /_hadoopBsp/giraph_yarn_application_1386468390622_0005/_cleanedUpDir > 2013-12-08 03:51:21,793 INFO [org.apache.giraph.master.MasterThread] master.BspServiceMaster (BspServiceMaster.java:cleanUpZooKeeper(1744)) - cleanedUpZooKeeper: Waiting for the children of /_hadoopBsp/giraph_yarn_application_1386468390622_0005/_cleanedUpDir to change since only got 2 nodes.{code} > As the last 2 entries show, instead of registering just 1 task ending, it registers 2 and thus it misses the condition on line 1740. > One solution would be to change the == in line 1740 to a >=. However, the actual issue seems to reside with the BspInputFormat.getMaxTasks() (BspInputFormat.java:51). This function assumes that in a pure yarn execution the total number of tasks will be equal to the maximum number of workers. However, based on GiraphApplicationMaster:167, this is not the case. An extra Master task is launched in addition to all the Worker tasks. BspInputFormat.getMaxTasks() should then return maxWorkers + 1 in the case of a pure yarn execution. > Compilation: > {code}mvn -Phadoop_yarn -Dhadoop.version=2.2.0 -DskipTests compile{code} > Execution command: > {code}$HADOOP_PREFIX/bin/hadoop jar ~/Projects/giraph/giraph-examples/target/giraph-examples-1.1.0-SNAPSHOT-for-hadoop-2.2.0-jar-with-dependencies.jar org.apache.giraph.GiraphRunner org.apache.giraph.examples.SimpleShortestPathsComputation -vif org.apache.giraph.io.formats.JsonLongDoubleFloatDoubleVertexInputFormat -vip giraph/input/tiny_graph.txt -vof org.apache.giraph.io.formats.IdWithValueTextOutputFormat -op giraph/output/shortestpahts -w 1 -ca giraph.zkList=localhost:2181 -yj giraph-examples-1.1.0-SNAPSHOT-for-hadoop-2.2.0-jar-with-dependencies.jar{code} -- This message was sent by Atlassian JIRA (v6.2#6252)