checkRunningState(); for (int i = 0; i < n; i++) { pred = this.dependingJobs.get(i); State s = pred.checkState(); if (s == State.WAITING || s == State.READY || s == State.RUNNING) { break; // a pred is still not completed, continue in WAITING this.message = "depending job " + i + " with jobID " + pred.getJobID() + " failed. " + pred.getMessage(); break;
/** * Add a new job. * * @param aJob * the new job */ synchronized public String addJob(CrunchControlledJob aJob) { String id = this.getNextJobID(); aJob.setJobID(id); aJob.setJobState(State.WAITING); this.addToQueue(aJob); return id; }
synchronized private void startReadyJobs() { Map<String, CrunchControlledJob> oldJobs = null; oldJobs = this.readyJobs; this.readyJobs = new Hashtable<String, CrunchControlledJob>(); for (CrunchControlledJob nextJob : oldJobs.values()) { // Submitting Job to Hadoop nextJob.submit(); this.addToQueue(nextJob); } }
public PipelineResult execute() { try { Thread controlThread = new Thread(control); controlThread.start(); while (!control.allFinished()) { Thread.sleep(1000); } control.stop(); } catch (InterruptedException e) { LOG.info(e); } List<CrunchControlledJob> failures = control.getFailedJobList(); if (!failures.isEmpty()) { System.err.println(failures.size() + " job failure(s) occurred:"); for (CrunchControlledJob job : failures) { System.err.println(job.getJobName() + "(" + job.getJobID() + "): " + job.getMessage()); } } List<PipelineResult.StageResult> stages = Lists.newArrayList(); for (CrunchControlledJob job : control.getSuccessfulJobList()) { try { stages.add(new PipelineResult.StageResult(job.getJobName(), job.getJob().getCounters())); } catch (Exception e) { LOG.error("Exception thrown fetching job counters for stage: " + job.getJobName(), e); } } return new PipelineResult(stages); } }
synchronized private void checkRunningJobs() throws IOException, InterruptedException { Map<String, CrunchControlledJob> oldJobs = null; oldJobs = this.runningJobs; this.runningJobs = new Hashtable<String, CrunchControlledJob>(); for (CrunchControlledJob nextJob : oldJobs.values()) { nextJob.checkState(); this.addToQueue(nextJob); } }
private static void addToQueue(CrunchControlledJob aJob, Map<String, CrunchControlledJob> queue) { synchronized (queue) { queue.put(aJob.getJobID(), aJob); } }
@Override public String toString() { StringBuffer sb = new StringBuffer(); sb.append("job name:\t").append(this.job.getJobName()).append("\n"); sb.append("job id:\t").append(this.controlID).append("\n"); sb.append("job state:\t").append(this.state).append("\n"); sb.append("job mapred id:\t").append(this.job.getJobID()).append("\n"); sb.append("job message:\t").append(this.message).append("\n"); if (this.dependingJobs == null || this.dependingJobs.size() == 0) { sb.append("job has no depending job:\t").append("\n"); } else { sb.append("job has ").append(this.dependingJobs.size()) .append(" dependeng jobs:\n"); for (int i = 0; i < this.dependingJobs.size(); i++) { sb.append("\t depending job ").append(i).append(":\t"); sb.append((this.dependingJobs.get(i)).getJobName()).append("\n"); } } return sb.toString(); }
private void addToQueue(CrunchControlledJob aJob) { Map<String, CrunchControlledJob> queue = getQueue(aJob.getJobState()); addToQueue(aJob, queue); }
synchronized private void checkWaitingJobs() throws IOException, InterruptedException { Map<String, CrunchControlledJob> oldJobs = null; oldJobs = this.waitingJobs; this.waitingJobs = new Hashtable<String, CrunchControlledJob>(); for (CrunchControlledJob nextJob : oldJobs.values()) { nextJob.checkState(); this.addToQueue(nextJob); } }
@Override protected synchronized void submit() { super.submit(); if (this.state == State.RUNNING) { log.info("Running job \"" + getJobName() + "\""); log.info("Job status available at: " + job.getTrackingURL()); } else { log.info("Error occurred starting job \"" + getJobName() + "\":"); log.info(getMessage()); } } }