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