Package org.apache.hadoop.tools.rumen

Rumen is a data extraction and analysis tool built for Apache Hadoop.

See: Description

Package org.apache.hadoop.tools.rumen Description

Rumen is a data extraction and analysis tool built for Apache Hadoop. Rumen mines job history logs to extract meaningful data and stores it into an easily-parsed format. The default output format of Rumen is JSON. Rumen uses the Jackson library to create JSON objects.

The following classes can be used to programmatically invoke Rumen:
  1. JobConfigurationParser
    A parser to parse and filter out interesting properties from job configuration.

    Sample code:
          
            // 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.

    Note: A single instance of JobConfigurationParser can be used to parse multiple job configuration files.
  2. JobHistoryParser
    A parser that parses job history files. It is an interface and actual implementations are defined as Enum in JobHistoryParserFactory. Note that RewindableInputStream
    is a wrapper class around InputStream to make the input stream rewindable.
    Sample code:
          
            // 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.

    Sample code:
          
            // 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.
  3. TopologyBuilder
    Builds the cluster topology based on the job history events. Every job history file consists of events. Each event can be represented using HistoryEvent. 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().

    Sample code:
          
            // 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();
          
          
  4. JobBuilder
    Summarizes a job history file. JobHistoryUtils 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.

    Sample code:
          
            // 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.
  5. DefaultOutputter
    Implements Outputter and writes JSON object in text format to the output file. DefaultOutputter can be initialized with the output filename.

    Sample code:
          
            // 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();
          
          
  6. JobTraceReader
    A reader for reading LoggedJob serialized using DefaultOutputter. LoggedJob provides various APIs for extracting job details. Following are the most commonly used ones
    • LoggedJob.getMapTasks() : Get the map tasks
    • LoggedJob.getReduceTasks() : Get the reduce tasks
    • LoggedJob.getOtherTasks() : Get the setup/cleanup tasks
    • LoggedJob.getOutcome() : Get the job's outcome
    • LoggedJob.getSubmitTime() : Get the job's submit time
    • LoggedJob.getFinishTime() : Get the job's finish time


    Sample code:
          
            // 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();
          
          
  7. ClusterTopologyReader
    A reader to read LoggedNetworkTopology serialized using DefaultOutputter. ClusterTopologyReader can be initialized using the serialized topology filename. ClusterTopologyReader.get() can be used to get the LoggedNetworkTopology.

    Sample code:
          
            // 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 © 2015 Apache Software Foundation. All rights reserved.