Browse Source

YARN-9837. Fixed reading YARN Service JSON spec file larger than 128k.
Contributed by Tarun Parimi

Eric Yang 5 years ago
parent
commit
dae22c962d

+ 2 - 12
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

@@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.service.utils;
 
 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;
@@ -34,7 +33,6 @@ import org.codehaus.jackson.map.SerializationConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.EOFException;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
@@ -177,17 +175,9 @@ public class JsonSerDeser<T> {
    * @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);
   }