Skip to content

Commit 75a398d

Browse files
mehakmeetarjun4084346
authored andcommitted
HADOOP-17229. No updation of bytes received counter value after response failure occurs in ABFS (#2264)
Contributed by Mehakmeet Singh (cherry picked from commit 0d85515)
1 parent 0ca5415 commit 75a398d

File tree

2 files changed

+34
-2
lines changed

2 files changed

+34
-2
lines changed

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

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -177,8 +177,12 @@ private boolean executeHttpOperation(final int retryCount) throws AzureBlobFileS
177177

178178
httpOperation.processResponse(buffer, bufferOffset, bufferLength);
179179
incrementCounter(AbfsStatistic.GET_RESPONSES, 1);
180-
incrementCounter(AbfsStatistic.BYTES_RECEIVED,
181-
httpOperation.getBytesReceived());
180+
//Only increment bytesReceived counter when the status code is 2XX.
181+
if (httpOperation.getStatusCode() >= HttpURLConnection.HTTP_OK
182+
&& httpOperation.getStatusCode() <= HttpURLConnection.HTTP_PARTIAL) {
183+
incrementCounter(AbfsStatistic.BYTES_RECEIVED,
184+
httpOperation.getBytesReceived());
185+
}
182186
} catch (IOException ex) {
183187
if (ex instanceof UnknownHostException) {
184188
LOG.warn(String.format("Unknown host name: %s. Retrying to resolve the host name...", httpOperation.getUrl().getHost()));

hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java

Lines changed: 28 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727

2828
import org.apache.hadoop.fs.FSDataInputStream;
2929
import org.apache.hadoop.fs.FSDataOutputStream;
30+
import org.apache.hadoop.fs.FileAlreadyExistsException;
3031
import org.apache.hadoop.io.IOUtils;
3132
import org.apache.hadoop.fs.Path;
3233
import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
@@ -250,4 +251,31 @@ public void testAbfsHttpResponseStatistics() throws IOException {
250251
}
251252
}
252253

254+
/**
255+
* Testing bytes_received counter value when a response failure occurs.
256+
*/
257+
@Test
258+
public void testAbfsHttpResponseFailure() throws IOException {
259+
describe("Test to check the values of bytes received counter when a "
260+
+ "response is failed");
261+
262+
AzureBlobFileSystem fs = getFileSystem();
263+
Path responseFailurePath = path(getMethodName());
264+
Map<String, Long> metricMap;
265+
FSDataOutputStream out = null;
266+
267+
try {
268+
//create an empty file
269+
out = fs.create(responseFailurePath);
270+
//Re-creating the file again on same path with false overwrite, this
271+
// would cause a response failure with status code 409.
272+
out = fs.create(responseFailurePath, false);
273+
} catch (FileAlreadyExistsException faee) {
274+
metricMap = fs.getInstrumentationMap();
275+
// Assert after catching the 409 error to check the counter values.
276+
assertAbfsStatistics(AbfsStatistic.BYTES_RECEIVED, 0, metricMap);
277+
} finally {
278+
IOUtils.cleanupWithLogger(LOG, out);
279+
}
280+
}
253281
}

0 commit comments

Comments
 (0)