HADOOP-15662. Better exception handling of DNS errors.

Contributed by Da Zhou.
This commit is contained in:
Da Zhou 2019-01-10 12:02:58 +00:00 committed by Steve Loughran
parent e8d1900369
commit 7211269142
No known key found for this signature in database
GPG Key ID: D22CF846DBB162A0
2 changed files with 37 additions and 2 deletions

View File

@ -21,6 +21,7 @@
import java.io.IOException;
import java.net.HttpURLConnection;
import java.net.URL;
import java.net.UnknownHostException;
import java.util.List;
import org.slf4j.Logger;
@ -33,6 +34,8 @@
import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
import org.apache.hadoop.fs.azurebfs.oauth2.AzureADAuthenticator.HttpException;
import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.UNKNOWN;
/**
* The AbfsRestOperation for Rest AbfsClient.
*/
@ -173,6 +176,15 @@ private boolean executeHttpOperation(final int retryCount) throws AzureBlobFileS
LOG.debug("HttpRequestFailure: " + method + "," + url, ex);
}
}
if (ex instanceof UnknownHostException) {
throw new AbfsRestOperationException(
UNKNOWN.getStatusCode(),
UNKNOWN.getErrorCode(),
String.format("Can not reach endpoint: %s, please check the account setting in configuration file", ex.getMessage()),
ex);
}
if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) {
throw new InvalidAbfsRestOperationException(ex);
}

View File

@ -18,11 +18,19 @@
package org.apache.hadoop.fs.azurebfs;
import java.util.UUID;
import org.junit.Assert;
import org.junit.Test;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
import org.junit.Assert;
import org.junit.Test;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/**
* Test continuation token which has equal sign.
@ -46,4 +54,19 @@ public void testContinuationTokenHavingEqualSign() throws Exception {
Assert.assertEquals("InvalidQueryParameterValue", ex.getErrorCode().getErrorCode());
}
}
@Test
public void verifyUnknownHost() throws Exception {
AbfsConfiguration conf = this.getConfiguration();
String accountName = this.getAccountName();
String fakeAccountName = "fake" + UUID.randomUUID() + accountName.substring(accountName.indexOf("."));
String fsDefaultFS = conf.get(FS_DEFAULT_NAME_KEY);
conf.set(FS_DEFAULT_NAME_KEY, fsDefaultFS.replace(accountName, fakeAccountName));
conf.set(FS_AZURE_ACCOUNT_KEY + "." + fakeAccountName, this.getAccountKey());
intercept(AbfsRestOperationException.class,
"Can not reach endpoint: " + fakeAccountName,
() -> FileSystem.get(conf.getRawConfiguration()));
}
}