YARN-9837. Fixed reading YARN Service JSON spec file larger than 128k.
Contributed by Tarun Parimi
This commit is contained in:
parent
c474e24c0b
commit
eefe9bc85c
@ -27,14 +27,12 @@
|
||||
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 T fromBytes(byte[] b) throws IOException {
|
||||
* @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);
|
||||
}
|
||||
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user