From 79707104188bcb62640e00176c80a6ff5c90701b Mon Sep 17 00:00:00 2001 From: Mehakmeet Singh Date: Tue, 8 Sep 2020 14:44:23 +0530 Subject: [PATCH] HADOOP-17229. No update of bytes received counter value after response failure occurs in ABFS (#2264) Contributed by Mehakmeet Singh Change-Id: Ia9ad1b87a460b10d27486bd00ee67c3cedd2b5b5 --- .../azurebfs/services/AbfsRestOperation.java | 8 ++++-- .../azurebfs/ITestAbfsNetworkStatistics.java | 28 +++++++++++++++++++ 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java index 936267aa50..88d5ad0f0a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java @@ -248,8 +248,12 @@ private boolean executeHttpOperation(final int retryCount) throws AzureBlobFileS httpOperation.processResponse(buffer, bufferOffset, bufferLength); incrementCounter(AbfsStatistic.GET_RESPONSES, 1); - incrementCounter(AbfsStatistic.BYTES_RECEIVED, - httpOperation.getBytesReceived()); + //Only increment bytesReceived counter when the status code is 2XX. + if (httpOperation.getStatusCode() >= HttpURLConnection.HTTP_OK + && httpOperation.getStatusCode() <= HttpURLConnection.HTTP_PARTIAL) { + incrementCounter(AbfsStatistic.BYTES_RECEIVED, + httpOperation.getBytesReceived()); + } } catch (IOException ex) { if (ex instanceof UnknownHostException) { LOG.warn(String.format("Unknown host name: %s. Retrying to resolve the host name...", httpOperation.getUrl().getHost())); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java index b2e1301152..e3a97b32c0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; @@ -278,4 +279,31 @@ public void testAbfsHttpResponseStatistics() throws IOException { } } + /** + * Testing bytes_received counter value when a response failure occurs. + */ + @Test + public void testAbfsHttpResponseFailure() throws IOException { + describe("Test to check the values of bytes received counter when a " + + "response is failed"); + + AzureBlobFileSystem fs = getFileSystem(); + Path responseFailurePath = path(getMethodName()); + Map metricMap; + FSDataOutputStream out = null; + + try { + //create an empty file + out = fs.create(responseFailurePath); + //Re-creating the file again on same path with false overwrite, this + // would cause a response failure with status code 409. + out = fs.create(responseFailurePath, false); + } catch (FileAlreadyExistsException faee) { + metricMap = fs.getInstrumentationMap(); + // Assert after catching the 409 error to check the counter values. + assertAbfsStatistics(AbfsStatistic.BYTES_RECEIVED, 0, metricMap); + } finally { + IOUtils.cleanupWithLogger(LOG, out); + } + } }