Forráskód Böngészése

HADOOP-17338 Intermittent S3AInputStream failures: Premature end of Content-Length delimited message body etc (#2692))

Backported from commit ID 3d2193cd648

Contributed by Yongjun Zhang
yzhangal 4 éve
szülő
commit
bac234b0a8

+ 42 - 14
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.fs.s3a;
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.services.s3.AmazonS3;
 import com.amazonaws.services.s3.model.GetObjectRequest;
+import com.amazonaws.services.s3.model.S3Object;
 import com.amazonaws.services.s3.model.S3ObjectInputStream;
 import com.amazonaws.services.s3.model.SSECustomerKey;
 import com.google.common.base.Preconditions;
@@ -72,6 +73,14 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
    * set
    */
   private volatile boolean closed;
+  /**
+   * wrappedStream is associated with an object (instance of S3Object). When
+   * the object is garbage collected, the associated wrappedStream will be
+   * closed. Keep a reference to this object to prevent the wrapperStream
+   * still in use from being closed unexpectedly due to garbage collection.
+   * See HADOOP-17338 for details.
+   */
+  private S3Object object;
   private S3ObjectInputStream wrappedStream;
   private final FileSystem.Statistics stats;
   private final AmazonS3 client;
@@ -157,7 +166,8 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
           StringUtils.isNotBlank(serverSideEncryptionKey)){
         request.setSSECustomerKey(new SSECustomerKey(serverSideEncryptionKey));
       }
-      wrappedStream = client.getObject(request).getObjectContent();
+      object = client.getObject(request);
+      wrappedStream = object.getObjectContent();
       contentRangeStart = targetPos;
       if (wrappedStream == null) {
         throw new IOException("Null IO stream from reopen of (" + reason +  ") "
@@ -345,9 +355,15 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
    * @throws IOException any exception thrown on the re-open attempt.
    */
   private void onReadFailure(IOException ioe, int length) throws IOException {
-    LOG.info("Got exception while trying to read from stream {}"
-        + " trying to recover: "+ ioe, uri);
-    LOG.debug("While trying to read from stream {}", uri, ioe);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Got exception while trying to read from stream {}, " +
+                "client: {} object: {}, trying to recover: ",
+                uri, client, object, ioe);
+    } else {
+      LOG.info("Got exception while trying to read from stream {}, " +
+                "client: {} object: {}, trying to recover: " + ioe,
+                uri, client, object);
+    }
     streamStatistics.readException();
     reopen("failure recovery", pos, length);
   }
@@ -450,14 +466,19 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
    * @param forceAbort force an abort; used if explicitly requested.
    */
   private void closeStream(String reason, long length, boolean forceAbort) {
-    if (wrappedStream != null) {
+    if (wrappedStream == null) {
+      // steam is already closed
+      return;
+    }
+
+    // if the amount of data remaining in the current request is greater
+    // than the readahead value: abort.
+    long remaining = remainingInCurrentRequest();
+    LOG.debug("Closing stream {}: {}", reason,
+        forceAbort ? "abort" : "soft");
+    boolean shouldAbort = forceAbort || remaining > readahead;
 
-      // if the amount of data remaining in the current request is greater
-      // than the readahead value: abort.
-      long remaining = remainingInCurrentRequest();
-      LOG.debug("Closing stream {}: {}", reason,
-          forceAbort ? "abort" : "soft");
-      boolean shouldAbort = forceAbort || remaining > readahead;
+    try {
       if (!shouldAbort) {
         try {
           // clean close. This will read to the end of the stream,
@@ -485,18 +506,25 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
       if (shouldAbort) {
         // Abort, rather than just close, the underlying stream.  Otherwise, the
         // remaining object payload is read from S3 while closing the stream.
-        LOG.debug("Aborting stream");
-        wrappedStream.abort();
+        LOG.debug("Aborting stream {}", uri);
+        try {
+          wrappedStream.abort();
+        } catch (Exception e) {
+          LOG.warn("When aborting {} stream after failing to close it for {}",
+              uri, reason, e);
+        }
         streamStatistics.streamClose(true, remaining);
       }
       LOG.debug("Stream {} {}: {}; remaining={} streamPos={},"
               + " nextReadPos={}," +
-          " request range {}-{} length={}",
+              " request range {}-{} length={}",
           uri, (shouldAbort ? "aborted" : "closed"), reason,
           remaining, pos, nextReadPos,
           contentRangeStart, contentRangeFinish,
           length);
+    } finally {
       wrappedStream = null;
+      object = null;
     }
   }