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 24ec2926647..584b71f1ee5 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 @@ -34,7 +34,6 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationExcep import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; -import org.apache.hadoop.fs.azurebfs.oauth2.AzureADAuthenticator.HttpException; /** * The AbfsRestOperation for Rest AbfsClient. @@ -233,7 +232,13 @@ public class AbfsRestOperation { hasRequestBody ? bufferLength : 0); break; } + } catch (IOException e) { + LOG.debug("Auth failure: {}, {}", method, url); + throw new AbfsRestOperationException(-1, null, + "Auth failure: " + e.getMessage(), e); + } + try { // dump the headers AbfsIoUtils.dumpHeadersToDebugLog("Request Headers", httpOperation.getConnection().getRequestProperties()); @@ -256,9 +261,7 @@ public class AbfsRestOperation { } } catch (UnknownHostException ex) { String hostname = null; - if (httpOperation != null) { - hostname = httpOperation.getHost(); - } + hostname = httpOperation.getHost(); LOG.warn("Unknown host name: %s. Retrying to resolve the host name...", hostname); if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) { @@ -267,24 +270,13 @@ public class AbfsRestOperation { return false; } catch (IOException ex) { if (LOG.isDebugEnabled()) { - if (httpOperation != null) { - LOG.debug("HttpRequestFailure: " + httpOperation.toString(), ex); - } else { - LOG.debug("HttpRequestFailure: " + method + "," + url, ex); - } + LOG.debug("HttpRequestFailure: {}, {}", httpOperation.toString(), ex); } if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) { throw new InvalidAbfsRestOperationException(ex); } - // once HttpException is thrown by AzureADAuthenticator, - // it indicates the policy in AzureADAuthenticator determined - // retry is not needed - if (ex instanceof HttpException) { - throw new AbfsRestOperationException((HttpException) ex); - } - return false; } finally { AbfsClientThrottlingIntercept.updateMetrics(operationType, httpOperation); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java index 1d86de7ebeb..a71e7bc815f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java @@ -20,21 +20,31 @@ package org.apache.hadoop.fs.azurebfs; 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; -import org.junit.Assert; -import org.junit.Test; - +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DOT; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME; import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * 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"; + public ITestAbfsRestOperationException() throws Exception { super(); } @@ -114,4 +124,35 @@ public class ITestAbfsRestOperationException extends AbstractAbfsIntegrationTest + ") done, does not match with fs.azure.custom.token.fetch.retry.count configured (" + numOfRetries + ")", RetryTestTokenProvider.reTryCount == numOfRetries); } + + @Test + public void testAuthFailException() throws Exception { + Configuration config = new Configuration(getRawConfiguration()); + String accountName = config + .get(FS_AZURE_ABFS_ACCOUNT_NAME); + // Setup to configure custom token provider + config.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME + DOT + + accountName, "Custom"); + config.set( + FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME + DOT + accountName, + RETRY_TEST_TOKEN_PROVIDER); + // Stop filesystem creation as it will lead to calls to store. + config.set(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, "false"); + + final AzureBlobFileSystem fs = getFileSystem(config); + try { + fs.getFileStatus(new Path("/")); + fail("Should fail at auth token fetch call"); + } catch (AbfsRestOperationException e) { + String errorDesc = "Should throw RestOp exception on AAD failure"; + Assertions.assertThat(e.getStatusCode()) + .describedAs("Incorrect status code. " + errorDesc).isEqualTo(-1); + Assertions.assertThat(e.getErrorCode()) + .describedAs("Incorrect error code. " + errorDesc) + .isEqualTo(AzureServiceErrorCode.UNKNOWN); + Assertions.assertThat(e.getErrorMessage()) + .describedAs("Incorrect error message. " + errorDesc) + .contains("Auth failure: "); + } + } } \ No newline at end of file