Author: edwardyoon Date: Tue Jan 31 02:07:50 2012 New Revision: 1238130 URL: http://svn.apache.org/viewvc?rev=1238130&view=rev Log: Kill job when some task failed
Modified: incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPJobClient.java incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/GroomServer.java incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/JobInProgress.java Modified: incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPJobClient.java URL: http://svn.apache.org/viewvc/incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPJobClient.java?rev=1238130&r1=1238129&r2=1238130&view=diff ============================================================================== --- incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPJobClient.java (original) +++ incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPJobClient.java Tue Jan 31 02:07:50 2012 @@ -602,7 +602,11 @@ public class BSPJobClient extends Config } } - LOG.info("The total number of supersteps: " + info.getSuperstepCount()); + if(job.isSuccessful()) { + LOG.info("The total number of supersteps: " + info.getSuperstepCount()); + } else { + LOG.info("Job failed."); + } // TODO job.getCounters().log(LOG); return job.isSuccessful(); } @@ -646,8 +650,12 @@ public class BSPJobClient extends Config running = jc.getJob(jobId); } - LOG.info("Job complete: " + jobId); - LOG.info("The total number of supersteps: " + running.getSuperstepCount()); + if (running.isSuccessful()) { + LOG.info("Job complete: " + jobId); + LOG.info("The total number of supersteps: " + running.getSuperstepCount()); + } else { + LOG.info("Job failed."); + } // TODO if error found, kill job // running.killJob(); Modified: incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/GroomServer.java URL: http://svn.apache.org/viewvc/incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/GroomServer.java?rev=1238130&r1=1238129&r2=1238130&view=diff ============================================================================== --- incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/GroomServer.java (original) +++ incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/GroomServer.java Tue Jan 31 02:07:50 2012 @@ -144,8 +144,6 @@ public class GroomServer implements Runn } if (actions != null) { - LOG.info("Launch " + actions.length + " tasks."); - assignedPeerNames = new HashMap<TaskAttemptID, Integer>(); int prevPort = Constants.DEFAULT_PEER_PORT; @@ -156,18 +154,20 @@ public class GroomServer implements Runn prevPort = BSPNetUtils.getNextAvailable(prevPort); assignedPeerNames.put(t.getTaskID(), prevPort); + LOG.info("Launch " + actions.length + " tasks."); startNewTask((LaunchTaskAction) action); } else { // TODO Use the cleanup thread // tasksToCleanup.put(action); + LOG.info("Kill " + actions.length + " tasks."); KillTaskAction killAction = (KillTaskAction) action; if (tasks.containsKey(killAction.getTaskID())) { TaskInProgress tip = tasks.get(killAction.getTaskID()); tip.taskStatus.setRunState(TaskStatus.State.FAILED); try { - tip.killAndCleanup(true); + tip.killAndCleanup(false); } catch (IOException ioe) { throw new DirectiveException("Error when killing a " + "TaskInProgress.", ioe); @@ -789,11 +789,9 @@ public class GroomServer implements Runn taskStatus.setRunState(TaskStatus.State.KILLED); } - if (taskStatus.getRunState() == TaskStatus.State.RUNNING) { - // runner could be null if task-cleanup attempt is not localized yet - if (runner != null) { - runner.killBsp(); - } + // runner could be null if task-cleanup attempt is not localized yet + if (runner != null) { + runner.killBsp(); } } Modified: incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/JobInProgress.java URL: http://svn.apache.org/viewvc/incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/JobInProgress.java?rev=1238130&r1=1238129&r2=1238130&view=diff ============================================================================== --- incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/JobInProgress.java (original) +++ incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/JobInProgress.java Tue Jan 31 02:07:50 2012 @@ -292,16 +292,20 @@ class JobInProgress { boolean allDone = true; for (TaskInProgress taskInProgress : tasks) { - if (!taskInProgress.isFailed()) { + if (taskInProgress.isFailed()) { allDone = false; break; } } - - // TODO + + // TODO + if (!allDone) { + // Kill job + this.kill(); + // Send KillTaskAction to GroomServer this.status = new JobStatus(this.status.getJobID(), this.profile - .getUser(), 0L, 0L, 0L, JobStatus.FAILED, superstepCounter); + .getUser(), 0L, 0L, 0L, JobStatus.KILLED, superstepCounter); this.finishTime = System.currentTimeMillis(); this.status.setFinishTime(this.finishTime);