diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/JsonSerDeser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/JsonSerDeser.java index 00b8e0cdbcb..254d6c5d379 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/JsonSerDeser.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/JsonSerDeser.java @@ -27,14 +27,12 @@ import com.fasterxml.jackson.databind.PropertyNamingStrategy; import com.fasterxml.jackson.databind.SerializationFeature; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IOUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.EOFException; import java.io.File; import java.io.FileNotFoundException; import java.io.FileOutputStream; @@ -176,17 +174,9 @@ public class JsonSerDeser { * @throws JsonParseException parse problems * @throws JsonMappingException O/J mapping problems */ - public T load(FileSystem fs, Path path) - throws IOException, JsonParseException, JsonMappingException { - FileStatus status = fs.getFileStatus(path); - long len = status.getLen(); - byte[] b = new byte[(int) len]; + public T load(FileSystem fs, Path path) throws IOException { FSDataInputStream dataInputStream = fs.open(path); - int count = dataInputStream.read(b); - if (count != len) { - throw new EOFException("Read of " + path +" finished prematurely"); - } - return fromBytes(b); + return fromStream(dataInputStream); }