001 /** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018 019 package org.apache.hadoop.mapreduce; 020 021 import java.io.IOException; 022 import java.net.URI; 023 import java.security.PrivilegedExceptionAction; 024 025 import org.apache.commons.logging.Log; 026 import org.apache.commons.logging.LogFactory; 027 import org.apache.hadoop.classification.InterfaceAudience; 028 import org.apache.hadoop.classification.InterfaceStability; 029 import org.apache.hadoop.classification.InterfaceAudience.Private; 030 import org.apache.hadoop.conf.Configuration; 031 import org.apache.hadoop.conf.Configuration.IntegerRanges; 032 import org.apache.hadoop.fs.FileSystem; 033 import org.apache.hadoop.fs.Path; 034 import org.apache.hadoop.io.RawComparator; 035 import org.apache.hadoop.mapred.JobConf; 036 import org.apache.hadoop.mapreduce.filecache.DistributedCache; 037 import org.apache.hadoop.mapreduce.protocol.ClientProtocol; 038 import org.apache.hadoop.mapreduce.task.JobContextImpl; 039 import org.apache.hadoop.mapreduce.util.ConfigUtil; 040 import org.apache.hadoop.util.StringUtils; 041 042 /** 043 * The job submitter's view of the Job. 044 * 045 * <p>It allows the user to configure the 046 * job, submit it, control its execution, and query the state. The set methods 047 * only work until the job is submitted, afterwards they will throw an 048 * IllegalStateException. </p> 049 * 050 * <p> 051 * Normally the user creates the application, describes various facets of the 052 * job via {@link Job} and then submits the job and monitor its progress.</p> 053 * 054 * <p>Here is an example on how to submit a job:</p> 055 * <p><blockquote><pre> 056 * // Create a new Job 057 * Job job = new Job(new Configuration()); 058 * job.setJarByClass(MyJob.class); 059 * 060 * // Specify various job-specific parameters 061 * job.setJobName("myjob"); 062 * 063 * job.setInputPath(new Path("in")); 064 * job.setOutputPath(new Path("out")); 065 * 066 * job.setMapperClass(MyJob.MyMapper.class); 067 * job.setReducerClass(MyJob.MyReducer.class); 068 * 069 * // Submit the job, then poll for progress until the job is complete 070 * job.waitForCompletion(true); 071 * </pre></blockquote></p> 072 * 073 * 074 */ 075 @InterfaceAudience.Public 076 @InterfaceStability.Evolving 077 public class Job extends JobContextImpl implements JobContext { 078 private static final Log LOG = LogFactory.getLog(Job.class); 079 080 @InterfaceStability.Evolving 081 public static enum JobState {DEFINE, RUNNING}; 082 private static final long MAX_JOBSTATUS_AGE = 1000 * 2; 083 public static final String OUTPUT_FILTER = "mapreduce.client.output.filter"; 084 /** Key in mapred-*.xml that sets completionPollInvervalMillis */ 085 public static final String COMPLETION_POLL_INTERVAL_KEY = 086 "mapreduce.client.completion.pollinterval"; 087 088 /** Default completionPollIntervalMillis is 5000 ms. */ 089 static final int DEFAULT_COMPLETION_POLL_INTERVAL = 5000; 090 /** Key in mapred-*.xml that sets progMonitorPollIntervalMillis */ 091 public static final String PROGRESS_MONITOR_POLL_INTERVAL_KEY = 092 "mapreduce.client.progressmonitor.pollinterval"; 093 /** Default progMonitorPollIntervalMillis is 1000 ms. */ 094 static final int DEFAULT_MONITOR_POLL_INTERVAL = 1000; 095 096 public static final String USED_GENERIC_PARSER = 097 "mapreduce.client.genericoptionsparser.used"; 098 public static final String SUBMIT_REPLICATION = 099 "mapreduce.client.submit.file.replication"; 100 private static final String TASKLOG_PULL_TIMEOUT_KEY = 101 "mapreduce.client.tasklog.timeout"; 102 private static final int DEFAULT_TASKLOG_TIMEOUT = 60000; 103 104 @InterfaceStability.Evolving 105 public static enum TaskStatusFilter { NONE, KILLED, FAILED, SUCCEEDED, ALL } 106 107 static { 108 ConfigUtil.loadResources(); 109 } 110 111 private JobState state = JobState.DEFINE; 112 private JobStatus status; 113 private long statustime; 114 private Cluster cluster; 115 116 @Deprecated 117 public Job() throws IOException { 118 this(new Configuration()); 119 } 120 121 @Deprecated 122 public Job(Configuration conf) throws IOException { 123 this(new JobConf(conf)); 124 } 125 126 @Deprecated 127 public Job(Configuration conf, String jobName) throws IOException { 128 this(conf); 129 setJobName(jobName); 130 } 131 132 Job(JobConf conf) throws IOException { 133 super(conf, null); 134 // propagate existing user credentials to job 135 this.credentials.mergeAll(this.ugi.getCredentials()); 136 this.cluster = null; 137 } 138 139 Job(JobStatus status, JobConf conf) throws IOException { 140 this(conf); 141 setJobID(status.getJobID()); 142 this.status = status; 143 state = JobState.RUNNING; 144 } 145 146 147 /** 148 * Creates a new {@link Job} with no particular {@link Cluster} . 149 * A Cluster will be created with a generic {@link Configuration}. 150 * 151 * @return the {@link Job} , with no connection to a cluster yet. 152 * @throws IOException 153 */ 154 public static Job getInstance() throws IOException { 155 // create with a null Cluster 156 return getInstance(new Configuration()); 157 } 158 159 /** 160 * Creates a new {@link Job} with no particular {@link Cluster} and a 161 * given {@link Configuration}. 162 * 163 * The <code>Job</code> makes a copy of the <code>Configuration</code> so 164 * that any necessary internal modifications do not reflect on the incoming 165 * parameter. 166 * 167 * A Cluster will be created from the conf parameter only when it's needed. 168 * 169 * @param conf the configuration 170 * @return the {@link Job} , with no connection to a cluster yet. 171 * @throws IOException 172 */ 173 public static Job getInstance(Configuration conf) throws IOException { 174 // create with a null Cluster 175 JobConf jobConf = new JobConf(conf); 176 return new Job(jobConf); 177 } 178 179 180 /** 181 * Creates a new {@link Job} with no particular {@link Cluster} and a given jobName. 182 * A Cluster will be created from the conf parameter only when it's needed. 183 * 184 * The <code>Job</code> makes a copy of the <code>Configuration</code> so 185 * that any necessary internal modifications do not reflect on the incoming 186 * parameter. 187 * 188 * @param conf the configuration 189 * @return the {@link Job} , with no connection to a cluster yet. 190 * @throws IOException 191 */ 192 public static Job getInstance(Configuration conf, String jobName) 193 throws IOException { 194 // create with a null Cluster 195 Job result = getInstance(conf); 196 result.setJobName(jobName); 197 return result; 198 } 199 200 /** 201 * Creates a new {@link Job} with no particular {@link Cluster} and given 202 * {@link Configuration} and {@link JobStatus}. 203 * A Cluster will be created from the conf parameter only when it's needed. 204 * 205 * The <code>Job</code> makes a copy of the <code>Configuration</code> so 206 * that any necessary internal modifications do not reflect on the incoming 207 * parameter. 208 * 209 * @param status job status 210 * @param conf job configuration 211 * @return the {@link Job} , with no connection to a cluster yet. 212 * @throws IOException 213 */ 214 public static Job getInstance(JobStatus status, Configuration conf) 215 throws IOException { 216 return new Job(status, new JobConf(conf)); 217 } 218 219 /** 220 * Creates a new {@link Job} with no particular {@link Cluster}. 221 * A Cluster will be created from the conf parameter only when it's needed. 222 * 223 * The <code>Job</code> makes a copy of the <code>Configuration</code> so 224 * that any necessary internal modifications do not reflect on the incoming 225 * parameter. 226 * 227 * @param ignored 228 * @return the {@link Job} , with no connection to a cluster yet. 229 * @throws IOException 230 * @deprecated Use {@link #getInstance()} 231 */ 232 @Deprecated 233 public static Job getInstance(Cluster ignored) throws IOException { 234 return getInstance(); 235 } 236 237 /** 238 * Creates a new {@link Job} with no particular {@link Cluster} and given 239 * {@link Configuration}. 240 * A Cluster will be created from the conf parameter only when it's needed. 241 * 242 * The <code>Job</code> makes a copy of the <code>Configuration</code> so 243 * that any necessary internal modifications do not reflect on the incoming 244 * parameter. 245 * 246 * @param ignored 247 * @param conf job configuration 248 * @return the {@link Job} , with no connection to a cluster yet. 249 * @throws IOException 250 * @deprecated Use {@link #getInstance(Configuration)} 251 */ 252 @Deprecated 253 public static Job getInstance(Cluster ignored, Configuration conf) 254 throws IOException { 255 return getInstance(conf); 256 } 257 258 /** 259 * Creates a new {@link Job} with no particular {@link Cluster} and given 260 * {@link Configuration} and {@link JobStatus}. 261 * A Cluster will be created from the conf parameter only when it's needed. 262 * 263 * The <code>Job</code> makes a copy of the <code>Configuration</code> so 264 * that any necessary internal modifications do not reflect on the incoming 265 * parameter. 266 * 267 * @param cluster cluster 268 * @param status job status 269 * @param conf job configuration 270 * @return the {@link Job} , with no connection to a cluster yet. 271 * @throws IOException 272 */ 273 @Private 274 public static Job getInstance(Cluster cluster, JobStatus status, 275 Configuration conf) throws IOException { 276 Job job = getInstance(status, conf); 277 job.setCluster(cluster); 278 return job; 279 } 280 281 private void ensureState(JobState state) throws IllegalStateException { 282 if (state != this.state) { 283 throw new IllegalStateException("Job in state "+ this.state + 284 " instead of " + state); 285 } 286 287 if (state == JobState.RUNNING && cluster == null) { 288 throw new IllegalStateException 289 ("Job in state " + this.state 290 + ", but it isn't attached to any job tracker!"); 291 } 292 } 293 294 /** 295 * Some methods rely on having a recent job status object. Refresh 296 * it, if necessary 297 */ 298 synchronized void ensureFreshStatus() 299 throws IOException { 300 if (System.currentTimeMillis() - statustime > MAX_JOBSTATUS_AGE) { 301 updateStatus(); 302 } 303 } 304 305 /** Some methods need to update status immediately. So, refresh 306 * immediately 307 * @throws IOException 308 */ 309 synchronized void updateStatus() throws IOException { 310 try { 311 this.status = ugi.doAs(new PrivilegedExceptionAction<JobStatus>() { 312 @Override 313 public JobStatus run() throws IOException, InterruptedException { 314 return cluster.getClient().getJobStatus(status.getJobID()); 315 } 316 }); 317 } 318 catch (InterruptedException ie) { 319 throw new IOException(ie); 320 } 321 if (this.status == null) { 322 throw new IOException("Job status not available "); 323 } 324 this.statustime = System.currentTimeMillis(); 325 } 326 327 public JobStatus getStatus() throws IOException, InterruptedException { 328 ensureState(JobState.RUNNING); 329 updateStatus(); 330 return status; 331 } 332 333 private void setStatus(JobStatus status) { 334 this.status = status; 335 } 336 337 /** 338 * Returns the current state of the Job. 339 * 340 * @return JobStatus#State 341 * @throws IOException 342 * @throws InterruptedException 343 */ 344 public JobStatus.State getJobState() 345 throws IOException, InterruptedException { 346 ensureState(JobState.RUNNING); 347 updateStatus(); 348 return status.getState(); 349 } 350 351 /** 352 * Set the boolean property for specifying which classpath takes precedence - 353 * the user's one or the system one, when the tasks are launched 354 * @param value pass true if user's classes should take precedence 355 */ 356 public void setUserClassesTakesPrecedence(boolean value) { 357 ensureState(JobState.DEFINE); 358 conf.setUserClassesTakesPrecedence(value); 359 } 360 361 /** 362 * Get the URL where some job progress information will be displayed. 363 * 364 * @return the URL where some job progress information will be displayed. 365 */ 366 public String getTrackingURL(){ 367 ensureState(JobState.RUNNING); 368 return status.getTrackingUrl().toString(); 369 } 370 371 /** 372 * Get the path of the submitted job configuration. 373 * 374 * @return the path of the submitted job configuration. 375 */ 376 public String getJobFile() { 377 ensureState(JobState.RUNNING); 378 return status.getJobFile(); 379 } 380 381 /** 382 * Get start time of the job. 383 * 384 * @return the start time of the job 385 */ 386 public long getStartTime() { 387 ensureState(JobState.RUNNING); 388 return status.getStartTime(); 389 } 390 391 /** 392 * Get finish time of the job. 393 * 394 * @return the finish time of the job 395 */ 396 public long getFinishTime() throws IOException, InterruptedException { 397 ensureState(JobState.RUNNING); 398 updateStatus(); 399 return status.getFinishTime(); 400 } 401 402 /** 403 * Get scheduling info of the job. 404 * 405 * @return the scheduling info of the job 406 */ 407 public String getSchedulingInfo() { 408 ensureState(JobState.RUNNING); 409 return status.getSchedulingInfo(); 410 } 411 412 /** 413 * Get scheduling info of the job. 414 * 415 * @return the scheduling info of the job 416 */ 417 public JobPriority getPriority() throws IOException, InterruptedException { 418 ensureState(JobState.RUNNING); 419 updateStatus(); 420 return status.getPriority(); 421 } 422 423 /** 424 * The user-specified job name. 425 */ 426 public String getJobName() { 427 if (state == JobState.DEFINE) { 428 return super.getJobName(); 429 } 430 ensureState(JobState.RUNNING); 431 return status.getJobName(); 432 } 433 434 public String getHistoryUrl() throws IOException, InterruptedException { 435 ensureState(JobState.RUNNING); 436 updateStatus(); 437 return status.getHistoryFile(); 438 } 439 440 public boolean isRetired() throws IOException, InterruptedException { 441 ensureState(JobState.RUNNING); 442 updateStatus(); 443 return status.isRetired(); 444 } 445 446 @Private 447 public Cluster getCluster() { 448 return cluster; 449 } 450 451 /** Only for mocks in unit tests. */ 452 @Private 453 private void setCluster(Cluster cluster) { 454 this.cluster = cluster; 455 } 456 457 /** 458 * Dump stats to screen. 459 */ 460 @Override 461 public String toString() { 462 ensureState(JobState.RUNNING); 463 String reasonforFailure = " "; 464 int numMaps = 0; 465 int numReduces = 0; 466 try { 467 updateStatus(); 468 if (status.getState().equals(JobStatus.State.FAILED)) 469 reasonforFailure = getTaskFailureEventString(); 470 numMaps = getTaskReports(TaskType.MAP).length; 471 numReduces = getTaskReports(TaskType.REDUCE).length; 472 } catch (IOException e) { 473 } catch (InterruptedException ie) { 474 } 475 StringBuffer sb = new StringBuffer(); 476 sb.append("Job: ").append(status.getJobID()).append("\n"); 477 sb.append("Job File: ").append(status.getJobFile()).append("\n"); 478 sb.append("Job Tracking URL : ").append(status.getTrackingUrl()); 479 sb.append("\n"); 480 sb.append("Uber job : ").append(status.isUber()).append("\n"); 481 sb.append("Number of maps: ").append(numMaps).append("\n"); 482 sb.append("Number of reduces: ").append(numReduces).append("\n"); 483 sb.append("map() completion: "); 484 sb.append(status.getMapProgress()).append("\n"); 485 sb.append("reduce() completion: "); 486 sb.append(status.getReduceProgress()).append("\n"); 487 sb.append("Job state: "); 488 sb.append(status.getState()).append("\n"); 489 sb.append("retired: ").append(status.isRetired()).append("\n"); 490 sb.append("reason for failure: ").append(reasonforFailure); 491 return sb.toString(); 492 } 493 494 /** 495 * @return taskid which caused job failure 496 * @throws IOException 497 * @throws InterruptedException 498 */ 499 String getTaskFailureEventString() throws IOException, 500 InterruptedException { 501 int failCount = 1; 502 TaskCompletionEvent lastEvent = null; 503 TaskCompletionEvent[] events = ugi.doAs(new 504 PrivilegedExceptionAction<TaskCompletionEvent[]>() { 505 @Override 506 public TaskCompletionEvent[] run() throws IOException, 507 InterruptedException { 508 return cluster.getClient().getTaskCompletionEvents( 509 status.getJobID(), 0, 10); 510 } 511 }); 512 for (TaskCompletionEvent event : events) { 513 if (event.getStatus().equals(TaskCompletionEvent.Status.FAILED)) { 514 failCount++; 515 lastEvent = event; 516 } 517 } 518 if (lastEvent == null) { 519 return "There are no failed tasks for the job. " 520 + "Job is failed due to some other reason and reason " 521 + "can be found in the logs."; 522 } 523 String[] taskAttemptID = lastEvent.getTaskAttemptId().toString().split("_", 2); 524 String taskID = taskAttemptID[1].substring(0, taskAttemptID[1].length()-2); 525 return (" task " + taskID + " failed " + 526 failCount + " times " + "For details check tasktracker at: " + 527 lastEvent.getTaskTrackerHttp()); 528 } 529 530 /** 531 * Get the information of the current state of the tasks of a job. 532 * 533 * @param type Type of the task 534 * @return the list of all of the map tips. 535 * @throws IOException 536 */ 537 public TaskReport[] getTaskReports(TaskType type) 538 throws IOException, InterruptedException { 539 ensureState(JobState.RUNNING); 540 final TaskType tmpType = type; 541 return ugi.doAs(new PrivilegedExceptionAction<TaskReport[]>() { 542 public TaskReport[] run() throws IOException, InterruptedException { 543 return cluster.getClient().getTaskReports(getJobID(), tmpType); 544 } 545 }); 546 } 547 548 /** 549 * Get the <i>progress</i> of the job's map-tasks, as a float between 0.0 550 * and 1.0. When all map tasks have completed, the function returns 1.0. 551 * 552 * @return the progress of the job's map-tasks. 553 * @throws IOException 554 */ 555 public float mapProgress() throws IOException { 556 ensureState(JobState.RUNNING); 557 ensureFreshStatus(); 558 return status.getMapProgress(); 559 } 560 561 /** 562 * Get the <i>progress</i> of the job's reduce-tasks, as a float between 0.0 563 * and 1.0. When all reduce tasks have completed, the function returns 1.0. 564 * 565 * @return the progress of the job's reduce-tasks. 566 * @throws IOException 567 */ 568 public float reduceProgress() throws IOException { 569 ensureState(JobState.RUNNING); 570 ensureFreshStatus(); 571 return status.getReduceProgress(); 572 } 573 574 /** 575 * Get the <i>progress</i> of the job's cleanup-tasks, as a float between 0.0 576 * and 1.0. When all cleanup tasks have completed, the function returns 1.0. 577 * 578 * @return the progress of the job's cleanup-tasks. 579 * @throws IOException 580 */ 581 public float cleanupProgress() throws IOException, InterruptedException { 582 ensureState(JobState.RUNNING); 583 ensureFreshStatus(); 584 return status.getCleanupProgress(); 585 } 586 587 /** 588 * Get the <i>progress</i> of the job's setup-tasks, as a float between 0.0 589 * and 1.0. When all setup tasks have completed, the function returns 1.0. 590 * 591 * @return the progress of the job's setup-tasks. 592 * @throws IOException 593 */ 594 public float setupProgress() throws IOException { 595 ensureState(JobState.RUNNING); 596 ensureFreshStatus(); 597 return status.getSetupProgress(); 598 } 599 600 /** 601 * Check if the job is finished or not. 602 * This is a non-blocking call. 603 * 604 * @return <code>true</code> if the job is complete, else <code>false</code>. 605 * @throws IOException 606 */ 607 public boolean isComplete() throws IOException { 608 ensureState(JobState.RUNNING); 609 updateStatus(); 610 return status.isJobComplete(); 611 } 612 613 /** 614 * Check if the job completed successfully. 615 * 616 * @return <code>true</code> if the job succeeded, else <code>false</code>. 617 * @throws IOException 618 */ 619 public boolean isSuccessful() throws IOException { 620 ensureState(JobState.RUNNING); 621 updateStatus(); 622 return status.getState() == JobStatus.State.SUCCEEDED; 623 } 624 625 /** 626 * Kill the running job. Blocks until all job tasks have been 627 * killed as well. If the job is no longer running, it simply returns. 628 * 629 * @throws IOException 630 */ 631 public void killJob() throws IOException { 632 ensureState(JobState.RUNNING); 633 try { 634 cluster.getClient().killJob(getJobID()); 635 } 636 catch (InterruptedException ie) { 637 throw new IOException(ie); 638 } 639 } 640 641 /** 642 * Set the priority of a running job. 643 * @param priority the new priority for the job. 644 * @throws IOException 645 */ 646 public void setPriority(JobPriority priority) 647 throws IOException, InterruptedException { 648 if (state == JobState.DEFINE) { 649 conf.setJobPriority( 650 org.apache.hadoop.mapred.JobPriority.valueOf(priority.name())); 651 } else { 652 ensureState(JobState.RUNNING); 653 final JobPriority tmpPriority = priority; 654 ugi.doAs(new PrivilegedExceptionAction<Object>() { 655 @Override 656 public Object run() throws IOException, InterruptedException { 657 cluster.getClient().setJobPriority(getJobID(), tmpPriority.toString()); 658 return null; 659 } 660 }); 661 } 662 } 663 664 /** 665 * Get events indicating completion (success/failure) of component tasks. 666 * 667 * @param startFrom index to start fetching events from 668 * @param numEvents number of events to fetch 669 * @return an array of {@link TaskCompletionEvent}s 670 * @throws IOException 671 */ 672 public TaskCompletionEvent[] getTaskCompletionEvents(final int startFrom, 673 final int numEvents) throws IOException, InterruptedException { 674 ensureState(JobState.RUNNING); 675 return ugi.doAs(new PrivilegedExceptionAction<TaskCompletionEvent[]>() { 676 @Override 677 public TaskCompletionEvent[] run() throws IOException, InterruptedException { 678 return cluster.getClient().getTaskCompletionEvents(getJobID(), 679 startFrom, numEvents); 680 } 681 }); 682 } 683 684 /** 685 * Get events indicating completion (success/failure) of component tasks. 686 * 687 * @param startFrom index to start fetching events from 688 * @return an array of {@link TaskCompletionEvent}s 689 * @throws IOException 690 */ 691 public TaskCompletionEvent[] getTaskCompletionEvents(final int startFrom) 692 throws IOException { 693 try { 694 return getTaskCompletionEvents(startFrom, 10); 695 } catch (InterruptedException ie) { 696 throw new IOException(ie); 697 } 698 } 699 700 /** 701 * Kill indicated task attempt. 702 * 703 * @param taskId the id of the task to be terminated. 704 * @throws IOException 705 */ 706 public boolean killTask(final TaskAttemptID taskId) 707 throws IOException { 708 ensureState(JobState.RUNNING); 709 try { 710 return ugi.doAs(new PrivilegedExceptionAction<Boolean>() { 711 public Boolean run() throws IOException, InterruptedException { 712 return cluster.getClient().killTask(taskId, false); 713 } 714 }); 715 } 716 catch (InterruptedException ie) { 717 throw new IOException(ie); 718 } 719 } 720 721 /** 722 * Fail indicated task attempt. 723 * 724 * @param taskId the id of the task to be terminated. 725 * @throws IOException 726 */ 727 public boolean failTask(final TaskAttemptID taskId) 728 throws IOException { 729 ensureState(JobState.RUNNING); 730 try { 731 return ugi.doAs(new PrivilegedExceptionAction<Boolean>() { 732 @Override 733 public Boolean run() throws IOException, InterruptedException { 734 return cluster.getClient().killTask(taskId, true); 735 } 736 }); 737 } 738 catch (InterruptedException ie) { 739 throw new IOException(ie); 740 } 741 } 742 743 /** 744 * Gets the counters for this job. May return null if the job has been 745 * retired and the job is no longer in the completed job store. 746 * 747 * @return the counters for this job. 748 * @throws IOException 749 */ 750 public Counters getCounters() 751 throws IOException { 752 ensureState(JobState.RUNNING); 753 try { 754 return ugi.doAs(new PrivilegedExceptionAction<Counters>() { 755 @Override 756 public Counters run() throws IOException, InterruptedException { 757 return cluster.getClient().getJobCounters(getJobID()); 758 } 759 }); 760 } 761 catch (InterruptedException ie) { 762 throw new IOException(ie); 763 } 764 } 765 766 /** 767 * Gets the diagnostic messages for a given task attempt. 768 * @param taskid 769 * @return the list of diagnostic messages for the task 770 * @throws IOException 771 */ 772 public String[] getTaskDiagnostics(final TaskAttemptID taskid) 773 throws IOException, InterruptedException { 774 ensureState(JobState.RUNNING); 775 return ugi.doAs(new PrivilegedExceptionAction<String[]>() { 776 @Override 777 public String[] run() throws IOException, InterruptedException { 778 return cluster.getClient().getTaskDiagnostics(taskid); 779 } 780 }); 781 } 782 783 /** 784 * Set the number of reduce tasks for the job. 785 * @param tasks the number of reduce tasks 786 * @throws IllegalStateException if the job is submitted 787 */ 788 public void setNumReduceTasks(int tasks) throws IllegalStateException { 789 ensureState(JobState.DEFINE); 790 conf.setNumReduceTasks(tasks); 791 } 792 793 /** 794 * Set the current working directory for the default file system. 795 * 796 * @param dir the new current working directory. 797 * @throws IllegalStateException if the job is submitted 798 */ 799 public void setWorkingDirectory(Path dir) throws IOException { 800 ensureState(JobState.DEFINE); 801 conf.setWorkingDirectory(dir); 802 } 803 804 /** 805 * Set the {@link InputFormat} for the job. 806 * @param cls the <code>InputFormat</code> to use 807 * @throws IllegalStateException if the job is submitted 808 */ 809 public void setInputFormatClass(Class<? extends InputFormat> cls 810 ) throws IllegalStateException { 811 ensureState(JobState.DEFINE); 812 conf.setClass(INPUT_FORMAT_CLASS_ATTR, cls, 813 InputFormat.class); 814 } 815 816 /** 817 * Set the {@link OutputFormat} for the job. 818 * @param cls the <code>OutputFormat</code> to use 819 * @throws IllegalStateException if the job is submitted 820 */ 821 public void setOutputFormatClass(Class<? extends OutputFormat> cls 822 ) throws IllegalStateException { 823 ensureState(JobState.DEFINE); 824 conf.setClass(OUTPUT_FORMAT_CLASS_ATTR, cls, 825 OutputFormat.class); 826 } 827 828 /** 829 * Set the {@link Mapper} for the job. 830 * @param cls the <code>Mapper</code> to use 831 * @throws IllegalStateException if the job is submitted 832 */ 833 public void setMapperClass(Class<? extends Mapper> cls 834 ) throws IllegalStateException { 835 ensureState(JobState.DEFINE); 836 conf.setClass(MAP_CLASS_ATTR, cls, Mapper.class); 837 } 838 839 /** 840 * Set the Jar by finding where a given class came from. 841 * @param cls the example class 842 */ 843 public void setJarByClass(Class<?> cls) { 844 ensureState(JobState.DEFINE); 845 conf.setJarByClass(cls); 846 } 847 848 /** 849 * Set the job jar 850 */ 851 public void setJar(String jar) { 852 ensureState(JobState.DEFINE); 853 conf.setJar(jar); 854 } 855 856 /** 857 * Set the reported username for this job. 858 * 859 * @param user the username for this job. 860 */ 861 public void setUser(String user) { 862 ensureState(JobState.DEFINE); 863 conf.setUser(user); 864 } 865 866 /** 867 * Set the combiner class for the job. 868 * @param cls the combiner to use 869 * @throws IllegalStateException if the job is submitted 870 */ 871 public void setCombinerClass(Class<? extends Reducer> cls 872 ) throws IllegalStateException { 873 ensureState(JobState.DEFINE); 874 conf.setClass(COMBINE_CLASS_ATTR, cls, Reducer.class); 875 } 876 877 /** 878 * Set the {@link Reducer} for the job. 879 * @param cls the <code>Reducer</code> to use 880 * @throws IllegalStateException if the job is submitted 881 */ 882 public void setReducerClass(Class<? extends Reducer> cls 883 ) throws IllegalStateException { 884 ensureState(JobState.DEFINE); 885 conf.setClass(REDUCE_CLASS_ATTR, cls, Reducer.class); 886 } 887 888 /** 889 * Set the {@link Partitioner} for the job. 890 * @param cls the <code>Partitioner</code> to use 891 * @throws IllegalStateException if the job is submitted 892 */ 893 public void setPartitionerClass(Class<? extends Partitioner> cls 894 ) throws IllegalStateException { 895 ensureState(JobState.DEFINE); 896 conf.setClass(PARTITIONER_CLASS_ATTR, cls, 897 Partitioner.class); 898 } 899 900 /** 901 * Set the key class for the map output data. This allows the user to 902 * specify the map output key class to be different than the final output 903 * value class. 904 * 905 * @param theClass the map output key class. 906 * @throws IllegalStateException if the job is submitted 907 */ 908 public void setMapOutputKeyClass(Class<?> theClass 909 ) throws IllegalStateException { 910 ensureState(JobState.DEFINE); 911 conf.setMapOutputKeyClass(theClass); 912 } 913 914 /** 915 * Set the value class for the map output data. This allows the user to 916 * specify the map output value class to be different than the final output 917 * value class. 918 * 919 * @param theClass the map output value class. 920 * @throws IllegalStateException if the job is submitted 921 */ 922 public void setMapOutputValueClass(Class<?> theClass 923 ) throws IllegalStateException { 924 ensureState(JobState.DEFINE); 925 conf.setMapOutputValueClass(theClass); 926 } 927 928 /** 929 * Set the key class for the job output data. 930 * 931 * @param theClass the key class for the job output data. 932 * @throws IllegalStateException if the job is submitted 933 */ 934 public void setOutputKeyClass(Class<?> theClass 935 ) throws IllegalStateException { 936 ensureState(JobState.DEFINE); 937 conf.setOutputKeyClass(theClass); 938 } 939 940 /** 941 * Set the value class for job outputs. 942 * 943 * @param theClass the value class for job outputs. 944 * @throws IllegalStateException if the job is submitted 945 */ 946 public void setOutputValueClass(Class<?> theClass 947 ) throws IllegalStateException { 948 ensureState(JobState.DEFINE); 949 conf.setOutputValueClass(theClass); 950 } 951 952 /** 953 * Define the comparator that controls which keys are grouped together 954 * for a single call to combiner, 955 * {@link Reducer#reduce(Object, Iterable, 956 * org.apache.hadoop.mapreduce.Reducer.Context)} 957 * 958 * @param cls the raw comparator to use 959 * @throws IllegalStateException if the job is submitted 960 */ 961 public void setCombinerKeyGroupingComparatorClass( 962 Class<? extends RawComparator> cls) throws IllegalStateException { 963 ensureState(JobState.DEFINE); 964 conf.setCombinerKeyGroupingComparator(cls); 965 } 966 967 /** 968 * Define the comparator that controls how the keys are sorted before they 969 * are passed to the {@link Reducer}. 970 * @param cls the raw comparator 971 * @throws IllegalStateException if the job is submitted 972 * @see #setCombinerKeyGroupingComparatorClass(Class) 973 */ 974 public void setSortComparatorClass(Class<? extends RawComparator> cls 975 ) throws IllegalStateException { 976 ensureState(JobState.DEFINE); 977 conf.setOutputKeyComparatorClass(cls); 978 } 979 980 /** 981 * Define the comparator that controls which keys are grouped together 982 * for a single call to 983 * {@link Reducer#reduce(Object, Iterable, 984 * org.apache.hadoop.mapreduce.Reducer.Context)} 985 * @param cls the raw comparator to use 986 * @throws IllegalStateException if the job is submitted 987 * @see #setCombinerKeyGroupingComparatorClass(Class) 988 */ 989 public void setGroupingComparatorClass(Class<? extends RawComparator> cls 990 ) throws IllegalStateException { 991 ensureState(JobState.DEFINE); 992 conf.setOutputValueGroupingComparator(cls); 993 } 994 995 /** 996 * Set the user-specified job name. 997 * 998 * @param name the job's new name. 999 * @throws IllegalStateException if the job is submitted 1000 */ 1001 public void setJobName(String name) throws IllegalStateException { 1002 ensureState(JobState.DEFINE); 1003 conf.setJobName(name); 1004 } 1005 1006 /** 1007 * Turn speculative execution on or off for this job. 1008 * 1009 * @param speculativeExecution <code>true</code> if speculative execution 1010 * should be turned on, else <code>false</code>. 1011 */ 1012 public void setSpeculativeExecution(boolean speculativeExecution) { 1013 ensureState(JobState.DEFINE); 1014 conf.setSpeculativeExecution(speculativeExecution); 1015 } 1016 1017 /** 1018 * Turn speculative execution on or off for this job for map tasks. 1019 * 1020 * @param speculativeExecution <code>true</code> if speculative execution 1021 * should be turned on for map tasks, 1022 * else <code>false</code>. 1023 */ 1024 public void setMapSpeculativeExecution(boolean speculativeExecution) { 1025 ensureState(JobState.DEFINE); 1026 conf.setMapSpeculativeExecution(speculativeExecution); 1027 } 1028 1029 /** 1030 * Turn speculative execution on or off for this job for reduce tasks. 1031 * 1032 * @param speculativeExecution <code>true</code> if speculative execution 1033 * should be turned on for reduce tasks, 1034 * else <code>false</code>. 1035 */ 1036 public void setReduceSpeculativeExecution(boolean speculativeExecution) { 1037 ensureState(JobState.DEFINE); 1038 conf.setReduceSpeculativeExecution(speculativeExecution); 1039 } 1040 1041 /** 1042 * Specify whether job-setup and job-cleanup is needed for the job 1043 * 1044 * @param needed If <code>true</code>, job-setup and job-cleanup will be 1045 * considered from {@link OutputCommitter} 1046 * else ignored. 1047 */ 1048 public void setJobSetupCleanupNeeded(boolean needed) { 1049 ensureState(JobState.DEFINE); 1050 conf.setBoolean(SETUP_CLEANUP_NEEDED, needed); 1051 } 1052 1053 /** 1054 * Set the given set of archives 1055 * @param archives The list of archives that need to be localized 1056 */ 1057 public void setCacheArchives(URI[] archives) { 1058 ensureState(JobState.DEFINE); 1059 DistributedCache.setCacheArchives(archives, conf); 1060 } 1061 1062 /** 1063 * Set the given set of files 1064 * @param files The list of files that need to be localized 1065 */ 1066 public void setCacheFiles(URI[] files) { 1067 ensureState(JobState.DEFINE); 1068 DistributedCache.setCacheFiles(files, conf); 1069 } 1070 1071 /** 1072 * Add a archives to be localized 1073 * @param uri The uri of the cache to be localized 1074 */ 1075 public void addCacheArchive(URI uri) { 1076 ensureState(JobState.DEFINE); 1077 DistributedCache.addCacheArchive(uri, conf); 1078 } 1079 1080 /** 1081 * Add a file to be localized 1082 * @param uri The uri of the cache to be localized 1083 */ 1084 public void addCacheFile(URI uri) { 1085 ensureState(JobState.DEFINE); 1086 DistributedCache.addCacheFile(uri, conf); 1087 } 1088 1089 /** 1090 * Add an file path to the current set of classpath entries It adds the file 1091 * to cache as well. 1092 * 1093 * Files added with this method will not be unpacked while being added to the 1094 * classpath. 1095 * To add archives to classpath, use the {@link #addArchiveToClassPath(Path)} 1096 * method instead. 1097 * 1098 * @param file Path of the file to be added 1099 */ 1100 public void addFileToClassPath(Path file) 1101 throws IOException { 1102 ensureState(JobState.DEFINE); 1103 DistributedCache.addFileToClassPath(file, conf, file.getFileSystem(conf)); 1104 } 1105 1106 /** 1107 * Add an archive path to the current set of classpath entries. It adds the 1108 * archive to cache as well. 1109 * 1110 * Archive files will be unpacked and added to the classpath 1111 * when being distributed. 1112 * 1113 * @param archive Path of the archive to be added 1114 */ 1115 public void addArchiveToClassPath(Path archive) 1116 throws IOException { 1117 ensureState(JobState.DEFINE); 1118 DistributedCache.addArchiveToClassPath(archive, conf, archive.getFileSystem(conf)); 1119 } 1120 1121 /** 1122 * Originally intended to enable symlinks, but currently symlinks cannot be 1123 * disabled. 1124 */ 1125 @Deprecated 1126 public void createSymlink() { 1127 ensureState(JobState.DEFINE); 1128 DistributedCache.createSymlink(conf); 1129 } 1130 1131 /** 1132 * Expert: Set the number of maximum attempts that will be made to run a 1133 * map task. 1134 * 1135 * @param n the number of attempts per map task. 1136 */ 1137 public void setMaxMapAttempts(int n) { 1138 ensureState(JobState.DEFINE); 1139 conf.setMaxMapAttempts(n); 1140 } 1141 1142 /** 1143 * Expert: Set the number of maximum attempts that will be made to run a 1144 * reduce task. 1145 * 1146 * @param n the number of attempts per reduce task. 1147 */ 1148 public void setMaxReduceAttempts(int n) { 1149 ensureState(JobState.DEFINE); 1150 conf.setMaxReduceAttempts(n); 1151 } 1152 1153 /** 1154 * Set whether the system should collect profiler information for some of 1155 * the tasks in this job? The information is stored in the user log 1156 * directory. 1157 * @param newValue true means it should be gathered 1158 */ 1159 public void setProfileEnabled(boolean newValue) { 1160 ensureState(JobState.DEFINE); 1161 conf.setProfileEnabled(newValue); 1162 } 1163 1164 /** 1165 * Set the profiler configuration arguments. If the string contains a '%s' it 1166 * will be replaced with the name of the profiling output file when the task 1167 * runs. 1168 * 1169 * This value is passed to the task child JVM on the command line. 1170 * 1171 * @param value the configuration string 1172 */ 1173 public void setProfileParams(String value) { 1174 ensureState(JobState.DEFINE); 1175 conf.setProfileParams(value); 1176 } 1177 1178 /** 1179 * Set the ranges of maps or reduces to profile. setProfileEnabled(true) 1180 * must also be called. 1181 * @param newValue a set of integer ranges of the map ids 1182 */ 1183 public void setProfileTaskRange(boolean isMap, String newValue) { 1184 ensureState(JobState.DEFINE); 1185 conf.setProfileTaskRange(isMap, newValue); 1186 } 1187 1188 private void ensureNotSet(String attr, String msg) throws IOException { 1189 if (conf.get(attr) != null) { 1190 throw new IOException(attr + " is incompatible with " + msg + " mode."); 1191 } 1192 } 1193 1194 /** 1195 * Sets the flag that will allow the JobTracker to cancel the HDFS delegation 1196 * tokens upon job completion. Defaults to true. 1197 */ 1198 public void setCancelDelegationTokenUponJobCompletion(boolean value) { 1199 ensureState(JobState.DEFINE); 1200 conf.setBoolean(JOB_CANCEL_DELEGATION_TOKEN, value); 1201 } 1202 1203 /** 1204 * Default to the new APIs unless they are explicitly set or the old mapper or 1205 * reduce attributes are used. 1206 * @throws IOException if the configuration is inconsistant 1207 */ 1208 private void setUseNewAPI() throws IOException { 1209 int numReduces = conf.getNumReduceTasks(); 1210 String oldMapperClass = "mapred.mapper.class"; 1211 String oldReduceClass = "mapred.reducer.class"; 1212 conf.setBooleanIfUnset("mapred.mapper.new-api", 1213 conf.get(oldMapperClass) == null); 1214 if (conf.getUseNewMapper()) { 1215 String mode = "new map API"; 1216 ensureNotSet("mapred.input.format.class", mode); 1217 ensureNotSet(oldMapperClass, mode); 1218 if (numReduces != 0) { 1219 ensureNotSet("mapred.partitioner.class", mode); 1220 } else { 1221 ensureNotSet("mapred.output.format.class", mode); 1222 } 1223 } else { 1224 String mode = "map compatability"; 1225 ensureNotSet(INPUT_FORMAT_CLASS_ATTR, mode); 1226 ensureNotSet(MAP_CLASS_ATTR, mode); 1227 if (numReduces != 0) { 1228 ensureNotSet(PARTITIONER_CLASS_ATTR, mode); 1229 } else { 1230 ensureNotSet(OUTPUT_FORMAT_CLASS_ATTR, mode); 1231 } 1232 } 1233 if (numReduces != 0) { 1234 conf.setBooleanIfUnset("mapred.reducer.new-api", 1235 conf.get(oldReduceClass) == null); 1236 if (conf.getUseNewReducer()) { 1237 String mode = "new reduce API"; 1238 ensureNotSet("mapred.output.format.class", mode); 1239 ensureNotSet(oldReduceClass, mode); 1240 } else { 1241 String mode = "reduce compatability"; 1242 ensureNotSet(OUTPUT_FORMAT_CLASS_ATTR, mode); 1243 ensureNotSet(REDUCE_CLASS_ATTR, mode); 1244 } 1245 } 1246 } 1247 1248 private synchronized void connect() 1249 throws IOException, InterruptedException, ClassNotFoundException { 1250 if (cluster == null) { 1251 cluster = 1252 ugi.doAs(new PrivilegedExceptionAction<Cluster>() { 1253 public Cluster run() 1254 throws IOException, InterruptedException, 1255 ClassNotFoundException { 1256 return new Cluster(getConfiguration()); 1257 } 1258 }); 1259 } 1260 } 1261 1262 boolean isConnected() { 1263 return cluster != null; 1264 } 1265 1266 /** Only for mocking via unit tests. */ 1267 @Private 1268 public JobSubmitter getJobSubmitter(FileSystem fs, 1269 ClientProtocol submitClient) throws IOException { 1270 return new JobSubmitter(fs, submitClient); 1271 } 1272 /** 1273 * Submit the job to the cluster and return immediately. 1274 * @throws IOException 1275 */ 1276 public void submit() 1277 throws IOException, InterruptedException, ClassNotFoundException { 1278 ensureState(JobState.DEFINE); 1279 setUseNewAPI(); 1280 connect(); 1281 final JobSubmitter submitter = 1282 getJobSubmitter(cluster.getFileSystem(), cluster.getClient()); 1283 status = ugi.doAs(new PrivilegedExceptionAction<JobStatus>() { 1284 public JobStatus run() throws IOException, InterruptedException, 1285 ClassNotFoundException { 1286 return submitter.submitJobInternal(Job.this, cluster); 1287 } 1288 }); 1289 state = JobState.RUNNING; 1290 LOG.info("The url to track the job: " + getTrackingURL()); 1291 } 1292 1293 /** 1294 * Submit the job to the cluster and wait for it to finish. 1295 * @param verbose print the progress to the user 1296 * @return true if the job succeeded 1297 * @throws IOException thrown if the communication with the 1298 * <code>JobTracker</code> is lost 1299 */ 1300 public boolean waitForCompletion(boolean verbose 1301 ) throws IOException, InterruptedException, 1302 ClassNotFoundException { 1303 if (state == JobState.DEFINE) { 1304 submit(); 1305 } 1306 if (verbose) { 1307 monitorAndPrintJob(); 1308 } else { 1309 // get the completion poll interval from the client. 1310 int completionPollIntervalMillis = 1311 Job.getCompletionPollInterval(cluster.getConf()); 1312 while (!isComplete()) { 1313 try { 1314 Thread.sleep(completionPollIntervalMillis); 1315 } catch (InterruptedException ie) { 1316 } 1317 } 1318 } 1319 return isSuccessful(); 1320 } 1321 1322 /** 1323 * Monitor a job and print status in real-time as progress is made and tasks 1324 * fail. 1325 * @return true if the job succeeded 1326 * @throws IOException if communication to the JobTracker fails 1327 */ 1328 public boolean monitorAndPrintJob() 1329 throws IOException, InterruptedException { 1330 String lastReport = null; 1331 Job.TaskStatusFilter filter; 1332 Configuration clientConf = getConfiguration(); 1333 filter = Job.getTaskOutputFilter(clientConf); 1334 JobID jobId = getJobID(); 1335 LOG.info("Running job: " + jobId); 1336 int eventCounter = 0; 1337 boolean profiling = getProfileEnabled(); 1338 IntegerRanges mapRanges = getProfileTaskRange(true); 1339 IntegerRanges reduceRanges = getProfileTaskRange(false); 1340 int progMonitorPollIntervalMillis = 1341 Job.getProgressPollInterval(clientConf); 1342 /* make sure to report full progress after the job is done */ 1343 boolean reportedAfterCompletion = false; 1344 boolean reportedUberMode = false; 1345 while (!isComplete() || !reportedAfterCompletion) { 1346 if (isComplete()) { 1347 reportedAfterCompletion = true; 1348 } else { 1349 Thread.sleep(progMonitorPollIntervalMillis); 1350 } 1351 if (status.getState() == JobStatus.State.PREP) { 1352 continue; 1353 } 1354 if (!reportedUberMode) { 1355 reportedUberMode = true; 1356 LOG.info("Job " + jobId + " running in uber mode : " + isUber()); 1357 } 1358 String report = 1359 (" map " + StringUtils.formatPercent(mapProgress(), 0)+ 1360 " reduce " + 1361 StringUtils.formatPercent(reduceProgress(), 0)); 1362 if (!report.equals(lastReport)) { 1363 LOG.info(report); 1364 lastReport = report; 1365 } 1366 1367 TaskCompletionEvent[] events = 1368 getTaskCompletionEvents(eventCounter, 10); 1369 eventCounter += events.length; 1370 printTaskEvents(events, filter, profiling, mapRanges, reduceRanges); 1371 } 1372 boolean success = isSuccessful(); 1373 if (success) { 1374 LOG.info("Job " + jobId + " completed successfully"); 1375 } else { 1376 LOG.info("Job " + jobId + " failed with state " + status.getState() + 1377 " due to: " + status.getFailureInfo()); 1378 } 1379 Counters counters = getCounters(); 1380 if (counters != null) { 1381 LOG.info(counters.toString()); 1382 } 1383 return success; 1384 } 1385 1386 /** 1387 * @return true if the profile parameters indicate that this is using 1388 * hprof, which generates profile files in a particular location 1389 * that we can retrieve to the client. 1390 */ 1391 private boolean shouldDownloadProfile() { 1392 // Check the argument string that was used to initialize profiling. 1393 // If this indicates hprof and file-based output, then we're ok to 1394 // download. 1395 String profileParams = getProfileParams(); 1396 1397 if (null == profileParams) { 1398 return false; 1399 } 1400 1401 // Split this on whitespace. 1402 String [] parts = profileParams.split("[ \\t]+"); 1403 1404 // If any of these indicate hprof, and the use of output files, return true. 1405 boolean hprofFound = false; 1406 boolean fileFound = false; 1407 for (String p : parts) { 1408 if (p.startsWith("-agentlib:hprof") || p.startsWith("-Xrunhprof")) { 1409 hprofFound = true; 1410 1411 // This contains a number of comma-delimited components, one of which 1412 // may specify the file to write to. Make sure this is present and 1413 // not empty. 1414 String [] subparts = p.split(","); 1415 for (String sub : subparts) { 1416 if (sub.startsWith("file=") && sub.length() != "file=".length()) { 1417 fileFound = true; 1418 } 1419 } 1420 } 1421 } 1422 1423 return hprofFound && fileFound; 1424 } 1425 1426 private void printTaskEvents(TaskCompletionEvent[] events, 1427 Job.TaskStatusFilter filter, boolean profiling, IntegerRanges mapRanges, 1428 IntegerRanges reduceRanges) throws IOException, InterruptedException { 1429 for (TaskCompletionEvent event : events) { 1430 switch (filter) { 1431 case NONE: 1432 break; 1433 case SUCCEEDED: 1434 if (event.getStatus() == 1435 TaskCompletionEvent.Status.SUCCEEDED) { 1436 LOG.info(event.toString()); 1437 } 1438 break; 1439 case FAILED: 1440 if (event.getStatus() == 1441 TaskCompletionEvent.Status.FAILED) { 1442 LOG.info(event.toString()); 1443 // Displaying the task diagnostic information 1444 TaskAttemptID taskId = event.getTaskAttemptId(); 1445 String[] taskDiagnostics = getTaskDiagnostics(taskId); 1446 if (taskDiagnostics != null) { 1447 for (String diagnostics : taskDiagnostics) { 1448 System.err.println(diagnostics); 1449 } 1450 } 1451 } 1452 break; 1453 case KILLED: 1454 if (event.getStatus() == TaskCompletionEvent.Status.KILLED){ 1455 LOG.info(event.toString()); 1456 } 1457 break; 1458 case ALL: 1459 LOG.info(event.toString()); 1460 break; 1461 } 1462 } 1463 } 1464 1465 /** The interval at which monitorAndPrintJob() prints status */ 1466 public static int getProgressPollInterval(Configuration conf) { 1467 // Read progress monitor poll interval from config. Default is 1 second. 1468 int progMonitorPollIntervalMillis = conf.getInt( 1469 PROGRESS_MONITOR_POLL_INTERVAL_KEY, DEFAULT_MONITOR_POLL_INTERVAL); 1470 if (progMonitorPollIntervalMillis < 1) { 1471 LOG.warn(PROGRESS_MONITOR_POLL_INTERVAL_KEY + 1472 " has been set to an invalid value; " 1473 + " replacing with " + DEFAULT_MONITOR_POLL_INTERVAL); 1474 progMonitorPollIntervalMillis = DEFAULT_MONITOR_POLL_INTERVAL; 1475 } 1476 return progMonitorPollIntervalMillis; 1477 } 1478 1479 /** The interval at which waitForCompletion() should check. */ 1480 public static int getCompletionPollInterval(Configuration conf) { 1481 int completionPollIntervalMillis = conf.getInt( 1482 COMPLETION_POLL_INTERVAL_KEY, DEFAULT_COMPLETION_POLL_INTERVAL); 1483 if (completionPollIntervalMillis < 1) { 1484 LOG.warn(COMPLETION_POLL_INTERVAL_KEY + 1485 " has been set to an invalid value; " 1486 + "replacing with " + DEFAULT_COMPLETION_POLL_INTERVAL); 1487 completionPollIntervalMillis = DEFAULT_COMPLETION_POLL_INTERVAL; 1488 } 1489 return completionPollIntervalMillis; 1490 } 1491 1492 /** 1493 * Get the task output filter. 1494 * 1495 * @param conf the configuration. 1496 * @return the filter level. 1497 */ 1498 public static TaskStatusFilter getTaskOutputFilter(Configuration conf) { 1499 return TaskStatusFilter.valueOf(conf.get(Job.OUTPUT_FILTER, "FAILED")); 1500 } 1501 1502 /** 1503 * Modify the Configuration to set the task output filter. 1504 * 1505 * @param conf the Configuration to modify. 1506 * @param newValue the value to set. 1507 */ 1508 public static void setTaskOutputFilter(Configuration conf, 1509 TaskStatusFilter newValue) { 1510 conf.set(Job.OUTPUT_FILTER, newValue.toString()); 1511 } 1512 1513 public boolean isUber() throws IOException, InterruptedException { 1514 ensureState(JobState.RUNNING); 1515 updateStatus(); 1516 return status.isUber(); 1517 } 1518 1519 }