|
@@ -418,15 +418,21 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|
|
int byteRead = invoker.retry("read", pathStr, true,
|
|
|
() -> {
|
|
|
int b;
|
|
|
+ // When exception happens before re-setting wrappedStream in "reopen" called
|
|
|
+ // by onReadFailure, then wrappedStream will be null. But the **retry** may
|
|
|
+ // re-execute this block and cause NPE if we don't check wrappedStream
|
|
|
+ if (wrappedStream == null) {
|
|
|
+ reopen("failure recovery", getPos(), 1, false);
|
|
|
+ }
|
|
|
try {
|
|
|
b = wrappedStream.read();
|
|
|
} catch (EOFException e) {
|
|
|
return -1;
|
|
|
} catch (SocketTimeoutException e) {
|
|
|
- onReadFailure(e, 1, true);
|
|
|
+ onReadFailure(e, true);
|
|
|
throw e;
|
|
|
} catch (IOException e) {
|
|
|
- onReadFailure(e, 1, false);
|
|
|
+ onReadFailure(e, false);
|
|
|
throw e;
|
|
|
}
|
|
|
return b;
|
|
@@ -444,15 +450,12 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Handle an IOE on a read by attempting to re-open the stream.
|
|
|
+ * Close the stream on read failure.
|
|
|
* The filesystem's readException count will be incremented.
|
|
|
* @param ioe exception caught.
|
|
|
- * @param length length of data being attempted to read
|
|
|
- * @throws IOException any exception thrown on the re-open attempt.
|
|
|
*/
|
|
|
@Retries.OnceTranslated
|
|
|
- private void onReadFailure(IOException ioe, int length, boolean forceAbort)
|
|
|
- throws IOException {
|
|
|
+ private void onReadFailure(IOException ioe, boolean forceAbort) {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Got exception while trying to read from stream {}, " +
|
|
|
"client: {} object: {}, trying to recover: ",
|
|
@@ -463,7 +466,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|
|
uri, client, object);
|
|
|
}
|
|
|
streamStatistics.readException();
|
|
|
- reopen("failure recovery", pos, length, forceAbort);
|
|
|
+ closeStream("failure recovery", contentRangeFinish, forceAbort);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -506,16 +509,22 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|
|
int bytesRead = invoker.retry("read", pathStr, true,
|
|
|
() -> {
|
|
|
int bytes;
|
|
|
+ // When exception happens before re-setting wrappedStream in "reopen" called
|
|
|
+ // by onReadFailure, then wrappedStream will be null. But the **retry** may
|
|
|
+ // re-execute this block and cause NPE if we don't check wrappedStream
|
|
|
+ if (wrappedStream == null) {
|
|
|
+ reopen("failure recovery", getPos(), 1, false);
|
|
|
+ }
|
|
|
try {
|
|
|
bytes = wrappedStream.read(buf, off, len);
|
|
|
} catch (EOFException e) {
|
|
|
// the base implementation swallows EOFs.
|
|
|
return -1;
|
|
|
} catch (SocketTimeoutException e) {
|
|
|
- onReadFailure(e, len, true);
|
|
|
+ onReadFailure(e, true);
|
|
|
throw e;
|
|
|
} catch (IOException e) {
|
|
|
- onReadFailure(e, len, false);
|
|
|
+ onReadFailure(e, false);
|
|
|
throw e;
|
|
|
}
|
|
|
return bytes;
|