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

Addresses transient failures in the following test classes:

* ITestAbfsStreamStatistics: Uses a filesystem level static instance to record read/write statistics, which also tracks these operations in other tests running in parallel. Marked for sequential-only run to avoid transient failure

* ITestAbfsRestOperationException: The use of a static member to track retry count causes transient failures when two tests of this class happen to run together. Switch to non-static variable for assertions on retry count

closes #3341

Contributed by Sumangala Patki
This commit is contained in:
sumangala-patki 2021-11-04 20:10:37 +05:30 committed by GitHub
parent 72718a8dc1
commit 82658a22d6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 51 additions and 13 deletions

View File

@ -555,6 +555,7 @@
<exclude>**/azurebfs/ITestAzureBlobFileSystemListStatus.java</exclude>
<exclude>**/azurebfs/extensions/ITestAbfsDelegationTokens.java</exclude>
<exclude>**/azurebfs/ITestSmallWriteOptimization.java</exclude>
<exclude>**/azurebfs/ITestAbfsStreamStatistics*.java</exclude>
<exclude>**/azurebfs/services/ITestReadBufferManager.java</exclude>
</excludes>
@ -597,6 +598,7 @@
<include>**/azurebfs/extensions/ITestAbfsDelegationTokens.java</include>
<include>**/azurebfs/ITestSmallWriteOptimization.java</include>
<include>**/azurebfs/services/ITestReadBufferManager.java</include>
<include>**/azurebfs/ITestAbfsStreamStatistics*.java</include>
</includes>
</configuration>
</execution>

View File

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

View File

@ -1223,4 +1223,9 @@ public class AbfsClient implements Closeable {
public <V> void addCallback(ListenableFuture<V> future, FutureCallback<V> callback) {
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.azurebfs.constants.FSOperationType;
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.services.AbfsClient;
import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
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.NativeAzureFileSystem;
import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
@ -241,6 +243,9 @@ public abstract class AbstractAbfsIntegrationTest extends
}
}
public AccessTokenProvider getAccessTokenProvider(final AzureBlobFileSystem fs) {
return TestAbfsClient.getAccessTokenProvider(fs.getAbfsStore().getClient());
}
public void loadConfiguredFileSystem() throws Exception {
// disable auto-creation of filesystem

View File

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

View File

@ -30,12 +30,12 @@ import org.slf4j.LoggerFactory;
*/
public class RetryTestTokenProvider implements CustomTokenProviderAdaptee {
// Need to track first token fetch otherwise will get counted as a retry too.
private static boolean isThisFirstTokenFetch = true;
public static int reTryCount = 0;
private static final Logger LOG = LoggerFactory.getLogger(
RetryTestTokenProvider.class);
private static final Logger LOG = LoggerFactory
.getLogger(RetryTestTokenProvider.class);
// Need to track first token fetch otherwise will get counted as a retry too.
private boolean isThisFirstTokenFetch = true;
private int retryCount = 0;
@Override
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;
reTryCount = 0;
retryCount = 0;
}
@Override
@ -53,7 +57,7 @@ public class RetryTestTokenProvider implements CustomTokenProviderAdaptee {
if (isThisFirstTokenFetch) {
isThisFirstTokenFetch = false;
} else {
reTryCount++;
retryCount++;
}
LOG.debug("RetryTestTokenProvider: Throw an exception in fetching tokens");
@ -64,4 +68,13 @@ public class RetryTestTokenProvider implements CustomTokenProviderAdaptee {
public Date getExpiryTime() {
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,
requestHeaders);
}
public static AccessTokenProvider getAccessTokenProvider(AbfsClient client) {
return client.getTokenProvider();
}
}