HADOOP-17836. Improve logging on ABFS error reporting (#3281)
Contributed by Steve Loughran.
This commit is contained in:
parent
e12bc4e1d8
commit
b75ced1e5d
|
@ -24,7 +24,8 @@ import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
|
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
|
@InterfaceAudience.Public
|
||||||
@InterfaceStability.Evolving
|
@InterfaceStability.Evolving
|
||||||
|
@ -34,7 +35,9 @@ public class InvalidAbfsRestOperationException extends AbfsRestOperationExceptio
|
||||||
super(
|
super(
|
||||||
AzureServiceErrorCode.UNKNOWN.getStatusCode(),
|
AzureServiceErrorCode.UNKNOWN.getStatusCode(),
|
||||||
AzureServiceErrorCode.UNKNOWN.getErrorCode(),
|
AzureServiceErrorCode.UNKNOWN.getErrorCode(),
|
||||||
"InvalidAbfsRestOperationException",
|
innerException != null
|
||||||
|
? innerException.toString()
|
||||||
|
: "InvalidAbfsRestOperationException",
|
||||||
innerException);
|
innerException);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -409,7 +409,9 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} catch (IOException ex) {
|
} catch (IOException ex) {
|
||||||
LOG.error("UnexpectedError: ", ex);
|
LOG.warn("IO/Network error: {} {}: {}",
|
||||||
|
method, getMaskedUrl(), ex.getMessage());
|
||||||
|
LOG.debug("IO Error: ", ex);
|
||||||
throw ex;
|
throw ex;
|
||||||
} finally {
|
} finally {
|
||||||
if (this.isTraceEnabled) {
|
if (this.isTraceEnabled) {
|
||||||
|
|
|
@ -303,7 +303,7 @@ public class AbfsRestOperation {
|
||||||
} catch (UnknownHostException ex) {
|
} catch (UnknownHostException ex) {
|
||||||
String hostname = null;
|
String hostname = null;
|
||||||
hostname = httpOperation.getHost();
|
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);
|
hostname);
|
||||||
if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) {
|
if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) {
|
||||||
throw new InvalidAbfsRestOperationException(ex);
|
throw new InvalidAbfsRestOperationException(ex);
|
||||||
|
@ -311,7 +311,7 @@ public class AbfsRestOperation {
|
||||||
return false;
|
return false;
|
||||||
} catch (IOException ex) {
|
} catch (IOException ex) {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("HttpRequestFailure: {}, {}", httpOperation.toString(), ex);
|
LOG.debug("HttpRequestFailure: {}, {}", httpOperation, ex);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) {
|
if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) {
|
||||||
|
@ -323,7 +323,7 @@ public class AbfsRestOperation {
|
||||||
intercept.updateMetrics(operationType, httpOperation);
|
intercept.updateMetrics(operationType, httpOperation);
|
||||||
}
|
}
|
||||||
|
|
||||||
LOG.debug("HttpRequest: {}: {}", operationType, httpOperation.toString());
|
LOG.debug("HttpRequest: {}: {}", operationType, httpOperation);
|
||||||
|
|
||||||
if (client.getRetryPolicy().shouldRetry(retryCount, httpOperation.getStatusCode())) {
|
if (client.getRetryPolicy().shouldRetry(retryCount, httpOperation.getStatusCode())) {
|
||||||
return false;
|
return false;
|
||||||
|
|
|
@ -455,8 +455,8 @@ final class ReadBufferManager {
|
||||||
*/
|
*/
|
||||||
void doneReading(final ReadBuffer buffer, final ReadBufferStatus result, final int bytesActuallyRead) {
|
void doneReading(final ReadBuffer buffer, final ReadBufferStatus result, final int bytesActuallyRead) {
|
||||||
if (LOGGER.isTraceEnabled()) {
|
if (LOGGER.isTraceEnabled()) {
|
||||||
LOGGER.trace("ReadBufferWorker completed file {} for offset {} bytes {}",
|
LOGGER.trace("ReadBufferWorker completed read file {} for offset {} outcome {} bytes {}",
|
||||||
buffer.getStream().getPath(), buffer.getOffset(), bytesActuallyRead);
|
buffer.getStream().getPath(), buffer.getOffset(), result, bytesActuallyRead);
|
||||||
}
|
}
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
// If this buffer has already been purged during
|
// If this buffer has already been purged during
|
||||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.fs.azurebfs.services;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.PathIOException;
|
||||||
import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
|
import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
|
||||||
|
|
||||||
class ReadBufferWorker implements Runnable {
|
class ReadBufferWorker implements Runnable {
|
||||||
|
@ -73,8 +74,11 @@ class ReadBufferWorker implements Runnable {
|
||||||
buffer.getTracingContext());
|
buffer.getTracingContext());
|
||||||
|
|
||||||
bufferManager.doneReading(buffer, ReadBufferStatus.AVAILABLE, bytesRead); // post result back to ReadBufferManager
|
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) {
|
} catch (Exception ex) {
|
||||||
buffer.setErrException(new IOException(ex));
|
buffer.setErrException(new PathIOException(buffer.getStream().getPath(), ex));
|
||||||
bufferManager.doneReading(buffer, ReadBufferStatus.READ_FAILED, 0);
|
bufferManager.doneReading(buffer, ReadBufferStatus.READ_FAILED, 0);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue