HADOOP-17229. No update of bytes received counter value after response failure occurs in ABFS (#2264)

Contributed by Mehakmeet Singh

Change-Id: Ia9ad1b87a460b10d27486bd00ee67c3cedd2b5b5
This commit is contained in:
Mehakmeet Singh 2020-09-08 14:44:23 +05:30 committed by Steve Loughran
parent 5710005a19
commit 7970710418
No known key found for this signature in database
GPG Key ID: D22CF846DBB162A0
2 changed files with 34 additions and 2 deletions

View File

@ -248,8 +248,12 @@ public class AbfsRestOperation {
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()));

View File

@ -27,6 +27,7 @@ import org.junit.Test;
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 class ITestAbfsNetworkStatistics extends AbstractAbfsIntegrationTest {
}
}
/**
* 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<String, Long> 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);
}
}
}