MAPREDUCE-6554. MRAppMaster servicestart failing with NPE in MRAppMaster#parsePreviousJobHistory. Contributed by Bibin A Chundatt

This commit is contained in:
Jason Lowe 2016-01-15 16:52:53 +00:00
parent fc6d3a3b23
commit 9fbd579ab5
2 changed files with 30 additions and 15 deletions

View File

@ -712,6 +712,9 @@ Release 2.7.3 - UNRELEASED
MAPREDUCE-6583. Clarify confusing sentence in MapReduce tutorial document.
(Kai Sasaki via aajisaka)
MAPREDUCE-6554. MRAppMaster servicestart failing with NPE in
MRAppMaster#parsePreviousJobHistory (Bibin A Chundatt via jlowe)
Release 2.7.2 - UNRELEASED
INCOMPATIBLE CHANGES
@ -1014,6 +1017,9 @@ Release 2.6.4 - UNRELEASED
MAPREDUCE-6363. [NNBench] Lease mismatch error when running with multiple
mappers. (Vlad Sharanhovich and Bibin A Chundatt via aajisaka)
MAPREDUCE-6554. MRAppMaster servicestart failing with NPE in
MRAppMaster#parsePreviousJobHistory (Bibin A Chundatt via jlowe)
Release 2.6.3 - 2015-12-17
INCOMPATIBLE CHANGES

View File

@ -20,9 +20,16 @@
import java.io.Closeable;
import java.io.DataInputStream;
import java.io.IOException;
import java.io.EOFException;
import java.io.IOException;
import org.apache.avro.AvroRuntimeException;
import org.apache.avro.Schema;
import org.apache.avro.io.DatumReader;
import org.apache.avro.io.Decoder;
import org.apache.avro.io.DecoderFactory;
import org.apache.avro.specific.SpecificData;
import org.apache.avro.specific.SpecificDatumReader;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileSystem;
@ -31,13 +38,6 @@
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.util.StringInterner;
import org.apache.avro.Schema;
import org.apache.avro.io.Decoder;
import org.apache.avro.io.DecoderFactory;
import org.apache.avro.io.DatumReader;
import org.apache.avro.specific.SpecificData;
import org.apache.avro.specific.SpecificDatumReader;
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class EventReader implements Closeable {
@ -69,14 +69,23 @@ public EventReader(DataInputStream in) throws IOException {
Schema myschema = new SpecificData(Event.class.getClassLoader()).getSchema(Event.class);
Schema.Parser parser = new Schema.Parser();
this.schema = parser.parse(in.readLine());
this.reader = new SpecificDatumReader(schema, myschema);
if (EventWriter.VERSION.equals(version)) {
this.decoder = DecoderFactory.get().jsonDecoder(schema, in);
} else if (EventWriter.VERSION_BINARY.equals(version)) {
this.decoder = DecoderFactory.get().binaryDecoder(in, null);
String eventschema = in.readLine();
if (null != eventschema) {
try {
this.schema = parser.parse(eventschema);
this.reader = new SpecificDatumReader(schema, myschema);
if (EventWriter.VERSION.equals(version)) {
this.decoder = DecoderFactory.get().jsonDecoder(schema, in);
} else if (EventWriter.VERSION_BINARY.equals(version)) {
this.decoder = DecoderFactory.get().binaryDecoder(in, null);
} else {
throw new IOException("Incompatible event log version: " + version);
}
} catch (AvroRuntimeException e) {
throw new IOException(e);
}
} else {
throw new IOException("Incompatible event log version: " + version);
throw new IOException("Event schema string not parsed since its null");
}
}