Skip to content

Commit

Permalink
HADOOP-18874: [ABFS] Add Server request ID in Exception Messages thro…
Browse files Browse the repository at this point in the history
…wn to the caller. (#6004)


Contributed by Anuj Modi
  • Loading branch information
anujmodi2021 authored Nov 6, 2023
1 parent d634dee commit 597ceaa
Show file tree
Hide file tree
Showing 2 changed files with 97 additions and 50 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -84,20 +84,22 @@ private static String formatMessage(final AbfsHttpOperation abfsHttpOperation) {
// HEAD request response doesn't have StorageErrorCode, StorageErrorMessage.
if (abfsHttpOperation.getMethod().equals("HEAD")) {
return String.format(
"Operation failed: \"%1$s\", %2$s, HEAD, %3$s",
abfsHttpOperation.getStatusDescription(),
abfsHttpOperation.getStatusCode(),
abfsHttpOperation.getMaskedUrl());
"Operation failed: \"%1$s\", %2$s, HEAD, %3$ss, rId: %4$s",
abfsHttpOperation.getStatusDescription(),
abfsHttpOperation.getStatusCode(),
abfsHttpOperation.getMaskedUrl(),
abfsHttpOperation.getRequestId());
}

return String.format(
"Operation failed: \"%1$s\", %2$s, %3$s, %4$s, %5$s, \"%6$s\"",
abfsHttpOperation.getStatusDescription(),
abfsHttpOperation.getStatusCode(),
abfsHttpOperation.getMethod(),
abfsHttpOperation.getMaskedUrl(),
abfsHttpOperation.getStorageErrorCode(),
// Remove break line to ensure the request id and timestamp can be shown in console.
abfsHttpOperation.getStorageErrorMessage().replaceAll("\\n", " "));
"Operation failed: \"%1$s\", %2$s, %3$s, %4$s, rId: %5$s, %6$s, \"%7$s\"",
abfsHttpOperation.getStatusDescription(),
abfsHttpOperation.getStatusCode(),
abfsHttpOperation.getMethod(),
abfsHttpOperation.getMaskedUrl(),
abfsHttpOperation.getRequestId(),
abfsHttpOperation.getStorageErrorCode(),
// Remove break line to ensure the request id and timestamp can be shown in console.
abfsHttpOperation.getStorageErrorMessage().replaceAll("\\n", " "));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,14 +21,12 @@
import java.io.IOException;

import org.assertj.core.api.Assertions;
import org.junit.Assert;
import org.junit.Test;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
import org.apache.hadoop.fs.azurebfs.oauth2.RetryTestTokenProvider;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;

Expand All @@ -43,7 +41,8 @@
* Verify the AbfsRestOperationException error message format.
* */
public class ITestAbfsRestOperationException extends AbstractAbfsIntegrationTest{
private static final String RETRY_TEST_TOKEN_PROVIDER = "org.apache.hadoop.fs.azurebfs.oauth2.RetryTestTokenProvider";
private static final String RETRY_TEST_TOKEN_PROVIDER =
"org.apache.hadoop.fs.azurebfs.oauth2.RetryTestTokenProvider";

public ITestAbfsRestOperationException() throws Exception {
super();
Expand All @@ -55,17 +54,35 @@ public void testAbfsRestOperationExceptionFormat() throws IOException {
Path nonExistedFilePath1 = new Path("nonExistedPath1");
Path nonExistedFilePath2 = new Path("nonExistedPath2");
try {
FileStatus fileStatus = fs.getFileStatus(nonExistedFilePath1);
fs.getFileStatus(nonExistedFilePath1);
} catch (Exception ex) {
String errorMessage = ex.getLocalizedMessage();
String[] errorFields = errorMessage.split(",");

Assert.assertEquals(4, errorFields.length);
// Expected Fields are: Message, StatusCode, Method, URL, ActivityId(rId)
Assertions.assertThat(errorFields)
.describedAs("Number of Fields in exception message are not as expected")
.hasSize(5);
// Check status message, status code, HTTP Request Type and URL.
Assert.assertEquals("Operation failed: \"The specified path does not exist.\"", errorFields[0].trim());
Assert.assertEquals("404", errorFields[1].trim());
Assert.assertEquals("HEAD", errorFields[2].trim());
Assert.assertTrue(errorFields[3].trim().startsWith("http"));
Assertions.assertThat(errorFields[0].trim())
.describedAs("Error Message Field in exception message is wrong")
.isEqualTo("Operation failed: \"The specified path does not exist.\"");
Assertions.assertThat(errorFields[1].trim())
.describedAs("Status Code Field in exception message "
+ "should be \"404\"")
.isEqualTo("404");
Assertions.assertThat(errorFields[2].trim())
.describedAs("Http Rest Method Field in exception message "
+ "should be \"HEAD\"")
.isEqualTo("HEAD");
Assertions.assertThat(errorFields[3].trim())
.describedAs("Url Field in exception message"
+ " should start with \"http\"")
.startsWith("http");
Assertions.assertThat(errorFields[4].trim())
.describedAs("ActivityId Field in exception message "
+ "should start with \"rId:\"")
.startsWith("rId:");
}

try {
Expand All @@ -74,18 +91,43 @@ public void testAbfsRestOperationExceptionFormat() throws IOException {
// verify its format
String errorMessage = ex.getLocalizedMessage();
String[] errorFields = errorMessage.split(",");
// Expected Fields are: Message, StatusCode, Method, URL, ActivityId(rId), StorageErrorCode, StorageErrorMessage.
Assertions.assertThat(errorFields)
.describedAs("fields in exception of %s", ex)
.hasSize(6);
.describedAs("Number of Fields in exception message are not as expected")
.hasSize(7);
// Check status message, status code, HTTP Request Type and URL.
Assert.assertEquals("Operation failed: \"The specified path does not exist.\"", errorFields[0].trim());
Assert.assertEquals("404", errorFields[1].trim());
Assert.assertEquals("GET", errorFields[2].trim());
Assert.assertTrue(errorFields[3].trim().startsWith("http"));
Assertions.assertThat(errorFields[0].trim())
.describedAs("Error Message Field in exception message is wrong")
.isEqualTo("Operation failed: \"The specified path does not exist.\"");
Assertions.assertThat(errorFields[1].trim())
.describedAs("Status Code Field in exception message"
+ " should be \"404\"")
.isEqualTo("404");
Assertions.assertThat(errorFields[2].trim())
.describedAs("Http Rest Method Field in exception message"
+ " should be \"GET\"")
.isEqualTo("GET");
Assertions.assertThat(errorFields[3].trim())
.describedAs("Url Field in exception message"
+ " should start with \"http\"")
.startsWith("http");
Assertions.assertThat(errorFields[4].trim())
.describedAs("ActivityId Field in exception message"
+ " should start with \"rId:\"")
.startsWith("rId:");
// Check storage error code and storage error message.
Assert.assertEquals("PathNotFound", errorFields[4].trim());
Assert.assertTrue(errorFields[5].contains("RequestId")
&& errorFields[5].contains("Time"));
Assertions.assertThat(errorFields[5].trim())
.describedAs("StorageErrorCode Field in exception message"
+ " should be \"PathNotFound\"")
.isEqualTo("PathNotFound");
Assertions.assertThat(errorFields[6].trim())
.describedAs("StorageErrorMessage Field in exception message"
+ " should contain \"RequestId\"")
.contains("RequestId");
Assertions.assertThat(errorFields[6].trim())
.describedAs("StorageErrorMessage Field in exception message"
+ " should contain \"Time\"")
.contains("Time");
}
}

Expand All @@ -101,32 +143,33 @@ public void testWithDifferentCustomTokenFetchRetry(int numOfRetries) throws Exce

Configuration config = new Configuration(this.getRawConfiguration());
String accountName = config.get("fs.azure.abfs.account.name");
// Setup to configure custom token provider
// Setup to configure custom token provider.
config.set("fs.azure.account.auth.type." + accountName, "Custom");
config.set("fs.azure.account.oauth.provider.type." + accountName, "org.apache.hadoop.fs"
+ ".azurebfs.oauth2.RetryTestTokenProvider");
config.set("fs.azure.custom.token.fetch.retry.count", Integer.toString(numOfRetries));
// Stop filesystem creation as it will lead to calls to store.
config.set("fs.azure.createRemoteFileSystemDuringInitialization", "false");

final AzureBlobFileSystem fs1 =
try (final AzureBlobFileSystem fs1 =
(AzureBlobFileSystem) FileSystem.newInstance(fs.getUri(),
config);
RetryTestTokenProvider retryTestTokenProvider
= RetryTestTokenProvider.getCurrentRetryTestProviderInstance(
getAccessTokenProvider(fs1));
retryTestTokenProvider.resetStatusToFirstTokenFetch();

intercept(Exception.class,
()-> {
fs1.getFileStatus(new Path("/"));
});

// Number of retries done should be as configured
Assert.assertEquals(
"Number of token fetch retries done does not match with fs.azure"
+ ".custom.token.fetch.retry.count configured", numOfRetries,
retryTestTokenProvider.getRetryCount());
config)) {
RetryTestTokenProvider retryTestTokenProvider
= RetryTestTokenProvider.getCurrentRetryTestProviderInstance(
getAccessTokenProvider(fs1));
retryTestTokenProvider.resetStatusToFirstTokenFetch();

intercept(Exception.class,
() -> {
fs1.getFileStatus(new Path("/"));
});

// Number of retries done should be as configured
Assertions.assertThat(retryTestTokenProvider.getRetryCount())
.describedAs("Number of token fetch retries done does not "
+ "match with fs.azure.custom.token.fetch.retry.count configured")
.isEqualTo(numOfRetries);
}
}

@Test
Expand All @@ -145,8 +188,10 @@ public void testAuthFailException() throws Exception {

final AzureBlobFileSystem fs = getFileSystem(config);
try {
fs.getFileStatus(new Path("/"));
fail("Should fail at auth token fetch call");
intercept(Exception.class,
() -> {
fs.getFileStatus(new Path("/"));
});
} catch (AbfsRestOperationException e) {
String errorDesc = "Should throw RestOp exception on AAD failure";
Assertions.assertThat(e.getStatusCode())
Expand Down

0 comments on commit 597ceaa

Please sign in to comment.