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
RewindableInputStream
InputStream
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.
TopologyBuilder
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()
.
// 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();
JobBuilder
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.
// 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.
DefaultOutputter
Outputter
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();
JobTraceReader
LoggedJob
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();
ClusterTopologyReader
LoggedNetworkTopology
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 © 2018 Apache Software Foundation. All rights reserved.