HADOOP-17873. ABFS: Fix transient failures in ITestAbfsStreamStatistics and ITestAbfsRestOperationException (#3699)

Successor for the reverted PR #3341, using the hadoop @VisibleForTesting attribute

Contributed by Sumangala Patki
This commit is contained in:
Sumangala Patki 2022-09-06 15:30:52 +05:30 committed by GitHub
parent be4c638e4c
commit 7bcf853ff4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 51 additions and 13 deletions

View File

@ -604,6 +604,7 @@
<exclude>**/azurebfs/ITestAzureBlobFileSystemListStatus.java</exclude> <exclude>**/azurebfs/ITestAzureBlobFileSystemListStatus.java</exclude>
<exclude>**/azurebfs/extensions/ITestAbfsDelegationTokens.java</exclude> <exclude>**/azurebfs/extensions/ITestAbfsDelegationTokens.java</exclude>
<exclude>**/azurebfs/ITestSmallWriteOptimization.java</exclude> <exclude>**/azurebfs/ITestSmallWriteOptimization.java</exclude>
<exclude>**/azurebfs/ITestAbfsStreamStatistics*.java</exclude>
<exclude>**/azurebfs/services/ITestReadBufferManager.java</exclude> <exclude>**/azurebfs/services/ITestReadBufferManager.java</exclude>
<exclude>**/azurebfs/commit/*.java</exclude> <exclude>**/azurebfs/commit/*.java</exclude>
</excludes> </excludes>
@ -647,6 +648,7 @@
<include>**/azurebfs/extensions/ITestAbfsDelegationTokens.java</include> <include>**/azurebfs/extensions/ITestAbfsDelegationTokens.java</include>
<include>**/azurebfs/ITestSmallWriteOptimization.java</include> <include>**/azurebfs/ITestSmallWriteOptimization.java</include>
<include>**/azurebfs/services/ITestReadBufferManager.java</include> <include>**/azurebfs/services/ITestReadBufferManager.java</include>
<include>**/azurebfs/ITestAbfsStreamStatistics*.java</include>
<include>**/azurebfs/commit/*.java</include> <include>**/azurebfs/commit/*.java</include>
</includes> </includes>
</configuration> </configuration>

View File

@ -22,6 +22,7 @@ package org.apache.hadoop.fs.azurebfs.oauth2;
import java.io.IOException; import java.io.IOException;
import java.net.URI; import java.net.URI;
import org.apache.hadoop.classification.VisibleForTesting;
import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.util.Preconditions;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -138,4 +139,9 @@ public final class CustomTokenProviderAdapter extends AccessTokenProvider
String suffix = ExtensionHelper.getUserAgentSuffix(adaptee, ""); String suffix = ExtensionHelper.getUserAgentSuffix(adaptee, "");
return suffix != null ? suffix : ""; return suffix != null ? suffix : "";
} }
@VisibleForTesting
protected CustomTokenProviderAdaptee getCustomTokenProviderAdaptee() {
return adaptee;
}
} }

View File

@ -1289,4 +1289,9 @@ public class AbfsClient implements Closeable {
public <V> void addCallback(ListenableFuture<V> future, FutureCallback<V> callback) { public <V> void addCallback(ListenableFuture<V> future, FutureCallback<V> callback) {
Futures.addCallback(future, callback, executorService); Futures.addCallback(future, callback, executorService);
} }
@VisibleForTesting
protected AccessTokenProvider getTokenProvider() {
return tokenProvider;
}
} }

View File

@ -37,10 +37,12 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager; import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager;
import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.azurebfs.services.AuthType;
import org.apache.hadoop.fs.azurebfs.services.TestAbfsClient;
import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore; import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore;
import org.apache.hadoop.fs.azure.NativeAzureFileSystem; import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation; import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
@ -251,6 +253,9 @@ public abstract class AbstractAbfsIntegrationTest extends
} }
} }
public AccessTokenProvider getAccessTokenProvider(final AzureBlobFileSystem fs) {
return TestAbfsClient.getAccessTokenProvider(fs.getAbfsStore().getClient());
}
public void loadConfiguredFileSystem() throws Exception { public void loadConfiguredFileSystem() throws Exception {
// disable auto-creation of filesystem // disable auto-creation of filesystem

View File

@ -112,7 +112,10 @@ public class ITestAbfsRestOperationException extends AbstractAbfsIntegrationTest
final AzureBlobFileSystem fs1 = final AzureBlobFileSystem fs1 =
(AzureBlobFileSystem) FileSystem.newInstance(fs.getUri(), (AzureBlobFileSystem) FileSystem.newInstance(fs.getUri(),
config); config);
RetryTestTokenProvider.ResetStatusToFirstTokenFetch(); RetryTestTokenProvider retryTestTokenProvider
= RetryTestTokenProvider.getCurrentRetryTestProviderInstance(
getAccessTokenProvider(fs1));
retryTestTokenProvider.resetStatusToFirstTokenFetch();
intercept(Exception.class, intercept(Exception.class,
()-> { ()-> {
@ -120,10 +123,10 @@ public class ITestAbfsRestOperationException extends AbstractAbfsIntegrationTest
}); });
// Number of retries done should be as configured // Number of retries done should be as configured
Assert.assertTrue( Assert.assertEquals(
"Number of token fetch retries (" + RetryTestTokenProvider.reTryCount "Number of token fetch retries done does not match with fs.azure"
+ ") done, does not match with fs.azure.custom.token.fetch.retry.count configured (" + numOfRetries + ".custom.token.fetch.retry.count configured", numOfRetries,
+ ")", RetryTestTokenProvider.reTryCount == numOfRetries); retryTestTokenProvider.getRetryCount());
} }
@Test @Test

View File

@ -30,12 +30,12 @@ import org.slf4j.LoggerFactory;
*/ */
public class RetryTestTokenProvider implements CustomTokenProviderAdaptee { public class RetryTestTokenProvider implements CustomTokenProviderAdaptee {
// Need to track first token fetch otherwise will get counted as a retry too. private static final Logger LOG = LoggerFactory.getLogger(
private static boolean isThisFirstTokenFetch = true; RetryTestTokenProvider.class);
public static int reTryCount = 0;
private static final Logger LOG = LoggerFactory // Need to track first token fetch otherwise will get counted as a retry too.
.getLogger(RetryTestTokenProvider.class); private boolean isThisFirstTokenFetch = true;
private int retryCount = 0;
@Override @Override
public void initialize(Configuration configuration, String accountName) public void initialize(Configuration configuration, String accountName)
@ -43,9 +43,13 @@ public class RetryTestTokenProvider implements CustomTokenProviderAdaptee {
} }
public static void ResetStatusToFirstTokenFetch() { /**
* Clear earlier retry details and reset RetryTestTokenProvider instance to
* state of first access token fetch call.
*/
public void resetStatusToFirstTokenFetch() {
isThisFirstTokenFetch = true; isThisFirstTokenFetch = true;
reTryCount = 0; retryCount = 0;
} }
@Override @Override
@ -53,7 +57,7 @@ public class RetryTestTokenProvider implements CustomTokenProviderAdaptee {
if (isThisFirstTokenFetch) { if (isThisFirstTokenFetch) {
isThisFirstTokenFetch = false; isThisFirstTokenFetch = false;
} else { } else {
reTryCount++; retryCount++;
} }
LOG.debug("RetryTestTokenProvider: Throw an exception in fetching tokens"); LOG.debug("RetryTestTokenProvider: Throw an exception in fetching tokens");
@ -64,4 +68,13 @@ public class RetryTestTokenProvider implements CustomTokenProviderAdaptee {
public Date getExpiryTime() { public Date getExpiryTime() {
return new Date(); return new Date();
} }
public static RetryTestTokenProvider getCurrentRetryTestProviderInstance(
AccessTokenProvider customTokenProvider) {
return (RetryTestTokenProvider) ((CustomTokenProviderAdapter) customTokenProvider).getCustomTokenProviderAdaptee();
}
public int getRetryCount() {
return retryCount;
}
} }

View File

@ -395,4 +395,8 @@ public final class TestAbfsClient {
url, url,
requestHeaders); requestHeaders);
} }
public static AccessTokenProvider getAccessTokenProvider(AbfsClient client) {
return client.getTokenProvider();
}
} }