HDFS-15050. Optimize log information when DFSInputStream meet CannotObtainBlockLengthException. Contributed by Xiaoqiao He.
Signed-off-by: Wei-Chiu Chuang <weichiu@apache.org> (cherry picked from commit 99af49878cc47f5750b6c0edadc8a4c2fcd7a7c6)
This commit is contained in:
parent
d19981fe48
commit
8ba4e13dcc
|
@ -52,4 +52,16 @@ public class CannotObtainBlockLengthException extends IOException {
|
||||||
super("Cannot obtain block length for " + locatedBlock);
|
super("Cannot obtain block length for " + locatedBlock);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs an {@code CannotObtainBlockLengthException} with the
|
||||||
|
* specified LocatedBlock and file that failed to obtain block length.
|
||||||
|
*
|
||||||
|
* @param locatedBlock
|
||||||
|
* The LocatedBlock instance which block length can not be obtained
|
||||||
|
* @param src The file which include this block
|
||||||
|
*/
|
||||||
|
public CannotObtainBlockLengthException(LocatedBlock locatedBlock,
|
||||||
|
String src) {
|
||||||
|
super("Cannot obtain block length for " + locatedBlock + " of " + src);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -365,7 +365,7 @@ public class DFSInputStream extends FSInputStream
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
throw new CannotObtainBlockLengthException(locatedblock);
|
throw new CannotObtainBlockLengthException(locatedblock, src);
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getFileLength() {
|
public long getFileLength() {
|
||||||
|
|
Loading…
Reference in New Issue