Skip to content

Commit 0d85515

Browse files
authored
HADOOP-17229. No updation of bytes received counter value after response failure occurs in ABFS (#2264)
Contributed by Mehakmeet Singh
1 parent 84ed6ad commit 0d85515

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
@@ -248,8 +248,12 @@ private boolean executeHttpOperation(final int retryCount) throws AzureBlobFileS
248248

249249
httpOperation.processResponse(buffer, bufferOffset, bufferLength);
250250
incrementCounter(AbfsStatistic.GET_RESPONSES, 1);
251-
incrementCounter(AbfsStatistic.BYTES_RECEIVED,
252-
httpOperation.getBytesReceived());
251+
//Only increment bytesReceived counter when the status code is 2XX.
252+
if (httpOperation.getStatusCode() >= HttpURLConnection.HTTP_OK
253+
&& httpOperation.getStatusCode() <= HttpURLConnection.HTTP_PARTIAL) {
254+
incrementCounter(AbfsStatistic.BYTES_RECEIVED,
255+
httpOperation.getBytesReceived());
256+
}
253257
} catch (IOException ex) {
254258
if (ex instanceof UnknownHostException) {
255259
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;
@@ -291,4 +292,31 @@ public void testAbfsHttpResponseStatistics() throws IOException {
291292
}
292293
}
293294

295+
/**
296+
* Testing bytes_received counter value when a response failure occurs.
297+
*/
298+
@Test
299+
public void testAbfsHttpResponseFailure() throws IOException {
300+
describe("Test to check the values of bytes received counter when a "
301+
+ "response is failed");
302+
303+
AzureBlobFileSystem fs = getFileSystem();
304+
Path responseFailurePath = path(getMethodName());
305+
Map<String, Long> metricMap;
306+
FSDataOutputStream out = null;
307+
308+
try {
309+
//create an empty file
310+
out = fs.create(responseFailurePath);
311+
//Re-creating the file again on same path with false overwrite, this
312+
// would cause a response failure with status code 409.
313+
out = fs.create(responseFailurePath, false);
314+
} catch (FileAlreadyExistsException faee) {
315+
metricMap = fs.getInstrumentationMap();
316+
// Assert after catching the 409 error to check the counter values.
317+
assertAbfsStatistics(AbfsStatistic.BYTES_RECEIVED, 0, metricMap);
318+
} finally {
319+
IOUtils.cleanupWithLogger(LOG, out);
320+
}
321+
}
294322
}

0 commit comments

Comments
 (0)