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

(cherry picked from commit 296ecbd5ec9f7d2eed84217a3318b4a3bcd7bd99)

Conflicts:

	hadoop-mapreduce-project/CHANGES.txt
This commit is contained in:
Jason Lowe 2016-01-15 17:04:34 +00:00
parent 840535f561
commit cbc3288b55
2 changed files with 23 additions and 11 deletions

View File

@ -27,6 +27,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 {
@ -72,9 +72,18 @@ public EventReader(DataInputStream in) throws IOException {
}
Schema myschema = new SpecificData(Event.class.getClassLoader()).getSchema(Event.class);
this.schema = Schema.parse(in.readLine());
this.reader = new SpecificDatumReader(schema, myschema);
this.decoder = DecoderFactory.get().jsonDecoder(schema, in);
String eventschema = in.readLine();
if (null != eventschema) {
try {
this.schema = Schema.parse(eventschema);
this.reader = new SpecificDatumReader(schema, myschema);
this.decoder = DecoderFactory.get().jsonDecoder(schema, in);
} catch (AvroRuntimeException e) {
throw new IOException(e);
}
} else {
throw new IOException("Event schema string not parsed since its null");
}
}
/**