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

Contributed by Steve Loughran.
This commit is contained in:
Steve Loughran 2021-08-18 11:39:17 +01:00
parent e12bc4e1d8
commit b75ced1e5d
No known key found for this signature in database
GPG Key ID: D22CF846DBB162A0
5 changed files with 18 additions and 9 deletions

View File

@ -24,7 +24,8 @@
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 InvalidAbfsRestOperationException(
super(
AzureServiceErrorCode.UNKNOWN.getStatusCode(),
AzureServiceErrorCode.UNKNOWN.getErrorCode(),
"InvalidAbfsRestOperationException",
innerException != null
? innerException.toString()
: "InvalidAbfsRestOperationException",
innerException);
}
}

View File

@ -409,7 +409,9 @@ public void processResponse(final byte[] buffer, final int offset, final int len
}
}
} 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) {

View File

@ -303,7 +303,7 @@ private boolean executeHttpOperation(final int retryCount,
} 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 @@ private boolean executeHttpOperation(final int retryCount,
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 @@ private boolean executeHttpOperation(final int retryCount,
intercept.updateMetrics(operationType, httpOperation);
}
LOG.debug("HttpRequest: {}: {}", operationType, httpOperation.toString());
LOG.debug("HttpRequest: {}: {}", operationType, httpOperation);
if (client.getRetryPolicy().shouldRetry(retryCount, httpOperation.getStatusCode())) {
return false;

View File

@ -455,8 +455,8 @@ ReadBuffer getNextBlockToRead() throws InterruptedException {
*/
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

View File

@ -21,6 +21,7 @@
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 @@ public void run() {
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);
}
}