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    package org.apache.hadoop.mapreduce.tools;
019    
020    import java.io.IOException;
021    import java.io.OutputStreamWriter;
022    import java.io.PrintWriter;
023    import java.util.ArrayList;
024    import java.util.Arrays;
025    import java.util.List;
026    import java.util.Set;
027    import java.util.HashSet;
028    
029    import org.apache.commons.lang.StringUtils;
030    import org.apache.commons.logging.Log;
031    import org.apache.commons.logging.LogFactory;
032    import org.apache.hadoop.classification.InterfaceAudience;
033    import org.apache.hadoop.classification.InterfaceStability;
034    import org.apache.hadoop.classification.InterfaceAudience.Private;
035    import org.apache.hadoop.conf.Configuration;
036    import org.apache.hadoop.conf.Configured;
037    import org.apache.hadoop.ipc.RemoteException;
038    import org.apache.hadoop.mapred.JobConf;
039    import org.apache.hadoop.mapred.TIPStatus;
040    import org.apache.hadoop.mapreduce.Cluster;
041    import org.apache.hadoop.mapreduce.Counters;
042    import org.apache.hadoop.mapreduce.Job;
043    import org.apache.hadoop.mapreduce.JobID;
044    import org.apache.hadoop.mapreduce.JobPriority;
045    import org.apache.hadoop.mapreduce.JobStatus;
046    import org.apache.hadoop.mapreduce.TaskAttemptID;
047    import org.apache.hadoop.mapreduce.TaskCompletionEvent;
048    import org.apache.hadoop.mapreduce.TaskReport;
049    import org.apache.hadoop.mapreduce.TaskTrackerInfo;
050    import org.apache.hadoop.mapreduce.TaskType;
051    import org.apache.hadoop.mapreduce.jobhistory.HistoryViewer;
052    import org.apache.hadoop.mapreduce.v2.LogParams;
053    import org.apache.hadoop.security.AccessControlException;
054    import org.apache.hadoop.util.ExitUtil;
055    import org.apache.hadoop.util.Tool;
056    import org.apache.hadoop.util.ToolRunner;
057    import org.apache.hadoop.yarn.logaggregation.LogDumper;
058    
059    import com.google.common.base.Charsets;
060    
061    /**
062     * Interprets the map reduce cli options 
063     */
064    @InterfaceAudience.Public
065    @InterfaceStability.Stable
066    public class CLI extends Configured implements Tool {
067      private static final Log LOG = LogFactory.getLog(CLI.class);
068      protected Cluster cluster;
069      private static final Set<String> taskTypes = new HashSet<String>(
070          Arrays.asList("MAP", "REDUCE"));
071    
072      public CLI() {
073      }
074      
075      public CLI(Configuration conf) {
076        setConf(conf);
077      }
078      
079      public int run(String[] argv) throws Exception {
080        int exitCode = -1;
081        if (argv.length < 1) {
082          displayUsage("");
083          return exitCode;
084        }    
085        // process arguments
086        String cmd = argv[0];
087        String submitJobFile = null;
088        String jobid = null;
089        String taskid = null;
090        String historyFile = null;
091        String counterGroupName = null;
092        String counterName = null;
093        JobPriority jp = null;
094        String taskType = null;
095        String taskState = null;
096        int fromEvent = 0;
097        int nEvents = 0;
098        boolean getStatus = false;
099        boolean getCounter = false;
100        boolean killJob = false;
101        boolean listEvents = false;
102        boolean viewHistory = false;
103        boolean viewAllHistory = false;
104        boolean listJobs = false;
105        boolean listAllJobs = false;
106        boolean listActiveTrackers = false;
107        boolean listBlacklistedTrackers = false;
108        boolean displayTasks = false;
109        boolean killTask = false;
110        boolean failTask = false;
111        boolean setJobPriority = false;
112        boolean logs = false;
113    
114        if ("-submit".equals(cmd)) {
115          if (argv.length != 2) {
116            displayUsage(cmd);
117            return exitCode;
118          }
119          submitJobFile = argv[1];
120        } else if ("-status".equals(cmd)) {
121          if (argv.length != 2) {
122            displayUsage(cmd);
123            return exitCode;
124          }
125          jobid = argv[1];
126          getStatus = true;
127        } else if("-counter".equals(cmd)) {
128          if (argv.length != 4) {
129            displayUsage(cmd);
130            return exitCode;
131          }
132          getCounter = true;
133          jobid = argv[1];
134          counterGroupName = argv[2];
135          counterName = argv[3];
136        } else if ("-kill".equals(cmd)) {
137          if (argv.length != 2) {
138            displayUsage(cmd);
139            return exitCode;
140          }
141          jobid = argv[1];
142          killJob = true;
143        } else if ("-set-priority".equals(cmd)) {
144          if (argv.length != 3) {
145            displayUsage(cmd);
146            return exitCode;
147          }
148          jobid = argv[1];
149          try {
150            jp = JobPriority.valueOf(argv[2]); 
151          } catch (IllegalArgumentException iae) {
152            LOG.info(iae);
153            displayUsage(cmd);
154            return exitCode;
155          }
156          setJobPriority = true; 
157        } else if ("-events".equals(cmd)) {
158          if (argv.length != 4) {
159            displayUsage(cmd);
160            return exitCode;
161          }
162          jobid = argv[1];
163          fromEvent = Integer.parseInt(argv[2]);
164          nEvents = Integer.parseInt(argv[3]);
165          listEvents = true;
166        } else if ("-history".equals(cmd)) {
167          if (argv.length != 2 && !(argv.length == 3 && "all".equals(argv[1]))) {
168             displayUsage(cmd);
169             return exitCode;
170          }
171          viewHistory = true;
172          if (argv.length == 3 && "all".equals(argv[1])) {
173            viewAllHistory = true;
174            historyFile = argv[2];
175          } else {
176            historyFile = argv[1];
177          }
178        } else if ("-list".equals(cmd)) {
179          if (argv.length != 1 && !(argv.length == 2 && "all".equals(argv[1]))) {
180            displayUsage(cmd);
181            return exitCode;
182          }
183          if (argv.length == 2 && "all".equals(argv[1])) {
184            listAllJobs = true;
185          } else {
186            listJobs = true;
187          }
188        } else if("-kill-task".equals(cmd)) {
189          if (argv.length != 2) {
190            displayUsage(cmd);
191            return exitCode;
192          }
193          killTask = true;
194          taskid = argv[1];
195        } else if("-fail-task".equals(cmd)) {
196          if (argv.length != 2) {
197            displayUsage(cmd);
198            return exitCode;
199          }
200          failTask = true;
201          taskid = argv[1];
202        } else if ("-list-active-trackers".equals(cmd)) {
203          if (argv.length != 1) {
204            displayUsage(cmd);
205            return exitCode;
206          }
207          listActiveTrackers = true;
208        } else if ("-list-blacklisted-trackers".equals(cmd)) {
209          if (argv.length != 1) {
210            displayUsage(cmd);
211            return exitCode;
212          }
213          listBlacklistedTrackers = true;
214        } else if ("-list-attempt-ids".equals(cmd)) {
215          if (argv.length != 4) {
216            displayUsage(cmd);
217            return exitCode;
218          }
219          jobid = argv[1];
220          taskType = argv[2];
221          taskState = argv[3];
222          displayTasks = true;
223          if (!taskTypes.contains(taskType.toUpperCase())) {
224            System.out.println("Error: Invalid task-type: "+taskType);
225            displayUsage(cmd);
226            return exitCode;
227          }
228        } else if ("-logs".equals(cmd)) {
229          if (argv.length == 2 || argv.length ==3) {
230            logs = true;
231            jobid = argv[1];
232            if (argv.length == 3) {
233              taskid = argv[2];
234            }  else {
235              taskid = null;
236            }
237          } else {
238            displayUsage(cmd);
239            return exitCode;
240          }
241        } else {
242          displayUsage(cmd);
243          return exitCode;
244        }
245    
246        // initialize cluster
247        cluster = createCluster();
248            
249        // Submit the request
250        try {
251          if (submitJobFile != null) {
252            Job job = Job.getInstance(new JobConf(submitJobFile));
253            job.submit();
254            System.out.println("Created job " + job.getJobID());
255            exitCode = 0;
256          } else if (getStatus) {
257            Job job = cluster.getJob(JobID.forName(jobid));
258            if (job == null) {
259              System.out.println("Could not find job " + jobid);
260            } else {
261              Counters counters = job.getCounters();
262              System.out.println();
263              System.out.println(job);
264              if (counters != null) {
265                System.out.println(counters);
266              } else {
267                System.out.println("Counters not available. Job is retired.");
268              }
269              exitCode = 0;
270            }
271          } else if (getCounter) {
272            Job job = cluster.getJob(JobID.forName(jobid));
273            if (job == null) {
274              System.out.println("Could not find job " + jobid);
275            } else {
276              Counters counters = job.getCounters();
277              if (counters == null) {
278                System.out.println("Counters not available for retired job " + 
279                jobid);
280                exitCode = -1;
281              } else {
282                System.out.println(getCounter(counters,
283                  counterGroupName, counterName));
284                exitCode = 0;
285              }
286            }
287          } else if (killJob) {
288            Job job = cluster.getJob(JobID.forName(jobid));
289            if (job == null) {
290              System.out.println("Could not find job " + jobid);
291            } else {
292              job.killJob();
293              System.out.println("Killed job " + jobid);
294              exitCode = 0;
295            }
296          } else if (setJobPriority) {
297            Job job = cluster.getJob(JobID.forName(jobid));
298            if (job == null) {
299              System.out.println("Could not find job " + jobid);
300            } else {
301              job.setPriority(jp);
302              System.out.println("Changed job priority.");
303              exitCode = 0;
304            } 
305          } else if (viewHistory) {
306            viewHistory(historyFile, viewAllHistory);
307            exitCode = 0;
308          } else if (listEvents) {
309            listEvents(cluster.getJob(JobID.forName(jobid)), fromEvent, nEvents);
310            exitCode = 0;
311          } else if (listJobs) {
312            listJobs(cluster);
313            exitCode = 0;
314          } else if (listAllJobs) {
315            listAllJobs(cluster);
316            exitCode = 0;
317          } else if (listActiveTrackers) {
318            listActiveTrackers(cluster);
319            exitCode = 0;
320          } else if (listBlacklistedTrackers) {
321            listBlacklistedTrackers(cluster);
322            exitCode = 0;
323          } else if (displayTasks) {
324            displayTasks(cluster.getJob(JobID.forName(jobid)), taskType, taskState);
325            exitCode = 0;
326          } else if(killTask) {
327            TaskAttemptID taskID = TaskAttemptID.forName(taskid);
328            Job job = cluster.getJob(taskID.getJobID());
329            if (job == null) {
330              System.out.println("Could not find job " + jobid);
331            } else if (job.killTask(taskID, false)) {
332              System.out.println("Killed task " + taskid);
333              exitCode = 0;
334            } else {
335              System.out.println("Could not kill task " + taskid);
336              exitCode = -1;
337            }
338          } else if(failTask) {
339            TaskAttemptID taskID = TaskAttemptID.forName(taskid);
340            Job job = cluster.getJob(taskID.getJobID());
341            if (job == null) {
342                System.out.println("Could not find job " + jobid);
343            } else if(job.killTask(taskID, true)) {
344              System.out.println("Killed task " + taskID + " by failing it");
345              exitCode = 0;
346            } else {
347              System.out.println("Could not fail task " + taskid);
348              exitCode = -1;
349            }
350          } else if (logs) {
351            try {
352            JobID jobID = JobID.forName(jobid);
353            TaskAttemptID taskAttemptID = TaskAttemptID.forName(taskid);
354            LogParams logParams = cluster.getLogParams(jobID, taskAttemptID);
355            LogDumper logDumper = new LogDumper();
356            logDumper.setConf(getConf());
357            exitCode = logDumper.dumpAContainersLogs(logParams.getApplicationId(),
358                logParams.getContainerId(), logParams.getNodeId(),
359                logParams.getOwner());
360            } catch (IOException e) {
361              if (e instanceof RemoteException) {
362                throw e;
363              } 
364              System.out.println(e.getMessage());
365            }
366          }
367        } catch (RemoteException re) {
368          IOException unwrappedException = re.unwrapRemoteException();
369          if (unwrappedException instanceof AccessControlException) {
370            System.out.println(unwrappedException.getMessage());
371          } else {
372            throw re;
373          }
374        } finally {
375          cluster.close();
376        }
377        return exitCode;
378      }
379    
380      Cluster createCluster() throws IOException {
381        return new Cluster(getConf());
382      }
383    
384      private String getJobPriorityNames() {
385        StringBuffer sb = new StringBuffer();
386        for (JobPriority p : JobPriority.values()) {
387          sb.append(p.name()).append(" ");
388        }
389        return sb.substring(0, sb.length()-1);
390      }
391    
392      private String getTaskTypes() {
393        return StringUtils.join(taskTypes, " ");
394      }
395    
396      /**
397       * Display usage of the command-line tool and terminate execution.
398       */
399      private void displayUsage(String cmd) {
400        String prefix = "Usage: CLI ";
401        String jobPriorityValues = getJobPriorityNames();
402        String taskStates = "running, completed";
403    
404        if ("-submit".equals(cmd)) {
405          System.err.println(prefix + "[" + cmd + " <job-file>]");
406        } else if ("-status".equals(cmd) || "-kill".equals(cmd)) {
407          System.err.println(prefix + "[" + cmd + " <job-id>]");
408        } else if ("-counter".equals(cmd)) {
409          System.err.println(prefix + "[" + cmd + 
410            " <job-id> <group-name> <counter-name>]");
411        } else if ("-events".equals(cmd)) {
412          System.err.println(prefix + "[" + cmd + 
413            " <job-id> <from-event-#> <#-of-events>]. Event #s start from 1.");
414        } else if ("-history".equals(cmd)) {
415          System.err.println(prefix + "[" + cmd + " <jobHistoryFile>]");
416        } else if ("-list".equals(cmd)) {
417          System.err.println(prefix + "[" + cmd + " [all]]");
418        } else if ("-kill-task".equals(cmd) || "-fail-task".equals(cmd)) {
419          System.err.println(prefix + "[" + cmd + " <task-attempt-id>]");
420        } else if ("-set-priority".equals(cmd)) {
421          System.err.println(prefix + "[" + cmd + " <job-id> <priority>]. " +
422              "Valid values for priorities are: " 
423              + jobPriorityValues); 
424        } else if ("-list-active-trackers".equals(cmd)) {
425          System.err.println(prefix + "[" + cmd + "]");
426        } else if ("-list-blacklisted-trackers".equals(cmd)) {
427          System.err.println(prefix + "[" + cmd + "]");
428        } else if ("-list-attempt-ids".equals(cmd)) {
429          System.err.println(prefix + "[" + cmd + 
430              " <job-id> <task-type> <task-state>]. " +
431              "Valid values for <task-type> are " + getTaskTypes() + ". " +
432              "Valid values for <task-state> are " + taskStates);
433        } else if ("-logs".equals(cmd)) {
434          System.err.println(prefix + "[" + cmd +
435              " <job-id> <task-attempt-id>]. " +
436              " <task-attempt-id> is optional to get task attempt logs.");      
437        } else {
438          System.err.printf(prefix + "<command> <args>%n");
439          System.err.printf("\t[-submit <job-file>]%n");
440          System.err.printf("\t[-status <job-id>]%n");
441          System.err.printf("\t[-counter <job-id> <group-name> <counter-name>]%n");
442          System.err.printf("\t[-kill <job-id>]%n");
443          System.err.printf("\t[-set-priority <job-id> <priority>]. " +
444            "Valid values for priorities are: " + jobPriorityValues + "%n");
445          System.err.printf("\t[-events <job-id> <from-event-#> <#-of-events>]%n");
446          System.err.printf("\t[-history <jobHistoryFile>]%n");
447          System.err.printf("\t[-list [all]]%n");
448          System.err.printf("\t[-list-active-trackers]%n");
449          System.err.printf("\t[-list-blacklisted-trackers]%n");
450          System.err.println("\t[-list-attempt-ids <job-id> <task-type> " +
451            "<task-state>]. " +
452            "Valid values for <task-type> are " + getTaskTypes() + ". " +
453            "Valid values for <task-state> are " + taskStates);
454          System.err.printf("\t[-kill-task <task-attempt-id>]%n");
455          System.err.printf("\t[-fail-task <task-attempt-id>]%n");
456          System.err.printf("\t[-logs <job-id> <task-attempt-id>]%n%n");
457          ToolRunner.printGenericCommandUsage(System.out);
458        }
459      }
460        
461      private void viewHistory(String historyFile, boolean all) 
462        throws IOException {
463        HistoryViewer historyViewer = new HistoryViewer(historyFile,
464                                            getConf(), all);
465        historyViewer.print();
466      }
467    
468      protected long getCounter(Counters counters, String counterGroupName,
469          String counterName) throws IOException {
470        return counters.findCounter(counterGroupName, counterName).getValue();
471      }
472      
473      /**
474       * List the events for the given job
475       * @param jobId the job id for the job's events to list
476       * @throws IOException
477       */
478      private void listEvents(Job job, int fromEventId, int numEvents)
479          throws IOException, InterruptedException {
480        TaskCompletionEvent[] events = job.
481          getTaskCompletionEvents(fromEventId, numEvents);
482        System.out.println("Task completion events for " + job.getJobID());
483        System.out.println("Number of events (from " + fromEventId + ") are: " 
484          + events.length);
485        for(TaskCompletionEvent event: events) {
486          System.out.println(event.getStatus() + " " + 
487            event.getTaskAttemptId() + " " + 
488            getTaskLogURL(event.getTaskAttemptId(), event.getTaskTrackerHttp()));
489        }
490      }
491    
492      protected static String getTaskLogURL(TaskAttemptID taskId, String baseUrl) {
493        return (baseUrl + "/tasklog?plaintext=true&attemptid=" + taskId); 
494      }
495      
496    
497      /**
498       * Dump a list of currently running jobs
499       * @throws IOException
500       */
501      private void listJobs(Cluster cluster) 
502          throws IOException, InterruptedException {
503        List<JobStatus> runningJobs = new ArrayList<JobStatus>();
504        for (JobStatus job : cluster.getAllJobStatuses()) {
505          if (!job.isJobComplete()) {
506            runningJobs.add(job);
507          }
508        }
509        displayJobList(runningJobs.toArray(new JobStatus[0]));
510      }
511        
512      /**
513       * Dump a list of all jobs submitted.
514       * @throws IOException
515       */
516      private void listAllJobs(Cluster cluster) 
517          throws IOException, InterruptedException {
518        displayJobList(cluster.getAllJobStatuses());
519      }
520      
521      /**
522       * Display the list of active trackers
523       */
524      private void listActiveTrackers(Cluster cluster) 
525          throws IOException, InterruptedException {
526        TaskTrackerInfo[] trackers = cluster.getActiveTaskTrackers();
527        for (TaskTrackerInfo tracker : trackers) {
528          System.out.println(tracker.getTaskTrackerName());
529        }
530      }
531    
532      /**
533       * Display the list of blacklisted trackers
534       */
535      private void listBlacklistedTrackers(Cluster cluster) 
536          throws IOException, InterruptedException {
537        TaskTrackerInfo[] trackers = cluster.getBlackListedTaskTrackers();
538        if (trackers.length > 0) {
539          System.out.println("BlackListedNode \t Reason");
540        }
541        for (TaskTrackerInfo tracker : trackers) {
542          System.out.println(tracker.getTaskTrackerName() + "\t" + 
543            tracker.getReasonForBlacklist());
544        }
545      }
546    
547      private void printTaskAttempts(TaskReport report) {
548        if (report.getCurrentStatus() == TIPStatus.COMPLETE) {
549          System.out.println(report.getSuccessfulTaskAttemptId());
550        } else if (report.getCurrentStatus() == TIPStatus.RUNNING) {
551          for (TaskAttemptID t : 
552            report.getRunningTaskAttemptIds()) {
553            System.out.println(t);
554          }
555        }
556      }
557    
558      /**
559       * Display the information about a job's tasks, of a particular type and
560       * in a particular state
561       * 
562       * @param job the job
563       * @param type the type of the task (map/reduce/setup/cleanup)
564       * @param state the state of the task 
565       * (pending/running/completed/failed/killed)
566       */
567      protected void displayTasks(Job job, String type, String state) 
568      throws IOException, InterruptedException {
569        TaskReport[] reports = job.getTaskReports(TaskType.valueOf(type.toUpperCase()));
570        for (TaskReport report : reports) {
571          TIPStatus status = report.getCurrentStatus();
572          if ((state.equals("pending") && status ==TIPStatus.PENDING) ||
573              (state.equals("running") && status ==TIPStatus.RUNNING) ||
574              (state.equals("completed") && status == TIPStatus.COMPLETE) ||
575              (state.equals("failed") && status == TIPStatus.FAILED) ||
576              (state.equals("killed") && status == TIPStatus.KILLED)) {
577            printTaskAttempts(report);
578          }
579        }
580      }
581    
582      public void displayJobList(JobStatus[] jobs) 
583          throws IOException, InterruptedException {
584        displayJobList(jobs, new PrintWriter(new OutputStreamWriter(System.out,
585            Charsets.UTF_8)));
586      }
587    
588      @Private
589      public static String headerPattern = "%23s\t%10s\t%14s\t%12s\t%12s\t%10s\t%15s\t%15s\t%8s\t%8s\t%10s\t%10s\n";
590      @Private
591      public static String dataPattern   = "%23s\t%10s\t%14d\t%12s\t%12s\t%10s\t%15s\t%15s\t%8s\t%8s\t%10s\t%10s\n";
592      private static String memPattern   = "%dM";
593      private static String UNAVAILABLE  = "N/A";
594    
595      @Private
596      public void displayJobList(JobStatus[] jobs, PrintWriter writer) {
597        writer.println("Total jobs:" + jobs.length);
598        writer.printf(headerPattern, "JobId", "State", "StartTime", "UserName",
599          "Queue", "Priority", "UsedContainers",
600          "RsvdContainers", "UsedMem", "RsvdMem", "NeededMem", "AM info");
601        for (JobStatus job : jobs) {
602          int numUsedSlots = job.getNumUsedSlots();
603          int numReservedSlots = job.getNumReservedSlots();
604          int usedMem = job.getUsedMem();
605          int rsvdMem = job.getReservedMem();
606          int neededMem = job.getNeededMem();
607          writer.printf(dataPattern,
608              job.getJobID().toString(), job.getState(), job.getStartTime(),
609              job.getUsername(), job.getQueue(), 
610              job.getPriority().name(),
611              numUsedSlots < 0 ? UNAVAILABLE : numUsedSlots,
612              numReservedSlots < 0 ? UNAVAILABLE : numReservedSlots,
613              usedMem < 0 ? UNAVAILABLE : String.format(memPattern, usedMem),
614              rsvdMem < 0 ? UNAVAILABLE : String.format(memPattern, rsvdMem),
615              neededMem < 0 ? UNAVAILABLE : String.format(memPattern, neededMem),
616              job.getSchedulingInfo());
617        }
618        writer.flush();
619      }
620      
621      public static void main(String[] argv) throws Exception {
622        int res = ToolRunner.run(new CLI(), argv);
623        ExitUtil.terminate(res);
624      }
625    }