Bläddra i källkod

HADOOP-17836. Improve logging on ABFS error reporting (#3281)

Contributed by Steve Loughran.
Steve Loughran 3 år sedan
förälder
incheckning
419e3e3ec5

+ 5 - 2
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java

@@ -24,7 +24,8 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
 
 /**
- * Exception to wrap invalid Azure service error responses.
+ * Exception to wrap invalid Azure service error responses and exceptions
+ * raised on network IO.
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
@@ -34,7 +35,9 @@ public class InvalidAbfsRestOperationException extends AbfsRestOperationExceptio
     super(
         AzureServiceErrorCode.UNKNOWN.getStatusCode(),
         AzureServiceErrorCode.UNKNOWN.getErrorCode(),
-        "InvalidAbfsRestOperationException",
+        innerException != null
+            ? innerException.toString()
+            : "InvalidAbfsRestOperationException",
         innerException);
   }
 }

+ 3 - 1
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java

@@ -409,7 +409,9 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
           }
         }
       } catch (IOException ex) {
-        LOG.error("UnexpectedError: ", ex);
+        LOG.warn("IO/Network error: {} {}: {}",
+            method, getMaskedUrl(), ex.getMessage());
+        LOG.debug("IO Error: ", ex);
         throw ex;
       } finally {
         if (this.isTraceEnabled) {

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

@@ -303,7 +303,7 @@ public class AbfsRestOperation {
     } catch (UnknownHostException ex) {
       String hostname = null;
       hostname = httpOperation.getHost();
-      LOG.warn("Unknown host name: %s. Retrying to resolve the host name...",
+      LOG.warn("Unknown host name: {}. Retrying to resolve the host name...",
           hostname);
       if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) {
         throw new InvalidAbfsRestOperationException(ex);
@@ -311,7 +311,7 @@ public class AbfsRestOperation {
       return false;
     } catch (IOException ex) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("HttpRequestFailure: {}, {}", httpOperation.toString(), ex);
+        LOG.debug("HttpRequestFailure: {}, {}", httpOperation, ex);
       }
 
       if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) {
@@ -323,7 +323,7 @@ public class AbfsRestOperation {
       intercept.updateMetrics(operationType, httpOperation);
     }
 
-    LOG.debug("HttpRequest: {}: {}", operationType, httpOperation.toString());
+    LOG.debug("HttpRequest: {}: {}", operationType, httpOperation);
 
     if (client.getRetryPolicy().shouldRetry(retryCount, httpOperation.getStatusCode())) {
       return false;

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

@@ -455,8 +455,8 @@ final class ReadBufferManager {
    */
   void doneReading(final ReadBuffer buffer, final ReadBufferStatus result, final int bytesActuallyRead) {
     if (LOGGER.isTraceEnabled()) {
-      LOGGER.trace("ReadBufferWorker completed file {} for offset {} bytes {}",
-          buffer.getStream().getPath(),  buffer.getOffset(), bytesActuallyRead);
+      LOGGER.trace("ReadBufferWorker completed read file {} for offset {} outcome {} bytes {}",
+          buffer.getStream().getPath(),  buffer.getOffset(), result, bytesActuallyRead);
     }
     synchronized (this) {
       // If this buffer has already been purged during

+ 5 - 1
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.fs.azurebfs.services;
 import java.io.IOException;
 import java.util.concurrent.CountDownLatch;
 
+import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
 
 class ReadBufferWorker implements Runnable {
@@ -73,8 +74,11 @@ class ReadBufferWorker implements Runnable {
                   buffer.getTracingContext());
 
           bufferManager.doneReading(buffer, ReadBufferStatus.AVAILABLE, bytesRead);  // post result back to ReadBufferManager
+        } catch (IOException ex) {
+          buffer.setErrException(ex);
+          bufferManager.doneReading(buffer, ReadBufferStatus.READ_FAILED, 0);
         } catch (Exception ex) {
-          buffer.setErrException(new IOException(ex));
+          buffer.setErrException(new PathIOException(buffer.getStream().getPath(), ex));
           bufferManager.doneReading(buffer, ReadBufferStatus.READ_FAILED, 0);
         }
       }