|
@@ -20,20 +20,43 @@ package org.apache.hadoop.fs.azurebfs.services;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.lang.reflect.Field;
|
|
|
+import java.net.HttpURLConnection;
|
|
|
+import java.net.ProtocolException;
|
|
|
import java.net.URL;
|
|
|
import java.util.List;
|
|
|
+import java.util.Random;
|
|
|
import java.util.regex.Pattern;
|
|
|
|
|
|
+import org.assertj.core.api.Assertions;
|
|
|
import org.junit.Test;
|
|
|
+import org.mockito.Mockito;
|
|
|
|
|
|
+import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
|
|
|
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
|
|
|
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
|
|
|
+import org.apache.hadoop.fs.azurebfs.TestAbfsConfigurationFieldsValidation;
|
|
|
+import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
|
|
|
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
|
|
|
+import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters;
|
|
|
import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
|
|
|
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
|
|
|
+import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat;
|
|
|
import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
|
|
|
|
|
|
-import static org.assertj.core.api.Assertions.assertThat;
|
|
|
+import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
|
|
|
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_ACTION;
|
|
|
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PATCH;
|
|
|
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT;
|
|
|
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE;
|
|
|
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT;
|
|
|
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE;
|
|
|
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_ACTION;
|
|
|
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_POSITION;
|
|
|
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME;
|
|
|
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
|
|
import static org.mockito.ArgumentMatchers.any;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
@@ -59,14 +82,19 @@ import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST
|
|
|
* Test useragent of abfs client.
|
|
|
*
|
|
|
*/
|
|
|
-public final class TestAbfsClient {
|
|
|
+public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
|
|
|
|
|
|
private static final String ACCOUNT_NAME = "bogusAccountName.dfs.core.windows.net";
|
|
|
private static final String FS_AZURE_USER_AGENT_PREFIX = "Partner Service";
|
|
|
+ private static final String TEST_PATH = "/testfile";
|
|
|
+ public static final int REDUCED_RETRY_COUNT = 2;
|
|
|
+ public static final int REDUCED_BACKOFF_INTERVAL = 100;
|
|
|
+ public static final int BUFFER_LENGTH = 5;
|
|
|
+ public static final int BUFFER_OFFSET = 0;
|
|
|
|
|
|
private final Pattern userAgentStringPattern;
|
|
|
|
|
|
- public TestAbfsClient(){
|
|
|
+ public ITestAbfsClient() throws Exception {
|
|
|
StringBuilder regEx = new StringBuilder();
|
|
|
regEx.append("^");
|
|
|
regEx.append(APN_VERSION);
|
|
@@ -124,7 +152,7 @@ public final class TestAbfsClient {
|
|
|
}
|
|
|
|
|
|
private void verifybBasicInfo(String userAgentStr) {
|
|
|
- assertThat(userAgentStr)
|
|
|
+ Assertions.assertThat(userAgentStr)
|
|
|
.describedAs("User-Agent string [" + userAgentStr
|
|
|
+ "] should be of the pattern: " + this.userAgentStringPattern.pattern())
|
|
|
.matches(this.userAgentStringPattern)
|
|
@@ -153,7 +181,7 @@ public final class TestAbfsClient {
|
|
|
String userAgentStr = getUserAgentString(abfsConfiguration, false);
|
|
|
|
|
|
verifybBasicInfo(userAgentStr);
|
|
|
- assertThat(userAgentStr)
|
|
|
+ Assertions.assertThat(userAgentStr)
|
|
|
.describedAs("User-Agent string should contain " + FS_AZURE_USER_AGENT_PREFIX)
|
|
|
.contains(FS_AZURE_USER_AGENT_PREFIX);
|
|
|
|
|
@@ -163,7 +191,7 @@ public final class TestAbfsClient {
|
|
|
userAgentStr = getUserAgentString(abfsConfiguration, false);
|
|
|
|
|
|
verifybBasicInfo(userAgentStr);
|
|
|
- assertThat(userAgentStr)
|
|
|
+ Assertions.assertThat(userAgentStr)
|
|
|
.describedAs("User-Agent string should not contain " + FS_AZURE_USER_AGENT_PREFIX)
|
|
|
.doesNotContain(FS_AZURE_USER_AGENT_PREFIX);
|
|
|
}
|
|
@@ -179,14 +207,14 @@ public final class TestAbfsClient {
|
|
|
String userAgentStr = getUserAgentString(abfsConfiguration, true);
|
|
|
|
|
|
verifybBasicInfo(userAgentStr);
|
|
|
- assertThat(userAgentStr)
|
|
|
+ Assertions.assertThat(userAgentStr)
|
|
|
.describedAs("User-Agent string should contain sslProvider")
|
|
|
.contains(DelegatingSSLSocketFactory.getDefaultFactory().getProviderName());
|
|
|
|
|
|
userAgentStr = getUserAgentString(abfsConfiguration, false);
|
|
|
|
|
|
verifybBasicInfo(userAgentStr);
|
|
|
- assertThat(userAgentStr)
|
|
|
+ Assertions.assertThat(userAgentStr)
|
|
|
.describedAs("User-Agent string should not contain sslProvider")
|
|
|
.doesNotContain(DelegatingSSLSocketFactory.getDefaultFactory().getProviderName());
|
|
|
}
|
|
@@ -202,7 +230,7 @@ public final class TestAbfsClient {
|
|
|
String userAgentStr = getUserAgentString(abfsConfiguration, false);
|
|
|
|
|
|
verifybBasicInfo(userAgentStr);
|
|
|
- assertThat(userAgentStr)
|
|
|
+ Assertions.assertThat(userAgentStr)
|
|
|
.describedAs("User-Agent string should contain cluster name")
|
|
|
.contains(clusterName);
|
|
|
|
|
@@ -212,7 +240,7 @@ public final class TestAbfsClient {
|
|
|
userAgentStr = getUserAgentString(abfsConfiguration, false);
|
|
|
|
|
|
verifybBasicInfo(userAgentStr);
|
|
|
- assertThat(userAgentStr)
|
|
|
+ Assertions.assertThat(userAgentStr)
|
|
|
.describedAs("User-Agent string should not contain cluster name")
|
|
|
.doesNotContain(clusterName)
|
|
|
.describedAs("User-Agent string should contain UNKNOWN as cluster name config is absent")
|
|
@@ -230,7 +258,7 @@ public final class TestAbfsClient {
|
|
|
String userAgentStr = getUserAgentString(abfsConfiguration, false);
|
|
|
|
|
|
verifybBasicInfo(userAgentStr);
|
|
|
- assertThat(userAgentStr)
|
|
|
+ Assertions.assertThat(userAgentStr)
|
|
|
.describedAs("User-Agent string should contain cluster type")
|
|
|
.contains(clusterType);
|
|
|
|
|
@@ -240,7 +268,7 @@ public final class TestAbfsClient {
|
|
|
userAgentStr = getUserAgentString(abfsConfiguration, false);
|
|
|
|
|
|
verifybBasicInfo(userAgentStr);
|
|
|
- assertThat(userAgentStr)
|
|
|
+ Assertions.assertThat(userAgentStr)
|
|
|
.describedAs("User-Agent string should not contain cluster type")
|
|
|
.doesNotContain(clusterType)
|
|
|
.describedAs("User-Agent string should contain UNKNOWN as cluster type config is absent")
|
|
@@ -311,24 +339,23 @@ public final class TestAbfsClient {
|
|
|
AbfsThrottlingInterceptFactory.getInstance(
|
|
|
abfsConfig.getAccountName().substring(0,
|
|
|
abfsConfig.getAccountName().indexOf(DOT)), abfsConfig));
|
|
|
-
|
|
|
// override baseurl
|
|
|
- client = TestAbfsClient.setAbfsClientField(client, "abfsConfiguration",
|
|
|
+ client = ITestAbfsClient.setAbfsClientField(client, "abfsConfiguration",
|
|
|
abfsConfig);
|
|
|
|
|
|
// override baseurl
|
|
|
- client = TestAbfsClient.setAbfsClientField(client, "baseUrl",
|
|
|
+ client = ITestAbfsClient.setAbfsClientField(client, "baseUrl",
|
|
|
baseAbfsClientInstance.getBaseUrl());
|
|
|
|
|
|
// override auth provider
|
|
|
if (currentAuthType == AuthType.SharedKey) {
|
|
|
- client = TestAbfsClient.setAbfsClientField(client, "sharedKeyCredentials",
|
|
|
+ client = ITestAbfsClient.setAbfsClientField(client, "sharedKeyCredentials",
|
|
|
new SharedKeyCredentials(
|
|
|
abfsConfig.getAccountName().substring(0,
|
|
|
abfsConfig.getAccountName().indexOf(DOT)),
|
|
|
abfsConfig.getStorageAccountKey()));
|
|
|
} else {
|
|
|
- client = TestAbfsClient.setAbfsClientField(client, "tokenProvider",
|
|
|
+ client = ITestAbfsClient.setAbfsClientField(client, "tokenProvider",
|
|
|
abfsConfig.getTokenProvider());
|
|
|
}
|
|
|
|
|
@@ -336,7 +363,7 @@ public final class TestAbfsClient {
|
|
|
String userAgent = "APN/1.0 Azure Blob FS/3.4.0-SNAPSHOT (PrivateBuild "
|
|
|
+ "JavaJRE 1.8.0_252; Linux 5.3.0-59-generic/amd64; openssl-1.0; "
|
|
|
+ "UNKNOWN/UNKNOWN) MSFT";
|
|
|
- client = TestAbfsClient.setAbfsClientField(client, "userAgent", userAgent);
|
|
|
+ client = ITestAbfsClient.setAbfsClientField(client, "userAgent", userAgent);
|
|
|
|
|
|
return client;
|
|
|
}
|
|
@@ -404,4 +431,156 @@ public final class TestAbfsClient {
|
|
|
public static AccessTokenProvider getAccessTokenProvider(AbfsClient client) {
|
|
|
return client.getTokenProvider();
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test helper method to get random bytes array.
|
|
|
+ * @param length The length of byte buffer.
|
|
|
+ * @return byte buffer.
|
|
|
+ */
|
|
|
+ private byte[] getRandomBytesArray(int length) {
|
|
|
+ final byte[] b = new byte[length];
|
|
|
+ new Random().nextBytes(b);
|
|
|
+ return b;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test to verify that client retries append request without
|
|
|
+ * expect header enabled if append with expect header enabled fails
|
|
|
+ * with 4xx kind of error.
|
|
|
+ * @throws Exception
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testExpectHundredContinue() throws Exception {
|
|
|
+ // Get the filesystem.
|
|
|
+ final AzureBlobFileSystem fs = getFileSystem();
|
|
|
+
|
|
|
+ final Configuration configuration = new Configuration();
|
|
|
+ configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
|
|
|
+ AbfsClient abfsClient = fs.getAbfsStore().getClient();
|
|
|
+
|
|
|
+ AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration,
|
|
|
+ configuration.get(FS_AZURE_ABFS_ACCOUNT_NAME));
|
|
|
+
|
|
|
+ // Update the configuration with reduced retry count and reduced backoff interval.
|
|
|
+ AbfsConfiguration abfsConfig
|
|
|
+ = TestAbfsConfigurationFieldsValidation.updateRetryConfigs(
|
|
|
+ abfsConfiguration,
|
|
|
+ REDUCED_RETRY_COUNT, REDUCED_BACKOFF_INTERVAL);
|
|
|
+
|
|
|
+ // Gets the client.
|
|
|
+ AbfsClient testClient = Mockito.spy(
|
|
|
+ ITestAbfsClient.createTestClientFromCurrentContext(
|
|
|
+ abfsClient,
|
|
|
+ abfsConfig));
|
|
|
+
|
|
|
+ // Create the append request params with expect header enabled initially.
|
|
|
+ AppendRequestParameters appendRequestParameters
|
|
|
+ = new AppendRequestParameters(
|
|
|
+ BUFFER_OFFSET, BUFFER_OFFSET, BUFFER_LENGTH,
|
|
|
+ AppendRequestParameters.Mode.APPEND_MODE, false, null, true);
|
|
|
+
|
|
|
+ byte[] buffer = getRandomBytesArray(BUFFER_LENGTH);
|
|
|
+
|
|
|
+ // Create a test container to upload the data.
|
|
|
+ Path testPath = path(TEST_PATH);
|
|
|
+ fs.create(testPath);
|
|
|
+ String finalTestPath = testPath.toString()
|
|
|
+ .substring(testPath.toString().lastIndexOf("/"));
|
|
|
+
|
|
|
+ // Creates a list of request headers.
|
|
|
+ final List<AbfsHttpHeader> requestHeaders
|
|
|
+ = ITestAbfsClient.getTestRequestHeaders(testClient);
|
|
|
+ requestHeaders.add(
|
|
|
+ new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH));
|
|
|
+ if (appendRequestParameters.isExpectHeaderEnabled()) {
|
|
|
+ requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE));
|
|
|
+ }
|
|
|
+
|
|
|
+ // Updates the query parameters.
|
|
|
+ final AbfsUriQueryBuilder abfsUriQueryBuilder
|
|
|
+ = testClient.createDefaultUriQueryBuilder();
|
|
|
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, APPEND_ACTION);
|
|
|
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION,
|
|
|
+ Long.toString(appendRequestParameters.getPosition()));
|
|
|
+
|
|
|
+ // Creates the url for the specified path.
|
|
|
+ URL url = testClient.createRequestUrl(finalTestPath, abfsUriQueryBuilder.toString());
|
|
|
+
|
|
|
+ // Create a mock of the AbfsRestOperation to set the urlConnection in the corresponding httpOperation.
|
|
|
+ AbfsRestOperation op = Mockito.spy(new AbfsRestOperation(
|
|
|
+ AbfsRestOperationType.Append,
|
|
|
+ testClient,
|
|
|
+ HTTP_METHOD_PUT,
|
|
|
+ url,
|
|
|
+ requestHeaders, buffer,
|
|
|
+ appendRequestParameters.getoffset(),
|
|
|
+ appendRequestParameters.getLength(), null));
|
|
|
+
|
|
|
+ AbfsHttpOperation abfsHttpOperation = Mockito.spy(new AbfsHttpOperation(url,
|
|
|
+ HTTP_METHOD_PUT, requestHeaders));
|
|
|
+
|
|
|
+ // Sets the expect request property if expect header is enabled.
|
|
|
+ if (appendRequestParameters.isExpectHeaderEnabled()) {
|
|
|
+ Mockito.doReturn(HUNDRED_CONTINUE).when(abfsHttpOperation)
|
|
|
+ .getConnProperty(EXPECT);
|
|
|
+ }
|
|
|
+
|
|
|
+ HttpURLConnection urlConnection = mock(HttpURLConnection.class);
|
|
|
+ Mockito.doNothing().when(urlConnection).setRequestProperty(Mockito
|
|
|
+ .any(), Mockito.any());
|
|
|
+ Mockito.doReturn(HTTP_METHOD_PUT).when(urlConnection).getRequestMethod();
|
|
|
+ Mockito.doReturn(url).when(urlConnection).getURL();
|
|
|
+ Mockito.doReturn(urlConnection).when(abfsHttpOperation).getConnection();
|
|
|
+
|
|
|
+ Mockito.doNothing().when(abfsHttpOperation).setRequestProperty(Mockito
|
|
|
+ .any(), Mockito.any());
|
|
|
+ Mockito.doReturn(url).when(abfsHttpOperation).getConnUrl();
|
|
|
+
|
|
|
+ // Give user error code 404 when processResponse is called.
|
|
|
+ Mockito.doReturn(HTTP_METHOD_PUT).when(abfsHttpOperation).getConnRequestMethod();
|
|
|
+ Mockito.doReturn(HTTP_NOT_FOUND).when(abfsHttpOperation).getConnResponseCode();
|
|
|
+ Mockito.doReturn("Resource Not Found")
|
|
|
+ .when(abfsHttpOperation)
|
|
|
+ .getConnResponseMessage();
|
|
|
+
|
|
|
+ // Make the getOutputStream throw IOException to see it returns from the sendRequest correctly.
|
|
|
+ Mockito.doThrow(new ProtocolException("Server rejected Operation"))
|
|
|
+ .when(abfsHttpOperation)
|
|
|
+ .getConnOutputStream();
|
|
|
+
|
|
|
+ // Sets the httpOperation for the rest operation.
|
|
|
+ Mockito.doReturn(abfsHttpOperation)
|
|
|
+ .when(op)
|
|
|
+ .createHttpOperation();
|
|
|
+
|
|
|
+ // Mock the restOperation for the client.
|
|
|
+ Mockito.doReturn(op)
|
|
|
+ .when(testClient)
|
|
|
+ .getAbfsRestOperationForAppend(Mockito.any(),
|
|
|
+ Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(),
|
|
|
+ Mockito.nullable(int.class), Mockito.nullable(int.class),
|
|
|
+ Mockito.any());
|
|
|
+
|
|
|
+ TracingContext tracingContext = Mockito.spy(new TracingContext("abcd",
|
|
|
+ "abcde", FSOperationType.APPEND,
|
|
|
+ TracingHeaderFormat.ALL_ID_FORMAT, null));
|
|
|
+
|
|
|
+ // Check that expect header is enabled before the append call.
|
|
|
+ Assertions.assertThat(appendRequestParameters.isExpectHeaderEnabled())
|
|
|
+ .describedAs("The expect header is not true before the append call")
|
|
|
+ .isTrue();
|
|
|
+
|
|
|
+ intercept(AzureBlobFileSystemException.class,
|
|
|
+ () -> testClient.append(finalTestPath, buffer, appendRequestParameters, null, tracingContext));
|
|
|
+
|
|
|
+ // Verify that the request was not exponentially retried because of user error.
|
|
|
+ Assertions.assertThat(tracingContext.getRetryCount())
|
|
|
+ .describedAs("The retry count is incorrect")
|
|
|
+ .isEqualTo(0);
|
|
|
+
|
|
|
+ // Verify that the same request was retried with expect header disabled.
|
|
|
+ Assertions.assertThat(appendRequestParameters.isExpectHeaderEnabled())
|
|
|
+ .describedAs("The expect header is not false")
|
|
|
+ .isFalse();
|
|
|
+ }
|
|
|
}
|