瀏覽代碼

HADOOP-17934 ABFS: Make sure the AbfsHttpOperation is non-null before using it (#3477)

Contributed by: Josh Elser
Josh Elser 3 年之前
父節點
當前提交
6f2fa87fc8

+ 21 - 2
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java

@@ -39,6 +39,7 @@ import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hadoop.thirdparty.com.google.common.base.Strings;
 import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback;
 import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures;
@@ -383,6 +384,10 @@ public class AbfsClient implements Closeable {
     try {
       op.execute(tracingContext);
     } catch (AzureBlobFileSystemException ex) {
+      // If we have no HTTP response, throw the original exception.
+      if (!op.hasResult()) {
+        throw ex;
+      }
       if (!isFile && op.getResult().getStatusCode() == HttpURLConnection.HTTP_CONFLICT) {
         String existingResource =
             op.getResult().getResponseHeader(X_MS_EXISTING_RESOURCE_TYPE);
@@ -506,6 +511,10 @@ public class AbfsClient implements Closeable {
     try {
       op.execute(tracingContext);
     } catch (AzureBlobFileSystemException e) {
+        // If we have no HTTP response, throw the original exception.
+        if (!op.hasResult()) {
+          throw e;
+        }
         final AbfsRestOperation idempotencyOp = renameIdempotencyCheckOp(
             renameRequestStartTime, op, destination, tracingContext);
         if (idempotencyOp.getResult().getStatusCode()
@@ -530,7 +539,7 @@ public class AbfsClient implements Closeable {
    * the one initiated from this ABFS filesytem instance as it was retried. This
    * should be a corner case hence going ahead with LMT check.
    * @param renameRequestStartTime startTime for the rename request
-   * @param op Rename request REST operation response
+   * @param op Rename request REST operation response with non-null HTTP response
    * @param destination rename destination path
    * @param tracingContext Tracks identifiers for request header
    * @return REST operation response post idempotency check
@@ -541,6 +550,7 @@ public class AbfsClient implements Closeable {
       final AbfsRestOperation op,
       final String destination,
       TracingContext tracingContext) throws AzureBlobFileSystemException {
+    Preconditions.checkArgument(op.hasResult(), "Operations has null HTTP response");
     if ((op.isARetriedRequest())
         && (op.getResult().getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND)) {
       // Server has returned HTTP 404, which means rename source no longer
@@ -612,6 +622,10 @@ public class AbfsClient implements Closeable {
     try {
       op.execute(tracingContext);
     } catch (AzureBlobFileSystemException e) {
+      // If we have no HTTP response, throw the original exception.
+      if (!op.hasResult()) {
+        throw e;
+      }
       if (reqParams.isAppendBlob()
           && appendSuccessCheckOp(op, path,
           (reqParams.getPosition() + reqParams.getLength()), tracingContext)) {
@@ -796,6 +810,10 @@ public class AbfsClient implements Closeable {
     try {
     op.execute(tracingContext);
     } catch (AzureBlobFileSystemException e) {
+      // If we have no HTTP response, throw the original exception.
+      if (!op.hasResult()) {
+        throw e;
+      }
       final AbfsRestOperation idempotencyOp = deleteIdempotencyCheckOp(op);
       if (idempotencyOp.getResult().getStatusCode()
           == op.getResult().getStatusCode()) {
@@ -822,10 +840,11 @@ public class AbfsClient implements Closeable {
    * delete issued from this filesystem instance.
    * These are few corner cases and usually returning a success at this stage
    * should help the job to continue.
-   * @param op Delete request REST operation response
+   * @param op Delete request REST operation response with non-null HTTP response
    * @return REST operation response post idempotency check
    */
   public AbfsRestOperation deleteIdempotencyCheckOp(final AbfsRestOperation op) {
+    Preconditions.checkArgument(op.hasResult(), "Operations has null HTTP response");
     if ((op.isARetriedRequest())
         && (op.getResult().getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND)
         && DEFAULT_DELETE_CONSIDERED_IDEMPOTENT) {

+ 8 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java

@@ -71,6 +71,14 @@ public class AbfsRestOperation {
   private AbfsHttpOperation result;
   private AbfsCounters abfsCounters;
 
+  /**
+   * Checks if there is non-null HTTP response.
+   * @return true if there is a non-null HTTP response from the ABFS call.
+   */
+  public boolean hasResult() {
+    return result != null;
+  }
+
   public AbfsHttpOperation getResult() {
     return result;
   }