See: Description
JobConfigurationParser
      
        // An example to parse and filter out job name
        
        String conf_filename = .. // assume the job configuration filename here
        
        // construct a list of interesting properties
        List<String> interestedProperties = new ArrayList<String>();
        interestedProperties.add("mapreduce.job.name");
        
        JobConfigurationParser jcp = 
          new JobConfigurationParser(interestedProperties);
        InputStream in = new FileInputStream(conf_filename);
        Properties parsedProperties = jcp.parse(in);
     
     
     Some of the commonly used interesting properties are enumerated in 
     JobConfPropertyNames. JobConfigurationParser 
        can be used to parse multiple job configuration files. 
     
   JobHistoryParser JobHistoryParserFactory. Note that
      RewindableInputStreamInputStream to make the input 
      stream rewindable.
      
      
      
        // An example to parse a current job history file i.e a job history 
        // file for which the version is known
        
        String filename = .. // assume the job history filename here
        
        InputStream in = new FileInputStream(filename);
        
        HistoryEvent event = null;
        
        JobHistoryParser parser = new CurrentJHParser(in);
        
        event = parser.nextEvent();
        // process all the events
        while (event != null) {
          // ... process all event
          event = parser.nextEvent();
        }
        
        // close the parser and the underlying stream
        parser.close();
      
      
      
      JobHistoryParserFactory provides a 
      JobHistoryParserFactory.getParser(org.apache.hadoop.tools.rumen.RewindableInputStream)
      API to get a parser for parsing the job history file. Note that this
      API can be used if the job history version is unknown.
      
        // An example to parse a job history for which the version is not 
        // known i.e using JobHistoryParserFactory.getParser()
        
        String filename = .. // assume the job history filename here
        
        InputStream in = new FileInputStream(filename);
        RewindableInputStream ris = new RewindableInputStream(in);
        
        // JobHistoryParserFactory will check and return a parser that can
        // parse the file
        JobHistoryParser parser = JobHistoryParserFactory.getParser(ris);
        
        // now use the parser to parse the events
        HistoryEvent event = parser.nextEvent();
        while (event != null) {
          // ... process the event
          event = parser.nextEvent();
        }
        
        parser.close();
      
      
      Note:
        Create one instance to parse a job history log and close it after use.
  TopologyBuilderHistoryEvent. 
      These events can be passed to TopologyBuilder using 
      TopologyBuilder.process(org.apache.hadoop.mapreduce.jobhistory.HistoryEvent).
      A cluster topology can be represented using LoggedNetworkTopology.
      Once all the job history events are processed, the cluster 
      topology can be obtained using TopologyBuilder.build().
      
      
      
        // Building topology for a job history file represented using 
        // 'filename' and the corresponding configuration file represented 
        // using 'conf_filename'
        String filename = .. // assume the job history filename here
        String conf_filename = .. // assume the job configuration filename here
        
        InputStream jobConfInputStream = new FileInputStream(filename);
        InputStream jobHistoryInputStream = new FileInputStream(conf_filename);
        
        TopologyBuilder tb = new TopologyBuilder();
        
        // construct a list of interesting properties
        List<String> interestingProperties = new ArrayList%lt;String>();
        // add the interesting properties here
        interestingProperties.add("mapreduce.job.name");
        
        JobConfigurationParser jcp = 
          new JobConfigurationParser(interestingProperties);
        
        // parse the configuration file
        tb.process(jcp.parse(jobConfInputStream));
        
        // read the job history file and pass it to the 
        // TopologyBuilder.
        JobHistoryParser parser = new CurrentJHParser(jobHistoryInputStream);
        HistoryEvent e;
        
        // read and process all the job history events
        while ((e = parser.nextEvent()) != null) {
          tb.process(e);
        }
        
        LoggedNetworkTopology topology = tb.build();
      
      
  JobBuilderJobHistoryUtils provides  
      JobHistoryUtils.extractJobID(String) 
      API for extracting job id from job history or job configuration files
      which can be used for instantiating JobBuilder. 
      JobBuilder generates a 
      LoggedJob object via 
      JobBuilder.build(). 
      See LoggedJob for more details.
      
      
      
        // An example to summarize a current job history file 'filename'
        // and the corresponding configuration file 'conf_filename'
        
        String filename = .. // assume the job history filename here
        String conf_filename = .. // assume the job configuration filename here
        
        InputStream jobConfInputStream = new FileInputStream(job_filename);
        InputStream jobHistoryInputStream = new FileInputStream(conf_filename);
        
        String jobID = TraceBuilder.extractJobID(job_filename);
        JobBuilder jb = new JobBuilder(jobID);
        
        // construct a list of interesting properties
        List<String> interestingProperties = new ArrayList%lt;String>();
        // add the interesting properties here
        interestingProperties.add("mapreduce.job.name");
        
        JobConfigurationParser jcp = 
          new JobConfigurationParser(interestingProperties);
        
        // parse the configuration file
        jb.process(jcp.parse(jobConfInputStream));
        
        // parse the job history file
        JobHistoryParser parser = new CurrentJHParser(jobHistoryInputStream);
        try {
          HistoryEvent e;
          // read and process all the job history events
          while ((e = parser.nextEvent()) != null) {
            jobBuilder.process(e);
          }
        } finally {
          parser.close();
        }
        
        LoggedJob job = jb.build();
      
      
     Note:
       The order of parsing the job configuration file or job history file is 
       not important. Create one instance to parse the history file and job 
       configuration.
   DefaultOutputterOutputter and writes 
      JSON object in text format to the output file. 
      DefaultOutputter can be 
      initialized with the output filename.
      
      
      
        // An example to summarize a current job history file represented by
        // 'filename' and the configuration filename represented using 
        // 'conf_filename'. Also output the job summary to 'out.json' along 
        // with the cluster topology to 'topology.json'.
        
        String filename = .. // assume the job history filename here
        String conf_filename = .. // assume the job configuration filename here
        
        Configuration conf = new Configuration();
        DefaultOutputter do = new DefaultOutputter();
        do.init("out.json", conf);
        
        InputStream jobConfInputStream = new FileInputStream(filename);
        InputStream jobHistoryInputStream = new FileInputStream(conf_filename);
        
        // extract the job-id from the filename
        String jobID = TraceBuilder.extractJobID(filename);
        JobBuilder jb = new JobBuilder(jobID);
        TopologyBuilder tb = new TopologyBuilder();
        
        // construct a list of interesting properties
        List<String> interestingProperties = new ArrayList%lt;String>();
        // add the interesting properties here
        interestingProperties.add("mapreduce.job.name");
        
        JobConfigurationParser jcp =
          new JobConfigurationParser(interestingProperties);
          
        // parse the configuration file
        tb.process(jcp.parse(jobConfInputStream));
        
        // read the job history file and pass it to the
        // TopologyBuilder.
        JobHistoryParser parser = new CurrentJHParser(jobHistoryInputStream);
        HistoryEvent e;
        while ((e = parser.nextEvent()) != null) {
          jb.process(e);
          tb.process(e);
        }
        
        LoggedJob j = jb.build();
        
        // serialize the job summary in json (text) format
        do.output(j);
        
        // close
        do.close();
        
        do.init("topology.json", conf);
        
        // get the job summary using TopologyBuilder
        LoggedNetworkTopology topology = topologyBuilder.build();
        
        // serialize the cluster topology in json (text) format
        do.output(topology);
        
        // close
        do.close();
      
      
   JobTraceReaderLoggedJob serialized using 
      DefaultOutputter. LoggedJob 
      provides various APIs for extracting job details. Following are the most
      commonly used ones
        LoggedJob.getMapTasks() : Get the map tasksLoggedJob.getReduceTasks() : Get the reduce tasksLoggedJob.getOtherTasks() : Get the setup/cleanup tasksLoggedJob.getOutcome() : Get the job's outcomeLoggedJob.getSubmitTime() : Get the job's submit timeLoggedJob.getFinishTime() : Get the job's finish time
      
        // An example to read job summary from a trace file 'out.json'.
        JobTraceReader reader = new JobTracerReader("out.json");
        LoggedJob job = reader.getNext();
        while (job != null) {
          // .... process job level information
          for (LoggedTask task : job.getMapTasks()) {
            // process all the map tasks in the job
            for (LoggedTaskAttempt attempt : task.getAttempts()) {
              // process all the map task attempts in the job
            }
          }
          
          // get the next job
          job = reader.getNext();
        }
        reader.close();
      
               
   ClusterTopologyReaderLoggedNetworkTopology serialized using 
      DefaultOutputter. ClusterTopologyReader can be 
      initialized using the serialized topology filename. 
      ClusterTopologyReader.get() can
      be used to get the 
      LoggedNetworkTopology. 
      
      
      
        // An example to read the cluster topology from a topology output file
        // 'topology.json'
        ClusterTopologyReader reader = new ClusterTopologyReader("topology.json");
        LoggedNetworkTopology topology  = reader.get();
        for (LoggedNetworkTopology t : topology.getChildren()) {
          // process the cluster topology
        }
        reader.close();
      
      
   Copyright © 2017 Apache Software Foundation. All rights reserved.