From ce05bd0627e1320945067227884fdc22a26a971c Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 29 Sep 2020 13:09:06 -0700 Subject: [PATCH 01/77] correlation-id : req-id : retry-count --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 18 ++++++++ .../azurebfs/constants/ConfigurationKeys.java | 5 +++ .../constants/FileSystemConfigurations.java | 3 ++ .../fs/azurebfs/services/AbfsClient.java | 8 ++++ .../azurebfs/services/AbfsHttpOperation.java | 11 +++-- .../azurebfs/services/AbfsRestOperation.java | 6 +++ .../fs/azurebfs/utils/TrackingContext.java | 44 +++++++++++++++++++ 7 files changed, 92 insertions(+), 3 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 72a8a435717e3..9a5f7efc8300c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -217,6 +217,10 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_FS_AZURE_USER_AGENT_PREFIX) private String userAgentId; + @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_CLIENT_CORRELATIONID, + DefaultValue = DEFAULT_FS_AZURE_CLIENT_CORRELATION_ID) + private String clientCorrelationID; + @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_CLUSTER_NAME, DefaultValue = DEFAULT_VALUE_UNKNOWN) private String clusterName; @@ -287,6 +291,20 @@ public String getAccountName() { return accountName; } + /** + * Gets client correlation ID provided in config. + * @return Client Correlation ID config value or empty string (default value) + * if length and character constraints are not met + */ + public String getClientCorrelationID() { + if ((clientCorrelationID.length() > MAX_CLIENT_CORRELATION_ID_LENGTH) || + (!clientCorrelationID.matches(CLIENT_CORRELATION_ID_PATTERN))) { + return DEFAULT_FS_AZURE_CLIENT_CORRELATION_ID; + } + + return clientCorrelationID; + } + /** * Appends an account name to a configuration key yielding the * account-specific form. diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index c15c470a4455f..b97b9daf846cf 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -85,6 +85,11 @@ public final class ConfigurationKeys { * Default value of this config is true. **/ public static final String FS_AZURE_DISABLE_OUTPUTSTREAM_FLUSH = "fs.azure.disable.outputstream.flush"; public static final String FS_AZURE_USER_AGENT_PREFIX_KEY = "fs.azure.user.agent.prefix"; + /** + * The client correlation ID provided over config that will be added to + * x-ms-client-request-Id header. Defaults to empty string if the length and + * character constraints are not satisfied. **/ + public static final String FS_AZURE_CLIENT_CORRELATIONID = "fs.azure.client.correlationid"; public static final String FS_AZURE_CLUSTER_NAME = "fs.azure.cluster.name"; public static final String FS_AZURE_CLUSTER_TYPE = "fs.azure.cluster.type"; public static final String FS_AZURE_SSL_CHANNEL_MODE_KEY = "fs.azure.ssl.channel.mode"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index 2b50a5ddfd40d..889d78a3f5f21 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -91,6 +91,9 @@ public final class FileSystemConfigurations { public static final String DEFAULT_FS_AZURE_USER_AGENT_PREFIX = EMPTY_STRING; public static final String DEFAULT_VALUE_UNKNOWN = "UNKNOWN"; + public static final String DEFAULT_FS_AZURE_CLIENT_CORRELATION_ID = EMPTY_STRING; + public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 65; + public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*"; public static final boolean DEFAULT_DELETE_CONSIDERED_IDEMPOTENT = true; public static final int DEFAULT_CLOCK_SKEW_WITH_SERVER_IN_MS = 5 * 60 * 1000; // 5 mins diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 0415857745ba6..592adbc0f1355 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -47,6 +47,7 @@ import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.hadoop.fs.azurebfs.utils.DateTimeUtils; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.apache.hadoop.io.IOUtils; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*; @@ -67,6 +68,7 @@ public class AbfsClient implements Closeable { private final String filesystem; private final AbfsConfiguration abfsConfiguration; private final String userAgent; + private TrackingContext trackingContext; private final AbfsPerfTracker abfsPerfTracker; private final String accountName; @@ -86,6 +88,8 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden this.retryPolicy = abfsClientContext.getExponentialRetryPolicy(); this.accountName = abfsConfiguration.getAccountName().substring(0, abfsConfiguration.getAccountName().indexOf(AbfsHttpConstants.DOT)); this.authType = abfsConfiguration.getAuthType(accountName); + this.trackingContext = new TrackingContext( + abfsConfiguration.getClientCorrelationID()); String sslProviderName = null; @@ -134,6 +138,10 @@ public String getFileSystem() { return filesystem; } + public TrackingContext getTrackingContext() { + return trackingContext; + } + protected AbfsPerfTracker getAbfsPerfTracker() { return abfsPerfTracker; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index a63c98261f10d..a2213ad5eb319 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -162,7 +162,7 @@ public String toString() { sb.append(","); sb.append(expectedAppendPos); sb.append(",cid="); - sb.append(clientRequestId); + sb.append(getClientRequestHeader()); sb.append(",rid="); sb.append(requestId); if (isTraceEnabled) { @@ -184,6 +184,11 @@ public String toString() { return sb.toString(); } + public String getClientRequestHeader() { + return this.connection + .getRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); + } + // Returns a trace message for the ABFS API logging service to consume public String getLogString() { String urlStr = null; @@ -239,7 +244,7 @@ public AbfsHttpOperation(final URL url, final String method, final List Date: Tue, 6 Oct 2020 01:43:06 -0700 Subject: [PATCH 02/77] adding IDs --- .../azurebfs/services/AbfsHttpOperation.java | 141 ++++++++++++------ .../azurebfs/services/AbfsRestOperation.java | 17 ++- .../fs/azurebfs/utils/TrackingContext.java | 43 +++++- .../hadoop/fs/azurebfs/ITestAbfsClient.java | 57 +++++++ .../MockDelegationSASTokenProvider.java | 5 +- .../fs/azurebfs/services/TestAbfsClient.java | 7 + 6 files changed, 208 insertions(+), 62 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index a2213ad5eb319..e8419aaff03af 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -32,6 +32,7 @@ import javax.net.ssl.SSLSocketFactory; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; + import org.codehaus.jackson.JsonFactory; import org.codehaus.jackson.JsonParser; import org.codehaus.jackson.JsonToken; @@ -44,47 +45,65 @@ import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsPerfLoggable; import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; /** * Represents an HTTP operation. */ public class AbfsHttpOperation implements AbfsPerfLoggable { - private static final Logger LOG = LoggerFactory.getLogger(AbfsHttpOperation.class); + + private static final Logger LOG = LoggerFactory.getLogger( + AbfsHttpOperation.class); private static final int CONNECT_TIMEOUT = 30 * 1000; + private static final int READ_TIMEOUT = 30 * 1000; private static final int CLEAN_UP_BUFFER_SIZE = 64 * 1024; private static final int ONE_THOUSAND = 1000; + private static final int ONE_MILLION = ONE_THOUSAND * ONE_THOUSAND; private final String method; + private final URL url; private HttpURLConnection connection; + private int statusCode; + private String statusDescription; + private String storageErrorCode = ""; - private String storageErrorMessage = ""; + + private String storageErrorMessage = ""; + private String clientRequestId = ""; - private String requestId = ""; + + private String requestId = ""; + private String expectedAppendPos = ""; + private ListResultSchema listResultSchema = null; // metrics private int bytesSent; + private long bytesReceived; // optional trace enabled metrics private final boolean isTraceEnabled; + private long connectionTimeMs; + private long sendRequestTimeMs; + private long recvResponseTimeMs; public static AbfsHttpOperation getAbfsHttpOperationWithFixedResult(final URL url, final String method, final int httpStatus) { - return new AbfsHttpOperation(url, method, httpStatus); + return new AbfsHttpOperation(url, method, httpStatus); } private AbfsHttpOperation(final URL url, final String method, @@ -95,7 +114,7 @@ private AbfsHttpOperation(final URL url, final String method, this.statusCode = httpStatus; } - protected HttpURLConnection getConnection() { + protected HttpURLConnection getConnection() { return connection; } @@ -195,37 +214,37 @@ public String getLogString() { try { urlStr = URLEncoder.encode(url.toString(), "UTF-8"); - } catch(UnsupportedEncodingException e) { + } catch (UnsupportedEncodingException e) { urlStr = "https%3A%2F%2Ffailed%2Fto%2Fencode%2Furl"; } final StringBuilder sb = new StringBuilder(); sb.append("s=") - .append(statusCode) - .append(" e=") - .append(storageErrorCode) - .append(" ci=") - .append(clientRequestId) - .append(" ri=") - .append(requestId); + .append(statusCode) + .append(" e=") + .append(storageErrorCode) + .append(" ci=") + .append(clientRequestId) + .append(" ri=") + .append(requestId); if (isTraceEnabled) { sb.append(" ct=") - .append(connectionTimeMs) - .append(" st=") - .append(sendRequestTimeMs) - .append(" rt=") - .append(recvResponseTimeMs); + .append(connectionTimeMs) + .append(" st=") + .append(sendRequestTimeMs) + .append(" rt=") + .append(recvResponseTimeMs); } sb.append(" bs=") - .append(bytesSent) - .append(" br=") - .append(bytesReceived) - .append(" m=") - .append(method) - .append(" u=") - .append(urlStr); + .append(bytesSent) + .append(" br=") + .append(bytesReceived) + .append(" m=") + .append(method) + .append(" u=") + .append(urlStr); return sb.toString(); } @@ -239,17 +258,27 @@ public String getLogString() { * * @throws IOException if an error occurs. */ - public AbfsHttpOperation(final URL url, final String method, final List requestHeaders) + public AbfsHttpOperation(final URL url, + final String method, + final List requestHeaders) throws IOException { + this(url, method, requestHeaders, new TrackingContext("")); + } + + public AbfsHttpOperation(final URL url, + final String method, + final List requestHeaders, + TrackingContext trackingContext) throws IOException { this.isTraceEnabled = LOG.isTraceEnabled(); this.url = url; this.method = method; -// this.clientRequestId = UUID.randomUUID().toString(); + trackingContext.setClientRequestID(); this.connection = openConnection(); if (this.connection instanceof HttpsURLConnection) { HttpsURLConnection secureConn = (HttpsURLConnection) this.connection; - SSLSocketFactory sslSocketFactory = DelegatingSSLSocketFactory.getDefaultFactory(); + SSLSocketFactory sslSocketFactory + = DelegatingSSLSocketFactory.getDefaultFactory(); if (sslSocketFactory != null) { secureConn.setSSLSocketFactory(sslSocketFactory); } @@ -264,10 +293,9 @@ public AbfsHttpOperation(final URL url, final String method, final List= HttpURLConnection.HTTP_OK @@ -260,6 +260,7 @@ private boolean executeHttpOperation(final int retryCount) throws AzureBlobFileS incrementCounter(AbfsStatistic.BYTES_RECEIVED, httpOperation.getBytesReceived()); } + } catch (IOException ex) { if (ex instanceof UnknownHostException) { LOG.warn(String.format("Unknown host name: %s. Retrying to resolve the host name...", httpOperation.getUrl().getHost())); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java index 557478aa3e71f..f1b2d63350159 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java @@ -20,13 +20,45 @@ import java.util.UUID; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.StringUtils; + public class TrackingContext { - private String clientCorrelationID; + + private final String clientCorrelationID; + + private final String fileSystemID; + private String clientRequestID; + + private String streamID; + private int retryCount; + private static final Logger LOG = LoggerFactory.getLogger( + org.apache.hadoop.fs.azurebfs.services.AbfsClient.class); + + public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; + + public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*"; + public TrackingContext(String clientCorrelationID) { - this.clientCorrelationID = clientCorrelationID; + //validation + if ((clientCorrelationID.length() > MAX_CLIENT_CORRELATION_ID_LENGTH) + || (!clientCorrelationID.matches(CLIENT_CORRELATION_ID_PATTERN))) { + this.clientCorrelationID = EMPTY_STRING; + LOG.debug( + "Invalid config provided; correlation id not included in header."); + } else if (clientCorrelationID.length() > 0) { + this.clientCorrelationID = clientCorrelationID + ":"; + } else { + this.clientCorrelationID = EMPTY_STRING; + } + fileSystemID = UUID.randomUUID().toString(); + streamID = EMPTY_STRING; } public void setRetryCount(int count) { @@ -37,8 +69,13 @@ public void setClientRequestID() { clientRequestID = UUID.randomUUID().toString(); } + public void setStreamID(String stream) { + streamID = stream + StringUtils.right(UUID.randomUUID().toString(), 12); + } + public String toString() { - return clientCorrelationID + ":" + clientRequestID + ":" + retryCount; + return clientCorrelationID + clientRequestID + ":" + fileSystemID + ":" + + streamID + ":" + retryCount; } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java index a4d645899049f..0bf4295958d80 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java @@ -32,14 +32,21 @@ import org.junit.Ignore; import org.junit.Test; +import com.google.common.base.Preconditions; + +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema; 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.apache.hadoop.fs.permission.FsPermission; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_CORRELATIONID; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -49,6 +56,9 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest { private static final int LIST_MAX_RESULTS = 500; private static final int LIST_MAX_RESULTS_SERVER = 5000; + private static final int HTTP_CREATED = 201; + private static final String[] CLIENT_CORRELATIONID_LIST = { + "valid-corr-id-123", "inval!d", ""}; public ITestAbfsClient() throws Exception { super(); @@ -87,6 +97,53 @@ public void testUnknownHost() throws Exception { () -> FileSystem.get(conf.getRawConfiguration())); } + @Test + public void testClientCorrelation() throws IOException { + checkRequest(CLIENT_CORRELATIONID_LIST[0], true); + checkRequest(CLIENT_CORRELATIONID_LIST[1], false); + checkRequest(CLIENT_CORRELATIONID_LIST[2], false); + } + + private String getOctalNotation(FsPermission fsPermission) { + Preconditions.checkNotNull(fsPermission, "fsPermission"); + return String.format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal()); + } + + private String getRelativePath(final Path path) { + Preconditions.checkNotNull(path, "path"); + return path.toUri().getPath(); + } + + public void checkRequest(String clientCorrelationId, boolean includeInHeader) + throws IOException { + Configuration config = new Configuration(this.getRawConfiguration()); + config.set(FS_AZURE_CLIENT_CORRELATIONID, clientCorrelationId); + + final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem + .newInstance(this.getFileSystem().getUri(), config); + AbfsClient client = fs.getAbfsClient(); + String path = getRelativePath(new Path("/testDir")); + boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(); + String permission = isNamespaceEnabled ? getOctalNotation(FsPermission.getDirDefault()) : null; + String umask = isNamespaceEnabled ? getOctalNotation(FsPermission.getUMask(fs.getConf())) : null; + AbfsRestOperation op = client.createPath(path, false, true, + permission, umask, false, null); + + int responseCode = op.getResult().getStatusCode(); + assertEquals("Status code", HTTP_CREATED, responseCode); + String responseHeader = op.getResult() + .getResponseHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); + if (includeInHeader) { + Assertions.assertThat(responseHeader) + .describedAs("Should contain request IDs") + .startsWith(clientCorrelationId); + } else if (clientCorrelationId.length() > 0){ + assertFalse( + "Invalid or empty correlationId value should not be included in header", + responseHeader.contains(clientCorrelationId)); + } + } + @Test public void testListPathWithValidListMaxResultsValues() throws IOException, ExecutionException, InterruptedException { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java index 121256c4dbcf7..2912285ade262 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java @@ -37,6 +37,7 @@ import org.apache.hadoop.fs.azurebfs.utils.Base64; import org.apache.hadoop.fs.azurebfs.utils.DelegationSASGenerator; import org.apache.hadoop.fs.azurebfs.utils.SASGenerator; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.apache.hadoop.security.AccessControlException; /** @@ -105,10 +106,10 @@ private byte[] getUserDelegationKey(String accountName, String appID, String app AbfsHttpOperation op = new AbfsHttpOperation(url, method, requestHeaders); byte[] requestBuffer = requestBody.toString().getBytes(StandardCharsets.UTF_8.toString()); - op.sendRequest(requestBuffer, 0, requestBuffer.length); + op.sendRequest(requestBuffer, 0, requestBuffer.length, new TrackingContext(CORRELATION_ID)); byte[] responseBuffer = new byte[4 * 1024]; - op.processResponse(responseBuffer, 0, responseBuffer.length); + op.processResponse(responseBuffer, 0, responseBuffer.length, new TrackingContext(CORRELATION_ID)); String responseBody = new String(responseBuffer, 0, (int) op.getBytesReceived(), StandardCharsets.UTF_8); int beginIndex = responseBody.indexOf("") + "".length(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index 7a7992d9bb475..1f9e8d3497e96 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -31,6 +31,7 @@ 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.TrackingContext; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import static org.assertj.core.api.Assertions.assertThat; @@ -334,6 +335,12 @@ public static AbfsClient getMockAbfsClient(AbfsClient baseAbfsClientInstance, + "UNKNOWN/UNKNOWN) MSFT"; client = TestAbfsClient.setAbfsClientField(client, "userAgent", userAgent); + // set tracking context + TrackingContext trackingContext = new TrackingContext("clientCorrID"); + client = TestAbfsClient.setAbfsClientField(client, "trackingContext", + trackingContext); + when(client.getTrackingContext()).thenReturn(trackingContext); + return client; } From e21f7c6b1604b515c1359532294b5f49552f36fc Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 6 Oct 2020 23:57:53 -0700 Subject: [PATCH 03/77] add op id --- .../fs/azurebfs/AzureBlobFileSystem.java | 15 ++++++++++ .../fs/azurebfs/services/AbfsInputStream.java | 6 ++++ .../azurebfs/services/AbfsOutputStream.java | 5 ++++ .../azurebfs/services/AbfsRestOperation.java | 1 + .../fs/azurebfs/utils/TrackingContext.java | 28 +++++++++++-------- 5 files changed, 43 insertions(+), 12 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 513150a6bd43b..58744cc963d80 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -31,6 +31,7 @@ import java.util.ArrayList; import java.util.EnumSet; import java.util.Map; +import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -94,6 +95,7 @@ public class AzureBlobFileSystem extends FileSystem { private Path workingDir; private AzureBlobFileSystemStore abfsStore; private boolean isClosed; + private String fileSystemID; private boolean delegationTokenEnabled = false; private AbfsDelegationTokenManager delegationTokenManager; @@ -142,6 +144,9 @@ public void initialize(URI uri, Configuration configuration) AbfsClientThrottlingIntercept.initializeSingleton(abfsConfiguration.isAutoThrottlingEnabled()); LOG.debug("Initializing AzureBlobFileSystem for {} complete", uri); + + fileSystemID = UUID.randomUUID().toString(); + abfsStore.getClient().getTrackingContext().setFileSystemID(fileSystemID); } @Override @@ -174,6 +179,7 @@ public FSDataInputStream open(final Path path, final int bufferSize) throws IOEx LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize); statIncrement(CALL_OPEN); Path qualifiedPath = makeQualified(path); + abfsStore.getClient().getTrackingContext().setOpName("read"); try { InputStream inputStream = abfsStore.openFileForRead(qualifiedPath, statistics); @@ -197,6 +203,7 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi trailingPeriodCheck(f); Path qualifiedPath = makeQualified(f); + abfsStore.getClient().getTrackingContext().setOpName("mkdir"); try { OutputStream outputStream = abfsStore.createFile(qualifiedPath, statistics, overwrite, @@ -261,6 +268,7 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr bufferSize); statIncrement(CALL_APPEND); Path qualifiedPath = makeQualified(f); + abfsStore.getClient().getTrackingContext().setOpName("append"); try { OutputStream outputStream = abfsStore.openFileForWrite(qualifiedPath, statistics, false); @@ -274,6 +282,7 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr public boolean rename(final Path src, final Path dst) throws IOException { LOG.debug("AzureBlobFileSystem.rename src: {} dst: {}", src, dst); statIncrement(CALL_RENAME); + abfsStore.getClient().getTrackingContext().setOpName("rename"); trailingPeriodCheck(dst); @@ -344,6 +353,7 @@ public boolean delete(final Path f, final boolean recursive) throws IOException "AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), recursive); statIncrement(CALL_DELETE); Path qualifiedPath = makeQualified(f); + abfsStore.getClient().getTrackingContext().setOpName("delete"); if (f.isRoot()) { if (!recursive) { @@ -369,6 +379,7 @@ public FileStatus[] listStatus(final Path f) throws IOException { "AzureBlobFileSystem.listStatus path: {}", f.toString()); statIncrement(CALL_LIST_STATUS); Path qualifiedPath = makeQualified(f); + abfsStore.getClient().getTrackingContext().setOpName("ListITR"); //get iter no. 3 digit try { FileStatus[] result = abfsStore.listStatus(qualifiedPath); @@ -428,6 +439,7 @@ public boolean mkdirs(final Path f, final FsPermission permission) throws IOExce "AzureBlobFileSystem.mkdirs path: {} permissions: {}", f, permission); statIncrement(CALL_MKDIRS); trailingPeriodCheck(f); + abfsStore.getClient().getTrackingContext().setOpName("mkdir"); final Path parentFolder = f.getParent(); if (parentFolder == null) { @@ -466,6 +478,7 @@ public FileStatus getFileStatus(final Path f) throws IOException { LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f); statIncrement(CALL_GET_FILE_STATUS); Path qualifiedPath = makeQualified(f); + abfsStore.getClient().getTrackingContext().setOpName("GFS"); try { return abfsStore.getFileStatus(qualifiedPath); @@ -535,6 +548,7 @@ public BlockLocation[] getFileBlockLocations(FileStatus file, if (file == null) { return null; } + abfsStore.getClient().getTrackingContext().setOpName("GFS"); if ((start < 0) || (len < 0)) { throw new IllegalArgumentException("Invalid start or len parameter"); @@ -590,6 +604,7 @@ public String getOwnerUserPrimaryGroup() { private boolean deleteRoot() throws IOException { LOG.debug("Deleting root content"); + abfsStore.getClient().getTrackingContext().setOpName("delete"); final ExecutorService executorService = Executors.newFixedThreadPool(10); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index ff3bd63cc7bf7..6aea25c77f1e6 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -22,6 +22,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.net.HttpURLConnection; +import java.util.UUID; import com.google.common.base.Preconditions; import com.google.common.annotations.VisibleForTesting; @@ -29,6 +30,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.CanUnbuffer; import org.apache.hadoop.fs.FSExceptionMessages; import org.apache.hadoop.fs.FSInputStream; @@ -67,6 +69,7 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer, private int limit = 0; // offset of next byte to be read into buffer from service (i.e., upper marker+1 // of valid bytes in buffer) private boolean closed = false; + private String inputStreamID; /** Stream statistics. */ private final AbfsInputStreamStatistics streamStatistics; @@ -92,6 +95,8 @@ public AbfsInputStream( this.cachedSasToken = new CachedSASToken( abfsInputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds()); this.streamStatistics = abfsInputStreamContext.getStreamStatistics(); + this.inputStreamID = StringUtils.right(UUID.randomUUID().toString(), 12); + client.getTrackingContext().setStreamID("IN" + inputStreamID); } public String getPath() { @@ -180,6 +185,7 @@ private int readOneBlock(final byte[] b, final int off, final int len) throws IO // Enable readAhead when reading sequentially if (-1 == fCursorAfterLastRead || fCursorAfterLastRead == fCursor || b.length >= bufferSize) { + client.getTrackingContext().setOpName("readAhead"); bytesRead = readInternal(fCursor, buffer, 0, bufferSize, false); } else { bytesRead = readInternal(fCursor, buffer, 0, b.length, true); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 1991638a66703..34270471227b1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -32,12 +32,14 @@ import java.util.concurrent.Callable; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.UUID; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.utils.CachedSASToken; @@ -78,6 +80,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable, StreamCa // SAS tokens can be re-used until they expire private CachedSASToken cachedSasToken; + private String outputStreamID; /** * Queue storing buffers with the size of the Azure block ready for @@ -134,6 +137,8 @@ public AbfsOutputStream( this.completionService = new ExecutorCompletionService<>(this.threadExecutor); this.cachedSasToken = new CachedSASToken( abfsOutputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds()); + this.outputStreamID = StringUtils.right(UUID.randomUUID().toString(), 12); + client.getTrackingContext().setStreamID("OUT" + outputStreamID); } /** 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 1403a4ae729b8..1ef6376db24ff 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 @@ -180,6 +180,7 @@ public void execute() throws AzureBlobFileSystemException { new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_ABFS_CLIENT_LATENCY, latencyHeader); requestHeaders.add(httpHeader); } +// client.getTrackingContext().setOperation(operationType); retryCount = 0; LOG.debug("First execution of REST operation - {}", operationType); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java index f1b2d63350159..37824c83b985e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java @@ -27,22 +27,16 @@ import org.apache.commons.lang3.StringUtils; public class TrackingContext { - private final String clientCorrelationID; - - private final String fileSystemID; - + private String fileSystemID = ""; private String clientRequestID; - - private String streamID; - + private String streamID = ""; private int retryCount; + private String opName = ""; private static final Logger LOG = LoggerFactory.getLogger( org.apache.hadoop.fs.azurebfs.services.AbfsClient.class); - public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; - public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*"; public TrackingContext(String clientCorrelationID) { @@ -57,10 +51,13 @@ public TrackingContext(String clientCorrelationID) { } else { this.clientCorrelationID = EMPTY_STRING; } - fileSystemID = UUID.randomUUID().toString(); streamID = EMPTY_STRING; } + public void setFileSystemID(String fileSystemID) { + this.fileSystemID = fileSystemID; + } + public void setRetryCount(int count) { retryCount = count; } @@ -70,12 +67,19 @@ public void setClientRequestID() { } public void setStreamID(String stream) { - streamID = stream + StringUtils.right(UUID.randomUUID().toString(), 12); + streamID = stream; + } + + public void setOpName(String op) { + opName = op; } +// public void setOperation(org.apache.hadoop.fs.azurebfs.services.AbfsRestOperationType op) { +// opName = op.name();//convert +// } public String toString() { return clientCorrelationID + clientRequestID + ":" + fileSystemID + ":" - + streamID + ":" + retryCount; + + streamID + ":" + opName + ":" + retryCount; } } From 67c53a3af0827014e4fffc7ae4ffe224edc2b2dd Mon Sep 17 00:00:00 2001 From: Sumangala Date: Wed, 7 Oct 2020 04:01:35 -0700 Subject: [PATCH 04/77] undo formatting --- .../azurebfs/services/AbfsHttpOperation.java | 89 +++++-------------- .../fs/azurebfs/utils/TrackingContext.java | 9 +- .../ITestAzureBlobFileSystemCheckAccess.java | 1 + 3 files changed, 31 insertions(+), 68 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index e8419aaff03af..c2d28f3e8c16b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -26,13 +26,11 @@ import java.net.URL; import java.net.URLEncoder; import java.util.List; -import java.util.UUID; import javax.net.ssl.HttpsURLConnection; import javax.net.ssl.SSLSocketFactory; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; - import org.codehaus.jackson.JsonFactory; import org.codehaus.jackson.JsonParser; import org.codehaus.jackson.JsonToken; @@ -51,54 +49,36 @@ * Represents an HTTP operation. */ public class AbfsHttpOperation implements AbfsPerfLoggable { - - private static final Logger LOG = LoggerFactory.getLogger( - AbfsHttpOperation.class); + private static final Logger LOG = LoggerFactory.getLogger(AbfsHttpOperation.class); private static final int CONNECT_TIMEOUT = 30 * 1000; - private static final int READ_TIMEOUT = 30 * 1000; private static final int CLEAN_UP_BUFFER_SIZE = 64 * 1024; private static final int ONE_THOUSAND = 1000; - private static final int ONE_MILLION = ONE_THOUSAND * ONE_THOUSAND; private final String method; - private final URL url; private HttpURLConnection connection; - private int statusCode; - private String statusDescription; - private String storageErrorCode = ""; - - private String storageErrorMessage = ""; - - private String clientRequestId = ""; - - private String requestId = ""; - + private String storageErrorMessage = ""; + private String requestId = ""; private String expectedAppendPos = ""; - private ListResultSchema listResultSchema = null; // metrics private int bytesSent; - private long bytesReceived; // optional trace enabled metrics private final boolean isTraceEnabled; - private long connectionTimeMs; - private long sendRequestTimeMs; - private long recvResponseTimeMs; public static AbfsHttpOperation getAbfsHttpOperationWithFixedResult(final URL url, @@ -114,7 +94,7 @@ private AbfsHttpOperation(final URL url, final String method, this.statusCode = httpStatus; } - protected HttpURLConnection getConnection() { + protected HttpURLConnection getConnection() { return connection; } @@ -143,7 +123,8 @@ public String getStorageErrorMessage() { } public String getClientRequestId() { - return clientRequestId; + return this.connection + .getRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); } public String getExpectedAppendPos() { @@ -181,7 +162,7 @@ public String toString() { sb.append(","); sb.append(expectedAppendPos); sb.append(",cid="); - sb.append(getClientRequestHeader()); + sb.append(getClientRequestId()); sb.append(",rid="); sb.append(requestId); if (isTraceEnabled) { @@ -203,18 +184,13 @@ public String toString() { return sb.toString(); } - public String getClientRequestHeader() { - return this.connection - .getRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); - } - // Returns a trace message for the ABFS API logging service to consume public String getLogString() { String urlStr = null; try { urlStr = URLEncoder.encode(url.toString(), "UTF-8"); - } catch (UnsupportedEncodingException e) { + } catch(UnsupportedEncodingException e) { urlStr = "https%3A%2F%2Ffailed%2Fto%2Fencode%2Furl"; } @@ -224,7 +200,7 @@ public String getLogString() { .append(" e=") .append(storageErrorCode) .append(" ci=") - .append(clientRequestId) + .append(getClientRequestId()) .append(" ri=") .append(requestId); @@ -258,16 +234,11 @@ public String getLogString() { * * @throws IOException if an error occurs. */ - public AbfsHttpOperation(final URL url, - final String method, - final List requestHeaders) throws IOException { + public AbfsHttpOperation(final URL url, final String method, final List requestHeaders) throws IOException { this(url, method, requestHeaders, new TrackingContext("")); } - public AbfsHttpOperation(final URL url, - final String method, - final List requestHeaders, - TrackingContext trackingContext) + public AbfsHttpOperation(final URL url, final String method, final List requestHeaders, TrackingContext trackingContext) throws IOException { this.isTraceEnabled = LOG.isTraceEnabled(); this.url = url; @@ -277,8 +248,7 @@ public AbfsHttpOperation(final URL url, this.connection = openConnection(); if (this.connection instanceof HttpsURLConnection) { HttpsURLConnection secureConn = (HttpsURLConnection) this.connection; - SSLSocketFactory sslSocketFactory - = DelegatingSSLSocketFactory.getDefaultFactory(); + SSLSocketFactory sslSocketFactory = DelegatingSSLSocketFactory.getDefaultFactory(); if (sslSocketFactory != null) { secureConn.setSSLSocketFactory(sslSocketFactory); } @@ -292,7 +262,7 @@ public AbfsHttpOperation(final URL url, for (AbfsHttpHeader header : requestHeaders) { this.connection.setRequestProperty(header.getName(), header.getValue()); } - + this.connection.setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, trackingContext.toString()); } /** @@ -306,10 +276,7 @@ public AbfsHttpOperation(final URL url, * * @throws IOException if an error occurs. */ - public void sendRequest(byte[] buffer, - int offset, - int length, - TrackingContext trackingContext) throws IOException { + public void sendRequest(byte[] buffer, int offset, int length) throws IOException { this.connection.setDoOutput(true); this.connection.setFixedLengthStreamingMode(length); if (buffer == null) { @@ -332,7 +299,6 @@ public void sendRequest(byte[] buffer, // accompanying statusCode this.bytesSent = length; outputStream.write(buffer, offset, length); - trackingContext.setStreamID("OUT"); } finally { if (this.isTraceEnabled) { this.sendRequestTimeMs = elapsedTimeMs(startTime); @@ -349,10 +315,7 @@ public void sendRequest(byte[] buffer, * * @throws IOException if an error occurs. */ - public void processResponse(final byte[] buffer, - final int offset, - final int length, - TrackingContext trackingContext) throws IOException { + public void processResponse(final byte[] buffer, final int offset, final int length) throws IOException { // get the response long startTime = 0; @@ -368,8 +331,7 @@ public void processResponse(final byte[] buffer, this.statusDescription = this.connection.getResponseMessage(); - this.requestId = this.connection.getHeaderField( - HttpHeaderConfigurations.X_MS_REQUEST_ID); + this.requestId = this.connection.getHeaderField(HttpHeaderConfigurations.X_MS_REQUEST_ID); if (this.requestId == null) { this.requestId = AbfsHttpConstants.EMPTY_STRING; } @@ -391,8 +353,7 @@ public void processResponse(final byte[] buffer, if (this.isTraceEnabled) { this.recvResponseTimeMs += elapsedTimeMs(startTime); } - this.bytesReceived = this.connection.getHeaderFieldLong( - HttpHeaderConfigurations.CONTENT_LENGTH, 0); + this.bytesReceived = this.connection.getHeaderFieldLong(HttpHeaderConfigurations.CONTENT_LENGTH, 0); } else { // consume the input stream to release resources int totalBytesRead = 0; @@ -403,18 +364,14 @@ public void processResponse(final byte[] buffer, } boolean endOfStream = false; - trackingContext.setStreamID("IN"); - // this is a list operation and need to retrieve the data // need a better solution - if (AbfsHttpConstants.HTTP_METHOD_GET.equals(this.method) - && buffer == null) { + if (AbfsHttpConstants.HTTP_METHOD_GET.equals(this.method) && buffer == null) { parseListFilesResponse(stream); } else { if (buffer != null) { while (totalBytesRead < length) { - int bytesRead = stream.read(buffer, offset + totalBytesRead, - length - totalBytesRead); + int bytesRead = stream.read(buffer, offset + totalBytesRead, length - totalBytesRead); if (bytesRead == -1) { endOfStream = true; break; @@ -533,8 +490,7 @@ private long elapsedTimeMs(final long startTime) { * @param stream InputStream contains the list results. * @throws IOException */ - private void parseListFilesResponse(final InputStream stream) - throws IOException { + private void parseListFilesResponse(final InputStream stream) throws IOException { if (stream == null) { return; } @@ -546,8 +502,7 @@ private void parseListFilesResponse(final InputStream stream) try { final ObjectMapper objectMapper = new ObjectMapper(); - this.listResultSchema = objectMapper.readValue(stream, - ListResultSchema.class); + this.listResultSchema = objectMapper.readValue(stream, ListResultSchema.class); } catch (IOException ex) { LOG.error("Unable to deserialize list results", ex); throw ex; @@ -561,4 +516,4 @@ private void parseListFilesResponse(final InputStream stream) private boolean isNullInputStream(InputStream stream) { return stream == null ? true : false; } -} +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java index 37824c83b985e..77fccc7546ef4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java @@ -33,6 +33,7 @@ public class TrackingContext { private String streamID = ""; private int retryCount; private String opName = ""; + private int iteration; private static final Logger LOG = LoggerFactory.getLogger( org.apache.hadoop.fs.azurebfs.services.AbfsClient.class); @@ -72,14 +73,20 @@ public void setStreamID(String stream) { public void setOpName(String op) { opName = op; + iteration = 0; } // public void setOperation(org.apache.hadoop.fs.azurebfs.services.AbfsRestOperationType op) { // opName = op.name();//convert // } + public void updateIteration() { + iteration++; + } public String toString() { +// if (iteration) + String operation = opName + (opName == "listITR" || opName == "ReadAhead"? iteration.toString() : ""); return clientCorrelationID + clientRequestID + ":" + fileSystemID + ":" - + streamID + ":" + opName + ":" + retryCount; + + streamID + ":" + operation + ":" + retryCount; } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java index 4189d666e7a70..393e3d4346b39 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java @@ -73,6 +73,7 @@ private void setTestUserFs() throws Exception { return; } String orgClientId = getConfiguration().get(FS_AZURE_BLOB_FS_CLIENT_ID); + System.out.println(orgClientId); String orgClientSecret = getConfiguration() .get(FS_AZURE_BLOB_FS_CLIENT_SECRET); Boolean orgCreateFileSystemDurungInit = getConfiguration() From 0e57f19f90c1b3325a22b8d67ab429ea4ff2f15a Mon Sep 17 00:00:00 2001 From: Sumangala Date: Wed, 7 Oct 2020 15:50:04 -0700 Subject: [PATCH 05/77] to pc --- .../fs/azurebfs/AzureBlobFileSystem.java | 7 ++- .../fs/azurebfs/AzureBlobFileSystemStore.java | 3 +- .../azurebfs/services/AbfsHttpOperation.java | 60 +++++++++---------- .../azurebfs/services/AbfsRestOperation.java | 13 +++- .../fs/azurebfs/utils/TrackingContext.java | 16 ++--- 5 files changed, 54 insertions(+), 45 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 58744cc963d80..67221f282de39 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -71,6 +71,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager; import org.apache.hadoop.fs.azurebfs.services.AbfsCounters; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.FsAction; @@ -146,7 +147,6 @@ public void initialize(URI uri, Configuration configuration) LOG.debug("Initializing AzureBlobFileSystem for {} complete", uri); fileSystemID = UUID.randomUUID().toString(); - abfsStore.getClient().getTrackingContext().setFileSystemID(fileSystemID); } @Override @@ -353,7 +353,8 @@ public boolean delete(final Path f, final boolean recursive) throws IOException "AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), recursive); statIncrement(CALL_DELETE); Path qualifiedPath = makeQualified(f); - abfsStore.getClient().getTrackingContext().setOpName("delete"); + TrackingContext trackingContext = new TrackingContext(fileSystemID, "DL"); +// abfsStore.getClient().getTrackingContext().setOpName("delete"); if (f.isRoot()) { if (!recursive) { @@ -364,7 +365,7 @@ public boolean delete(final Path f, final boolean recursive) throws IOException } try { - abfsStore.delete(qualifiedPath, recursive); + abfsStore.delete(qualifiedPath, recursive, trackingContext); return true; } catch (AzureBlobFileSystemException ex) { checkException(f, ex, AzureServiceErrorCode.PATH_NOT_FOUND); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index d2a1d538f6380..fac6270e11817 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -727,11 +727,12 @@ public void rename(final Path source, final Path destination) throws } while (shouldContinue); } - public void delete(final Path path, final boolean recursive) + public void delete(final Path path, final boolean recursive, TrackingContext trackingContext) throws AzureBlobFileSystemException { final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue = true; + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); LOG.debug("delete filesystem: {} path: {} recursive: {}", client.getFileSystem(), diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index c2d28f3e8c16b..9677121e3e382 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -196,31 +196,31 @@ public String getLogString() { final StringBuilder sb = new StringBuilder(); sb.append("s=") - .append(statusCode) - .append(" e=") - .append(storageErrorCode) - .append(" ci=") - .append(getClientRequestId()) - .append(" ri=") - .append(requestId); + .append(statusCode) + .append(" e=") + .append(storageErrorCode) + .append(" ci=") + .append(getClientRequestId()) + .append(" ri=") + .append(requestId); if (isTraceEnabled) { sb.append(" ct=") - .append(connectionTimeMs) - .append(" st=") - .append(sendRequestTimeMs) - .append(" rt=") - .append(recvResponseTimeMs); + .append(connectionTimeMs) + .append(" st=") + .append(sendRequestTimeMs) + .append(" rt=") + .append(recvResponseTimeMs); } sb.append(" bs=") - .append(bytesSent) - .append(" br=") - .append(bytesReceived) - .append(" m=") - .append(method) - .append(" u=") - .append(urlStr); + .append(bytesSent) + .append(" br=") + .append(bytesReceived) + .append(" m=") + .append(method) + .append(" u=") + .append(urlStr); return sb.toString(); } @@ -453,17 +453,17 @@ private void processStorageErrorResponse() { jp.nextToken(); fieldValue = jp.getText(); switch (fieldName) { - case "code": - storageErrorCode = fieldValue; - break; - case "message": - storageErrorMessage = fieldValue; - break; - case "ExpectedAppendPos": - expectedAppendPos = fieldValue; - break; - default: - break; + case "code": + storageErrorCode = fieldValue; + break; + case "message": + storageErrorMessage = fieldValue; + break; + case "ExpectedAppendPos": + expectedAppendPos = fieldValue; + break; + default: + break; } } jp.nextToken(); 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 1ef6376db24ff..2b4f5b2ec9fe2 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 @@ -35,6 +35,7 @@ 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; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; /** * The AbfsRestOperation for Rest AbfsClient. @@ -51,6 +52,8 @@ public class AbfsRestOperation { // all the custom HTTP request headers provided by the caller private final List requestHeaders; + private final TrackingContext trackingContext; + // This is a simple operation class, where all the upload methods have a // request body and all the download methods have a response body. private final boolean hasRequestBody; @@ -107,8 +110,9 @@ String getSasToken() { final AbfsClient client, final String method, final URL url, - final List requestHeaders) { - this(operationType, client, method, url, requestHeaders, null); + final List requestHeaders, + final TrackingContext trackingContext) { + this(operationType, client, method, url, requestHeaders, trackingContext, null); } /** @@ -125,12 +129,14 @@ String getSasToken() { final String method, final URL url, final List requestHeaders, + final TrackingContext trackingContext, final String sasToken) { this.operationType = operationType; this.client = client; this.method = method; this.url = url; this.requestHeaders = requestHeaders; + this.trackingContext = trackingContext; this.hasRequestBody = (AbfsHttpConstants.HTTP_METHOD_PUT.equals(method) || AbfsHttpConstants.HTTP_METHOD_PATCH.equals(method)); this.sasToken = sasToken; @@ -156,11 +162,12 @@ String getSasToken() { String method, URL url, List requestHeaders, + final TrackingContext trackingContext, byte[] buffer, int bufferOffset, int bufferLength, String sasToken) { - this(operationType, client, method, url, requestHeaders, sasToken); + this(operationType, client, method, url, requestHeaders, trackingContext, sasToken); this.buffer = buffer; this.bufferOffset = bufferOffset; this.bufferLength = bufferLength; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java index 77fccc7546ef4..cedca8b5f4632 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java @@ -27,12 +27,12 @@ import org.apache.commons.lang3.StringUtils; public class TrackingContext { - private final String clientCorrelationID; + private String clientCorrelationID; private String fileSystemID = ""; private String clientRequestID; private String streamID = ""; private int retryCount; - private String opName = ""; + private String hadoopOpName = ""; private int iteration; private static final Logger LOG = LoggerFactory.getLogger( @@ -40,7 +40,12 @@ public class TrackingContext { public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*"; - public TrackingContext(String clientCorrelationID) { + public TrackingContext(String fileSystemID, String hadoopOpName) { + this.fileSystemID = fileSystemID; + this.hadoopOpName = hadoopOpName; + streamID = EMPTY_STRING; + } + public void setClientCorrelationID(String clientCorrelationID) { //validation if ((clientCorrelationID.length() > MAX_CLIENT_CORRELATION_ID_LENGTH) || (!clientCorrelationID.matches(CLIENT_CORRELATION_ID_PATTERN))) { @@ -52,7 +57,6 @@ public TrackingContext(String clientCorrelationID) { } else { this.clientCorrelationID = EMPTY_STRING; } - streamID = EMPTY_STRING; } public void setFileSystemID(String fileSystemID) { @@ -71,10 +75,6 @@ public void setStreamID(String stream) { streamID = stream; } - public void setOpName(String op) { - opName = op; - iteration = 0; - } // public void setOperation(org.apache.hadoop.fs.azurebfs.services.AbfsRestOperationType op) { // opName = op.name();//convert // } From 20c916d962fb10e9c9209c332a2aa00818c3f1aa Mon Sep 17 00:00:00 2001 From: sumangala Date: Thu, 8 Oct 2020 07:45:33 +0530 Subject: [PATCH 06/77] tc -> ops --- .../fs/azurebfs/AzureBlobFileSystem.java | 80 +++++--- .../fs/azurebfs/AzureBlobFileSystemStore.java | 191 +++++++++++------- .../fs/azurebfs/services/AbfsClient.java | 91 +++++---- .../fs/azurebfs/services/AbfsInputStream.java | 5 +- .../azurebfs/services/AbfsOutputStream.java | 5 +- .../azurebfs/services/AbfsRestOperation.java | 35 ++-- .../fs/azurebfs/utils/TrackingContext.java | 9 +- .../azurebfs/AbstractAbfsIntegrationTest.java | 5 +- .../fs/azurebfs/ITestGetNameSpaceEnabled.java | 8 +- 9 files changed, 252 insertions(+), 177 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 67221f282de39..0f80673932c6e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -174,15 +174,19 @@ public URI getUri() { return this.uri; } + public String getFileSystemID() { + return fileSystemID; + } + @Override public FSDataInputStream open(final Path path, final int bufferSize) throws IOException { LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize); statIncrement(CALL_OPEN); + TrackingContext trackingContext = new TrackingContext(fileSystemID, "FS"); Path qualifiedPath = makeQualified(path); - abfsStore.getClient().getTrackingContext().setOpName("read"); try { - InputStream inputStream = abfsStore.openFileForRead(qualifiedPath, statistics); + InputStream inputStream = abfsStore.openFileForRead(qualifiedPath, statistics, trackingContext); return new FSDataInputStream(inputStream); } catch(AzureBlobFileSystemException ex) { checkException(path, ex); @@ -198,16 +202,17 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi permission, overwrite, blockSize); + TrackingContext trackingContext = new TrackingContext(fileSystemID, "LS"); statIncrement(CALL_CREATE); trailingPeriodCheck(f); Path qualifiedPath = makeQualified(f); - abfsStore.getClient().getTrackingContext().setOpName("mkdir"); try { OutputStream outputStream = abfsStore.createFile(qualifiedPath, statistics, overwrite, - permission == null ? FsPermission.getFileDefault() : permission, FsPermission.getUMask(getConf())); + permission == null ? FsPermission.getFileDefault() : permission, FsPermission.getUMask(getConf()), + trackingContext); statIncrement(FILES_CREATED); return new FSDataOutputStream(outputStream, statistics); } catch(AzureBlobFileSystemException ex) { @@ -267,11 +272,12 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr f.toString(), bufferSize); statIncrement(CALL_APPEND); + TrackingContext trackingContext = new TrackingContext(fileSystemID, "AP"); Path qualifiedPath = makeQualified(f); - abfsStore.getClient().getTrackingContext().setOpName("append"); try { - OutputStream outputStream = abfsStore.openFileForWrite(qualifiedPath, statistics, false); + OutputStream outputStream = abfsStore.openFileForWrite(qualifiedPath, statistics, false, + trackingContext); return new FSDataOutputStream(outputStream, statistics); } catch(AzureBlobFileSystemException ex) { checkException(f, ex); @@ -282,7 +288,7 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr public boolean rename(final Path src, final Path dst) throws IOException { LOG.debug("AzureBlobFileSystem.rename src: {} dst: {}", src, dst); statIncrement(CALL_RENAME); - abfsStore.getClient().getTrackingContext().setOpName("rename"); + TrackingContext trackingContext = new TrackingContext(fileSystemID, "RN"); trailingPeriodCheck(dst); @@ -312,7 +318,7 @@ public boolean rename(final Path src, final Path dst) throws IOException { } // Non-HNS account need to check dst status on driver side. - if (!abfsStore.getIsNamespaceEnabled() && dstFileStatus == null) { + if (!abfsStore.getIsNamespaceEnabled(trackingContext) && dstFileStatus == null) { dstFileStatus = tryGetFileStatus(qualifiedDstPath); } @@ -329,7 +335,7 @@ public boolean rename(final Path src, final Path dst) throws IOException { qualifiedDstPath = makeQualified(adjustedDst); - abfsStore.rename(qualifiedSrcPath, qualifiedDstPath); + abfsStore.rename(qualifiedSrcPath, qualifiedDstPath, trackingContext); return true; } catch(AzureBlobFileSystemException ex) { LOG.debug("Rename operation failed. ", ex); @@ -354,7 +360,6 @@ public boolean delete(final Path f, final boolean recursive) throws IOException statIncrement(CALL_DELETE); Path qualifiedPath = makeQualified(f); TrackingContext trackingContext = new TrackingContext(fileSystemID, "DL"); -// abfsStore.getClient().getTrackingContext().setOpName("delete"); if (f.isRoot()) { if (!recursive) { @@ -380,10 +385,10 @@ public FileStatus[] listStatus(final Path f) throws IOException { "AzureBlobFileSystem.listStatus path: {}", f.toString()); statIncrement(CALL_LIST_STATUS); Path qualifiedPath = makeQualified(f); - abfsStore.getClient().getTrackingContext().setOpName("ListITR"); //get iter no. 3 digit + TrackingContext trackingContext = new TrackingContext(fileSystemID, "LS"); try { - FileStatus[] result = abfsStore.listStatus(qualifiedPath); + FileStatus[] result = abfsStore.listStatus(qualifiedPath, trackingContext); return result; } catch (AzureBlobFileSystemException ex) { checkException(f, ex); @@ -440,7 +445,7 @@ public boolean mkdirs(final Path f, final FsPermission permission) throws IOExce "AzureBlobFileSystem.mkdirs path: {} permissions: {}", f, permission); statIncrement(CALL_MKDIRS); trailingPeriodCheck(f); - abfsStore.getClient().getTrackingContext().setOpName("mkdir"); + TrackingContext trackingContext = new TrackingContext(fileSystemID, "MK"); final Path parentFolder = f.getParent(); if (parentFolder == null) { @@ -452,7 +457,7 @@ public boolean mkdirs(final Path f, final FsPermission permission) throws IOExce try { abfsStore.createDirectory(qualifiedPath, permission == null ? FsPermission.getDirDefault() : permission, - FsPermission.getUMask(getConf())); + FsPermission.getUMask(getConf()), trackingContext); statIncrement(DIRECTORIES_CREATED); return true; } catch (AzureBlobFileSystemException ex) { @@ -479,10 +484,10 @@ public FileStatus getFileStatus(final Path f) throws IOException { LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f); statIncrement(CALL_GET_FILE_STATUS); Path qualifiedPath = makeQualified(f); - abfsStore.getClient().getTrackingContext().setOpName("GFS"); + TrackingContext trackingContext = new TrackingContext(fileSystemID, "FS"); try { - return abfsStore.getFileStatus(qualifiedPath); + return abfsStore.getFileStatus(qualifiedPath, trackingContext); } catch(AzureBlobFileSystemException ex) { checkException(f, ex); return null; @@ -549,7 +554,6 @@ public BlockLocation[] getFileBlockLocations(FileStatus file, if (file == null) { return null; } - abfsStore.getClient().getTrackingContext().setOpName("GFS"); if ((start < 0) || (len < 0)) { throw new IllegalArgumentException("Invalid start or len parameter"); @@ -605,7 +609,6 @@ public String getOwnerUserPrimaryGroup() { private boolean deleteRoot() throws IOException { LOG.debug("Deleting root content"); - abfsStore.getClient().getTrackingContext().setOpName("delete"); final ExecutorService executorService = Executors.newFixedThreadPool(10); @@ -692,6 +695,7 @@ public void setOwner(final Path path, final String owner, final String group) public void setXAttr(final Path path, final String name, final byte[] value, final EnumSet flag) throws IOException { LOG.debug("AzureBlobFileSystem.setXAttr path: {}", path); + TrackingContext trackingContext = new TrackingContext(fileSystemID, "AT"); if (name == null || name.isEmpty() || value == null) { throw new IllegalArgumentException("A valid name and value must be specified."); @@ -700,14 +704,14 @@ public void setXAttr(final Path path, final String name, final byte[] value, fin Path qualifiedPath = makeQualified(path); try { - Hashtable properties = abfsStore.getPathStatus(qualifiedPath); + Hashtable properties = abfsStore.getPathStatus(qualifiedPath, trackingContext); String xAttrName = ensureValidAttributeName(name); boolean xAttrExists = properties.containsKey(xAttrName); XAttrSetFlag.validate(name, xAttrExists, flag); String xAttrValue = abfsStore.decodeAttribute(value); properties.put(xAttrName, xAttrValue); - abfsStore.setPathProperties(qualifiedPath, properties); + abfsStore.setPathProperties(qualifiedPath, properties, trackingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -731,12 +735,13 @@ public byte[] getXAttr(final Path path, final String name) if (name == null || name.isEmpty()) { throw new IllegalArgumentException("A valid name must be specified."); } + TrackingContext trackingContext = new TrackingContext(fileSystemID, "AT"); Path qualifiedPath = makeQualified(path); byte[] value = null; try { - Hashtable properties = abfsStore.getPathStatus(qualifiedPath); + Hashtable properties = abfsStore.getPathStatus(qualifiedPath, trackingContext); String xAttrName = ensureValidAttributeName(name); if (properties.containsKey(xAttrName)) { String xAttrValue = properties.get(xAttrName); @@ -767,6 +772,7 @@ public void setPermission(final Path path, final FsPermission permission) super.setPermission(path, permission); return; } + TrackingContext trackingContext = new TrackingContext(fileSystemID, "PR"); if (permission == null) { throw new IllegalArgumentException("The permission can't be null"); @@ -776,7 +782,7 @@ public void setPermission(final Path path, final FsPermission permission) try { abfsStore.setPermission(qualifiedPath, - permission); + permission, trackingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -796,6 +802,7 @@ public void setPermission(final Path path, final FsPermission permission) public void modifyAclEntries(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.modifyAclEntries path: {}", path); + TrackingContext trackingContext = new TrackingContext(fileSystemID, "AC"); if (!getIsNamespaceEnabled()) { throw new UnsupportedOperationException( @@ -811,7 +818,7 @@ public void modifyAclEntries(final Path path, final List aclSpec) try { abfsStore.modifyAclEntries(qualifiedPath, - aclSpec); + aclSpec, trackingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -829,6 +836,7 @@ public void modifyAclEntries(final Path path, final List aclSpec) public void removeAclEntries(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.removeAclEntries path: {}", path); + TrackingContext trackingContext = new TrackingContext(fileSystemID, "AC"); if (!getIsNamespaceEnabled()) { throw new UnsupportedOperationException( @@ -843,7 +851,7 @@ public void removeAclEntries(final Path path, final List aclSpec) Path qualifiedPath = makeQualified(path); try { - abfsStore.removeAclEntries(qualifiedPath, aclSpec); + abfsStore.removeAclEntries(qualifiedPath, aclSpec, trackingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -864,11 +872,12 @@ public void removeDefaultAcl(final Path path) throws IOException { "removeDefaultAcl is only supported by storage accounts with the " + "hierarchical namespace enabled."); } + TrackingContext trackingContext = new TrackingContext(fileSystemID, "AC"); Path qualifiedPath = makeQualified(path); try { - abfsStore.removeDefaultAcl(qualifiedPath); + abfsStore.removeDefaultAcl(qualifiedPath, trackingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -885,6 +894,7 @@ public void removeDefaultAcl(final Path path) throws IOException { @Override public void removeAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeAcl path: {}", path); + TrackingContext trackingContext = new TrackingContext(fileSystemID, "AC"); if (!getIsNamespaceEnabled()) { throw new UnsupportedOperationException( @@ -895,7 +905,7 @@ public void removeAcl(final Path path) throws IOException { Path qualifiedPath = makeQualified(path); try { - abfsStore.removeAcl(qualifiedPath); + abfsStore.removeAcl(qualifiedPath, trackingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -921,6 +931,7 @@ public void setAcl(final Path path, final List aclSpec) "setAcl is only supported by storage accounts with the hierarchical " + "namespace enabled."); } + TrackingContext trackingContext = new TrackingContext(fileSystemID, "AC"); if (aclSpec == null || aclSpec.size() == 0) { throw new IllegalArgumentException("The aclSpec argument is invalid."); @@ -929,7 +940,7 @@ public void setAcl(final Path path, final List aclSpec) Path qualifiedPath = makeQualified(path); try { - abfsStore.setAcl(qualifiedPath, aclSpec); + abfsStore.setAcl(qualifiedPath, aclSpec, trackingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -951,11 +962,12 @@ public AclStatus getAclStatus(final Path path) throws IOException { "getAclStatus is only supported by storage account with the " + "hierarchical namespace enabled."); } + TrackingContext trackingContext = new TrackingContext(fileSystemID, "AC"); Path qualifiedPath = makeQualified(path); try { - return abfsStore.getAclStatus(qualifiedPath); + return abfsStore.getAclStatus(qualifiedPath, trackingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); return null; @@ -977,9 +989,10 @@ public AclStatus getAclStatus(final Path path) throws IOException { @Override public void access(final Path path, final FsAction mode) throws IOException { LOG.debug("AzureBlobFileSystem.access path : {}, mode : {}", path, mode); + TrackingContext trackingContext = new TrackingContext(fileSystemID, "LS"); Path qualifiedPath = makeQualified(path); try { - this.abfsStore.access(qualifiedPath, mode); + this.abfsStore.access(qualifiedPath, mode, trackingContext); } catch (AzureBlobFileSystemException ex) { checkCheckAccessException(path, ex); } @@ -1011,8 +1024,9 @@ private FileStatus tryGetFileStatus(final Path f) { private boolean fileSystemExists() throws IOException { LOG.debug( "AzureBlobFileSystem.fileSystemExists uri: {}", uri); + TrackingContext trackingContext = new TrackingContext(fileSystemID, "FS"); try { - abfsStore.getFilesystemProperties(); + abfsStore.getFilesystemProperties(trackingContext); } catch (AzureBlobFileSystemException ex) { try { checkException(null, ex); @@ -1030,8 +1044,9 @@ private boolean fileSystemExists() throws IOException { private void createFileSystem() throws IOException { LOG.debug( "AzureBlobFileSystem.createFileSystem uri: {}", uri); + TrackingContext trackingContext = new TrackingContext(fileSystemID, "FS"); try { - abfsStore.createFilesystem(); + abfsStore.createFilesystem(trackingContext); } catch (AzureBlobFileSystemException ex) { checkException(null, ex); } @@ -1254,7 +1269,8 @@ AbfsDelegationTokenManager getDelegationTokenManager() { @VisibleForTesting boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException { - return abfsStore.getIsNamespaceEnabled(); + TrackingContext trackingContext = new TrackingContext(fileSystemID, "NS"); + return abfsStore.getIsNamespaceEnabled(trackingContext); } @VisibleForTesting diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index fac6270e11817..2bcb8c3422e54 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -47,6 +47,7 @@ import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.UUID; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; @@ -105,6 +106,7 @@ import org.apache.hadoop.fs.azurebfs.utils.Base64; import org.apache.hadoop.fs.azurebfs.utils.CRC64; import org.apache.hadoop.fs.azurebfs.utils.DateTimeUtils; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.apache.hadoop.fs.azurebfs.utils.UriUtils; import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclStatus; @@ -278,18 +280,19 @@ private String[] authorityParts(URI uri) throws InvalidUriAuthorityException, In return authorityParts; } - public boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException { + public boolean getIsNamespaceEnabled(TrackingContext trackingContext) throws AzureBlobFileSystemException { try { return this.isNamespaceEnabled.toBoolean(); } catch (TrileanConversionException e) { LOG.debug("isNamespaceEnabled is UNKNOWN; fall back and determine through" + " getAcl server call", e); } + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); LOG.debug("Get root ACL status"); try (AbfsPerfInfo perfInfo = startTracking("getIsNamespaceEnabled", "getAclStatus")) { - AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.ROOT_PATH); + AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.ROOT_PATH, trackingContext); perfInfo.registerResult(op.getResult()); isNamespaceEnabled = Trilean.getTrilean(true); perfInfo.registerSuccess(true); @@ -342,15 +345,17 @@ public AbfsConfiguration getAbfsConfiguration() { return this.abfsConfiguration; } - public Hashtable getFilesystemProperties() throws AzureBlobFileSystemException { + public Hashtable getFilesystemProperties(TrackingContext trackingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("getFilesystemProperties", "getFilesystemProperties")) { LOG.debug("getFilesystemProperties for filesystem: {}", client.getFileSystem()); + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); + final Hashtable parsedXmsProperties; - final AbfsRestOperation op = client.getFilesystemProperties(); + final AbfsRestOperation op = client.getFilesystemProperties(trackingContext); perfInfo.registerResult(op.getResult()); final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); @@ -387,14 +392,17 @@ public void setFilesystemProperties(final Hashtable properties) } } - public Hashtable getPathStatus(final Path path) throws AzureBlobFileSystemException { + public Hashtable getPathStatus(final Path path, TrackingContext trackingContext) + throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("getPathStatus", "getPathStatus")){ LOG.debug("getPathStatus for filesystem: {} path: {}", client.getFileSystem(), path); + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); final Hashtable parsedXmsProperties; - final AbfsRestOperation op = client.getPathStatus(getRelativePath(path), true); + final AbfsRestOperation op = client.getPathStatus(getRelativePath(path), true, + trackingContext); perfInfo.registerResult(op.getResult()); final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); @@ -407,12 +415,14 @@ public Hashtable getPathStatus(final Path path) throws AzureBlob } } - public void setPathProperties(final Path path, final Hashtable properties) throws AzureBlobFileSystemException { + public void setPathProperties(final Path path, final Hashtable properties, + TrackingContext trackingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("setPathProperties", "setPathProperties")){ LOG.debug("setFilesystemProperties for filesystem: {} path: {} with properties: {}", client.getFileSystem(), path, properties); + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); final String commaSeparatedProperties; try { @@ -420,27 +430,30 @@ public void setPathProperties(final Path path, final Hashtable p } catch (CharacterCodingException ex) { throw new InvalidAbfsRestOperationException(ex); } - final AbfsRestOperation op = client.setPathProperties(getRelativePath(path), commaSeparatedProperties); + final AbfsRestOperation op = client.setPathProperties(getRelativePath(path), commaSeparatedProperties, + trackingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } - public void createFilesystem() throws AzureBlobFileSystemException { + public void createFilesystem(TrackingContext trackingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("createFilesystem", "createFilesystem")){ LOG.debug("createFilesystem for filesystem: {}", client.getFileSystem()); + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); - final AbfsRestOperation op = client.createFilesystem(); + final AbfsRestOperation op = client.createFilesystem(trackingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } - public void deleteFilesystem() throws AzureBlobFileSystemException { + public void deleteFilesystem(TrackingContext trackingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("deleteFilesystem", "deleteFilesystem")) { LOG.debug("deleteFilesystem for filesystem: {}", client.getFileSystem()); + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); - final AbfsRestOperation op = client.deleteFilesystem(); + final AbfsRestOperation op = client.deleteFilesystem(trackingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } @@ -448,7 +461,7 @@ public void deleteFilesystem() throws AzureBlobFileSystemException { public OutputStream createFile(final Path path, final FileSystem.Statistics statistics, final boolean overwrite, final FsPermission permission, - final FsPermission umask) throws AzureBlobFileSystemException { + final FsPermission umask, TrackingContext trackingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(); LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}", @@ -458,6 +471,7 @@ public OutputStream createFile(final Path path, permission, umask, isNamespaceEnabled); + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); String relativePath = getRelativePath(path); boolean isAppendBlob = false; @@ -480,7 +494,7 @@ public OutputStream createFile(final Path path, statistics, isNamespaceEnabled ? getOctalNotation(permission) : null, isNamespaceEnabled ? getOctalNotation(umask) : null, - isAppendBlob + isAppendBlob, trackingContext ); } else { @@ -489,16 +503,19 @@ public OutputStream createFile(final Path path, isNamespaceEnabled ? getOctalNotation(permission) : null, isNamespaceEnabled ? getOctalNotation(umask) : null, isAppendBlob, - null); + null, trackingContext); } perfInfo.registerResult(op.getResult()).registerSuccess(true); - return new AbfsOutputStream( - client, - statistics, - relativePath, - 0, - populateAbfsOutputStreamContext(isAppendBlob)); + AbfsOutputStream out = new AbfsOutputStream( + client, + statistics, + relativePath, + 0, + populateAbfsOutputStreamContext(isAppendBlob)); + trackingContext.setStreamID(out.getOutputStreamID()); + + return out; } } @@ -517,7 +534,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa final FileSystem.Statistics statistics, final String permission, final String umask, - final boolean isAppendBlob) throws AzureBlobFileSystemException { + final boolean isAppendBlob, TrackingContext trackingContext) throws AzureBlobFileSystemException { AbfsRestOperation op; try { @@ -525,12 +542,12 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa // avoided for cases when no pre-existing file is present (major portion // of create file traffic falls into the case of no pre-existing file). op = client.createPath(relativePath, true, - false, permission, umask, isAppendBlob, null); + false, permission, umask, isAppendBlob, null, trackingContext); } catch (AbfsRestOperationException e) { if (e.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { // File pre-exists, fetch eTag try { - op = client.getPathStatus(relativePath, false); + op = client.getPathStatus(relativePath, false, trackingContext); } catch (AbfsRestOperationException ex) { if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { // Is a parallel access case, as file which was found to be @@ -549,7 +566,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa try { // overwrite only if eTag matches with the file properties fetched befpre op = client.createPath(relativePath, true, - true, permission, umask, isAppendBlob, eTag); + true, permission, umask, isAppendBlob, eTag, trackingContext); } catch (AbfsRestOperationException ex) { if (ex.getStatusCode() == HttpURLConnection.HTTP_PRECON_FAILED) { // Is a parallel access case, as file with eTag was just queried @@ -586,7 +603,8 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext(boolean isAppend .build(); } - public void createDirectory(final Path path, final FsPermission permission, final FsPermission umask) + public void createDirectory(final Path path, final FsPermission permission, final FsPermission umask, + TrackingContext trackingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("createDirectory", "createPath")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(); @@ -599,21 +617,24 @@ public void createDirectory(final Path path, final FsPermission permission, fina final AbfsRestOperation op = client.createPath(getRelativePath(path), false, true, isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null, false, null); + isNamespaceEnabled ? getOctalNotation(umask) : null, false, null, + trackingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } - public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statistics statistics) + public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statistics statistics, + TrackingContext trackingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("openFileForRead", "getPathStatus")) { LOG.debug("openFileForRead filesystem: {} path: {}", client.getFileSystem(), path); + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getPathStatus(relativePath, false); + final AbfsRestOperation op = client.getPathStatus(relativePath, false, trackingContext); perfInfo.registerResult(op.getResult()); final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); @@ -631,10 +652,12 @@ public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statist perfInfo.registerSuccess(true); // Add statistics for InputStream - return new AbfsInputStream(client, statistics, + AbfsInputStream in = new AbfsInputStream(client, statistics, relativePath, contentLength, populateAbfsInputStreamContext(), eTag); + trackingContext.setStreamID(in.getInputStreamID()); + return } } @@ -647,17 +670,19 @@ private AbfsInputStreamContext populateAbfsInputStreamContext() { .build(); } - public OutputStream openFileForWrite(final Path path, final FileSystem.Statistics statistics, final boolean overwrite) throws + public OutputStream openFileForWrite(final Path path, final FileSystem.Statistics statistics, + final boolean overwrite, TrackingContext trackingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("openFileForWrite", "getPathStatus")) { LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {}", client.getFileSystem(), path, overwrite); + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getPathStatus(relativePath, false); + final AbfsRestOperation op = client.getPathStatus(relativePath, false, trackingContext); perfInfo.registerResult(op.getResult()); final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); @@ -679,21 +704,24 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic if (isAppendBlobKey(path.toString())) { isAppendBlob = true; } - - return new AbfsOutputStream( - client, - statistics, - relativePath, - offset, - populateAbfsOutputStreamContext(isAppendBlob)); + AbfsOutputStream out = new AbfsOutputStream( + client, + statistics, + relativePath, + offset, + populateAbfsOutputStreamContext(isAppendBlob)); + trackingContext.setStreamID(out.getOutputStreamID()); + + return out; } } - public void rename(final Path source, final Path destination) throws + public void rename(final Path source, final Path destination, TrackingContext trackingContext) throws AzureBlobFileSystemException { final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue; + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); if (isAtomicRenameKey(source.getName())) { LOG.warn("The atomic rename feature is not supported by the ABFS scheme; however rename," @@ -713,7 +741,7 @@ public void rename(final Path source, final Path destination) throws do { try (AbfsPerfInfo perfInfo = startTracking("rename", "renamePath")) { AbfsRestOperation op = client.renamePath(sourceRelativePath, - destinationRelativePath, continuation); + destinationRelativePath, continuation, trackingContext); perfInfo.registerResult(op.getResult()); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); perfInfo.registerSuccess(true); @@ -746,12 +774,13 @@ public void delete(final Path path, final boolean recursive, TrackingContext tra do { try (AbfsPerfInfo perfInfo = startTracking("delete", "deletePath")) { AbfsRestOperation op = client.deletePath( - relativePath, recursive, continuation); + relativePath, recursive, continuation, trackingContext); perfInfo.registerResult(op.getResult()); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); perfInfo.registerSuccess(true); countAggregate++; shouldContinue = continuation != null && !continuation.isEmpty(); + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); if (!shouldContinue) { perfInfo.registerAggregates(startAggregate, countAggregate); @@ -760,26 +789,27 @@ public void delete(final Path path, final boolean recursive, TrackingContext tra } while (shouldContinue); } - public FileStatus getFileStatus(final Path path) throws IOException { + public FileStatus getFileStatus(final Path path, TrackingContext trackingContext) throws IOException { try (AbfsPerfInfo perfInfo = startTracking("getFileStatus", "undetermined")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(); LOG.debug("getFileStatus filesystem: {} path: {} isNamespaceEnabled: {}", client.getFileSystem(), path, isNamespaceEnabled); + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); final AbfsRestOperation op; if (path.isRoot()) { if (isNamespaceEnabled) { perfInfo.registerCallee("getAclStatus"); - op = client.getAclStatus(getRelativePath(path)); + op = client.getAclStatus(getRelativePath(path), trackingContext); } else { perfInfo.registerCallee("getFilesystemProperties"); - op = client.getFilesystemProperties(); + op = client.getFilesystemProperties(trackingContext); } } else { perfInfo.registerCallee("getPathStatus"); - op = client.getPathStatus(getRelativePath(path), false); + op = client.getPathStatus(getRelativePath(path), false, trackingContext); } perfInfo.registerResult(op.getResult()); @@ -833,8 +863,9 @@ public FileStatus getFileStatus(final Path path) throws IOException { * @param path The list path. * @return the entries in the path. * */ - public FileStatus[] listStatus(final Path path) throws IOException { - return listStatus(path, null); + public FileStatus[] listStatus(final Path path, TrackingContext trackingContext) throws IOException { + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); + return listStatus(path, null, trackingContext); } /** @@ -849,10 +880,12 @@ public FileStatus[] listStatus(final Path path) throws IOException { * @return the entries in the path start from "startFrom" in lexical order. * */ @InterfaceStability.Unstable - public FileStatus[] listStatus(final Path path, final String startFrom) throws IOException { + public FileStatus[] listStatus(final Path path, final String startFrom, + TrackingContext trackingContext) throws IOException { final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue = true; + trackingContext.updateIteration(); LOG.debug("listStatus filesystem: {} path: {}, startFrom: {}", client.getFileSystem(), @@ -873,7 +906,7 @@ public FileStatus[] listStatus(final Path path, final String startFrom) throws I do { try (AbfsPerfInfo perfInfo = startTracking("listStatus", "listPath")) { AbfsRestOperation op = client.listPath(relativePath, false, - abfsConfiguration.getListMaxResults(), continuation); + abfsConfiguration.getListMaxResults(), continuation, trackingContext); perfInfo.registerResult(op.getResult()); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); ListResultSchema retrievedSchema = op.getResult().getListResultSchema(); @@ -990,12 +1023,14 @@ private String generateContinuationTokenForNonXns(String path, final String firs return encodedTokenBuilder.toString(); } - public void setOwner(final Path path, final String owner, final String group) throws + public void setOwner(final Path path, final String owner, final String group, + TrackingContext trackingContext) throws AzureBlobFileSystemException { if (!getIsNamespaceEnabled()) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfo = startTracking("setOwner", "setOwner")) { @@ -1011,18 +1046,20 @@ public void setOwner(final Path path, final String owner, final String group) th final AbfsRestOperation op = client.setOwner(getRelativePath(path), transformedOwner, - transformedGroup); + transformedGroup, trackingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } - public void setPermission(final Path path, final FsPermission permission) throws + public void setPermission(final Path path, final FsPermission permission, + TrackingContext trackingContext) throws AzureBlobFileSystemException { if (!getIsNamespaceEnabled()) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfo = startTracking("setPermission", "setPermission")) { @@ -1033,18 +1070,20 @@ public void setPermission(final Path path, final FsPermission permission) throws permission); final AbfsRestOperation op = client.setPermission(getRelativePath(path), - String.format(AbfsHttpConstants.PERMISSION_FORMAT, permission.toOctal())); + String.format(AbfsHttpConstants.PERMISSION_FORMAT, permission.toOctal()), trackingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } - public void modifyAclEntries(final Path path, final List aclSpec) throws + public void modifyAclEntries(final Path path, final List aclSpec, + TrackingContext trackingContext) throws AzureBlobFileSystemException { if (!getIsNamespaceEnabled()) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfoGet = startTracking("modifyAclEntries", "getAclStatus")) { @@ -1060,7 +1099,7 @@ public void modifyAclEntries(final Path path, final List aclSpec) thro String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getAclStatus(relativePath, useUpn); + final AbfsRestOperation op = client.getAclStatus(relativePath, useUpn, trackingContext); perfInfoGet.registerResult(op.getResult()); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); @@ -1081,11 +1120,13 @@ public void modifyAclEntries(final Path path, final List aclSpec) thro } } - public void removeAclEntries(final Path path, final List aclSpec) throws AzureBlobFileSystemException { + public void removeAclEntries(final Path path, final List aclSpec, + TrackingContext trackingContext) throws AzureBlobFileSystemException { if (!getIsNamespaceEnabled()) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfoGet = startTracking("removeAclEntries", "getAclStatus")) { @@ -1101,7 +1142,7 @@ public void removeAclEntries(final Path path, final List aclSpec) thro String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getAclStatus(relativePath, isUpnFormat); + final AbfsRestOperation op = client.getAclStatus(relativePath, isUpnFormat, trackingContext); perfInfoGet.registerResult(op.getResult()); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); @@ -1114,7 +1155,7 @@ public void removeAclEntries(final Path path, final List aclSpec) thro try (AbfsPerfInfo perfInfoSet = startTracking("removeAclEntries", "setAcl")) { final AbfsRestOperation setAclOp = client.setAcl(relativePath, - AbfsAclHelper.serializeAclSpec(aclEntries), eTag); + AbfsAclHelper.serializeAclSpec(aclEntries), eTag, trackingContext); perfInfoSet.registerResult(setAclOp.getResult()) .registerSuccess(true) .registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT); @@ -1122,11 +1163,12 @@ public void removeAclEntries(final Path path, final List aclSpec) thro } } - public void removeDefaultAcl(final Path path) throws AzureBlobFileSystemException { + public void removeDefaultAcl(final Path path, TrackingContext trackingContext) throws AzureBlobFileSystemException { if (!getIsNamespaceEnabled()) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfoGet = startTracking("removeDefaultAcl", "getAclStatus")) { @@ -1137,7 +1179,7 @@ public void removeDefaultAcl(final Path path) throws AzureBlobFileSystemExceptio String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getAclStatus(relativePath); + final AbfsRestOperation op = client.getAclStatus(relativePath, trackingContext); perfInfoGet.registerResult(op.getResult()); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); final Map aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); @@ -1156,7 +1198,7 @@ public void removeDefaultAcl(final Path path) throws AzureBlobFileSystemExceptio try (AbfsPerfInfo perfInfoSet = startTracking("removeDefaultAcl", "setAcl")) { final AbfsRestOperation setAclOp = client.setAcl(relativePath, - AbfsAclHelper.serializeAclSpec(aclEntries), eTag); + AbfsAclHelper.serializeAclSpec(aclEntries), eTag, trackingContext); perfInfoSet.registerResult(setAclOp.getResult()) .registerSuccess(true) .registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT); @@ -1164,11 +1206,12 @@ public void removeDefaultAcl(final Path path) throws AzureBlobFileSystemExceptio } } - public void removeAcl(final Path path) throws AzureBlobFileSystemException { + public void removeAcl(final Path path, TrackingContext trackingContext) throws AzureBlobFileSystemException { if (!getIsNamespaceEnabled()) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfoGet = startTracking("removeAcl", "getAclStatus")){ @@ -1179,7 +1222,7 @@ public void removeAcl(final Path path) throws AzureBlobFileSystemException { String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getAclStatus(relativePath); + final AbfsRestOperation op = client.getAclStatus(relativePath, trackingContext); perfInfoGet.registerResult(op.getResult()); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); @@ -1195,7 +1238,7 @@ public void removeAcl(final Path path) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfoSet = startTracking("removeAcl", "setAcl")) { final AbfsRestOperation setAclOp = client.setAcl(relativePath, - AbfsAclHelper.serializeAclSpec(newAclEntries), eTag); + AbfsAclHelper.serializeAclSpec(newAclEntries), eTag, trackingContext); perfInfoSet.registerResult(setAclOp.getResult()) .registerSuccess(true) .registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT); @@ -1203,11 +1246,13 @@ public void removeAcl(final Path path) throws AzureBlobFileSystemException { } } - public void setAcl(final Path path, final List aclSpec) throws AzureBlobFileSystemException { + public void setAcl(final Path path, final List aclSpec, + TrackingContext trackingContext) throws AzureBlobFileSystemException { if (!getIsNamespaceEnabled()) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfoGet = startTracking("setAcl", "getAclStatus")) { @@ -1223,7 +1268,7 @@ public void setAcl(final Path path, final List aclSpec) throws AzureBl String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getAclStatus(relativePath, isUpnFormat); + final AbfsRestOperation op = client.getAclStatus(relativePath, isUpnFormat, trackingContext); perfInfoGet.registerResult(op.getResult()); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); @@ -1236,7 +1281,7 @@ public void setAcl(final Path path, final List aclSpec) throws AzureBl try (AbfsPerfInfo perfInfoSet = startTracking("setAcl", "setAcl")) { final AbfsRestOperation setAclOp = client.setAcl(relativePath, - AbfsAclHelper.serializeAclSpec(aclEntries), eTag); + AbfsAclHelper.serializeAclSpec(aclEntries), eTag, trackingContext); perfInfoSet.registerResult(setAclOp.getResult()) .registerSuccess(true) .registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT); @@ -1244,11 +1289,12 @@ public void setAcl(final Path path, final List aclSpec) throws AzureBl } } - public AclStatus getAclStatus(final Path path) throws IOException { + public AclStatus getAclStatus(final Path path, TrackingContext trackingContext) throws IOException { if (!getIsNamespaceEnabled()) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfo = startTracking("getAclStatus", "getAclStatus")) { @@ -1257,7 +1303,7 @@ public AclStatus getAclStatus(final Path path) throws IOException { client.getFileSystem(), path); - AbfsRestOperation op = client.getAclStatus(getRelativePath(path)); + AbfsRestOperation op = client.getAclStatus(getRelativePath(path), trackingContext); AbfsHttpOperation result = op.getResult(); perfInfo.registerResult(result); @@ -1290,7 +1336,7 @@ public AclStatus getAclStatus(final Path path) throws IOException { } } - public void access(final Path path, final FsAction mode) + public void access(final Path path, final FsAction mode, TrackingContext trackingContext) throws AzureBlobFileSystemException { LOG.debug("access for filesystem: {}, path: {}, mode: {}", this.client.getFileSystem(), path, mode); @@ -1300,9 +1346,10 @@ public void access(final Path path, final FsAction mode) + " used is not namespace enabled"); return; } + trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfo = startTracking("access", "checkAccess")) { final AbfsRestOperation op = this.client - .checkAccess(getRelativePath(path), mode.SYMBOL); + .checkAccess(getRelativePath(path), mode.SYMBOL, trackingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 592adbc0f1355..f8e52afc37d7b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -68,7 +68,6 @@ public class AbfsClient implements Closeable { private final String filesystem; private final AbfsConfiguration abfsConfiguration; private final String userAgent; - private TrackingContext trackingContext; private final AbfsPerfTracker abfsPerfTracker; private final String accountName; @@ -88,8 +87,6 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden this.retryPolicy = abfsClientContext.getExponentialRetryPolicy(); this.accountName = abfsConfiguration.getAccountName().substring(0, abfsConfiguration.getAccountName().indexOf(AbfsHttpConstants.DOT)); this.authType = abfsConfiguration.getAuthType(accountName); - this.trackingContext = new TrackingContext( - abfsConfiguration.getClientCorrelationID()); String sslProviderName = null; @@ -138,10 +135,6 @@ public String getFileSystem() { return filesystem; } - public TrackingContext getTrackingContext() { - return trackingContext; - } - protected AbfsPerfTracker getAbfsPerfTracker() { return abfsPerfTracker; } @@ -172,7 +165,7 @@ AbfsUriQueryBuilder createDefaultUriQueryBuilder() { return abfsUriQueryBuilder; } - public AbfsRestOperation createFilesystem() throws AzureBlobFileSystemException { + public AbfsRestOperation createFilesystem(TrackingContext trackingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); @@ -185,11 +178,11 @@ public AbfsRestOperation createFilesystem() throws AzureBlobFileSystemException HTTP_METHOD_PUT, url, requestHeaders); - op.execute(); + op.execute(trackingContext); return op; } - public AbfsRestOperation setFilesystemProperties(final String properties) throws AzureBlobFileSystemException { + public AbfsRestOperation setFilesystemProperties(final String properties, TrackingContext trackingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. @@ -209,12 +202,13 @@ public AbfsRestOperation setFilesystemProperties(final String properties) throws HTTP_METHOD_PUT, url, requestHeaders); - op.execute(); + op.execute(trackingContext); return op; } public AbfsRestOperation listPath(final String relativePath, final boolean recursive, final int listMaxResults, - final String continuation) throws AzureBlobFileSystemException { + final String continuation, TrackingContext trackingContext) + throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); @@ -233,11 +227,11 @@ public AbfsRestOperation listPath(final String relativePath, final boolean recur HTTP_METHOD_GET, url, requestHeaders); - op.execute(); + op.execute(trackingContext); return op; } - public AbfsRestOperation getFilesystemProperties() throws AzureBlobFileSystemException { + public AbfsRestOperation getFilesystemProperties(TrackingContext trackingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); @@ -250,11 +244,11 @@ public AbfsRestOperation getFilesystemProperties() throws AzureBlobFileSystemExc HTTP_METHOD_HEAD, url, requestHeaders); - op.execute(); + op.execute(trackingContext); return op; } - public AbfsRestOperation deleteFilesystem() throws AzureBlobFileSystemException { + public AbfsRestOperation deleteFilesystem(TrackingContext trackingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); @@ -267,13 +261,14 @@ public AbfsRestOperation deleteFilesystem() throws AzureBlobFileSystemException HTTP_METHOD_DELETE, url, requestHeaders); - op.execute(); + op.execute(trackingContext); return op; } public AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite, final String permission, final String umask, - final boolean isAppendBlob, final String eTag) throws AzureBlobFileSystemException { + final boolean isAppendBlob, final String eTag, + TrackingContext trackingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); if (!overwrite) { requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, AbfsHttpConstants.STAR)); @@ -309,11 +304,12 @@ public AbfsRestOperation createPath(final String path, final boolean isFile, fin HTTP_METHOD_PUT, url, requestHeaders); - op.execute(); + op.execute(trackingContext); return op; } - public AbfsRestOperation renamePath(String source, final String destination, final String continuation) + public AbfsRestOperation renamePath(String source, final String destination, final String continuation, + TrackingContext trackingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); @@ -341,7 +337,7 @@ public AbfsRestOperation renamePath(String source, final String destination, fin requestHeaders); Instant renameRequestStartTime = Instant.now(); try { - op.execute(); + op.execute(trackingContext); } catch (AzureBlobFileSystemException e) { final AbfsRestOperation idempotencyOp = renameIdempotencyCheckOp( renameRequestStartTime, op, destination); @@ -426,7 +422,7 @@ public AbfsRestOperation append(final String path, final long position, final by url, requestHeaders, buffer, offset, length, sasTokenForReuse); try { - op.execute(); + op.execute(trackingContext); } catch (AzureBlobFileSystemException e) { if (isAppendBlob && appendSuccessCheckOp(op, path, (position + length))) { final AbfsRestOperation successOp = new AbfsRestOperation( @@ -490,11 +486,12 @@ public AbfsRestOperation flush(final String path, final long position, boolean r HTTP_METHOD_PUT, url, requestHeaders, sasTokenForReuse); - op.execute(); + op.execute(trackingContext); return op; } - public AbfsRestOperation setPathProperties(final String path, final String properties) + public AbfsRestOperation setPathProperties(final String path, final String properties, + TrackingContext trackingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use @@ -515,11 +512,12 @@ public AbfsRestOperation setPathProperties(final String path, final String prope HTTP_METHOD_PUT, url, requestHeaders); - op.execute(); + op.execute(trackingContext); return op; } - public AbfsRestOperation getPathStatus(final String path, final boolean includeProperties) throws AzureBlobFileSystemException { + public AbfsRestOperation getPathStatus(final String path, final boolean includeProperties, + TrackingContext trackingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); @@ -541,7 +539,7 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP HTTP_METHOD_HEAD, url, requestHeaders); - op.execute(); + op.execute(trackingContext); return op; } @@ -568,12 +566,13 @@ public AbfsRestOperation read(final String path, final long position, final byte buffer, bufferOffset, bufferLength, sasTokenForReuse); - op.execute(); + op.execute(trackingContext); return op; } - public AbfsRestOperation deletePath(final String path, final boolean recursive, final String continuation) + public AbfsRestOperation deletePath(final String path, final boolean recursive, final String continuation, + TrackingContext trackingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); @@ -589,9 +588,10 @@ public AbfsRestOperation deletePath(final String path, final boolean recursive, this, HTTP_METHOD_DELETE, url, - requestHeaders); + requestHeaders, + trackingContext); try { - op.execute(); + op.execute(trackingContext); } catch (AzureBlobFileSystemException e) { final AbfsRestOperation idempotencyOp = deleteIdempotencyCheckOp(op); if (idempotencyOp.getResult().getStatusCode() @@ -641,7 +641,8 @@ public AbfsRestOperation deleteIdempotencyCheckOp(final AbfsRestOperation op) { return op; } - public AbfsRestOperation setOwner(final String path, final String owner, final String group) + public AbfsRestOperation setOwner(final String path, final String owner, final String group, + TrackingContext trackingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use @@ -667,11 +668,12 @@ public AbfsRestOperation setOwner(final String path, final String owner, final S AbfsHttpConstants.HTTP_METHOD_PUT, url, requestHeaders); - op.execute(); + op.execute(trackingContext); return op; } - public AbfsRestOperation setPermission(final String path, final String permission) + public AbfsRestOperation setPermission(final String path, final String permission, + TrackingContext trackingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use @@ -692,7 +694,7 @@ public AbfsRestOperation setPermission(final String path, final String permissio AbfsHttpConstants.HTTP_METHOD_PUT, url, requestHeaders); - op.execute(); + op.execute(trackingContext); return op; } @@ -700,7 +702,8 @@ public AbfsRestOperation setAcl(final String path, final String aclSpecString) t return setAcl(path, aclSpecString, AbfsHttpConstants.EMPTY_STRING); } - public AbfsRestOperation setAcl(final String path, final String aclSpecString, final String eTag) + public AbfsRestOperation setAcl(final String path, final String aclSpecString, final String eTag, + TrackingContext trackingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use @@ -725,15 +728,17 @@ public AbfsRestOperation setAcl(final String path, final String aclSpecString, f AbfsHttpConstants.HTTP_METHOD_PUT, url, requestHeaders); - op.execute(); + op.execute(trackingContext); return op; } - public AbfsRestOperation getAclStatus(final String path) throws AzureBlobFileSystemException { - return getAclStatus(path, abfsConfiguration.isUpnUsed()); + public AbfsRestOperation getAclStatus(final String path, TrackingContext trackingContext) + throws AzureBlobFileSystemException { + return getAclStatus(path, abfsConfiguration.isUpnUsed(), trackingContext); } - public AbfsRestOperation getAclStatus(final String path, final boolean useUPN) throws AzureBlobFileSystemException { + public AbfsRestOperation getAclStatus(final String path, final boolean useUPN, + TrackingContext trackingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); @@ -748,7 +753,7 @@ public AbfsRestOperation getAclStatus(final String path, final boolean useUPN) t AbfsHttpConstants.HTTP_METHOD_HEAD, url, requestHeaders); - op.execute(); + op.execute(trackingContext); return op; } @@ -761,7 +766,7 @@ public AbfsRestOperation getAclStatus(final String path, final boolean useUPN) t * @return The {@link AbfsRestOperation} object for the operation * @throws AzureBlobFileSystemException in case of bad requests */ - public AbfsRestOperation checkAccess(String path, String rwx) + public AbfsRestOperation checkAccess(String path, String rwx, TrackingContext trackingContext) throws AzureBlobFileSystemException { AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, CHECK_ACCESS); @@ -771,7 +776,7 @@ public AbfsRestOperation checkAccess(String path, String rwx) AbfsRestOperation op = new AbfsRestOperation( AbfsRestOperationType.CheckAccess, this, AbfsHttpConstants.HTTP_METHOD_HEAD, url, createDefaultHeaders()); - op.execute(); + op.execute(trackingContext); return op; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 6aea25c77f1e6..128be0b38006c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -96,13 +96,16 @@ public AbfsInputStream( abfsInputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds()); this.streamStatistics = abfsInputStreamContext.getStreamStatistics(); this.inputStreamID = StringUtils.right(UUID.randomUUID().toString(), 12); - client.getTrackingContext().setStreamID("IN" + inputStreamID); } public String getPath() { return path; } + public String getInputStreamID() { + return inputStreamID; + } + @Override public int read() throws IOException { byte[] b = new byte[1]; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 34270471227b1..c1cc00b728e96 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -138,7 +138,6 @@ public AbfsOutputStream( this.cachedSasToken = new CachedSASToken( abfsOutputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds()); this.outputStreamID = StringUtils.right(UUID.randomUUID().toString(), 12); - client.getTrackingContext().setStreamID("OUT" + outputStreamID); } /** @@ -173,6 +172,10 @@ public void write(final int byteVal) throws IOException { write(new byte[]{(byte) (byteVal & 0xFF)}); } + public String getOutputStreamID() { + return outputStreamID; + } + /** * Writes length bytes from the specified byte array starting at off to * this output stream. 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 2b4f5b2ec9fe2..d57ad418ea0b5 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 @@ -52,8 +52,6 @@ public class AbfsRestOperation { // all the custom HTTP request headers provided by the caller private final List requestHeaders; - private final TrackingContext trackingContext; - // This is a simple operation class, where all the upload methods have a // request body and all the download methods have a response body. private final boolean hasRequestBody; @@ -110,9 +108,8 @@ String getSasToken() { final AbfsClient client, final String method, final URL url, - final List requestHeaders, - final TrackingContext trackingContext) { - this(operationType, client, method, url, requestHeaders, trackingContext, null); + final List requestHeaders) { + this(operationType, client, method, url, requestHeaders, null); } /** @@ -129,14 +126,12 @@ String getSasToken() { final String method, final URL url, final List requestHeaders, - final TrackingContext trackingContext, final String sasToken) { this.operationType = operationType; this.client = client; this.method = method; this.url = url; this.requestHeaders = requestHeaders; - this.trackingContext = trackingContext; this.hasRequestBody = (AbfsHttpConstants.HTTP_METHOD_PUT.equals(method) || AbfsHttpConstants.HTTP_METHOD_PATCH.equals(method)); this.sasToken = sasToken; @@ -162,12 +157,11 @@ String getSasToken() { String method, URL url, List requestHeaders, - final TrackingContext trackingContext, byte[] buffer, int bufferOffset, int bufferLength, String sasToken) { - this(operationType, client, method, url, requestHeaders, trackingContext, sasToken); + this(operationType, client, method, url, requestHeaders, sasToken); this.buffer = buffer; this.bufferOffset = bufferOffset; this.bufferLength = bufferLength; @@ -179,7 +173,7 @@ String getSasToken() { * HTTP operations. */ @VisibleForTesting - public void execute() throws AzureBlobFileSystemException { + public void execute(TrackingContext trackingContext) throws AzureBlobFileSystemException { // see if we have latency reports from the previous requests String latencyHeader = this.client.getAbfsPerfTracker().getClientLatency(); if (latencyHeader != null && !latencyHeader.isEmpty()) { @@ -187,14 +181,13 @@ public void execute() throws AzureBlobFileSystemException { new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_ABFS_CLIENT_LATENCY, latencyHeader); requestHeaders.add(httpHeader); } -// client.getTrackingContext().setOperation(operationType); retryCount = 0; LOG.debug("First execution of REST operation - {}", operationType); - while (!executeHttpOperation(retryCount)) { + while (!executeHttpOperation(retryCount, trackingContext)) { try { ++retryCount; - this.client.getTrackingContext().setRetryCount(retryCount); + trackingContext.setRetryCount(retryCount); LOG.debug("Retrying REST operation {}. RetryCount = {}", operationType, retryCount); Thread.sleep(client.getRetryPolicy().getRetryInterval(retryCount)); @@ -216,12 +209,16 @@ public void execute() throws AzureBlobFileSystemException { * fails, there may be a retry. The retryCount is incremented with each * attempt. */ - private boolean executeHttpOperation(final int retryCount) throws AzureBlobFileSystemException { + private boolean executeHttpOperation(final int retryCount, TrackingContext trackingContext) + throws AzureBlobFileSystemException { AbfsHttpOperation httpOperation = null; try { // initialize the HTTP request and open the connection - httpOperation = new AbfsHttpOperation(url, method, requestHeaders, client.getTrackingContext()); + httpOperation = new AbfsHttpOperation(url, method, requestHeaders, trackingContext); incrementCounter(AbfsStatistic.CONNECTIONS_MADE, 1); + httpOperation.getConnection() + .setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, + trackingContext.toString()); switch(client.getAuthType()) { case Custom: @@ -243,10 +240,6 @@ private boolean executeHttpOperation(final int retryCount) throws AzureBlobFileS break; } - httpOperation.getConnection() - .setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, - client.getTrackingContext().toString()); - // dump the headers AbfsIoUtils.dumpHeadersToDebugLog("Request Headers", httpOperation.getConnection().getRequestProperties()); @@ -254,13 +247,13 @@ private boolean executeHttpOperation(final int retryCount) throws AzureBlobFileS if (hasRequestBody) { // HttpUrlConnection requires - httpOperation.sendRequest(buffer, bufferOffset, bufferLength, client.getTrackingContext()); + httpOperation.sendRequest(buffer, bufferOffset, bufferLength); incrementCounter(AbfsStatistic.SEND_REQUESTS, 1); incrementCounter(AbfsStatistic.BYTES_SENT, bufferLength); } - httpOperation.processResponse(buffer, bufferOffset, bufferLength, client.getTrackingContext()); + httpOperation.processResponse(buffer, bufferOffset, bufferLength); incrementCounter(AbfsStatistic.GET_RESPONSES, 1); //Only increment bytesReceived counter when the status code is 2XX. if (httpOperation.getStatusCode() >= HttpURLConnection.HTTP_OK diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java index cedca8b5f4632..285db50db197c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java @@ -30,6 +30,7 @@ public class TrackingContext { private String clientCorrelationID; private String fileSystemID = ""; private String clientRequestID; + private String primaryRequestID; private String streamID = ""; private int retryCount; private String hadoopOpName = ""; @@ -44,6 +45,9 @@ public TrackingContext(String fileSystemID, String hadoopOpName) { this.fileSystemID = fileSystemID; this.hadoopOpName = hadoopOpName; streamID = EMPTY_STRING; + retryCount = 0; + iteration = 0; + primaryRequestID = ""; } public void setClientCorrelationID(String clientCorrelationID) { //validation @@ -80,12 +84,13 @@ public void setStreamID(String stream) { // } public void updateIteration() { iteration++; + primaryRequestID = clientRequestID + ":"; } public String toString() { // if (iteration) - String operation = opName + (opName == "listITR" || opName == "ReadAhead"? iteration.toString() : ""); - return clientCorrelationID + clientRequestID + ":" + fileSystemID + ":" + String operation = hadoopOpName + (hadoopOpName == "LS" || hadoopOpName == "ReadAhead"? iteration.toString() : ""); + return clientCorrelationID + clientRequestID + ":" + fileSystemID + ":" + primaryRequestID + streamID + ":" + operation + ":" + retryCount; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index 34b3615c1b526..a8cef1352f300 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -25,6 +25,7 @@ import java.util.UUID; import java.util.concurrent.Callable; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -184,12 +185,12 @@ public void teardown() throws Exception { if (usingFilesystemForSASTests) { abfsConfig.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey.name()); AzureBlobFileSystem tempFs = (AzureBlobFileSystem) FileSystem.newInstance(rawConfig); - tempFs.getAbfsStore().deleteFilesystem(); + tempFs.getAbfsStore().deleteFilesystem(new TrackingContext(abfs.getFileSystemID(), "DL")); } else if (!useConfiguredFileSystem) { // Delete all uniquely created filesystem from the account final AzureBlobFileSystemStore abfsStore = abfs.getAbfsStore(); - abfsStore.deleteFilesystem(); + abfsStore.deleteFilesystem(new TrackingContext(abfs.getFileSystemID(), "DL"); AbfsRestOperationException ex = intercept(AbfsRestOperationException.class, new Callable>() { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java index 29de126c4cc40..a020f4282c0cc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java @@ -22,6 +22,7 @@ import java.util.UUID; import org.apache.hadoop.fs.azurebfs.enums.Trilean; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.junit.Assume; import org.junit.Test; import org.assertj.core.api.Assertions; @@ -54,6 +55,7 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest { private static final String TRUE_STR = "true"; private static final String FALSE_STR = "false"; + private final String fileSystemID = "test-filesystem-id"; private boolean isUsingXNSAccount; public ITestGetNameSpaceEnabled() throws Exception { @@ -82,7 +84,7 @@ public void testGetIsNamespaceEnabledWhenConfigIsTrue() throws Exception { Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs( "getIsNamespaceEnabled should return true when the " + "config is set as true").isTrue(); - fs.getAbfsStore().deleteFilesystem(); + fs.getAbfsStore().deleteFilesystem(new TrackingContext(fileSystemID, "DL"); unsetAndAssert(); } @@ -92,7 +94,7 @@ public void testGetIsNamespaceEnabledWhenConfigIsFalse() throws Exception { Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs( "getIsNamespaceEnabled should return false when the " + "config is set as false").isFalse(); - fs.getAbfsStore().deleteFilesystem(); + fs.getAbfsStore().deleteFilesystem(new TrackingContext(fileSystemID, "DL"); unsetAndAssert(); } @@ -105,7 +107,7 @@ private void unsetAndAssert() throws Exception { "getIsNamespaceEnabled should return the value " + "configured for fs.azure.test.namespace.enabled") .isEqualTo(expectedValue); - fs.getAbfsStore().deleteFilesystem(); + fs.getAbfsStore().deleteFilesystem(new TrackingContext(fileSystemID, "DL")); } private AzureBlobFileSystem getNewFSWithHnsConf( From 5e97c55b77905e441037b0af76db85c69da5f810 Mon Sep 17 00:00:00 2001 From: sumangala Date: Thu, 8 Oct 2020 16:00:11 +0530 Subject: [PATCH 07/77] other IDs --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 11 ++--- .../fs/azurebfs/AzureBlobFileSystem.java | 4 +- .../fs/azurebfs/AzureBlobFileSystemStore.java | 44 ++++++++++--------- .../constants/FileSystemConfigurations.java | 3 -- .../fs/azurebfs/services/AbfsClient.java | 5 ++- .../azurebfs/services/AbfsHttpOperation.java | 12 ++--- .../fs/azurebfs/services/AbfsInputStream.java | 24 +++++++--- .../azurebfs/services/AbfsOutputStream.java | 18 +++++--- .../azurebfs/services/AbfsRestOperation.java | 5 ++- .../fs/azurebfs/utils/TrackingContext.java | 9 +++- .../hadoop/fs/azurebfs/ITestAbfsClient.java | 4 +- 11 files changed, 84 insertions(+), 55 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 9a5f7efc8300c..3977a396dd1cb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -67,6 +67,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.*; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.*; @@ -218,7 +219,7 @@ public class AbfsConfiguration{ private String userAgentId; @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_CLIENT_CORRELATIONID, - DefaultValue = DEFAULT_FS_AZURE_CLIENT_CORRELATION_ID) + DefaultValue = EMPTY_STRING) private String clientCorrelationID; @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_CLUSTER_NAME, @@ -293,15 +294,9 @@ public String getAccountName() { /** * Gets client correlation ID provided in config. - * @return Client Correlation ID config value or empty string (default value) - * if length and character constraints are not met + * @return Client Correlation ID config */ public String getClientCorrelationID() { - if ((clientCorrelationID.length() > MAX_CLIENT_CORRELATION_ID_LENGTH) || - (!clientCorrelationID.matches(CLIENT_CORRELATION_ID_PATTERN))) { - return DEFAULT_FS_AZURE_CLIENT_CORRELATION_ID; - } - return clientCorrelationID; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 0f80673932c6e..72fc5a1ee11ac 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -665,6 +665,7 @@ public void setOwner(final Path path, final String owner, final String group) super.setOwner(path, owner, group); return; } + TrackingContext trackingContext = new TrackingContext(fileSystemID, "PO") if ((owner == null || owner.isEmpty()) && (group == null || group.isEmpty())) { throw new IllegalArgumentException("A valid owner or group must be specified."); @@ -675,7 +676,8 @@ public void setOwner(final Path path, final String owner, final String group) try { abfsStore.setOwner(qualifiedPath, owner, - group); + group, + trackingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 2bcb8c3422e54..f80f2029454db 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -387,7 +387,8 @@ public void setFilesystemProperties(final Hashtable properties) throw new InvalidAbfsRestOperationException(ex); } - final AbfsRestOperation op = client.setFilesystemProperties(commaSeparatedProperties); + final AbfsRestOperation op = client.setFilesystemProperties(commaSeparatedProperties, + new TrackingContext("test-filesystem-id", "FS")); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } @@ -463,7 +464,7 @@ public OutputStream createFile(final Path path, final boolean overwrite, final FsPermission permission, final FsPermission umask, TrackingContext trackingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) { - boolean isNamespaceEnabled = getIsNamespaceEnabled(); + boolean isNamespaceEnabled = getIsNamespaceEnabled(trackingContext); LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}", client.getFileSystem(), path, @@ -512,8 +513,9 @@ public OutputStream createFile(final Path path, statistics, relativePath, 0, - populateAbfsOutputStreamContext(isAppendBlob)); - trackingContext.setStreamID(out.getOutputStreamID()); + populateAbfsOutputStreamContext(isAppendBlob), + trackingContext); +// trackingContext.setStreamID(out.getOutputStreamID()); return out; } @@ -548,6 +550,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa // File pre-exists, fetch eTag try { op = client.getPathStatus(relativePath, false, trackingContext); + trackingContext.setPrimaryRequestID(); } catch (AbfsRestOperationException ex) { if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { // Is a parallel access case, as file which was found to be @@ -607,7 +610,7 @@ public void createDirectory(final Path path, final FsPermission permission, fina TrackingContext trackingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("createDirectory", "createPath")) { - boolean isNamespaceEnabled = getIsNamespaceEnabled(); + boolean isNamespaceEnabled = getIsNamespaceEnabled(trackingContext); LOG.debug("createDirectory filesystem: {} path: {} permission: {} umask: {} isNamespaceEnabled: {}", client.getFileSystem(), path, @@ -656,8 +659,8 @@ public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statist relativePath, contentLength, populateAbfsInputStreamContext(), eTag); - trackingContext.setStreamID(in.getInputStreamID()); - return +// trackingContext.setStreamID(in.getInputStreamID()); + return in; } } @@ -709,8 +712,9 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic statistics, relativePath, offset, - populateAbfsOutputStreamContext(isAppendBlob)); - trackingContext.setStreamID(out.getOutputStreamID()); + populateAbfsOutputStreamContext(isAppendBlob), + trackingContext); +// trackingContext.setStreamID(out.getOutputStreamID()); return out; } @@ -791,7 +795,7 @@ public void delete(final Path path, final boolean recursive, TrackingContext tra public FileStatus getFileStatus(final Path path, TrackingContext trackingContext) throws IOException { try (AbfsPerfInfo perfInfo = startTracking("getFileStatus", "undetermined")) { - boolean isNamespaceEnabled = getIsNamespaceEnabled(); + boolean isNamespaceEnabled = getIsNamespaceEnabled(trackingContext); LOG.debug("getFileStatus filesystem: {} path: {} isNamespaceEnabled: {}", client.getFileSystem(), path, @@ -897,7 +901,7 @@ public FileStatus[] listStatus(final Path path, final String startFrom, // generate continuation token if a valid startFrom is provided. if (startFrom != null && !startFrom.isEmpty()) { - continuation = getIsNamespaceEnabled() + continuation = getIsNamespaceEnabled(trackingContext) ? generateContinuationTokenForXns(startFrom) : generateContinuationTokenForNonXns(relativePath, startFrom); } @@ -1026,7 +1030,7 @@ private String generateContinuationTokenForNonXns(String path, final String firs public void setOwner(final Path path, final String owner, final String group, TrackingContext trackingContext) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled()) { + if (!getIsNamespaceEnabled(trackingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } @@ -1055,7 +1059,7 @@ public void setOwner(final Path path, final String owner, final String group, public void setPermission(final Path path, final FsPermission permission, TrackingContext trackingContext) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled()) { + if (!getIsNamespaceEnabled(trackingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } @@ -1079,7 +1083,7 @@ public void setPermission(final Path path, final FsPermission permission, public void modifyAclEntries(final Path path, final List aclSpec, TrackingContext trackingContext) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled()) { + if (!getIsNamespaceEnabled(trackingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } @@ -1122,7 +1126,7 @@ public void modifyAclEntries(final Path path, final List aclSpec, public void removeAclEntries(final Path path, final List aclSpec, TrackingContext trackingContext) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled()) { + if (!getIsNamespaceEnabled(trackingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } @@ -1164,7 +1168,7 @@ public void removeAclEntries(final Path path, final List aclSpec, } public void removeDefaultAcl(final Path path, TrackingContext trackingContext) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled()) { + if (!getIsNamespaceEnabled(trackingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } @@ -1207,7 +1211,7 @@ public void removeDefaultAcl(final Path path, TrackingContext trackingContext) t } public void removeAcl(final Path path, TrackingContext trackingContext) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled()) { + if (!getIsNamespaceEnabled(trackingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } @@ -1248,7 +1252,7 @@ public void removeAcl(final Path path, TrackingContext trackingContext) throws A public void setAcl(final Path path, final List aclSpec, TrackingContext trackingContext) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled()) { + if (!getIsNamespaceEnabled(trackingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } @@ -1290,7 +1294,7 @@ public void setAcl(final Path path, final List aclSpec, } public AclStatus getAclStatus(final Path path, TrackingContext trackingContext) throws IOException { - if (!getIsNamespaceEnabled()) { + if (!getIsNamespaceEnabled(trackingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } @@ -1341,7 +1345,7 @@ public void access(final Path path, final FsAction mode, TrackingContext trackin LOG.debug("access for filesystem: {}, path: {}, mode: {}", this.client.getFileSystem(), path, mode); if (!this.abfsConfiguration.isCheckAccessEnabled() - || !getIsNamespaceEnabled()) { + || !getIsNamespaceEnabled(trackingContext)) { LOG.debug("Returning; either check access is not enabled or the account" + " used is not namespace enabled"); return; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index 889d78a3f5f21..2b50a5ddfd40d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -91,9 +91,6 @@ public final class FileSystemConfigurations { public static final String DEFAULT_FS_AZURE_USER_AGENT_PREFIX = EMPTY_STRING; public static final String DEFAULT_VALUE_UNKNOWN = "UNKNOWN"; - public static final String DEFAULT_FS_AZURE_CLIENT_CORRELATION_ID = EMPTY_STRING; - public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 65; - public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*"; public static final boolean DEFAULT_DELETE_CONSIDERED_IDEMPOTENT = true; public static final int DEFAULT_CLOCK_SKEW_WITH_SERVER_IN_MS = 5 * 60 * 1000; // 5 mins diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index f8e52afc37d7b..a3476fbe2c66a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -698,8 +698,9 @@ public AbfsRestOperation setPermission(final String path, final String permissio return op; } - public AbfsRestOperation setAcl(final String path, final String aclSpecString) throws AzureBlobFileSystemException { - return setAcl(path, aclSpecString, AbfsHttpConstants.EMPTY_STRING); + public AbfsRestOperation setAcl(final String path, final String aclSpecString, + TrackingContext trackingContext) throws AzureBlobFileSystemException { + return setAcl(path, aclSpecString, AbfsHttpConstants.EMPTY_STRING, trackingContext); } public AbfsRestOperation setAcl(final String path, final String aclSpecString, final String eTag, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index 9677121e3e382..22c43f78d2dfc 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -234,16 +234,16 @@ public String getLogString() { * * @throws IOException if an error occurs. */ - public AbfsHttpOperation(final URL url, final String method, final List requestHeaders) throws IOException { - this(url, method, requestHeaders, new TrackingContext("")); - } +// public AbfsHttpOperation(final URL url, final String method, final List requestHeaders) throws IOException { +// this(url, method, requestHeaders, new TrackingContext("", "OP")); +// } - public AbfsHttpOperation(final URL url, final String method, final List requestHeaders, TrackingContext trackingContext) + public AbfsHttpOperation(final URL url, final String method, final List requestHeaders) throws IOException { this.isTraceEnabled = LOG.isTraceEnabled(); this.url = url; this.method = method; - trackingContext.setClientRequestID(); +// trackingContext.setClientRequestID(); this.connection = openConnection(); if (this.connection instanceof HttpsURLConnection) { @@ -262,7 +262,7 @@ public AbfsHttpOperation(final URL url, final String method, final List= bufferSize) { - client.getTrackingContext().setOpName("readAhead"); bytesRead = readInternal(fCursor, buffer, 0, bufferSize, false); } else { bytesRead = readInternal(fCursor, buffer, 0, b.length, true); @@ -242,6 +254,8 @@ private int readInternal(final long position, final byte[] b, final int offset, ReadBufferManager.getBufferManager().queueReadAhead(this, nextOffset, (int) nextSize); nextOffset = nextOffset + nextSize; numReadAheads--; + System.out.println("in the readahead loop " + numReadAheads.toString()); + trackingContext.setPrimaryRequestID(); } // try reading from buffers first diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index c1cc00b728e96..f00453d211275 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -36,6 +36,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -97,12 +98,22 @@ public class AbfsOutputStream extends OutputStream implements Syncable, StreamCa private static final Logger LOG = LoggerFactory.getLogger(AbfsOutputStream.class); + public AbfsOutputStream(final AbfsClient client, + final Statistics statistics, + final String path, + final long position, + AbfsOutputStreamContext abfsOutputStreamContext) { + this(client, statistics, path, position, abfsOutputStreamContext, + new TrackingContext("test-filesystem-id", "OP")); + } + public AbfsOutputStream( final AbfsClient client, final Statistics statistics, final String path, final long position, - AbfsOutputStreamContext abfsOutputStreamContext) { + AbfsOutputStreamContext abfsOutputStreamContext, + TrackingContext trackingContext) { this.client = client; this.statistics = statistics; this.path = path; @@ -138,6 +149,7 @@ public AbfsOutputStream( this.cachedSasToken = new CachedSASToken( abfsOutputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds()); this.outputStreamID = StringUtils.right(UUID.randomUUID().toString(), 12); + trackingContext.setStreamID(outputStreamID); } /** @@ -172,10 +184,6 @@ public void write(final int byteVal) throws IOException { write(new byte[]{(byte) (byteVal & 0xFF)}); } - public String getOutputStreamID() { - return outputStreamID; - } - /** * Writes length bytes from the specified byte array starting at off to * this output stream. 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 d57ad418ea0b5..a28d20c6fb509 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 @@ -187,7 +187,7 @@ public void execute(TrackingContext trackingContext) throws AzureBlobFileSystemE while (!executeHttpOperation(retryCount, trackingContext)) { try { ++retryCount; - trackingContext.setRetryCount(retryCount); + trackingContext.updateRetryCount(); LOG.debug("Retrying REST operation {}. RetryCount = {}", operationType, retryCount); Thread.sleep(client.getRetryPolicy().getRetryInterval(retryCount)); @@ -214,7 +214,8 @@ private boolean executeHttpOperation(final int retryCount, TrackingContext track AbfsHttpOperation httpOperation = null; try { // initialize the HTTP request and open the connection - httpOperation = new AbfsHttpOperation(url, method, requestHeaders, trackingContext); + trackingContext.setClientRequestID(); + httpOperation = new AbfsHttpOperation(url, method, requestHeaders); incrementCounter(AbfsStatistic.CONNECTIONS_MADE, 1); httpOperation.getConnection() .setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java index 285db50db197c..408f26d7f91c8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java @@ -67,8 +67,9 @@ public void setFileSystemID(String fileSystemID) { this.fileSystemID = fileSystemID; } - public void setRetryCount(int count) { - retryCount = count; + public void updateRetryCount() { +// retryCount = count; + retryCount++; } public void setClientRequestID() { @@ -87,6 +88,10 @@ public void updateIteration() { primaryRequestID = clientRequestID + ":"; } + public void setPrimaryRequestID() { + this.primaryRequestID = clientRequestID; + } + public String toString() { // if (iteration) String operation = hadoopOpName + (hadoopOpName == "LS" || hadoopOpName == "ReadAhead"? iteration.toString() : ""); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java index 0bf4295958d80..1fc4581fc5223 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java @@ -27,6 +27,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Ignore; @@ -127,7 +128,8 @@ public void checkRequest(String clientCorrelationId, boolean includeInHeader) String permission = isNamespaceEnabled ? getOctalNotation(FsPermission.getDirDefault()) : null; String umask = isNamespaceEnabled ? getOctalNotation(FsPermission.getUMask(fs.getConf())) : null; AbfsRestOperation op = client.createPath(path, false, true, - permission, umask, false, null); + permission, umask, false, null, + new TrackingContext(fs.getFileSystemID(), "CR")); int responseCode = op.getResult().getStatusCode(); assertEquals("Status code", HTTP_CREATED, responseCode); From 2df54bfb9008324ebee052c052b1c85f88f37417 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Thu, 8 Oct 2020 09:05:41 -0700 Subject: [PATCH 08/77] debug --- .../fs/azurebfs/AzureBlobFileSystem.java | 2 +- .../fs/azurebfs/AzureBlobFileSystemStore.java | 2 +- .../fs/azurebfs/services/AbfsClient.java | 31 ++++++++++++------- .../fs/azurebfs/services/AbfsInputStream.java | 5 +-- .../azurebfs/services/AbfsOutputStream.java | 6 ++-- .../fs/azurebfs/utils/TrackingContext.java | 8 ++++- .../azurebfs/AbstractAbfsIntegrationTest.java | 10 +++--- .../azurebfs/ITestFileSystemProperties.java | 4 ++- .../fs/azurebfs/ITestGetNameSpaceEnabled.java | 4 +-- 9 files changed, 47 insertions(+), 25 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 72fc5a1ee11ac..ff725bbc68f5d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -665,7 +665,7 @@ public void setOwner(final Path path, final String owner, final String group) super.setOwner(path, owner, group); return; } - TrackingContext trackingContext = new TrackingContext(fileSystemID, "PO") + TrackingContext trackingContext = new TrackingContext(fileSystemID, "PO"); if ((owner == null || owner.isEmpty()) && (group == null || group.isEmpty())) { throw new IllegalArgumentException("A valid owner or group must be specified."); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index f80f2029454db..87d5d451ceaa9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -1116,7 +1116,7 @@ public void modifyAclEntries(final Path path, final List aclSpec, try (AbfsPerfInfo perfInfoSet = startTracking("modifyAclEntries", "setAcl")) { final AbfsRestOperation setAclOp = client.setAcl(relativePath, - AbfsAclHelper.serializeAclSpec(aclEntries), eTag); + AbfsAclHelper.serializeAclSpec(aclEntries), eTag, trackingContext); perfInfoSet.registerResult(setAclOp.getResult()) .registerSuccess(true) .registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index a3476fbe2c66a..3a319211148fa 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -340,7 +340,7 @@ public AbfsRestOperation renamePath(String source, final String destination, fin op.execute(trackingContext); } catch (AzureBlobFileSystemException e) { final AbfsRestOperation idempotencyOp = renameIdempotencyCheckOp( - renameRequestStartTime, op, destination); + renameRequestStartTime, op, destination, trackingContext); if (idempotencyOp.getResult().getStatusCode() == op.getResult().getStatusCode()) { // idempotency did not return different result @@ -371,7 +371,8 @@ public AbfsRestOperation renamePath(String source, final String destination, fin public AbfsRestOperation renameIdempotencyCheckOp( final Instant renameRequestStartTime, final AbfsRestOperation op, - final String destination) throws AzureBlobFileSystemException { + final String destination, + TrackingContext trackingContext) throws AzureBlobFileSystemException { if ((op.isARetriedRequest()) && (op.getResult().getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND)) { // Server has returned HTTP 404, which means rename source no longer @@ -380,7 +381,7 @@ public AbfsRestOperation renameIdempotencyCheckOp( try { final AbfsRestOperation destStatusOp = getPathStatus(destination, - false); + false , trackingContext); if (destStatusOp.getResult().getStatusCode() == HttpURLConnection.HTTP_OK) { String lmt = destStatusOp.getResult().getResponseHeader( @@ -400,7 +401,14 @@ public AbfsRestOperation renameIdempotencyCheckOp( } public AbfsRestOperation append(final String path, final long position, final byte[] buffer, final int offset, - final int length, final String cachedSasToken, final boolean isAppendBlob) throws AzureBlobFileSystemException { + final int length, final String cachedSasToken, final boolean isAppendBlob) throws AzureBlobFileSystemException { + return append(path, position, buffer, offset, length, cachedSasToken, + isAppendBlob, new TrackingContext("test-filesystem-id", "AP")); + } + + public AbfsRestOperation append(final String path, final long position, final byte[] buffer, final int offset, + final int length, final String cachedSasToken, final boolean isAppendBlob, + TrackingContext trackingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. @@ -424,7 +432,7 @@ public AbfsRestOperation append(final String path, final long position, final by try { op.execute(trackingContext); } catch (AzureBlobFileSystemException e) { - if (isAppendBlob && appendSuccessCheckOp(op, path, (position + length))) { + if (isAppendBlob && appendSuccessCheckOp(op, path, (position + length), trackingContext)) { final AbfsRestOperation successOp = new AbfsRestOperation( AbfsRestOperationType.Append, this, @@ -446,10 +454,10 @@ public AbfsRestOperation append(final String path, final long position, final by // Hence, we pass/succeed the appendblob append call // in case we are doing a retry after checking the length of the file public boolean appendSuccessCheckOp(AbfsRestOperation op, final String path, - final long length) throws AzureBlobFileSystemException { + final long length, TrackingContext trackingContext) throws AzureBlobFileSystemException { if ((op.isARetriedRequest()) && (op.getResult().getStatusCode() == HttpURLConnection.HTTP_BAD_REQUEST)) { - final AbfsRestOperation destStatusOp = getPathStatus(path, false); + final AbfsRestOperation destStatusOp = getPathStatus(path, false, trackingContext); if (destStatusOp.getResult().getStatusCode() == HttpURLConnection.HTTP_OK) { String fileLength = destStatusOp.getResult().getResponseHeader( HttpHeaderConfigurations.CONTENT_LENGTH); @@ -462,7 +470,8 @@ public boolean appendSuccessCheckOp(AbfsRestOperation op, final String path, } public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData, - boolean isClose, final String cachedSasToken) + boolean isClose, final String cachedSasToken, + TrackingContext trackingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use @@ -544,7 +553,8 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP } public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset, - final int bufferLength, final String eTag, String cachedSasToken) throws AzureBlobFileSystemException { + final int bufferLength, final String eTag, String cachedSasToken, + TrackingContext trackingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); requestHeaders.add(new AbfsHttpHeader(RANGE, String.format("bytes=%d-%d", position, position + bufferLength - 1))); @@ -588,8 +598,7 @@ public AbfsRestOperation deletePath(final String path, final boolean recursive, this, HTTP_METHOD_DELETE, url, - requestHeaders, - trackingContext); + requestHeaders); try { op.execute(trackingContext); } catch (AzureBlobFileSystemException e) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 0cd5b711ec7af..e96de8dc0252e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -254,7 +254,7 @@ private int readInternal(final long position, final byte[] b, final int offset, ReadBufferManager.getBufferManager().queueReadAhead(this, nextOffset, (int) nextSize); nextOffset = nextOffset + nextSize; numReadAheads--; - System.out.println("in the readahead loop " + numReadAheads.toString()); + System.out.println("in the readahead loop " + Integer.toString(numReadAheads)); trackingContext.setPrimaryRequestID(); } @@ -302,7 +302,8 @@ int readRemote(long position, byte[] b, int offset, int length) throws IOExcepti AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "readRemote", "read")) { LOG.trace("Trigger client.read for path={} position={} offset={} length={}", path, position, offset, length); - op = client.read(path, position, b, offset, length, tolerateOobAppends ? "*" : eTag, cachedSasToken.get()); + op = client.read(path, position, b, offset, length, tolerateOobAppends ? "*" : eTag, cachedSasToken.get(), + new TrackingContext("test-filesystem-id", inputStreamID, "RD")); cachedSasToken.update(op.getSasToken()); if (streamStatistics != null) { streamStatistics.remoteReadOperation(); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index f00453d211275..405df0838faae 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -351,7 +351,8 @@ private void writeAppendBlobCurrentBufferToService() throws IOException { try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "writeCurrentBufferToService", "append")) { AbfsRestOperation op = client.append(path, offset, bytes, 0, - bytesLength, cachedSasToken.get(), this.isAppendBlob); + bytesLength, cachedSasToken.get(), this.isAppendBlob, + new TrackingContext("test-filesystem-id", outputStreamID, "AP")); cachedSasToken.update(op.getSasToken()); outputStreamStatistics.uploadSuccessful(bytesLength); perfInfo.registerResult(op.getResult()); @@ -465,7 +466,8 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset, AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "flushWrittenBytesToServiceInternal", "flush")) { - AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, isClose, cachedSasToken.get()); + AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, isClose, cachedSasToken.get(), + new TrackingContext("test-filesystem-id", outputStreamID, "OP")); cachedSasToken.update(op.getSasToken()); perfInfo.registerResult(op.getResult()).registerSuccess(true); } catch (AzureBlobFileSystemException ex) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java index 408f26d7f91c8..bb4b93890005b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java @@ -49,6 +49,12 @@ public TrackingContext(String fileSystemID, String hadoopOpName) { iteration = 0; primaryRequestID = ""; } + + public TrackingContext(String fileSystemID, String streamID, String hadoopOpName) { + this(fileSystemID, hadoopOpName); + this.streamID = streamID; + } + public void setClientCorrelationID(String clientCorrelationID) { //validation if ((clientCorrelationID.length() > MAX_CLIENT_CORRELATION_ID_LENGTH) @@ -94,7 +100,7 @@ public void setPrimaryRequestID() { public String toString() { // if (iteration) - String operation = hadoopOpName + (hadoopOpName == "LS" || hadoopOpName == "ReadAhead"? iteration.toString() : ""); + String operation = hadoopOpName + (hadoopOpName == "LS" || hadoopOpName == "ReadAhead"? Integer.toString(iteration).toString() : ""); return clientCorrelationID + clientRequestID + ":" + fileSystemID + ":" + primaryRequestID + streamID + ":" + operation + ":" + retryCount; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index a8cef1352f300..9ac2b8bee0f7d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -181,22 +181,23 @@ public void teardown() throws Exception { if (abfs == null) { return; } + TrackingContext trackingContext = new TrackingContext(abfs.getFileSystemID(), "DL"); if (usingFilesystemForSASTests) { abfsConfig.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey.name()); AzureBlobFileSystem tempFs = (AzureBlobFileSystem) FileSystem.newInstance(rawConfig); - tempFs.getAbfsStore().deleteFilesystem(new TrackingContext(abfs.getFileSystemID(), "DL")); + tempFs.getAbfsStore().deleteFilesystem(trackingContext); } else if (!useConfiguredFileSystem) { // Delete all uniquely created filesystem from the account final AzureBlobFileSystemStore abfsStore = abfs.getAbfsStore(); - abfsStore.deleteFilesystem(new TrackingContext(abfs.getFileSystemID(), "DL"); + abfsStore.deleteFilesystem(trackingContext); AbfsRestOperationException ex = intercept(AbfsRestOperationException.class, new Callable>() { @Override public Hashtable call() throws Exception { - return abfsStore.getFilesystemProperties(); + return abfsStore.getFilesystemProperties(trackingContext); } }); if (FILE_SYSTEM_NOT_FOUND.getStatusCode() != ex.getStatusCode()) { @@ -430,7 +431,8 @@ protected AbfsOutputStream createAbfsOutputStreamWithFlushEnabled( abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false); return (AbfsOutputStream) abfss.createFile(path, fs.getFsStatistics(), - true, FsPermission.getDefault(), FsPermission.getUMask(fs.getConf())); + true, FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()), + new TrackingContext(fs.getFileSystemID(), "OP")); } /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java index ba9b639adb602..d9ea6653c4b1e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; /** * Test FileSystemProperties. @@ -65,7 +66,8 @@ public void testBase64FileSystemProperties() throws Exception { final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: value }"); fs.getAbfsStore().setFilesystemProperties(properties); - Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties(); + Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties( + new TrackingContext(fs.getFileSystemID(), "FS")); assertEquals(properties, fetchedProperties); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java index a020f4282c0cc..29cb6ed73e190 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java @@ -84,7 +84,7 @@ public void testGetIsNamespaceEnabledWhenConfigIsTrue() throws Exception { Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs( "getIsNamespaceEnabled should return true when the " + "config is set as true").isTrue(); - fs.getAbfsStore().deleteFilesystem(new TrackingContext(fileSystemID, "DL"); + fs.getAbfsStore().deleteFilesystem(new TrackingContext(fileSystemID, "DL")); unsetAndAssert(); } @@ -94,7 +94,7 @@ public void testGetIsNamespaceEnabledWhenConfigIsFalse() throws Exception { Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs( "getIsNamespaceEnabled should return false when the " + "config is set as false").isFalse(); - fs.getAbfsStore().deleteFilesystem(new TrackingContext(fileSystemID, "DL"); + fs.getAbfsStore().deleteFilesystem(new TrackingContext(fileSystemID, "DL")); unsetAndAssert(); } From d3ba55099abebafe7b6624f8aff17710c0b54fe9 Mon Sep 17 00:00:00 2001 From: sumangala Date: Fri, 9 Oct 2020 08:48:48 +0530 Subject: [PATCH 09/77] debug --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 6 +++- .../fs/azurebfs/utils/TrackingContext.java | 2 +- .../hadoop/fs/azurebfs/ITestAbfsClient.java | 6 ++-- .../ITestAbfsInputStreamStatistics.java | 4 ++- .../ITestAzureBlobFileSystemCreate.java | 13 ++++---- .../ITestAzureBlobFileSystemDelete.java | 11 +++++-- .../ITestAzureBlobFileSystemOauth.java | 12 +++++--- .../ITestAzureBlobFileSystemRename.java | 13 +++++--- .../fs/azurebfs/ITestSharedKeyAuth.java | 4 ++- .../fs/azurebfs/services/TestAbfsClient.java | 8 ++--- .../services/TestAbfsInputStream.java | 22 +++++++++----- .../services/TestAbfsOutputStream.java | 30 ++++++++++++------- 12 files changed, 88 insertions(+), 43 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 87d5d451ceaa9..28eb4aad96219 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -280,7 +280,11 @@ private String[] authorityParts(URI uri) throws InvalidUriAuthorityException, In return authorityParts; } - public boolean getIsNamespaceEnabled(TrackingContext trackingContext) throws AzureBlobFileSystemException { + public boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException { + return getIsNamespaceEnabled(new TrackingContext("test-filesystem-id", "NS")); + } + + public boolean getIsNamespaceEnabled(TrackingContext trackingContext) throws AzureBlobFileSystemException { try { return this.isNamespaceEnabled.toBoolean(); } catch (TrileanConversionException e) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java index bb4b93890005b..9e737585c5ace 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java @@ -28,7 +28,7 @@ public class TrackingContext { private String clientCorrelationID; - private String fileSystemID = ""; + private String fileSystemID; private String clientRequestID; private String primaryRequestID; private String streamID = ""; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java index 1fc4581fc5223..f47a064c80c25 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java @@ -73,7 +73,8 @@ public void testContinuationTokenHavingEqualSign() throws Exception { AbfsClient abfsClient = fs.getAbfsClient(); try { - AbfsRestOperation op = abfsClient.listPath("/", true, LIST_MAX_RESULTS, "==========="); + AbfsRestOperation op = abfsClient.listPath("/", true, LIST_MAX_RESULTS, "===========", + new TrackingContext(fs.getFileSystemID(), "PA")); Assert.assertTrue(false); } catch (AbfsRestOperationException ex) { Assert.assertEquals("InvalidQueryParameterValue", ex.getErrorCode().getErrorCode()); @@ -194,7 +195,8 @@ public void testListPathWithInvalidListMaxResultsValues() throws Exception { private List listPath(String directory) throws IOException { return getFileSystem().getAbfsClient() - .listPath(directory, false, getListMaxResults(), null).getResult() + .listPath(directory, false, getListMaxResults(), null, + new TrackingContext(getFileSystem().getFileSystemID(), "PA")).getResult() .getListResultSchema().paths(); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java index 52dfdf2a61ca8..03b573fa5b79c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java @@ -20,6 +20,7 @@ import java.io.IOException; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.assertj.core.api.Assertions; import org.junit.Test; import org.slf4j.Logger; @@ -269,7 +270,8 @@ public void testWithNullStreamStatistics() throws IOException { // AbfsRestOperation Instance required for eTag. AbfsRestOperation abfsRestOperation = - fs.getAbfsClient().getPathStatus(nullStatFilePath.toUri().getPath(), false); + fs.getAbfsClient().getPathStatus(nullStatFilePath.toUri().getPath(), false, + new TrackingContext(fs.getFileSystemID(), "PA")); // AbfsInputStream with no StreamStatistics. in = new AbfsInputStream(fs.getAbfsClient(), null, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 09304d1ec218d..4c1c6b9e88515 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -25,6 +25,7 @@ import java.util.EnumSet; import java.util.UUID; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.junit.Test; import org.apache.hadoop.conf.Configuration; @@ -343,10 +344,11 @@ public void testNegativeScenariosForCreateOverwriteDisabled() = TestAbfsClient.getMockAbfsClient( fs.getAbfsStore().getClient(), fs.getAbfsStore().getAbfsConfiguration()); + TrackingContext trackingContext = new TrackingContext(fs.getFileSystemID(), "CR") AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); abfsStore = setAzureBlobSystemStoreField(abfsStore, "client", mockClient); - boolean isNamespaceEnabled = abfsStore.getIsNamespaceEnabled(); + boolean isNamespaceEnabled = abfsStore.getIsNamespaceEnabled(trackingContext); AbfsRestOperation successOp = mock( AbfsRestOperation.class); @@ -377,14 +379,14 @@ public void testNegativeScenariosForCreateOverwriteDisabled() .createPath(any(String.class), eq(true), eq(false), isNamespaceEnabled ? any(String.class) : eq(null), isNamespaceEnabled ? any(String.class) : eq(null), - any(boolean.class), eq(null)); + any(boolean.class), eq(null), trackingContext); doThrow(fileNotFoundResponseEx) // Scn1: GFS fails with Http404 .doThrow(serverErrorResponseEx) // Scn2: GFS fails with Http500 .doReturn(successOp) // Scn3: create overwrite=true fails with Http412 .doReturn(successOp) // Scn4: create overwrite=true fails with Http500 .when(mockClient) - .getPathStatus(any(String.class), eq(false)); + .getPathStatus(any(String.class), eq(false), trackingContext); // mock for overwrite=true doThrow( @@ -395,7 +397,7 @@ public void testNegativeScenariosForCreateOverwriteDisabled() .createPath(any(String.class), eq(true), eq(true), isNamespaceEnabled ? any(String.class) : eq(null), isNamespaceEnabled ? any(String.class) : eq(null), - any(boolean.class), eq(null)); + any(boolean.class), eq(null), trackingContext); // Scn1: GFS fails with Http404 // Sequence of events expected: @@ -461,7 +463,8 @@ private void validateCreateFileException(final Class ex Path testPath = new Path("testFile"); intercept( exceptionClass, - () -> abfsStore.createFile(testPath, null, true, permission, umask)); + () -> abfsStore.createFile(testPath, null, true, permission, umask, + new TrackingContext("test-filesystem-id", "CR"))); } private AbfsRestOperationException getMockAbfsRestOperationException(int status) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index 2f2a6191ed48a..84181ccfeb7e3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -26,6 +26,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.Test; @@ -212,12 +213,15 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { fs.getAbfsStore().getClient(), this.getConfiguration()); + TrackingContext trackingContext = new TrackingContext(fs.getFileSystemID(), "DL"); + // Case 1: Not a retried case should throw error back intercept(AbfsRestOperationException.class, () -> client.deletePath( "/NonExistingPath", false, - null)); + null), + trackingContext); // mock idempotency check to mimic retried case AbfsClient mockClient = TestAbfsClient.getMockAbfsClient( @@ -233,12 +237,13 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { doReturn(idempotencyRetOp).when(mockClient).deleteIdempotencyCheckOp(any()); when(mockClient.deletePath("/NonExistingPath", false, - null)).thenCallRealMethod(); + null, trackingContext)).thenCallRealMethod(); Assertions.assertThat(mockClient.deletePath( "/NonExistingPath", false, - null) + null, + trackingContext) .getResult() .getStatusCode()) .describedAs("Idempotency check reports successful " diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java index e517f685784e7..145666c54a43d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java @@ -22,6 +22,7 @@ import java.io.InputStream; import java.util.Map; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.junit.Assume; import org.junit.Test; import org.slf4j.Logger; @@ -123,6 +124,7 @@ public void testBlobDataReader() throws Exception { Assume.assumeTrue("Reader client id not provided", clientId != null); String secret = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET); Assume.assumeTrue("Reader client secret not provided", secret != null); + TrackingContext trackingContext = new TrackingContext(getFileSystem().getFileSystemID(), "AU") prepareFiles(); final AzureBlobFileSystem fs = getBlobReader(); @@ -130,25 +132,27 @@ public void testBlobDataReader() throws Exception { // Use abfsStore in this test to verify the ERROR code in AbfsRestOperationException AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); // TEST READ FS - Map properties = abfsStore.getFilesystemProperties(); + Map properties = abfsStore.getFilesystemProperties(trackingContext); // TEST READ FOLDER assertTrue(fs.exists(EXISTED_FOLDER_PATH)); // TEST DELETE FOLDER try { - abfsStore.delete(EXISTED_FOLDER_PATH, true); + abfsStore.delete(EXISTED_FOLDER_PATH, true, trackingContext); } catch (AbfsRestOperationException e) { assertEquals(AzureServiceErrorCode.AUTHORIZATION_PERMISSION_MISS_MATCH, e.getErrorCode()); } // TEST READ FILE - try (InputStream inputStream = abfsStore.openFileForRead(EXISTED_FILE_PATH, null)) { + try (InputStream inputStream = abfsStore.openFileForRead(EXISTED_FILE_PATH, null, + trackingContext)) { assertTrue(inputStream.read() != 0); } // TEST WRITE FILE try { - abfsStore.openFileForWrite(EXISTED_FILE_PATH, fs.getFsStatistics(), true); + abfsStore.openFileForWrite(EXISTED_FILE_PATH, fs.getFsStatistics(), true, + trackingContext); } catch (AbfsRestOperationException e) { assertEquals(AzureServiceErrorCode.AUTHORIZATION_PERMISSION_MISS_MATCH, e.getErrorCode()); } finally { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java index 2adf70ca6457d..ea612a4e125ea 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java @@ -26,6 +26,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.assertj.core.api.Assertions; import org.junit.Test; import org.junit.Assert; @@ -237,11 +238,13 @@ private void testRenameIdempotencyTriggerChecks( fs.getAbfsStore().getClient(), this.getConfiguration()); + TrackingContext trackingContext = new TrackingContext(fs.getFileSystemID(), "RN"); + AbfsRestOperation idempotencyRetOp = mock(AbfsRestOperation.class); when(idempotencyRetOp.getResult()).thenReturn(idempotencyRetHttpOp); doReturn(idempotencyRetOp).when(client).renameIdempotencyCheckOp(any(), - any(), any()); - when(client.renamePath(any(), any(), any())).thenCallRealMethod(); + any(), any(), trackingContext); + when(client.renamePath(any(), any(), any(), trackingContext)).thenCallRealMethod(); // rename on non-existing source file will trigger idempotency check if (idempotencyRetHttpOp.getStatusCode() == HTTP_OK) { @@ -249,7 +252,8 @@ private void testRenameIdempotencyTriggerChecks( Assertions.assertThat(client.renamePath( "/NonExistingsourcepath", "/destpath", - null) + null, + trackingContext) .getResult() .getStatusCode()) .describedAs("Idempotency check reports recent successful " @@ -261,7 +265,8 @@ private void testRenameIdempotencyTriggerChecks( () -> client.renamePath( "/NonExistingsourcepath", "/destpath", - "")); + "", + trackingContext)); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java index ab55ffa3fe3c6..5a31eef2ab43d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.fs.azurebfs; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.junit.Assume; import org.junit.Test; @@ -54,7 +55,8 @@ public void testWithWrongSharedKey() throws Exception { + "Authorization header is formed correctly including the " + "signature.\", 403", () -> { - abfsClient.getAclStatus("/"); + abfsClient.getAclStatus("/", new TrackingContext(this.getFileSystem().getFileSystemID(), + "AC")); }); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index 1f9e8d3497e96..9489a940f6bfd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -336,10 +336,10 @@ public static AbfsClient getMockAbfsClient(AbfsClient baseAbfsClientInstance, client = TestAbfsClient.setAbfsClientField(client, "userAgent", userAgent); // set tracking context - TrackingContext trackingContext = new TrackingContext("clientCorrID"); - client = TestAbfsClient.setAbfsClientField(client, "trackingContext", - trackingContext); - when(client.getTrackingContext()).thenReturn(trackingContext); +// TrackingContext trackingContext = new TrackingContext("clientCorrID", "TS"); +// client = TestAbfsClient.setAbfsClientField(client, "trackingContext", +// trackingContext); +// when(client.getTrackingContext()).thenReturn(trackingContext); return client; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index c9dacd6eb0642..8adee3e91b351 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -20,6 +20,7 @@ import java.io.IOException; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.junit.Assert; import org.junit.Test; @@ -105,7 +106,8 @@ private void verifyReadCallCount(AbfsClient client, int count) throws Thread.sleep(1000); verify(client, times(count)).read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), - any(String.class), any(String.class)); + any(String.class), any(String.class), + new TrackingContext(getFileSystem().getFileSystemID(), "RD")); } private void checkEvictedStatus(AbfsInputStream inputStream, int position, boolean expectedToThrowException) @@ -160,7 +162,8 @@ public void testFailedReadAhead() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class)); + any(String.class), + new TrackingContext(getFileSystem().getFileSystemID(), "RD")); AbfsInputStream inputStream = getAbfsInputStream(client, "testFailedReadAhead.txt"); @@ -207,7 +210,8 @@ public void testOlderReadAheadFailure() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class)); + any(String.class), + new TrackingContext(getFileSystem().getFileSystemID(), "RD")); AbfsInputStream inputStream = getAbfsInputStream(client, "testOlderReadAheadFailure.txt"); @@ -261,7 +265,8 @@ public void testSuccessfulReadAhead() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class)); + any(String.class), + new TrackingContext(getFileSystem().getFileSystemID(), "RD")); AbfsInputStream inputStream = getAbfsInputStream(client, "testSuccessfulReadAhead.txt"); @@ -306,7 +311,8 @@ public void testReadAheadManagerForFailedReadAhead() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class)); + any(String.class), + new TrackingContext(getFileSystem().getFileSystemID(), "RD")); AbfsInputStream inputStream = getAbfsInputStream(client, "testReadAheadManagerForFailedReadAhead.txt"); @@ -359,7 +365,8 @@ public void testReadAheadManagerForOlderReadAheadFailure() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class)); + any(String.class), + new TrackingContext(getFileSystem().getFileSystemID(), "RD")); AbfsInputStream inputStream = getAbfsInputStream(client, "testReadAheadManagerForOlderReadAheadFailure.txt"); @@ -413,7 +420,8 @@ public void testReadAheadManagerForSuccessfulReadAhead() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class)); + any(String.class), + new TrackingContext(getFileSystem().getFileSystemID(), "RD")); AbfsInputStream inputStream = getAbfsInputStream(client, "testSuccessfulReadAhead.txt"); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index aab0248c407ed..5d5b98c927cfd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -23,6 +23,7 @@ import java.util.HashSet; import java.util.Random; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.junit.Test; import org.mockito.ArgumentCaptor; @@ -84,7 +85,8 @@ public void verifyShortWriteRequest() throws Exception { AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any())).thenReturn(op); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), + new TrackingContext("test-filesystem-id", "OP"))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); final byte[] b = new byte[WRITE_SIZE]; @@ -133,7 +135,8 @@ public void verifyWriteRequest() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any())).thenReturn(op); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), + new TrackingContext("test-filesystem-id", "OP"))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); final byte[] b = new byte[WRITE_SIZE]; @@ -168,7 +171,8 @@ public void verifyWriteRequest() throws Exception { ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); verify(client, times(1)).flush(acFlushString.capture(), acFlushLong.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture()); + acFlushSASToken.capture(), + new TrackingContext("test-filesystem-id", "OP")); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushString.getAllValues()); assertThat(Arrays.asList(Long.valueOf(5*WRITE_SIZE))).describedAs("position").isEqualTo(acFlushLong.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); @@ -192,7 +196,8 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any())).thenReturn(op); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), + new TrackingContext("test-filesystem-id", "OP"))).thenReturn(op); when(op.getSasToken()).thenReturn("testToken"); when(op.getResult()).thenReturn(httpOp); @@ -228,7 +233,8 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); verify(client, times(1)).flush(acFlushString.capture(), acFlushLong.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture()); + acFlushSASToken.capture(), + new TrackingContext("test-filesystem-id", "OP")); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushString.getAllValues()); assertThat(Arrays.asList(Long.valueOf(2*BUFFER_SIZE))).describedAs("position").isEqualTo(acFlushLong.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); @@ -253,7 +259,8 @@ public void verifyWriteRequestOfBufferSize() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any())).thenReturn(op); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), + new TrackingContext("test-filesystem-id", "OP"))).thenReturn(op); when(op.getSasToken()).thenReturn("testToken"); when(op.getResult()).thenReturn(httpOp); @@ -300,7 +307,8 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any())).thenReturn(op); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), + new TrackingContext("test-filesystem-id", "OP"))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, true)); final byte[] b = new byte[BUFFER_SIZE]; @@ -345,7 +353,8 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any())).thenReturn(op); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), + new TrackingContext("test-filesystem-id", "OP"))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); final byte[] b = new byte[BUFFER_SIZE]; @@ -379,7 +388,7 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); verify(client, times(1)).flush(acFlushString.capture(), acFlushLong.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture()); + acFlushSASToken.capture(), new TrackingContext("test-filesystem-id", "OP")); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushString.getAllValues()); assertThat(Arrays.asList(Long.valueOf(2*BUFFER_SIZE))).describedAs("position").isEqualTo(acFlushLong.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); @@ -402,7 +411,8 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any())).thenReturn(op); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), + new TrackingContext("test-filesystem-id", "OP"))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); final byte[] b = new byte[BUFFER_SIZE]; From 70e7d0f38fc26e7edd05b2e744390d03ae0ed387 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Thu, 8 Oct 2020 22:31:46 -0700 Subject: [PATCH 10/77] builds --- .../ITestAbfsInputStreamStatistics.java | 12 ++++++++---- .../ITestAzureBlobFileSystemCreate.java | 2 +- .../ITestAzureBlobFileSystemDelete.java | 4 ++-- .../ITestAzureBlobFileSystemOauth.java | 2 +- .../ITestAzureBlobFileSystemRename.java | 3 ++- ...BlobFileSystemStoreListStatusWithRange.java | 3 ++- .../fs/azurebfs/ITestFileSystemProperties.java | 18 ++++++++++++------ .../fs/azurebfs/ITestGetNameSpaceEnabled.java | 11 +++++++---- .../MockDelegationSASTokenProvider.java | 4 ++-- .../azurebfs/services/TestAbfsInputStream.java | 10 +++++----- 10 files changed, 42 insertions(+), 27 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java index 03b573fa5b79c..f8f5fb0aecf97 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java @@ -63,7 +63,8 @@ public void testInitValues() throws IOException { try { outputStream = createAbfsOutputStreamWithFlushEnabled(fs, initValuesPath); - inputStream = abfss.openFileForRead(initValuesPath, fs.getFsStatistics()); + inputStream = abfss.openFileForRead(initValuesPath, fs.getFsStatistics(), + new TrackingContext("test-filesystem-id", "NS")); AbfsInputStreamStatisticsImpl stats = (AbfsInputStreamStatisticsImpl) inputStream.getStreamStatistics(); @@ -107,7 +108,8 @@ public void testSeekStatistics() throws IOException { //Writing a default buffer in a file. out.write(defBuffer); out.hflush(); - in = abfss.openFileForRead(seekStatPath, fs.getFsStatistics()); + in = abfss.openFileForRead(seekStatPath, fs.getFsStatistics(), + new TrackingContext(fs.getFileSystemID(), "IN")); /* * Writing 1MB buffer to the file, this would make the fCursor(Current @@ -198,7 +200,8 @@ public void testReadStatistics() throws IOException { */ out.write(defBuffer); out.hflush(); - in = abfss.openFileForRead(readStatPath, fs.getFsStatistics()); + in = abfss.openFileForRead(readStatPath, fs.getFsStatistics(), + new TrackingContext("test-filesystem-id", "NS")); /* * Doing file read 10 times. @@ -320,7 +323,8 @@ public void testReadAheadCounters() throws IOException { out.write(defBuffer); out.close(); - in = abfss.openFileForRead(readAheadCountersPath, fs.getFsStatistics()); + in = abfss.openFileForRead(readAheadCountersPath, fs.getFsStatistics(), + new TrackingContext("test-filesystem-id", "NS")); /* * Reading 1KB after each i * KB positions. Hence the reads are from 0 diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 4c1c6b9e88515..191f181cbaeae 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -344,7 +344,7 @@ public void testNegativeScenariosForCreateOverwriteDisabled() = TestAbfsClient.getMockAbfsClient( fs.getAbfsStore().getClient(), fs.getAbfsStore().getAbfsConfiguration()); - TrackingContext trackingContext = new TrackingContext(fs.getFileSystemID(), "CR") + TrackingContext trackingContext = new TrackingContext(fs.getFileSystemID(), "CR"); AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); abfsStore = setAzureBlobSystemStoreField(abfsStore, "client", mockClient); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index 84181ccfeb7e3..ad03fc82ec01b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -220,8 +220,8 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { () -> client.deletePath( "/NonExistingPath", false, - null), - trackingContext); + null, + trackingContext)); // mock idempotency check to mimic retried case AbfsClient mockClient = TestAbfsClient.getMockAbfsClient( diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java index 145666c54a43d..0d94152985707 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java @@ -124,7 +124,7 @@ public void testBlobDataReader() throws Exception { Assume.assumeTrue("Reader client id not provided", clientId != null); String secret = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET); Assume.assumeTrue("Reader client secret not provided", secret != null); - TrackingContext trackingContext = new TrackingContext(getFileSystem().getFileSystemID(), "AU") + TrackingContext trackingContext = new TrackingContext(getFileSystem().getFileSystemID(), "AU"); prepareFiles(); final AzureBlobFileSystem fs = getBlobReader(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java index ea612a4e125ea..b4ec055c8e424 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java @@ -326,7 +326,8 @@ private void testRenameTimeout( Assertions.assertThat(testClient.renameIdempotencyCheckOp( renameRequestStartTime, op, - destinationPath.toUri().getPath()) + destinationPath.toUri().getPath(), + new TrackingContext("test-fs-id", "RN")) .getResult() .getStatusCode()) .describedAs(assertMessage) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java index 849bb6ba0987b..07e9da1bfd90e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java @@ -107,7 +107,8 @@ public ITestAzureBlobFileSystemStoreListStatusWithRange() throws Exception { @Test public void testListWithRange() throws IOException { try { - FileStatus[] listResult = store.listStatus(new Path(path), startFrom); + FileStatus[] listResult = store.listStatus(new Path(path), startFrom, + new org.apache.hadoop.fs.azurebfs.utils.TrackingContext("test-filesystem-id", "LS")); if (!expectedResult) { Assert.fail("Excepting failure with IllegalArgumentException"); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java index d9ea6653c4b1e..7c487a906fb4b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java @@ -78,9 +78,10 @@ public void testBase64PathProperties() throws Exception { final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueTest }"); touch(TEST_PATH); - fs.getAbfsStore().setPathProperties(TEST_PATH, properties); + TrackingContext trackingContext = new TrackingContext(fs.getFileSystemID(), "PA"); + fs.getAbfsStore().setPathProperties(TEST_PATH, properties, trackingContext); Hashtable fetchedProperties = - fs.getAbfsStore().getPathStatus(TEST_PATH); + fs.getAbfsStore().getPathStatus(TEST_PATH, trackingContext); assertEquals(properties, fetchedProperties); } @@ -88,10 +89,12 @@ public void testBase64PathProperties() throws Exception { @Test (expected = Exception.class) public void testBase64InvalidFileSystemProperties() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); + TrackingContext trackingContext = new TrackingContext(fs.getFileSystemID(), "PA"); final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueæ­² }"); fs.getAbfsStore().setFilesystemProperties(properties); - Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties(); + Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties( + trackingContext); assertEquals(properties, fetchedProperties); } @@ -101,9 +104,11 @@ public void testBase64InvalidPathProperties() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueTestå…© }"); + TrackingContext trackingContext = new TrackingContext(fs.getFileSystemID(), "PA"); touch(TEST_PATH); - fs.getAbfsStore().setPathProperties(TEST_PATH, properties); - Hashtable fetchedProperties = fs.getAbfsStore().getPathStatus(TEST_PATH); + fs.getAbfsStore().setPathProperties(TEST_PATH, properties, trackingContext); + Hashtable fetchedProperties = fs.getAbfsStore().getPathStatus(TEST_PATH, + trackingContext); assertEquals(properties, fetchedProperties); } @@ -114,7 +119,8 @@ public void testSetFileSystemProperties() throws Exception { final Hashtable properties = new Hashtable<>(); properties.put("containerForDevTest", "true"); fs.getAbfsStore().setFilesystemProperties(properties); - Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties(); + Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties( + new TrackingContext(fs.getFileSystemID(), "FS")); assertEquals(properties, fetchedProperties); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java index 29cb6ed73e190..42257f9ef4137 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java @@ -180,7 +180,8 @@ private void ensureGetAclCallIsMadeOnceForInvalidConf(String invalidConf) .setNamespaceEnabled(Trilean.getTrilean(invalidConf)); AbfsClient mockClient = callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient(); - verify(mockClient, times(1)).getAclStatus(anyString()); + verify(mockClient, times(1)).getAclStatus(anyString(), + new TrackingContext("test-filesystem-id", "NS")); } private void ensureGetAclCallIsNeverMadeForValidConf(String validConf) @@ -189,14 +190,16 @@ private void ensureGetAclCallIsNeverMadeForValidConf(String validConf) .setNamespaceEnabled(Trilean.getTrilean(validConf)); AbfsClient mockClient = callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient(); - verify(mockClient, never()).getAclStatus(anyString()); + verify(mockClient, never()).getAclStatus(anyString(), + new TrackingContext("test-filesystem-id", "NS")); } private void unsetConfAndEnsureGetAclCallIsMadeOnce() throws IOException { this.getFileSystem().getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN); AbfsClient mockClient = callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient(); - verify(mockClient, times(1)).getAclStatus(anyString()); + verify(mockClient, times(1)).getAclStatus(anyString(), + new TrackingContext("test-filesystem-id", "NS")); } private AbfsClient callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient() @@ -205,7 +208,7 @@ private AbfsClient callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient() final AzureBlobFileSystemStore abfsStore = abfs.getAbfsStore(); final AbfsClient mockClient = mock(AbfsClient.class); doReturn(mock(AbfsRestOperation.class)).when(mockClient) - .getAclStatus(anyString()); + .getAclStatus(anyString(), new TrackingContext("test-filesystem-id", "NS")); abfsStore.setClient(mockClient); abfs.getIsNamespaceEnabled(); return mockClient; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java index 2912285ade262..e6ae3ac84543b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java @@ -106,10 +106,10 @@ private byte[] getUserDelegationKey(String accountName, String appID, String app AbfsHttpOperation op = new AbfsHttpOperation(url, method, requestHeaders); byte[] requestBuffer = requestBody.toString().getBytes(StandardCharsets.UTF_8.toString()); - op.sendRequest(requestBuffer, 0, requestBuffer.length, new TrackingContext(CORRELATION_ID)); + op.sendRequest(requestBuffer, 0, requestBuffer.length); byte[] responseBuffer = new byte[4 * 1024]; - op.processResponse(responseBuffer, 0, responseBuffer.length, new TrackingContext(CORRELATION_ID)); + op.processResponse(responseBuffer, 0, responseBuffer.length); String responseBody = new String(responseBuffer, 0, (int) op.getBytesReceived(), StandardCharsets.UTF_8); int beginIndex = responseBody.indexOf("") + "".length(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index 8adee3e91b351..bab773c90a03a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -107,7 +107,7 @@ private void verifyReadCallCount(AbfsClient client, int count) throws verify(client, times(count)).read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - new TrackingContext(getFileSystem().getFileSystemID(), "RD")); + new TrackingContext("test-fileSystem-id", "RD")); } private void checkEvictedStatus(AbfsInputStream inputStream, int position, boolean expectedToThrowException) @@ -211,7 +211,7 @@ public void testOlderReadAheadFailure() throws Exception { .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - new TrackingContext(getFileSystem().getFileSystemID(), "RD")); + new TrackingContext("test-fileSystemID", "RD")); AbfsInputStream inputStream = getAbfsInputStream(client, "testOlderReadAheadFailure.txt"); @@ -312,7 +312,7 @@ public void testReadAheadManagerForFailedReadAhead() throws Exception { .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - new TrackingContext(getFileSystem().getFileSystemID(), "RD")); + new TrackingContext("test-FileSystemID", "RD")); AbfsInputStream inputStream = getAbfsInputStream(client, "testReadAheadManagerForFailedReadAhead.txt"); @@ -366,7 +366,7 @@ public void testReadAheadManagerForOlderReadAheadFailure() throws Exception { .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - new TrackingContext(getFileSystem().getFileSystemID(), "RD")); + new TrackingContext("test-fileSystemID", "RD")); AbfsInputStream inputStream = getAbfsInputStream(client, "testReadAheadManagerForOlderReadAheadFailure.txt"); @@ -421,7 +421,7 @@ public void testReadAheadManagerForSuccessfulReadAhead() throws Exception { .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - new TrackingContext(getFileSystem().getFileSystemID(), "RD")); + new TrackingContext("test-FileSystemID", "RD")); AbfsInputStream inputStream = getAbfsInputStream(client, "testSuccessfulReadAhead.txt"); From 3cb2ccbacf66a9743019161bc7ead1236d42d0b8 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Sun, 11 Oct 2020 21:05:10 -0700 Subject: [PATCH 11/77] primary req id --- .../fs/azurebfs/AzureBlobFileSystem.java | 5 ++-- .../fs/azurebfs/AzureBlobFileSystemStore.java | 1 + .../azurebfs/services/AbfsHttpOperation.java | 5 ++++ .../fs/azurebfs/services/AbfsInputStream.java | 2 +- .../azurebfs/services/ReadBufferWorker.java | 1 + .../fs/azurebfs/utils/TrackingContext.java | 14 ++++++++--- .../hadoop/fs/azurebfs/ITestAbfsClient.java | 24 ++++++++++++------- .../ITestAzureBlobFileSystemOauth.java | 2 +- .../fs/azurebfs/ITestSharedKeyAuth.java | 2 +- .../services/TestAbfsInputStream.java | 14 +++++------ 10 files changed, 46 insertions(+), 24 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index ff725bbc68f5d..71cdbbed2f268 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -108,6 +108,8 @@ public void initialize(URI uri, Configuration configuration) uri = ensureAuthority(uri, configuration); super.initialize(uri, configuration); setConf(configuration); + fileSystemID = UUID.randomUUID().toString(); + System.out.println("init fs id" + fileSystemID); LOG.debug("Initializing AzureBlobFileSystem for {}", uri); @@ -145,8 +147,6 @@ public void initialize(URI uri, Configuration configuration) AbfsClientThrottlingIntercept.initializeSingleton(abfsConfiguration.isAutoThrottlingEnabled()); LOG.debug("Initializing AzureBlobFileSystem for {} complete", uri); - - fileSystemID = UUID.randomUUID().toString(); } @Override @@ -964,6 +964,7 @@ public AclStatus getAclStatus(final Path path) throws IOException { "getAclStatus is only supported by storage account with the " + "hierarchical namespace enabled."); } + System.out.println("fs id is " + fileSystemID); TrackingContext trackingContext = new TrackingContext(fileSystemID, "AC"); Path qualifiedPath = makeQualified(path); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 28eb4aad96219..2c5a08af203df 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -873,6 +873,7 @@ public FileStatus getFileStatus(final Path path, TrackingContext trackingContext * */ public FileStatus[] listStatus(final Path path, TrackingContext trackingContext) throws IOException { trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); + trackingContext.setPrimaryRequestID(); return listStatus(path, null, trackingContext); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index 22c43f78d2dfc..3e7b67f6ced61 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -151,6 +151,11 @@ public String getResponseHeader(String httpHeader) { return connection.getHeaderField(httpHeader); } + @com.google.common.annotations.VisibleForTesting + public String getRequestHeader(String httpHeader) { + return connection.getRequestProperties().get(httpHeader).toString(); + } + // Returns a trace message for the request @Override public String toString() { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index e96de8dc0252e..ca3cef3ddac26 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -71,7 +71,7 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer, // of valid bytes in buffer) private boolean closed = false; private String inputStreamID; - private TrackingContext trackingContext; + public TrackingContext trackingContext; /** Stream statistics. */ private final AbfsInputStreamStatistics streamStatistics; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java index 41acd7e06f132..61d32a3679a2b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java @@ -62,6 +62,7 @@ public void run() { if (buffer != null) { try { // do the actual read, from the file. + buffer.getStream().trackingContext.setPrimaryRequestID(); int bytesRead = buffer.getStream().readRemote( buffer.getOffset(), buffer.getBuffer(), diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java index 9e737585c5ace..f3f741ff97944 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java @@ -55,6 +55,14 @@ public TrackingContext(String fileSystemID, String streamID, String hadoopOpName this.streamID = streamID; } + public TrackingContext(TrackingContext originalTrackingContext) { + this.fileSystemID = originalTrackingContext.fileSystemID; + this.streamID = originalTrackingContext.streamID; + this.clientCorrelationID = originalTrackingContext.clientCorrelationID; + this.primaryRequestID = originalTrackingContext.clientRequestID; + this.clientRequestID = UUID.randomUUID().toString(); + } + public void setClientCorrelationID(String clientCorrelationID) { //validation if ((clientCorrelationID.length() > MAX_CLIENT_CORRELATION_ID_LENGTH) @@ -91,7 +99,7 @@ public void setStreamID(String stream) { // } public void updateIteration() { iteration++; - primaryRequestID = clientRequestID + ":"; +// primaryRequestID = clientRequestID + ":"; } public void setPrimaryRequestID() { @@ -100,9 +108,9 @@ public void setPrimaryRequestID() { public String toString() { // if (iteration) - String operation = hadoopOpName + (hadoopOpName == "LS" || hadoopOpName == "ReadAhead"? Integer.toString(iteration).toString() : ""); + String operation = hadoopOpName + (hadoopOpName == "LS"? Integer.toString(iteration).toString() : ""); return clientCorrelationID + clientRequestID + ":" + fileSystemID + ":" + primaryRequestID - + streamID + ":" + operation + ":" + retryCount; + + ":" + streamID + ":" + operation + ":" + retryCount; } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java index f47a064c80c25..8d69cd6f231cd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java @@ -124,6 +124,7 @@ public void checkRequest(String clientCorrelationId, boolean includeInHeader) final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem .newInstance(this.getFileSystem().getUri(), config); AbfsClient client = fs.getAbfsClient(); +// TrackingContext String path = getRelativePath(new Path("/testDir")); boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(); String permission = isNamespaceEnabled ? getOctalNotation(FsPermission.getDirDefault()) : null; @@ -134,16 +135,21 @@ public void checkRequest(String clientCorrelationId, boolean includeInHeader) int responseCode = op.getResult().getStatusCode(); assertEquals("Status code", HTTP_CREATED, responseCode); - String responseHeader = op.getResult() - .getResponseHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); + String requestHeader = op.getResult().getRequestHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); + List clientRequestIds = java.util.Arrays.asList( + requestHeader.replace("[","") + .replace("]", "") + .split(":")); if (includeInHeader) { - Assertions.assertThat(responseHeader) - .describedAs("Should contain request IDs") - .startsWith(clientCorrelationId); + assertEquals("There should be 7 items in the header when valid clientCorrelationId is set", + 7, clientRequestIds.size()); + assertTrue("clientCorrelation should be included in the header", + clientRequestIds.contains(clientCorrelationId)); } else if (clientCorrelationId.length() > 0){ - assertFalse( - "Invalid or empty correlationId value should not be included in header", - responseHeader.contains(clientCorrelationId)); + assertEquals("There should be only 6 item in the header when invalid clientCorrelationId is set", + 6, clientRequestIds.size()); + assertFalse("Invalid or empty correlationId value should not be included in header", + clientRequestIds.contains(clientCorrelationId)); } } @@ -196,7 +202,7 @@ private List listPath(String directory) throws IOException { return getFileSystem().getAbfsClient() .listPath(directory, false, getListMaxResults(), null, - new TrackingContext(getFileSystem().getFileSystemID(), "PA")).getResult() + new TrackingContext("test-fileSystemID", "PA")).getResult() .getListResultSchema().paths(); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java index 0d94152985707..997a681f081ae 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java @@ -124,7 +124,7 @@ public void testBlobDataReader() throws Exception { Assume.assumeTrue("Reader client id not provided", clientId != null); String secret = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET); Assume.assumeTrue("Reader client secret not provided", secret != null); - TrackingContext trackingContext = new TrackingContext(getFileSystem().getFileSystemID(), "AU"); + TrackingContext trackingContext = new TrackingContext("test-fileSystemID", "AU"); prepareFiles(); final AzureBlobFileSystem fs = getBlobReader(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java index 5a31eef2ab43d..901bee4444f0c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java @@ -55,7 +55,7 @@ public void testWithWrongSharedKey() throws Exception { + "Authorization header is formed correctly including the " + "signature.\", 403", () -> { - abfsClient.getAclStatus("/", new TrackingContext(this.getFileSystem().getFileSystemID(), + abfsClient.getAclStatus("/", new TrackingContext("test-fileSystemID", "AC")); }); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index bab773c90a03a..f0875596c470a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -107,7 +107,7 @@ private void verifyReadCallCount(AbfsClient client, int count) throws verify(client, times(count)).read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - new TrackingContext("test-fileSystem-id", "RD")); + any(TrackingContext.class)); } private void checkEvictedStatus(AbfsInputStream inputStream, int position, boolean expectedToThrowException) @@ -163,7 +163,7 @@ public void testFailedReadAhead() throws Exception { .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - new TrackingContext(getFileSystem().getFileSystemID(), "RD")); + any(TrackingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testFailedReadAhead.txt"); @@ -211,7 +211,7 @@ public void testOlderReadAheadFailure() throws Exception { .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - new TrackingContext("test-fileSystemID", "RD")); + any(TrackingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testOlderReadAheadFailure.txt"); @@ -266,7 +266,7 @@ public void testSuccessfulReadAhead() throws Exception { .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - new TrackingContext(getFileSystem().getFileSystemID(), "RD")); + any(TrackingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testSuccessfulReadAhead.txt"); @@ -312,7 +312,7 @@ public void testReadAheadManagerForFailedReadAhead() throws Exception { .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - new TrackingContext("test-FileSystemID", "RD")); + any(TrackingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testReadAheadManagerForFailedReadAhead.txt"); @@ -366,7 +366,7 @@ public void testReadAheadManagerForOlderReadAheadFailure() throws Exception { .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - new TrackingContext("test-fileSystemID", "RD")); + any(TrackingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testReadAheadManagerForOlderReadAheadFailure.txt"); @@ -421,7 +421,7 @@ public void testReadAheadManagerForSuccessfulReadAhead() throws Exception { .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - new TrackingContext("test-FileSystemID", "RD")); + any(TrackingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testSuccessfulReadAhead.txt"); From c3ddf824e24998ef0a4d7efd15d032a1bc98ba74 Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 12 Oct 2020 20:53:11 +0530 Subject: [PATCH 12/77] readahead --- .../fs/azurebfs/services/AbfsInputStream.java | 14 ++++++++------ .../hadoop/fs/azurebfs/services/ReadBuffer.java | 10 ++++++++++ .../fs/azurebfs/services/ReadBufferManager.java | 6 +++++- .../fs/azurebfs/services/ReadBufferWorker.java | 4 ++-- .../fs/azurebfs/utils/TrackingContext.java | 17 +++++++++-------- .../hadoop/fs/azurebfs/ITestAbfsClient.java | 2 ++ .../azurebfs/services/TestAbfsInputStream.java | 6 +++--- 7 files changed, 39 insertions(+), 20 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index ca3cef3ddac26..952209302e037 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -27,6 +27,7 @@ import com.google.common.base.Preconditions; import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.codec.binary.StringUtils; import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -247,15 +248,16 @@ private int readInternal(final long position, final byte[] b, final int offset, long nextSize; long nextOffset = position; LOG.debug("read ahead enabled issuing readheads num = {}", numReadAheads); + trackingContext.setPrimaryRequestID(); while (numReadAheads > 0 && nextOffset < contentLength) { nextSize = Math.min((long) bufferSize, contentLength - nextOffset); LOG.debug("issuing read ahead requestedOffset = {} requested size {}", nextOffset, nextSize); - ReadBufferManager.getBufferManager().queueReadAhead(this, nextOffset, (int) nextSize); + ReadBufferManager.getBufferManager().queueReadAhead(this, nextOffset, (int) nextSize, + trackingContext); nextOffset = nextOffset + nextSize; numReadAheads--; System.out.println("in the readahead loop " + Integer.toString(numReadAheads)); - trackingContext.setPrimaryRequestID(); } // try reading from buffers first @@ -271,15 +273,15 @@ private int readInternal(final long position, final byte[] b, final int offset, } // got nothing from read-ahead, do our own read now - receivedBytes = readRemote(position, b, offset, length); + receivedBytes = readRemote(position, b, offset, length, new TrackingContext(trackingContext)); return receivedBytes; } else { LOG.debug("read ahead disabled, reading remote"); - return readRemote(position, b, offset, length); + return readRemote(position, b, offset, length, new TrackingContext(trackingContext)); } } - int readRemote(long position, byte[] b, int offset, int length) throws IOException { + int readRemote(long position, byte[] b, int offset, int length, TrackingContext trackingContext) throws IOException { if (position < 0) { throw new IllegalArgumentException("attempting to read from negative offset"); } @@ -303,7 +305,7 @@ int readRemote(long position, byte[] b, int offset, int length) throws IOExcepti try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "readRemote", "read")) { LOG.trace("Trigger client.read for path={} position={} offset={} length={}", path, position, offset, length); op = client.read(path, position, b, offset, length, tolerateOobAppends ? "*" : eTag, cachedSasToken.get(), - new TrackingContext("test-filesystem-id", inputStreamID, "RD")); + trackingContext); cachedSasToken.update(op.getSasToken()); if (streamStatistics != null) { streamStatistics.remoteReadOperation(); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java index 5d55726222de7..c79d4099612d6 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java @@ -22,6 +22,7 @@ import java.util.concurrent.CountDownLatch; import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import static org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus.READ_FAILED; @@ -36,6 +37,7 @@ class ReadBuffer { private ReadBufferStatus status; // status of the buffer private CountDownLatch latch = null; // signaled when the buffer is done reading, so any client // waiting on this buffer gets unblocked + private TrackingContext trackingContext; // fields to help with eviction logic private long timeStamp = 0; // tick at which buffer became available to read @@ -53,6 +55,14 @@ public void setStream(AbfsInputStream stream) { this.stream = stream; } + public void setTrackingContext(TrackingContext trackingContext) { + this.trackingContext = new TrackingContext(trackingContext); + } + + public TrackingContext getTrackingContext() { + return trackingContext; + } + public long getOffset() { return offset; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java index 73c23b0155133..467c7864c7eec 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java @@ -18,6 +18,8 @@ package org.apache.hadoop.fs.azurebfs.services; import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.graalvm.compiler.core.common.alloc.Trace; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,7 +97,8 @@ private ReadBufferManager() { * @param requestedOffset The offset in the file which shoukd be read * @param requestedLength The length to read */ - void queueReadAhead(final AbfsInputStream stream, final long requestedOffset, final int requestedLength) { + void queueReadAhead(final AbfsInputStream stream, final long requestedOffset, final int requestedLength, + TrackingContext trackingContext) { if (LOGGER.isTraceEnabled()) { LOGGER.trace("Start Queueing readAhead for {} offset {} length {}", stream.getPath(), requestedOffset, requestedLength); @@ -116,6 +119,7 @@ void queueReadAhead(final AbfsInputStream stream, final long requestedOffset, fi buffer.setRequestedLength(requestedLength); buffer.setStatus(ReadBufferStatus.NOT_AVAILABLE); buffer.setLatch(new CountDownLatch(1)); + buffer.setTrackingContext(trackingContext); Integer bufferIndex = freeList.pop(); // will return a value, since we have checked size > 0 already diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java index 61d32a3679a2b..b70adc3b6732b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java @@ -62,7 +62,6 @@ public void run() { if (buffer != null) { try { // do the actual read, from the file. - buffer.getStream().trackingContext.setPrimaryRequestID(); int bytesRead = buffer.getStream().readRemote( buffer.getOffset(), buffer.getBuffer(), @@ -70,7 +69,8 @@ public void run() { // If AbfsInputStream was created with bigger buffer size than // read-ahead buffer size, make sure a valid length is passed // for remote read - Math.min(buffer.getRequestedLength(), buffer.getBuffer().length)); + Math.min(buffer.getRequestedLength(), buffer.getBuffer().length), + buffer.getTrackingContext()); bufferManager.doneReading(buffer, ReadBufferStatus.AVAILABLE, bytesRead); // post result back to ReadBufferManager } catch (Exception ex) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java index f3f741ff97944..28b3c62b91961 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java @@ -34,7 +34,6 @@ public class TrackingContext { private String streamID = ""; private int retryCount; private String hadoopOpName = ""; - private int iteration; private static final Logger LOG = LoggerFactory.getLogger( org.apache.hadoop.fs.azurebfs.services.AbfsClient.class); @@ -46,7 +45,6 @@ public TrackingContext(String fileSystemID, String hadoopOpName) { this.hadoopOpName = hadoopOpName; streamID = EMPTY_STRING; retryCount = 0; - iteration = 0; primaryRequestID = ""; } @@ -59,8 +57,10 @@ public TrackingContext(TrackingContext originalTrackingContext) { this.fileSystemID = originalTrackingContext.fileSystemID; this.streamID = originalTrackingContext.streamID; this.clientCorrelationID = originalTrackingContext.clientCorrelationID; - this.primaryRequestID = originalTrackingContext.clientRequestID; + this.primaryRequestID = originalTrackingContext.primaryRequestID; this.clientRequestID = UUID.randomUUID().toString(); + this.hadoopOpName = originalTrackingContext.hadoopOpName; + this.retryCount = 0; } public void setClientCorrelationID(String clientCorrelationID) { @@ -97,18 +97,19 @@ public void setStreamID(String stream) { // public void setOperation(org.apache.hadoop.fs.azurebfs.services.AbfsRestOperationType op) { // opName = op.name();//convert // } - public void updateIteration() { - iteration++; +// public void updateIteration() { +// iteration++; // primaryRequestID = clientRequestID + ":"; - } +// } public void setPrimaryRequestID() { - this.primaryRequestID = clientRequestID; +// this.primaryRequestID = clientRequestID; + primaryRequestID = StringUtils.right(UUID.randomUUID().toString(), 12); } public String toString() { // if (iteration) - String operation = hadoopOpName + (hadoopOpName == "LS"? Integer.toString(iteration).toString() : ""); + String operation = hadoopOpName; // + (hadoopOpName == "LS"? Integer.toString(iteration).toString() : ""); return clientCorrelationID + clientRequestID + ":" + fileSystemID + ":" + primaryRequestID + ":" + streamID + ":" + operation + ":" + retryCount; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java index 8d69cd6f231cd..f600c7eeb3116 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java @@ -135,6 +135,8 @@ public void checkRequest(String clientCorrelationId, boolean includeInHeader) int responseCode = op.getResult().getStatusCode(); assertEquals("Status code", HTTP_CREATED, responseCode); + + op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); String requestHeader = op.getResult().getRequestHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); List clientRequestIds = java.util.Arrays.asList( requestHeader.replace("[","") diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index f0875596c470a..2095baa10dcb6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -92,11 +92,11 @@ private AbfsInputStream getAbfsInputStream(AbfsClient mockAbfsClient, String fil private void queueReadAheads(AbfsInputStream inputStream) { // Mimic AbfsInputStream readAhead queue requests ReadBufferManager.getBufferManager() - .queueReadAhead(inputStream, 0, ONE_KB); + .queueReadAhead(inputStream, 0, ONE_KB, inputStream.trackingContext); ReadBufferManager.getBufferManager() - .queueReadAhead(inputStream, ONE_KB, ONE_KB); + .queueReadAhead(inputStream, ONE_KB, ONE_KB, inputStream.trackingContext); ReadBufferManager.getBufferManager() - .queueReadAhead(inputStream, TWO_KB, TWO_KB); + .queueReadAhead(inputStream, TWO_KB, TWO_KB, inputStream.trackingContext); } private void verifyReadCallCount(AbfsClient client, int count) throws From 2e882d4cfaa531ebfc1922223d60cbde315dfcd9 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 13 Oct 2020 05:24:14 -0700 Subject: [PATCH 13/77] dependent & client req id (readaheads) --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 2 +- .../fs/azurebfs/services/AbfsClient.java | 5 ++- .../fs/azurebfs/services/AbfsInputStream.java | 10 ++++-- .../azurebfs/services/ReadBufferManager.java | 2 +- .../azurebfs/services/ReadBufferWorker.java | 1 + .../fs/azurebfs/utils/TrackingContext.java | 34 +++++++++++++++++-- .../azurebfs/ITestAbfsReadWriteAndSeek.java | 22 ++++++++++-- .../src/test/resources/log4j.properties | 1 + 8 files changed, 66 insertions(+), 11 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 2c5a08af203df..639febc2ceb8f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -894,7 +894,7 @@ public FileStatus[] listStatus(final Path path, final String startFrom, final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue = true; - trackingContext.updateIteration(); +// trackingContext.updateIteration(); LOG.debug("listStatus filesystem: {} path: {}, startFrom: {}", client.getFileSystem(), diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 3a319211148fa..9810553ef31d3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -555,6 +555,7 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset, final int bufferLength, final String eTag, String cachedSasToken, TrackingContext trackingContext) throws AzureBlobFileSystemException { + System.out.println("in read"); final List requestHeaders = createDefaultHeaders(); requestHeaders.add(new AbfsHttpHeader(RANGE, String.format("bytes=%d-%d", position, position + bufferLength - 1))); @@ -566,7 +567,7 @@ public AbfsRestOperation read(final String path, final long position, final byte abfsUriQueryBuilder, cachedSasToken); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - + System.out.println("in the read method of client"); final AbfsRestOperation op = new AbfsRestOperation( AbfsRestOperationType.ReadFile, this, @@ -576,7 +577,9 @@ public AbfsRestOperation read(final String path, final long position, final byte buffer, bufferOffset, bufferLength, sasTokenForReuse); + System.out.println("op for read"); op.execute(trackingContext); + System.out.println("executed read"); return op; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 952209302e037..30913ee71a6a3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -27,12 +27,12 @@ import com.google.common.base.Preconditions; import com.google.common.annotations.VisibleForTesting; -import org.apache.commons.codec.binary.StringUtils; +//import org.apache.commons.codec.binary.StringUtils; import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.commons.lang3.StringUtils; +//import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.CanUnbuffer; import org.apache.hadoop.fs.FSExceptionMessages; import org.apache.hadoop.fs.FSInputStream; @@ -111,7 +111,7 @@ public AbfsInputStream( this.cachedSasToken = new CachedSASToken( abfsInputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds()); this.streamStatistics = abfsInputStreamContext.getStreamStatistics(); - this.inputStreamID = StringUtils.right(UUID.randomUUID().toString(), 12); + this.inputStreamID = org.apache.commons.lang3.StringUtils.right(UUID.randomUUID().toString(), 12); trackingContext.setStreamID(inputStreamID); this.trackingContext = trackingContext; } @@ -249,10 +249,12 @@ private int readInternal(final long position, final byte[] b, final int offset, long nextOffset = position; LOG.debug("read ahead enabled issuing readheads num = {}", numReadAheads); trackingContext.setPrimaryRequestID(); + System.out.println("set preq id"); while (numReadAheads > 0 && nextOffset < contentLength) { nextSize = Math.min((long) bufferSize, contentLength - nextOffset); LOG.debug("issuing read ahead requestedOffset = {} requested size {}", nextOffset, nextSize); + trackingContext.firstRequest = numReadAheads == readAheadQueueDepth; ReadBufferManager.getBufferManager().queueReadAhead(this, nextOffset, (int) nextSize, trackingContext); nextOffset = nextOffset + nextSize; @@ -271,6 +273,7 @@ private int readInternal(final long position, final byte[] b, final int offset, } return receivedBytes; } + trackingContext.reset(); // got nothing from read-ahead, do our own read now receivedBytes = readRemote(position, b, offset, length, new TrackingContext(trackingContext)); @@ -304,6 +307,7 @@ int readRemote(long position, byte[] b, int offset, int length, TrackingContext AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "readRemote", "read")) { LOG.trace("Trigger client.read for path={} position={} offset={} length={}", path, position, offset, length); + System.out.println("going to client read"); op = client.read(path, position, b, offset, length, tolerateOobAppends ? "*" : eTag, cachedSasToken.get(), trackingContext); cachedSasToken.update(op.getSasToken()); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java index 467c7864c7eec..42307fbe4db0b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java @@ -19,7 +19,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; -import org.graalvm.compiler.core.common.alloc.Trace; +//import org.graalvm.compiler.core.common.alloc.Trace; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java index b70adc3b6732b..3bb5bb54005f2 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java @@ -62,6 +62,7 @@ public void run() { if (buffer != null) { try { // do the actual read, from the file. + System.out.println(buffer.getTrackingContext().toString()); int bytesRead = buffer.getStream().readRemote( buffer.getOffset(), buffer.getBuffer(), diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java index 28b3c62b91961..9ad397c7d46b6 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java @@ -29,11 +29,12 @@ public class TrackingContext { private String clientCorrelationID; private String fileSystemID; - private String clientRequestID; + private String clientRequestID = ""; //UUID.fromString("00000000-0000-0000-0000-000000000000"); private String primaryRequestID; private String streamID = ""; private int retryCount; private String hadoopOpName = ""; + public boolean firstRequest = false; private static final Logger LOG = LoggerFactory.getLogger( org.apache.hadoop.fs.azurebfs.services.AbfsClient.class); @@ -46,6 +47,7 @@ public TrackingContext(String fileSystemID, String hadoopOpName) { streamID = EMPTY_STRING; retryCount = 0; primaryRequestID = ""; + firstRequest = false; } public TrackingContext(String fileSystemID, String streamID, String hadoopOpName) { @@ -57,10 +59,18 @@ public TrackingContext(TrackingContext originalTrackingContext) { this.fileSystemID = originalTrackingContext.fileSystemID; this.streamID = originalTrackingContext.streamID; this.clientCorrelationID = originalTrackingContext.clientCorrelationID; - this.primaryRequestID = originalTrackingContext.primaryRequestID; - this.clientRequestID = UUID.randomUUID().toString(); this.hadoopOpName = originalTrackingContext.hadoopOpName; this.retryCount = 0; + this.firstRequest = originalTrackingContext.firstRequest; + + if (originalTrackingContext.firstRequest) { + this.clientRequestID = originalTrackingContext.primaryRequestID; + this.primaryRequestID = EMPTY_STRING; + } + else { + this.primaryRequestID = originalTrackingContext.primaryRequestID; + this.clientRequestID = UUID.randomUUID().toString(); + } } public void setClientCorrelationID(String clientCorrelationID) { @@ -86,10 +96,28 @@ public void updateRetryCount() { retryCount++; } +// public void setClientRequestID(boolean forceUpdate) { +// if(forceUpdate || clientRequestID.isEmpty()) { +// clientRequestID = UUID.randomUUID().toString(); +// } +// } + public void setClientRequestID() { + if(firstRequest) { + return; + } clientRequestID = UUID.randomUUID().toString(); } + public String getPrimaryRequestID() { + return primaryRequestID; + } + + public void reset() { + primaryRequestID = EMPTY_STRING; + retryCount = 0; + } + public void setStreamID(String stream) { streamID = stream; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index 52abb097ef311..2e26b7680f88a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -62,14 +62,32 @@ public void testReadAndWriteWithDifferentBufferSizesAndSeek() throws Exception { testReadWriteAndSeek(size); } - private void testReadWriteAndSeek(int bufferSize) throws Exception { + @Test + public void testReadAheadRequestID() throws java.io.IOException { final AzureBlobFileSystem fs = getFileSystem(); final AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration(); + int bufferSize = 32; + abfsConfiguration.setReadBufferSize(bufferSize); + final byte[] b = new byte[bufferSize * 10]; + new Random().nextBytes(b); + try (FSDataOutputStream stream = fs.create(TEST_PATH)) { + stream.write(b); + } + + final byte[] readBuffer = new byte[4 * bufferSize]; + int result; + try (FSDataInputStream inputStream = fs.open(TEST_PATH)) { + result = inputStream.read(readBuffer, 0, bufferSize*4); + } + } + + private void testReadWriteAndSeek(int bufferSize) throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration(); abfsConfiguration.setWriteBufferSize(bufferSize); abfsConfiguration.setReadBufferSize(bufferSize); - final byte[] b = new byte[2 * bufferSize]; new Random().nextBytes(b); diff --git a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties index 9f72d03653306..efef7d8f36f81 100644 --- a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties @@ -26,6 +26,7 @@ log4j.logger.org.apache.hadoop.fs.azure.AzureFileSystemThreadPoolExecutor=DEBUG log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG log4j.logger.org.apache.hadoop.fs.azurebfs.contracts.services.TracingService=TRACE log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsClient=DEBUG +#log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsInputStream=DEBUG; # after here: turn off log messages from other parts of the system # which only clutter test reports. From eeae13ee9a93dab4ea6713333842b543065529b3 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 13 Oct 2020 06:13:43 -0700 Subject: [PATCH 14/77] liststatus ok --- .../hadoop/fs/azurebfs/AzureBlobFileSystemStore.java | 11 ++++++++--- .../hadoop/fs/azurebfs/utils/TrackingContext.java | 1 + 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 639febc2ceb8f..900028bea20fe 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -873,7 +873,6 @@ public FileStatus getFileStatus(final Path path, TrackingContext trackingContext * */ public FileStatus[] listStatus(final Path path, TrackingContext trackingContext) throws IOException { trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); - trackingContext.setPrimaryRequestID(); return listStatus(path, null, trackingContext); } @@ -890,7 +889,7 @@ public FileStatus[] listStatus(final Path path, TrackingContext trackingContext) * */ @InterfaceStability.Unstable public FileStatus[] listStatus(final Path path, final String startFrom, - TrackingContext trackingContext) throws IOException { + TrackingContext originalTrackingContext) throws IOException { final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue = true; @@ -904,6 +903,8 @@ public FileStatus[] listStatus(final Path path, final String startFrom, final String relativePath = getRelativePath(path); String continuation = null; + TrackingContext trackingContext = new TrackingContext(originalTrackingContext); + // generate continuation token if a valid startFrom is provided. if (startFrom != null && !startFrom.isEmpty()) { continuation = getIsNamespaceEnabled(trackingContext) @@ -911,11 +912,14 @@ public FileStatus[] listStatus(final Path path, final String startFrom, : generateContinuationTokenForNonXns(relativePath, startFrom); } + trackingContext.setPrimaryRequestID(); + trackingContext.firstRequest = true; + ArrayList fileStatuses = new ArrayList<>(); do { try (AbfsPerfInfo perfInfo = startTracking("listStatus", "listPath")) { AbfsRestOperation op = client.listPath(relativePath, false, - abfsConfiguration.getListMaxResults(), continuation, trackingContext); + abfsConfiguration.getListMaxResults(), continuation, new TrackingContext(trackingContext)); perfInfo.registerResult(op.getResult()); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); ListResultSchema retrievedSchema = op.getResult().getListResultSchema(); @@ -971,6 +975,7 @@ public FileStatus[] listStatus(final Path path, final String startFrom, perfInfo.registerAggregates(startAggregate, countAggregate); } } + trackingContext.firstRequest = false; } while (shouldContinue); return fileStatuses.toArray(new FileStatus[fileStatuses.size()]); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java index 9ad397c7d46b6..93976a13685a3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java @@ -116,6 +116,7 @@ public String getPrimaryRequestID() { public void reset() { primaryRequestID = EMPTY_STRING; retryCount = 0; + firstRequest = false; } public void setStreamID(String stream) { From d4343fae35638e25952a3ce1cc8b472fb726a83b Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 13 Oct 2020 06:54:26 -0700 Subject: [PATCH 15/77] create overwrite case ok --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 900028bea20fe..77bc517828e03 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -543,18 +543,24 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa final boolean isAppendBlob, TrackingContext trackingContext) throws AzureBlobFileSystemException { AbfsRestOperation op; + trackingContext.setPrimaryRequestID(); + trackingContext.firstRequest = true; + try { // Trigger a create with overwrite=false first so that eTag fetch can be // avoided for cases when no pre-existing file is present (major portion // of create file traffic falls into the case of no pre-existing file). + + op = client.createPath(relativePath, true, - false, permission, umask, isAppendBlob, null, trackingContext); + false, permission, umask, isAppendBlob, null, new TrackingContext(trackingContext)); } catch (AbfsRestOperationException e) { + trackingContext.firstRequest = false; if (e.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { // File pre-exists, fetch eTag try { - op = client.getPathStatus(relativePath, false, trackingContext); - trackingContext.setPrimaryRequestID(); + op = client.getPathStatus(relativePath, false, new TrackingContext(trackingContext)); +// trackingContext.setPrimaryRequestID(); } catch (AbfsRestOperationException ex) { if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { // Is a parallel access case, as file which was found to be @@ -573,7 +579,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa try { // overwrite only if eTag matches with the file properties fetched befpre op = client.createPath(relativePath, true, - true, permission, umask, isAppendBlob, eTag, trackingContext); + true, permission, umask, isAppendBlob, eTag, new TrackingContext(trackingContext)); } catch (AbfsRestOperationException ex) { if (ex.getStatusCode() == HttpURLConnection.HTTP_PRECON_FAILED) { // Is a parallel access case, as file with eTag was just queried From 0ef9a27aa46e442f52bd57e4a336c94b5ea2e54d Mon Sep 17 00:00:00 2001 From: Sumangala Date: Wed, 14 Oct 2020 20:09:17 -0700 Subject: [PATCH 16/77] fixed some errors --- hadoop-tools/hadoop-azure/pom.xml | 17 +++ .../fs/azurebfs/AzureBlobFileSystemStore.java | 5 +- .../fs/azurebfs/services/AbfsInputStream.java | 7 +- .../azurebfs/services/AbfsOutputStream.java | 11 +- .../azurebfs/services/AbfsRestOperation.java | 4 +- .../fs/azurebfs/utils/TrackingContext.java | 8 +- .../hadoop/fs/azurebfs/ITestAbfsClient.java | 2 +- .../ITestAzureBlobFileSystemCreate.java | 6 +- .../ITestClientCorrelationHeader.java | 107 ++++++++++++++++++ pom.xml | 19 ++++ 10 files changed, 167 insertions(+), 19 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml index 6f23986c37ca0..60c3fa1eec458 100644 --- a/hadoop-tools/hadoop-azure/pom.xml +++ b/hadoop-tools/hadoop-azure/pom.xml @@ -45,7 +45,24 @@ 7200 10 1000 + + 2.0.2 + + + + org.powermock + powermock-module-junit4 + ${powermock.version} + test + + + org.powermock + powermock-api-mockito + ${powermock.version} + test + + diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 77bc517828e03..45636e27a67a0 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -751,11 +751,13 @@ public void rename(final Path source, final Path destination, TrackingContext tr String sourceRelativePath = getRelativePath(source); String destinationRelativePath = getRelativePath(destination); + trackingContext.setPrimaryRequestID(); + trackingContext.firstRequest = true; do { try (AbfsPerfInfo perfInfo = startTracking("rename", "renamePath")) { AbfsRestOperation op = client.renamePath(sourceRelativePath, - destinationRelativePath, continuation, trackingContext); + destinationRelativePath, continuation, new TrackingContext(trackingContext)); perfInfo.registerResult(op.getResult()); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); perfInfo.registerSuccess(true); @@ -766,6 +768,7 @@ public void rename(final Path source, final Path destination, TrackingContext tr perfInfo.registerAggregates(startAggregate, countAggregate); } } + trackingContext.firstRequest = false; } while (shouldContinue); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 30913ee71a6a3..7bc1b84af98c7 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -112,10 +112,9 @@ public AbfsInputStream( abfsInputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds()); this.streamStatistics = abfsInputStreamContext.getStreamStatistics(); this.inputStreamID = org.apache.commons.lang3.StringUtils.right(UUID.randomUUID().toString(), 12); - trackingContext.setStreamID(inputStreamID); - this.trackingContext = trackingContext; + this.trackingContext = new TrackingContext(trackingContext); + this.trackingContext.setStreamID(inputStreamID); } - public String getPath() { return path; } @@ -273,7 +272,7 @@ private int readInternal(final long position, final byte[] b, final int offset, } return receivedBytes; } - trackingContext.reset(); + trackingContext.reset(); //remove to enable remote read to have same preq id // got nothing from read-ahead, do our own read now receivedBytes = readRemote(position, b, offset, length, new TrackingContext(trackingContext)); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 405df0838faae..d2cfb291c70f4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -82,6 +82,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable, StreamCa // SAS tokens can be re-used until they expire private CachedSASToken cachedSasToken; private String outputStreamID; + private TrackingContext trackingContext; /** * Queue storing buffers with the size of the Azure block ready for @@ -149,7 +150,8 @@ public AbfsOutputStream( this.cachedSasToken = new CachedSASToken( abfsOutputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds()); this.outputStreamID = StringUtils.right(UUID.randomUUID().toString(), 12); - trackingContext.setStreamID(outputStreamID); + this.trackingContext = new TrackingContext(trackingContext); + this.trackingContext.setStreamID(outputStreamID); } /** @@ -352,7 +354,7 @@ private void writeAppendBlobCurrentBufferToService() throws IOException { "writeCurrentBufferToService", "append")) { AbfsRestOperation op = client.append(path, offset, bytes, 0, bytesLength, cachedSasToken.get(), this.isAppendBlob, - new TrackingContext("test-filesystem-id", outputStreamID, "AP")); + new TrackingContext(trackingContext)); cachedSasToken.update(op.getSasToken()); outputStreamStatistics.uploadSuccessful(bytesLength); perfInfo.registerResult(op.getResult()); @@ -405,7 +407,8 @@ public Void call() throws Exception { try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "writeCurrentBufferToService", "append")) { AbfsRestOperation op = client.append(path, offset, bytes, 0, - bytesLength, cachedSasToken.get(), false); + bytesLength, cachedSasToken.get(), false, + new TrackingContext(trackingContext)); cachedSasToken.update(op.getSasToken()); perfInfo.registerResult(op.getResult()); byteBufferPool.putBuffer(ByteBuffer.wrap(bytes)); @@ -467,7 +470,7 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset, try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "flushWrittenBytesToServiceInternal", "flush")) { AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, isClose, cachedSasToken.get(), - new TrackingContext("test-filesystem-id", outputStreamID, "OP")); + new TrackingContext(trackingContext)); cachedSasToken.update(op.getSasToken()); perfInfo.registerResult(op.getResult()).registerSuccess(true); } catch (AzureBlobFileSystemException ex) { 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 a28d20c6fb509..29f8524cba217 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 @@ -104,7 +104,7 @@ String getSasToken() { * @param url The full URL including query string parameters. * @param requestHeaders The HTTP request headers. */ - AbfsRestOperation(final AbfsRestOperationType operationType, + public AbfsRestOperation(final AbfsRestOperationType operationType, final AbfsClient client, final String method, final URL url, @@ -209,7 +209,7 @@ public void execute(TrackingContext trackingContext) throws AzureBlobFileSystemE * fails, there may be a retry. The retryCount is incremented with each * attempt. */ - private boolean executeHttpOperation(final int retryCount, TrackingContext trackingContext) + public boolean executeHttpOperation(final int retryCount, TrackingContext trackingContext) throws AzureBlobFileSystemException { AbfsHttpOperation httpOperation = null; try { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java index 93976a13685a3..5d8c0d894963a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java @@ -94,6 +94,7 @@ public void setFileSystemID(String fileSystemID) { public void updateRetryCount() { // retryCount = count; retryCount++; + firstRequest = false; //this is necessary for retry in case of first request of continuation ops } // public void setClientRequestID(boolean forceUpdate) { @@ -103,10 +104,9 @@ public void updateRetryCount() { // } public void setClientRequestID() { - if(firstRequest) { - return; + if (clientRequestID.isEmpty() || !firstRequest) { + clientRequestID = UUID.randomUUID().toString(); } - clientRequestID = UUID.randomUUID().toString(); } public String getPrimaryRequestID() { @@ -133,7 +133,7 @@ public void setStreamID(String stream) { public void setPrimaryRequestID() { // this.primaryRequestID = clientRequestID; - primaryRequestID = StringUtils.right(UUID.randomUUID().toString(), 12); + primaryRequestID = UUID.randomUUID().toString(); } public String toString() { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java index f600c7eeb3116..51229d69e3a37 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java @@ -204,7 +204,7 @@ private List listPath(String directory) throws IOException { return getFileSystem().getAbfsClient() .listPath(directory, false, getListMaxResults(), null, - new TrackingContext("test-fileSystemID", "PA")).getResult() + new TrackingContext(getFileSystem().getFileSystemID(), "PA")).getResult() .getListResultSchema().paths(); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 191f181cbaeae..8ad2ce2316f95 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -379,14 +379,14 @@ public void testNegativeScenariosForCreateOverwriteDisabled() .createPath(any(String.class), eq(true), eq(false), isNamespaceEnabled ? any(String.class) : eq(null), isNamespaceEnabled ? any(String.class) : eq(null), - any(boolean.class), eq(null), trackingContext); + any(boolean.class), eq(null), any(TrackingContext.class)); doThrow(fileNotFoundResponseEx) // Scn1: GFS fails with Http404 .doThrow(serverErrorResponseEx) // Scn2: GFS fails with Http500 .doReturn(successOp) // Scn3: create overwrite=true fails with Http412 .doReturn(successOp) // Scn4: create overwrite=true fails with Http500 .when(mockClient) - .getPathStatus(any(String.class), eq(false), trackingContext); + .getPathStatus(any(String.class), eq(false), any(TrackingContext.class)); // mock for overwrite=true doThrow( @@ -397,7 +397,7 @@ public void testNegativeScenariosForCreateOverwriteDisabled() .createPath(any(String.class), eq(true), eq(true), isNamespaceEnabled ? any(String.class) : eq(null), isNamespaceEnabled ? any(String.class) : eq(null), - any(boolean.class), eq(null), trackingContext); + any(boolean.class), eq(null), any(TrackingContext.class)); // Scn1: GFS fails with Http404 // Sequence of events expected: diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java new file mode 100644 index 0000000000000..d9e17ddb8100d --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java @@ -0,0 +1,107 @@ +package org.apache.hadoop.fs.azurebfs; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.http.HttpException; + +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; +import org.junit.Test; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +//import static org.powermock.api.easymock.PowerMock.*; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +public final class ITestClientCorrelationHeader extends AbstractAbfsIntegrationTest { + AzureBlobFileSystem fs = getFileSystem(); + + public ITestClientCorrelationHeader() throws Exception { + super(); + } + + public void testPrimaryRequestID() { + // readahead + // liststatus + // createoverwrite + // rename + + // check if placed in the correct slot and matches client req id + // how to verify for readahead - concurrent requests? + + //check client correlation id - response vs request header? + } + + public void testStreamID() { + // create one input and one output stream. All req on a given stream should have same stream id throughout + // placed correctly? should be in slot 4 while parsing ( + } + + public void testFilesystemID() { + // all requests using a given filesystem instance should have same fs id + // in slot 0 + } + + @Test +// @PrepareForTest(AbfsRestOperation.class) + public void testRetryCount() throws IOException { +// AbfsRestOperation op = PowerMock.spy(AbfsRestOperation.class, "executeHttpOperation"); +// PowerMockito.doThrow(new AbfsRestOperationException(new HttpException())) +// .when(op, "executeHttpOperation", any(int), +// any(TrackingContext.class)); + + AbfsClient client = mock(AbfsClient.class); + + AbfsRestOperation restop = new AbfsRestOperation( + org.apache.hadoop.fs.azurebfs.services.AbfsRestOperationType.CreatePath, + client, + HTTP_METHOD_PUT, + new java.net.URL("url"), + new java.util.ArrayList()); + AbfsRestOperation op = spy(restop); + + +// AbfsRestOperation op = spy(restop); + when(op.executeHttpOperation(any(int.class), any(TrackingContext.class))) + .thenReturn(false); +// .thenThrow(AzureBlobFileSystemException.class); + + op.execute(new org.apache.hadoop.fs.azurebfs.utils.TrackingContext("fsid","op")); + + String path = getRelativePath(new Path("/testDir")); + boolean isNamespaceEnabled = true;//fs.getIsNamespaceEnabled(); + String permission = null;// isNamespaceEnabled ? getOctalNotation(FsPermission.getDirDefault()) : null; + String umask = null; //isNamespaceEnabled ? getOctalNotation(FsPermission.getUMask(fs.getConf())) : null; + +// AbfsClient client = mock(AbfsClient.class); + when(client.createPath(any(String.class), eq(true), eq(false), + eq(null), eq(null), + any(boolean.class), eq(null), any(TrackingContext.class))).thenReturn(op); + AbfsRestOperation op1 = client.createPath(path, false, true, + permission, umask, false, null, + new TrackingContext("fsid", "CR")); + + } + + private String getOctalNotation(FsPermission fsPermission) { + Preconditions.checkNotNull(fsPermission, "fsPermission"); + return String.format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal()); + } + + private String getRelativePath(final Path path) { + Preconditions.checkNotNull(path, "path"); + return path.toUri().getPath(); + } + +} \ No newline at end of file diff --git a/pom.xml b/pom.xml index 7e94cfb8ad0b0..5565562425e32 100644 --- a/pom.xml +++ b/pom.xml @@ -77,10 +77,27 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x Apache Software Foundation https://www.apache.org + + + + + org.powermock + powermock-module-junit4 + ${powermock.version} + test + + + org.powermock + powermock-api-mockito + ${powermock.version} + test + + 3.4.0-SNAPSHOT + 2.0.2 apache.snapshots.https Apache Development Snapshot Repository @@ -118,6 +135,8 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x org.fusesource.leveldbjni + + hadoop-project From 107df060d5bca839cb0e266d6631d862e2fcfe0a Mon Sep 17 00:00:00 2001 From: sumangala Date: Fri, 16 Oct 2020 00:44:22 +0530 Subject: [PATCH 17/77] tc changes --- hadoop-tools/hadoop-azure/pom.xml | 16 -- .../fs/azurebfs/AzureBlobFileSystem.java | 135 ++++++----- .../fs/azurebfs/AzureBlobFileSystemStore.java | 213 +++++++----------- .../fs/azurebfs/services/AbfsClient.java | 100 ++++---- .../azurebfs/services/AbfsHttpOperation.java | 8 +- .../fs/azurebfs/services/AbfsInputStream.java | 45 ++-- .../azurebfs/services/AbfsOutputStream.java | 25 +- .../azurebfs/services/AbfsRestOperation.java | 16 +- .../fs/azurebfs/services/ReadBuffer.java | 12 +- .../azurebfs/services/ReadBufferManager.java | 6 +- .../azurebfs/services/ReadBufferWorker.java | 3 +- .../fs/azurebfs/utils/TracingContext.java | 107 +++++++++ .../fs/azurebfs/utils/TrackingContext.java | 146 ------------ .../azurebfs/AbstractAbfsIntegrationTest.java | 16 +- .../hadoop/fs/azurebfs/ITestAbfsClient.java | 39 ++-- .../ITestAbfsInputStreamStatistics.java | 18 +- .../ITestAzureBlobFileSystemCheckAccess.java | 1 - .../ITestAzureBlobFileSystemCreate.java | 14 +- .../ITestAzureBlobFileSystemDelete.java | 10 +- .../ITestAzureBlobFileSystemOauth.java | 12 +- .../ITestAzureBlobFileSystemRename.java | 14 +- ...lobFileSystemStoreListStatusWithRange.java | 4 +- .../ITestClientCorrelationHeader.java | 14 +- .../azurebfs/ITestFileSystemProperties.java | 35 +-- .../fs/azurebfs/ITestGetNameSpaceEnabled.java | 16 +- .../fs/azurebfs/ITestSharedKeyAuth.java | 6 +- .../MockDelegationSASTokenProvider.java | 2 +- .../fs/azurebfs/services/TestAbfsClient.java | 10 +- .../services/TestAbfsInputStream.java | 28 +-- .../services/TestAbfsOutputStream.java | 62 +++-- pom.xml | 17 -- 31 files changed, 537 insertions(+), 613 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml index 60c3fa1eec458..450feb5afba1e 100644 --- a/hadoop-tools/hadoop-azure/pom.xml +++ b/hadoop-tools/hadoop-azure/pom.xml @@ -46,23 +46,7 @@ 10 1000 - 2.0.2 - - - - org.powermock - powermock-module-junit4 - ${powermock.version} - test - - - org.powermock - powermock-api-mockito - ${powermock.version} - test - - diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 71cdbbed2f268..cdbebd8e10a10 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -71,7 +71,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager; import org.apache.hadoop.fs.azurebfs.services.AbfsCounters; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.FsAction; @@ -101,6 +101,7 @@ public class AzureBlobFileSystem extends FileSystem { private boolean delegationTokenEnabled = false; private AbfsDelegationTokenManager delegationTokenManager; private AbfsCounters abfsCounters; + private String clientCorrelationID; @Override public void initialize(URI uri, Configuration configuration) @@ -109,7 +110,7 @@ public void initialize(URI uri, Configuration configuration) super.initialize(uri, configuration); setConf(configuration); fileSystemID = UUID.randomUUID().toString(); - System.out.println("init fs id" + fileSystemID); + clientCorrelationID = abfsStore.getAbfsConfiguration().getClientCorrelationID(); LOG.debug("Initializing AzureBlobFileSystem for {}", uri); @@ -182,11 +183,11 @@ public String getFileSystemID() { public FSDataInputStream open(final Path path, final int bufferSize) throws IOException { LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize); statIncrement(CALL_OPEN); - TrackingContext trackingContext = new TrackingContext(fileSystemID, "FS"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "FS"); Path qualifiedPath = makeQualified(path); try { - InputStream inputStream = abfsStore.openFileForRead(qualifiedPath, statistics, trackingContext); + InputStream inputStream = abfsStore.openFileForRead(qualifiedPath, statistics, tracingContext); return new FSDataInputStream(inputStream); } catch(AzureBlobFileSystemException ex) { checkException(path, ex); @@ -202,7 +203,7 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi permission, overwrite, blockSize); - TrackingContext trackingContext = new TrackingContext(fileSystemID, "LS"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "LS"); statIncrement(CALL_CREATE); trailingPeriodCheck(f); @@ -212,7 +213,7 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi try { OutputStream outputStream = abfsStore.createFile(qualifiedPath, statistics, overwrite, permission == null ? FsPermission.getFileDefault() : permission, FsPermission.getUMask(getConf()), - trackingContext); + tracingContext); statIncrement(FILES_CREATED); return new FSDataOutputStream(outputStream, statistics); } catch(AzureBlobFileSystemException ex) { @@ -272,12 +273,12 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr f.toString(), bufferSize); statIncrement(CALL_APPEND); - TrackingContext trackingContext = new TrackingContext(fileSystemID, "AP"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AP"); Path qualifiedPath = makeQualified(f); try { OutputStream outputStream = abfsStore.openFileForWrite(qualifiedPath, statistics, false, - trackingContext); + tracingContext); return new FSDataOutputStream(outputStream, statistics); } catch(AzureBlobFileSystemException ex) { checkException(f, ex); @@ -288,7 +289,8 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr public boolean rename(final Path src, final Path dst) throws IOException { LOG.debug("AzureBlobFileSystem.rename src: {} dst: {}", src, dst); statIncrement(CALL_RENAME); - TrackingContext trackingContext = new TrackingContext(fileSystemID, "RN"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "RN"); + tracingContext.setPrimaryRequestID(); trailingPeriodCheck(dst); @@ -318,7 +320,7 @@ public boolean rename(final Path src, final Path dst) throws IOException { } // Non-HNS account need to check dst status on driver side. - if (!abfsStore.getIsNamespaceEnabled(trackingContext) && dstFileStatus == null) { + if (!abfsStore.getIsNamespaceEnabled(tracingContext) && dstFileStatus == null) { dstFileStatus = tryGetFileStatus(qualifiedDstPath); } @@ -335,7 +337,7 @@ public boolean rename(final Path src, final Path dst) throws IOException { qualifiedDstPath = makeQualified(adjustedDst); - abfsStore.rename(qualifiedSrcPath, qualifiedDstPath, trackingContext); + abfsStore.rename(qualifiedSrcPath, qualifiedDstPath, tracingContext); return true; } catch(AzureBlobFileSystemException ex) { LOG.debug("Rename operation failed. ", ex); @@ -359,7 +361,7 @@ public boolean delete(final Path f, final boolean recursive) throws IOException "AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), recursive); statIncrement(CALL_DELETE); Path qualifiedPath = makeQualified(f); - TrackingContext trackingContext = new TrackingContext(fileSystemID, "DL"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "DL"); if (f.isRoot()) { if (!recursive) { @@ -370,7 +372,7 @@ public boolean delete(final Path f, final boolean recursive) throws IOException } try { - abfsStore.delete(qualifiedPath, recursive, trackingContext); + abfsStore.delete(qualifiedPath, recursive, tracingContext); return true; } catch (AzureBlobFileSystemException ex) { checkException(f, ex, AzureServiceErrorCode.PATH_NOT_FOUND); @@ -385,10 +387,11 @@ public FileStatus[] listStatus(final Path f) throws IOException { "AzureBlobFileSystem.listStatus path: {}", f.toString()); statIncrement(CALL_LIST_STATUS); Path qualifiedPath = makeQualified(f); - TrackingContext trackingContext = new TrackingContext(fileSystemID, "LS"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "LS"); + tracingContext.setPrimaryRequestID(); try { - FileStatus[] result = abfsStore.listStatus(qualifiedPath, trackingContext); + FileStatus[] result = abfsStore.listStatus(qualifiedPath, tracingContext); return result; } catch (AzureBlobFileSystemException ex) { checkException(f, ex); @@ -445,7 +448,8 @@ public boolean mkdirs(final Path f, final FsPermission permission) throws IOExce "AzureBlobFileSystem.mkdirs path: {} permissions: {}", f, permission); statIncrement(CALL_MKDIRS); trailingPeriodCheck(f); - TrackingContext trackingContext = new TrackingContext(fileSystemID, "MK"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "MK"); + tracingContext.setPrimaryRequestID(); final Path parentFolder = f.getParent(); if (parentFolder == null) { @@ -457,7 +461,7 @@ public boolean mkdirs(final Path f, final FsPermission permission) throws IOExce try { abfsStore.createDirectory(qualifiedPath, permission == null ? FsPermission.getDirDefault() : permission, - FsPermission.getUMask(getConf()), trackingContext); + FsPermission.getUMask(getConf()), tracingContext); statIncrement(DIRECTORIES_CREATED); return true; } catch (AzureBlobFileSystemException ex) { @@ -484,10 +488,10 @@ public FileStatus getFileStatus(final Path f) throws IOException { LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f); statIncrement(CALL_GET_FILE_STATUS); Path qualifiedPath = makeQualified(f); - TrackingContext trackingContext = new TrackingContext(fileSystemID, "FS"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "FS"); try { - return abfsStore.getFileStatus(qualifiedPath, trackingContext); + return abfsStore.getFileStatus(qualifiedPath, tracingContext); } catch(AzureBlobFileSystemException ex) { checkException(f, ex); return null; @@ -661,11 +665,13 @@ public void setOwner(final Path path, final String owner, final String group) throws IOException { LOG.debug( "AzureBlobFileSystem.setOwner path: {}", path); - if (!getIsNamespaceEnabled()) { + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "PA"); + tracingContext.setPrimaryRequestID(); + + if (!getIsNamespaceEnabled(tracingContext)) { super.setOwner(path, owner, group); return; } - TrackingContext trackingContext = new TrackingContext(fileSystemID, "PO"); if ((owner == null || owner.isEmpty()) && (group == null || group.isEmpty())) { throw new IllegalArgumentException("A valid owner or group must be specified."); @@ -677,7 +683,7 @@ public void setOwner(final Path path, final String owner, final String group) abfsStore.setOwner(qualifiedPath, owner, group, - trackingContext); + tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -697,7 +703,8 @@ public void setOwner(final Path path, final String owner, final String group) public void setXAttr(final Path path, final String name, final byte[] value, final EnumSet flag) throws IOException { LOG.debug("AzureBlobFileSystem.setXAttr path: {}", path); - TrackingContext trackingContext = new TrackingContext(fileSystemID, "AT"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AT"); + tracingContext.setPrimaryRequestID(); if (name == null || name.isEmpty() || value == null) { throw new IllegalArgumentException("A valid name and value must be specified."); @@ -706,14 +713,14 @@ public void setXAttr(final Path path, final String name, final byte[] value, fin Path qualifiedPath = makeQualified(path); try { - Hashtable properties = abfsStore.getPathStatus(qualifiedPath, trackingContext); + Hashtable properties = abfsStore.getPathStatus(qualifiedPath, tracingContext); String xAttrName = ensureValidAttributeName(name); boolean xAttrExists = properties.containsKey(xAttrName); XAttrSetFlag.validate(name, xAttrExists, flag); String xAttrValue = abfsStore.decodeAttribute(value); properties.put(xAttrName, xAttrValue); - abfsStore.setPathProperties(qualifiedPath, properties, trackingContext); + abfsStore.setPathProperties(qualifiedPath, properties, tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -737,13 +744,14 @@ public byte[] getXAttr(final Path path, final String name) if (name == null || name.isEmpty()) { throw new IllegalArgumentException("A valid name must be specified."); } - TrackingContext trackingContext = new TrackingContext(fileSystemID, "AT"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AT"); + tracingContext.setPrimaryRequestID(); Path qualifiedPath = makeQualified(path); byte[] value = null; try { - Hashtable properties = abfsStore.getPathStatus(qualifiedPath, trackingContext); + Hashtable properties = abfsStore.getPathStatus(qualifiedPath, tracingContext); String xAttrName = ensureValidAttributeName(name); if (properties.containsKey(xAttrName)) { String xAttrValue = properties.get(xAttrName); @@ -770,11 +778,13 @@ private static String ensureValidAttributeName(String attribute) { public void setPermission(final Path path, final FsPermission permission) throws IOException { LOG.debug("AzureBlobFileSystem.setPermission path: {}", path); - if (!getIsNamespaceEnabled()) { + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "PR"); + tracingContext.setPrimaryRequestID(); + + if (!getIsNamespaceEnabled(tracingContext)) { super.setPermission(path, permission); return; } - TrackingContext trackingContext = new TrackingContext(fileSystemID, "PR"); if (permission == null) { throw new IllegalArgumentException("The permission can't be null"); @@ -784,7 +794,7 @@ public void setPermission(final Path path, final FsPermission permission) try { abfsStore.setPermission(qualifiedPath, - permission, trackingContext); + permission, tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -804,9 +814,10 @@ public void setPermission(final Path path, final FsPermission permission) public void modifyAclEntries(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.modifyAclEntries path: {}", path); - TrackingContext trackingContext = new TrackingContext(fileSystemID, "AC"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AC"); + tracingContext.setPrimaryRequestID(); - if (!getIsNamespaceEnabled()) { + if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "modifyAclEntries is only supported by storage accounts with the " + "hierarchical namespace enabled."); @@ -820,7 +831,7 @@ public void modifyAclEntries(final Path path, final List aclSpec) try { abfsStore.modifyAclEntries(qualifiedPath, - aclSpec, trackingContext); + aclSpec, tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -838,9 +849,10 @@ public void modifyAclEntries(final Path path, final List aclSpec) public void removeAclEntries(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.removeAclEntries path: {}", path); - TrackingContext trackingContext = new TrackingContext(fileSystemID, "AC"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AC"); + tracingContext.setPrimaryRequestID(); - if (!getIsNamespaceEnabled()) { + if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "removeAclEntries is only supported by storage accounts with the " + "hierarchical namespace enabled."); @@ -853,7 +865,7 @@ public void removeAclEntries(final Path path, final List aclSpec) Path qualifiedPath = makeQualified(path); try { - abfsStore.removeAclEntries(qualifiedPath, aclSpec, trackingContext); + abfsStore.removeAclEntries(qualifiedPath, aclSpec, tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -868,18 +880,18 @@ public void removeAclEntries(final Path path, final List aclSpec) @Override public void removeDefaultAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeDefaultAcl path: {}", path); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AC"); - if (!getIsNamespaceEnabled()) { + if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "removeDefaultAcl is only supported by storage accounts with the " + "hierarchical namespace enabled."); } - TrackingContext trackingContext = new TrackingContext(fileSystemID, "AC"); Path qualifiedPath = makeQualified(path); try { - abfsStore.removeDefaultAcl(qualifiedPath, trackingContext); + abfsStore.removeDefaultAcl(qualifiedPath, tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -896,9 +908,10 @@ public void removeDefaultAcl(final Path path) throws IOException { @Override public void removeAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeAcl path: {}", path); - TrackingContext trackingContext = new TrackingContext(fileSystemID, "AC"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AC"); + tracingContext.setPrimaryRequestID(); - if (!getIsNamespaceEnabled()) { + if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "removeAcl is only supported by storage accounts with the " + "hierarchical namespace enabled."); @@ -907,7 +920,7 @@ public void removeAcl(final Path path) throws IOException { Path qualifiedPath = makeQualified(path); try { - abfsStore.removeAcl(qualifiedPath, trackingContext); + abfsStore.removeAcl(qualifiedPath, tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -927,13 +940,14 @@ public void removeAcl(final Path path) throws IOException { public void setAcl(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.setAcl path: {}", path); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AC"); + tracingContext.setPrimaryRequestID(); - if (!getIsNamespaceEnabled()) { + if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "setAcl is only supported by storage accounts with the hierarchical " + "namespace enabled."); } - TrackingContext trackingContext = new TrackingContext(fileSystemID, "AC"); if (aclSpec == null || aclSpec.size() == 0) { throw new IllegalArgumentException("The aclSpec argument is invalid."); @@ -942,7 +956,7 @@ public void setAcl(final Path path, final List aclSpec) Path qualifiedPath = makeQualified(path); try { - abfsStore.setAcl(qualifiedPath, aclSpec, trackingContext); + abfsStore.setAcl(qualifiedPath, aclSpec, tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -958,19 +972,19 @@ public void setAcl(final Path path, final List aclSpec) @Override public AclStatus getAclStatus(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.getAclStatus path: {}", path); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AC"); + tracingContext.setPrimaryRequestID(); - if (!getIsNamespaceEnabled()) { + if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "getAclStatus is only supported by storage account with the " + "hierarchical namespace enabled."); } - System.out.println("fs id is " + fileSystemID); - TrackingContext trackingContext = new TrackingContext(fileSystemID, "AC"); Path qualifiedPath = makeQualified(path); try { - return abfsStore.getAclStatus(qualifiedPath, trackingContext); + return abfsStore.getAclStatus(qualifiedPath, tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); return null; @@ -992,10 +1006,10 @@ public AclStatus getAclStatus(final Path path) throws IOException { @Override public void access(final Path path, final FsAction mode) throws IOException { LOG.debug("AzureBlobFileSystem.access path : {}, mode : {}", path, mode); - TrackingContext trackingContext = new TrackingContext(fileSystemID, "LS"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "LS"); Path qualifiedPath = makeQualified(path); try { - this.abfsStore.access(qualifiedPath, mode, trackingContext); + this.abfsStore.access(qualifiedPath, mode, tracingContext); } catch (AzureBlobFileSystemException ex) { checkCheckAccessException(path, ex); } @@ -1027,9 +1041,9 @@ private FileStatus tryGetFileStatus(final Path f) { private boolean fileSystemExists() throws IOException { LOG.debug( "AzureBlobFileSystem.fileSystemExists uri: {}", uri); - TrackingContext trackingContext = new TrackingContext(fileSystemID, "FS"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "FS"); try { - abfsStore.getFilesystemProperties(trackingContext); + abfsStore.getFilesystemProperties(tracingContext); } catch (AzureBlobFileSystemException ex) { try { checkException(null, ex); @@ -1047,9 +1061,9 @@ private boolean fileSystemExists() throws IOException { private void createFileSystem() throws IOException { LOG.debug( "AzureBlobFileSystem.createFileSystem uri: {}", uri); - TrackingContext trackingContext = new TrackingContext(fileSystemID, "FS"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "FS"); try { - abfsStore.createFilesystem(trackingContext); + abfsStore.createFilesystem(tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(null, ex); } @@ -1270,10 +1284,14 @@ AbfsDelegationTokenManager getDelegationTokenManager() { return delegationTokenManager; } + boolean getIsNamespaceEnabled(TracingContext tracingContext) throws AzureBlobFileSystemException { + return abfsStore.getIsNamespaceEnabled(tracingContext); + } + @VisibleForTesting boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException { - TrackingContext trackingContext = new TrackingContext(fileSystemID, "NS"); - return abfsStore.getIsNamespaceEnabled(trackingContext); + return abfsStore.getIsNamespaceEnabled( + new TracingContext(clientCorrelationID, fileSystemID, "NS")); } @VisibleForTesting @@ -1286,12 +1304,13 @@ public boolean hasPathCapability(final Path path, final String capability) throws IOException { // qualify the path to make sure that it refers to the current FS. final Path p = makeQualified(path); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "PA"); switch (validatePathCapabilityArgs(p, capability)) { case CommonPathCapabilities.FS_PERMISSIONS: case CommonPathCapabilities.FS_APPEND: return true; case CommonPathCapabilities.FS_ACLS: - return getIsNamespaceEnabled(); + return getIsNamespaceEnabled(tracingContext); default: return super.hasPathCapability(p, capability); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 45636e27a67a0..7520576279ae6 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -106,7 +106,7 @@ import org.apache.hadoop.fs.azurebfs.utils.Base64; import org.apache.hadoop.fs.azurebfs.utils.CRC64; import org.apache.hadoop.fs.azurebfs.utils.DateTimeUtils; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.UriUtils; import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclStatus; @@ -281,22 +281,22 @@ private String[] authorityParts(URI uri) throws InvalidUriAuthorityException, In } public boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException { - return getIsNamespaceEnabled(new TrackingContext("test-filesystem-id", "NS")); + return getIsNamespaceEnabled(new TracingContext(abfsConfiguration.getClientCorrelationID(), + "test-filesystem-id", "NS")); } - public boolean getIsNamespaceEnabled(TrackingContext trackingContext) throws AzureBlobFileSystemException { + public boolean getIsNamespaceEnabled(TracingContext tracingContext) throws AzureBlobFileSystemException { try { return this.isNamespaceEnabled.toBoolean(); } catch (TrileanConversionException e) { LOG.debug("isNamespaceEnabled is UNKNOWN; fall back and determine through" + " getAcl server call", e); } - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); LOG.debug("Get root ACL status"); try (AbfsPerfInfo perfInfo = startTracking("getIsNamespaceEnabled", "getAclStatus")) { - AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.ROOT_PATH, trackingContext); + AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.ROOT_PATH, tracingContext); perfInfo.registerResult(op.getResult()); isNamespaceEnabled = Trilean.getTrilean(true); perfInfo.registerSuccess(true); @@ -349,17 +349,15 @@ public AbfsConfiguration getAbfsConfiguration() { return this.abfsConfiguration; } - public Hashtable getFilesystemProperties(TrackingContext trackingContext) throws AzureBlobFileSystemException { + public Hashtable getFilesystemProperties(TracingContext tracingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("getFilesystemProperties", "getFilesystemProperties")) { LOG.debug("getFilesystemProperties for filesystem: {}", client.getFileSystem()); - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); - final Hashtable parsedXmsProperties; - final AbfsRestOperation op = client.getFilesystemProperties(trackingContext); + final AbfsRestOperation op = client.getFilesystemProperties(tracingContext); perfInfo.registerResult(op.getResult()); final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); @@ -371,7 +369,7 @@ public Hashtable getFilesystemProperties(TrackingContext trackin } } - public void setFilesystemProperties(final Hashtable properties) + public void setFilesystemProperties(final Hashtable properties, TracingContext tracingContext) throws AzureBlobFileSystemException { if (properties == null || properties.isEmpty()) { LOG.trace("setFilesystemProperties no properties present"); @@ -391,23 +389,21 @@ public void setFilesystemProperties(final Hashtable properties) throw new InvalidAbfsRestOperationException(ex); } - final AbfsRestOperation op = client.setFilesystemProperties(commaSeparatedProperties, - new TrackingContext("test-filesystem-id", "FS")); + final AbfsRestOperation op = client.setFilesystemProperties(commaSeparatedProperties, tracingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } - public Hashtable getPathStatus(final Path path, TrackingContext trackingContext) + public Hashtable getPathStatus(final Path path, TracingContext tracingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("getPathStatus", "getPathStatus")){ LOG.debug("getPathStatus for filesystem: {} path: {}", client.getFileSystem(), path); - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); final Hashtable parsedXmsProperties; final AbfsRestOperation op = client.getPathStatus(getRelativePath(path), true, - trackingContext); + tracingContext); perfInfo.registerResult(op.getResult()); final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); @@ -421,13 +417,12 @@ public Hashtable getPathStatus(final Path path, TrackingContext } public void setPathProperties(final Path path, final Hashtable properties, - TrackingContext trackingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("setPathProperties", "setPathProperties")){ LOG.debug("setFilesystemProperties for filesystem: {} path: {} with properties: {}", client.getFileSystem(), path, properties); - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); final String commaSeparatedProperties; try { @@ -436,29 +431,27 @@ public void setPathProperties(final Path path, final Hashtable p throw new InvalidAbfsRestOperationException(ex); } final AbfsRestOperation op = client.setPathProperties(getRelativePath(path), commaSeparatedProperties, - trackingContext); + tracingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } - public void createFilesystem(TrackingContext trackingContext) throws AzureBlobFileSystemException { + public void createFilesystem(TracingContext tracingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("createFilesystem", "createFilesystem")){ LOG.debug("createFilesystem for filesystem: {}", client.getFileSystem()); - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); - final AbfsRestOperation op = client.createFilesystem(trackingContext); + final AbfsRestOperation op = client.createFilesystem(tracingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } - public void deleteFilesystem(TrackingContext trackingContext) throws AzureBlobFileSystemException { + public void deleteFilesystem(TracingContext tracingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("deleteFilesystem", "deleteFilesystem")) { LOG.debug("deleteFilesystem for filesystem: {}", client.getFileSystem()); - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); - final AbfsRestOperation op = client.deleteFilesystem(trackingContext); + final AbfsRestOperation op = client.deleteFilesystem(tracingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } @@ -466,9 +459,9 @@ public void deleteFilesystem(TrackingContext trackingContext) throws AzureBlobFi public OutputStream createFile(final Path path, final FileSystem.Statistics statistics, final boolean overwrite, final FsPermission permission, - final FsPermission umask, TrackingContext trackingContext) throws AzureBlobFileSystemException { + final FsPermission umask, TracingContext tracingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) { - boolean isNamespaceEnabled = getIsNamespaceEnabled(trackingContext); + boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}", client.getFileSystem(), path, @@ -476,7 +469,6 @@ public OutputStream createFile(final Path path, permission, umask, isNamespaceEnabled); - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); String relativePath = getRelativePath(path); boolean isAppendBlob = false; @@ -499,7 +491,7 @@ public OutputStream createFile(final Path path, statistics, isNamespaceEnabled ? getOctalNotation(permission) : null, isNamespaceEnabled ? getOctalNotation(umask) : null, - isAppendBlob, trackingContext + isAppendBlob, new TracingContext(tracingContext) ); } else { @@ -508,20 +500,17 @@ public OutputStream createFile(final Path path, isNamespaceEnabled ? getOctalNotation(permission) : null, isNamespaceEnabled ? getOctalNotation(umask) : null, isAppendBlob, - null, trackingContext); + null, tracingContext); } perfInfo.registerResult(op.getResult()).registerSuccess(true); - AbfsOutputStream out = new AbfsOutputStream( + return new AbfsOutputStream( client, statistics, relativePath, 0, populateAbfsOutputStreamContext(isAppendBlob), - trackingContext); -// trackingContext.setStreamID(out.getOutputStreamID()); - - return out; + tracingContext); } } @@ -540,27 +529,23 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa final FileSystem.Statistics statistics, final String permission, final String umask, - final boolean isAppendBlob, TrackingContext trackingContext) throws AzureBlobFileSystemException { + final boolean isAppendBlob, TracingContext tracingContext) throws AzureBlobFileSystemException { AbfsRestOperation op; - trackingContext.setPrimaryRequestID(); - trackingContext.firstRequest = true; + tracingContext.setPrimaryRequestID(); try { // Trigger a create with overwrite=false first so that eTag fetch can be // avoided for cases when no pre-existing file is present (major portion // of create file traffic falls into the case of no pre-existing file). - op = client.createPath(relativePath, true, - false, permission, umask, isAppendBlob, null, new TrackingContext(trackingContext)); + false, permission, umask, isAppendBlob, null, new TracingContext(tracingContext)); } catch (AbfsRestOperationException e) { - trackingContext.firstRequest = false; if (e.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { // File pre-exists, fetch eTag try { - op = client.getPathStatus(relativePath, false, new TrackingContext(trackingContext)); -// trackingContext.setPrimaryRequestID(); + op = client.getPathStatus(relativePath, false, new TracingContext(tracingContext)); } catch (AbfsRestOperationException ex) { if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { // Is a parallel access case, as file which was found to be @@ -579,7 +564,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa try { // overwrite only if eTag matches with the file properties fetched befpre op = client.createPath(relativePath, true, - true, permission, umask, isAppendBlob, eTag, new TrackingContext(trackingContext)); + true, permission, umask, isAppendBlob, eTag, new TracingContext(tracingContext)); } catch (AbfsRestOperationException ex) { if (ex.getStatusCode() == HttpURLConnection.HTTP_PRECON_FAILED) { // Is a parallel access case, as file with eTag was just queried @@ -617,10 +602,10 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext(boolean isAppend } public void createDirectory(final Path path, final FsPermission permission, final FsPermission umask, - TrackingContext trackingContext) + TracingContext tracingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("createDirectory", "createPath")) { - boolean isNamespaceEnabled = getIsNamespaceEnabled(trackingContext); + boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); LOG.debug("createDirectory filesystem: {} path: {} permission: {} umask: {} isNamespaceEnabled: {}", client.getFileSystem(), path, @@ -631,23 +616,22 @@ public void createDirectory(final Path path, final FsPermission permission, fina final AbfsRestOperation op = client.createPath(getRelativePath(path), false, true, isNamespaceEnabled ? getOctalNotation(permission) : null, isNamespaceEnabled ? getOctalNotation(umask) : null, false, null, - trackingContext); + tracingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statistics statistics, - TrackingContext trackingContext) + TracingContext tracingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("openFileForRead", "getPathStatus")) { LOG.debug("openFileForRead filesystem: {} path: {}", client.getFileSystem(), path); - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getPathStatus(relativePath, false, trackingContext); + final AbfsRestOperation op = client.getPathStatus(relativePath, false, tracingContext); perfInfo.registerResult(op.getResult()); final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); @@ -668,8 +652,8 @@ public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statist AbfsInputStream in = new AbfsInputStream(client, statistics, relativePath, contentLength, populateAbfsInputStreamContext(), - eTag); -// trackingContext.setStreamID(in.getInputStreamID()); + eTag, tracingContext); +// tracingContext.setStreamID(in.getInputStreamID()); return in; } } @@ -684,18 +668,17 @@ private AbfsInputStreamContext populateAbfsInputStreamContext() { } public OutputStream openFileForWrite(final Path path, final FileSystem.Statistics statistics, - final boolean overwrite, TrackingContext trackingContext) throws + final boolean overwrite, TracingContext tracingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("openFileForWrite", "getPathStatus")) { LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {}", client.getFileSystem(), path, overwrite); - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getPathStatus(relativePath, false, trackingContext); + final AbfsRestOperation op = client.getPathStatus(relativePath, false, tracingContext); perfInfo.registerResult(op.getResult()); final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); @@ -723,19 +706,18 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic relativePath, offset, populateAbfsOutputStreamContext(isAppendBlob), - trackingContext); -// trackingContext.setStreamID(out.getOutputStreamID()); + tracingContext); +// tracingContext.setStreamID(out.getOutputStreamID()); return out; } } - public void rename(final Path source, final Path destination, TrackingContext trackingContext) throws + public void rename(final Path source, final Path destination, TracingContext tracingContext) throws AzureBlobFileSystemException { final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue; - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); if (isAtomicRenameKey(source.getName())) { LOG.warn("The atomic rename feature is not supported by the ABFS scheme; however rename," @@ -751,13 +733,11 @@ public void rename(final Path source, final Path destination, TrackingContext tr String sourceRelativePath = getRelativePath(source); String destinationRelativePath = getRelativePath(destination); - trackingContext.setPrimaryRequestID(); - trackingContext.firstRequest = true; do { try (AbfsPerfInfo perfInfo = startTracking("rename", "renamePath")) { AbfsRestOperation op = client.renamePath(sourceRelativePath, - destinationRelativePath, continuation, new TrackingContext(trackingContext)); + destinationRelativePath, continuation, new TracingContext(tracingContext)); perfInfo.registerResult(op.getResult()); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); perfInfo.registerSuccess(true); @@ -768,16 +748,14 @@ public void rename(final Path source, final Path destination, TrackingContext tr perfInfo.registerAggregates(startAggregate, countAggregate); } } - trackingContext.firstRequest = false; } while (shouldContinue); } - public void delete(final Path path, final boolean recursive, TrackingContext trackingContext) + public void delete(final Path path, final boolean recursive, TracingContext tracingContext) throws AzureBlobFileSystemException { final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue = true; - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); LOG.debug("delete filesystem: {} path: {} recursive: {}", client.getFileSystem(), @@ -791,13 +769,12 @@ public void delete(final Path path, final boolean recursive, TrackingContext tra do { try (AbfsPerfInfo perfInfo = startTracking("delete", "deletePath")) { AbfsRestOperation op = client.deletePath( - relativePath, recursive, continuation, trackingContext); + relativePath, recursive, continuation, tracingContext); perfInfo.registerResult(op.getResult()); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); perfInfo.registerSuccess(true); countAggregate++; shouldContinue = continuation != null && !continuation.isEmpty(); - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); if (!shouldContinue) { perfInfo.registerAggregates(startAggregate, countAggregate); @@ -806,27 +783,26 @@ public void delete(final Path path, final boolean recursive, TrackingContext tra } while (shouldContinue); } - public FileStatus getFileStatus(final Path path, TrackingContext trackingContext) throws IOException { + public FileStatus getFileStatus(final Path path, TracingContext tracingContext) throws IOException { try (AbfsPerfInfo perfInfo = startTracking("getFileStatus", "undetermined")) { - boolean isNamespaceEnabled = getIsNamespaceEnabled(trackingContext); + boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); LOG.debug("getFileStatus filesystem: {} path: {} isNamespaceEnabled: {}", client.getFileSystem(), path, isNamespaceEnabled); - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); final AbfsRestOperation op; if (path.isRoot()) { if (isNamespaceEnabled) { perfInfo.registerCallee("getAclStatus"); - op = client.getAclStatus(getRelativePath(path), trackingContext); + op = client.getAclStatus(getRelativePath(path), tracingContext); } else { perfInfo.registerCallee("getFilesystemProperties"); - op = client.getFilesystemProperties(trackingContext); + op = client.getFilesystemProperties(tracingContext); } } else { perfInfo.registerCallee("getPathStatus"); - op = client.getPathStatus(getRelativePath(path), false, trackingContext); + op = client.getPathStatus(getRelativePath(path), false, tracingContext); } perfInfo.registerResult(op.getResult()); @@ -880,9 +856,8 @@ public FileStatus getFileStatus(final Path path, TrackingContext trackingContext * @param path The list path. * @return the entries in the path. * */ - public FileStatus[] listStatus(final Path path, TrackingContext trackingContext) throws IOException { - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); - return listStatus(path, null, trackingContext); + public FileStatus[] listStatus(final Path path, TracingContext tracingContext) throws IOException { + return listStatus(path, null, tracingContext); } /** @@ -898,11 +873,11 @@ public FileStatus[] listStatus(final Path path, TrackingContext trackingContext) * */ @InterfaceStability.Unstable public FileStatus[] listStatus(final Path path, final String startFrom, - TrackingContext originalTrackingContext) throws IOException { + TracingContext originalTracingContext) throws IOException { final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue = true; -// trackingContext.updateIteration(); +// tracingContext.updateIteration(); LOG.debug("listStatus filesystem: {} path: {}, startFrom: {}", client.getFileSystem(), @@ -912,23 +887,19 @@ public FileStatus[] listStatus(final Path path, final String startFrom, final String relativePath = getRelativePath(path); String continuation = null; - TrackingContext trackingContext = new TrackingContext(originalTrackingContext); - // generate continuation token if a valid startFrom is provided. if (startFrom != null && !startFrom.isEmpty()) { - continuation = getIsNamespaceEnabled(trackingContext) + continuation = getIsNamespaceEnabled(originalTracingContext) ? generateContinuationTokenForXns(startFrom) : generateContinuationTokenForNonXns(relativePath, startFrom); } - - trackingContext.setPrimaryRequestID(); - trackingContext.firstRequest = true; + TracingContext tracingContext = new TracingContext(originalTracingContext); ArrayList fileStatuses = new ArrayList<>(); do { try (AbfsPerfInfo perfInfo = startTracking("listStatus", "listPath")) { AbfsRestOperation op = client.listPath(relativePath, false, - abfsConfiguration.getListMaxResults(), continuation, new TrackingContext(trackingContext)); + abfsConfiguration.getListMaxResults(), continuation, new TracingContext(tracingContext)); perfInfo.registerResult(op.getResult()); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); ListResultSchema retrievedSchema = op.getResult().getListResultSchema(); @@ -984,7 +955,6 @@ public FileStatus[] listStatus(final Path path, final String startFrom, perfInfo.registerAggregates(startAggregate, countAggregate); } } - trackingContext.firstRequest = false; } while (shouldContinue); return fileStatuses.toArray(new FileStatus[fileStatuses.size()]); @@ -1047,13 +1017,12 @@ private String generateContinuationTokenForNonXns(String path, final String firs } public void setOwner(final Path path, final String owner, final String group, - TrackingContext trackingContext) throws + TracingContext tracingContext) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled(trackingContext)) { + if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfo = startTracking("setOwner", "setOwner")) { @@ -1069,20 +1038,19 @@ public void setOwner(final Path path, final String owner, final String group, final AbfsRestOperation op = client.setOwner(getRelativePath(path), transformedOwner, - transformedGroup, trackingContext); + transformedGroup, tracingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } public void setPermission(final Path path, final FsPermission permission, - TrackingContext trackingContext) throws + TracingContext tracingContext) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled(trackingContext)) { + if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfo = startTracking("setPermission", "setPermission")) { @@ -1093,20 +1061,19 @@ public void setPermission(final Path path, final FsPermission permission, permission); final AbfsRestOperation op = client.setPermission(getRelativePath(path), - String.format(AbfsHttpConstants.PERMISSION_FORMAT, permission.toOctal()), trackingContext); + String.format(AbfsHttpConstants.PERMISSION_FORMAT, permission.toOctal()), tracingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } public void modifyAclEntries(final Path path, final List aclSpec, - TrackingContext trackingContext) throws + TracingContext tracingContext) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled(trackingContext)) { + if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfoGet = startTracking("modifyAclEntries", "getAclStatus")) { @@ -1122,7 +1089,7 @@ public void modifyAclEntries(final Path path, final List aclSpec, String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getAclStatus(relativePath, useUpn, trackingContext); + final AbfsRestOperation op = client.getAclStatus(relativePath, useUpn, tracingContext); perfInfoGet.registerResult(op.getResult()); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); @@ -1135,7 +1102,7 @@ public void modifyAclEntries(final Path path, final List aclSpec, try (AbfsPerfInfo perfInfoSet = startTracking("modifyAclEntries", "setAcl")) { final AbfsRestOperation setAclOp = client.setAcl(relativePath, - AbfsAclHelper.serializeAclSpec(aclEntries), eTag, trackingContext); + AbfsAclHelper.serializeAclSpec(aclEntries), eTag, tracingContext); perfInfoSet.registerResult(setAclOp.getResult()) .registerSuccess(true) .registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT); @@ -1144,12 +1111,11 @@ public void modifyAclEntries(final Path path, final List aclSpec, } public void removeAclEntries(final Path path, final List aclSpec, - TrackingContext trackingContext) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled(trackingContext)) { + TracingContext tracingContext) throws AzureBlobFileSystemException { + if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfoGet = startTracking("removeAclEntries", "getAclStatus")) { @@ -1165,7 +1131,7 @@ public void removeAclEntries(final Path path, final List aclSpec, String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getAclStatus(relativePath, isUpnFormat, trackingContext); + final AbfsRestOperation op = client.getAclStatus(relativePath, isUpnFormat, tracingContext); perfInfoGet.registerResult(op.getResult()); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); @@ -1178,7 +1144,7 @@ public void removeAclEntries(final Path path, final List aclSpec, try (AbfsPerfInfo perfInfoSet = startTracking("removeAclEntries", "setAcl")) { final AbfsRestOperation setAclOp = client.setAcl(relativePath, - AbfsAclHelper.serializeAclSpec(aclEntries), eTag, trackingContext); + AbfsAclHelper.serializeAclSpec(aclEntries), eTag, tracingContext); perfInfoSet.registerResult(setAclOp.getResult()) .registerSuccess(true) .registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT); @@ -1186,12 +1152,11 @@ public void removeAclEntries(final Path path, final List aclSpec, } } - public void removeDefaultAcl(final Path path, TrackingContext trackingContext) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled(trackingContext)) { + public void removeDefaultAcl(final Path path, TracingContext tracingContext) throws AzureBlobFileSystemException { + if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfoGet = startTracking("removeDefaultAcl", "getAclStatus")) { @@ -1202,7 +1167,7 @@ public void removeDefaultAcl(final Path path, TrackingContext trackingContext) t String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getAclStatus(relativePath, trackingContext); + final AbfsRestOperation op = client.getAclStatus(relativePath, tracingContext); perfInfoGet.registerResult(op.getResult()); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); final Map aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); @@ -1221,7 +1186,7 @@ public void removeDefaultAcl(final Path path, TrackingContext trackingContext) t try (AbfsPerfInfo perfInfoSet = startTracking("removeDefaultAcl", "setAcl")) { final AbfsRestOperation setAclOp = client.setAcl(relativePath, - AbfsAclHelper.serializeAclSpec(aclEntries), eTag, trackingContext); + AbfsAclHelper.serializeAclSpec(aclEntries), eTag, tracingContext); perfInfoSet.registerResult(setAclOp.getResult()) .registerSuccess(true) .registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT); @@ -1229,12 +1194,11 @@ public void removeDefaultAcl(final Path path, TrackingContext trackingContext) t } } - public void removeAcl(final Path path, TrackingContext trackingContext) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled(trackingContext)) { + public void removeAcl(final Path path, TracingContext tracingContext) throws AzureBlobFileSystemException { + if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfoGet = startTracking("removeAcl", "getAclStatus")){ @@ -1245,7 +1209,7 @@ public void removeAcl(final Path path, TrackingContext trackingContext) throws A String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getAclStatus(relativePath, trackingContext); + final AbfsRestOperation op = client.getAclStatus(relativePath, tracingContext); perfInfoGet.registerResult(op.getResult()); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); @@ -1261,7 +1225,7 @@ public void removeAcl(final Path path, TrackingContext trackingContext) throws A try (AbfsPerfInfo perfInfoSet = startTracking("removeAcl", "setAcl")) { final AbfsRestOperation setAclOp = client.setAcl(relativePath, - AbfsAclHelper.serializeAclSpec(newAclEntries), eTag, trackingContext); + AbfsAclHelper.serializeAclSpec(newAclEntries), eTag, tracingContext); perfInfoSet.registerResult(setAclOp.getResult()) .registerSuccess(true) .registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT); @@ -1270,12 +1234,11 @@ public void removeAcl(final Path path, TrackingContext trackingContext) throws A } public void setAcl(final Path path, final List aclSpec, - TrackingContext trackingContext) throws AzureBlobFileSystemException { - if (!getIsNamespaceEnabled(trackingContext)) { + TracingContext tracingContext) throws AzureBlobFileSystemException { + if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfoGet = startTracking("setAcl", "getAclStatus")) { @@ -1291,7 +1254,7 @@ public void setAcl(final Path path, final List aclSpec, String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getAclStatus(relativePath, isUpnFormat, trackingContext); + final AbfsRestOperation op = client.getAclStatus(relativePath, isUpnFormat, tracingContext); perfInfoGet.registerResult(op.getResult()); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); @@ -1304,7 +1267,7 @@ public void setAcl(final Path path, final List aclSpec, try (AbfsPerfInfo perfInfoSet = startTracking("setAcl", "setAcl")) { final AbfsRestOperation setAclOp = client.setAcl(relativePath, - AbfsAclHelper.serializeAclSpec(aclEntries), eTag, trackingContext); + AbfsAclHelper.serializeAclSpec(aclEntries), eTag, tracingContext); perfInfoSet.registerResult(setAclOp.getResult()) .registerSuccess(true) .registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT); @@ -1312,12 +1275,11 @@ public void setAcl(final Path path, final List aclSpec, } } - public AclStatus getAclStatus(final Path path, TrackingContext trackingContext) throws IOException { - if (!getIsNamespaceEnabled(trackingContext)) { + public AclStatus getAclStatus(final Path path, TracingContext tracingContext) throws IOException { + if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); } - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfo = startTracking("getAclStatus", "getAclStatus")) { @@ -1326,7 +1288,7 @@ public AclStatus getAclStatus(final Path path, TrackingContext trackingContext) client.getFileSystem(), path); - AbfsRestOperation op = client.getAclStatus(getRelativePath(path), trackingContext); + AbfsRestOperation op = client.getAclStatus(getRelativePath(path), tracingContext); AbfsHttpOperation result = op.getResult(); perfInfo.registerResult(result); @@ -1359,20 +1321,19 @@ public AclStatus getAclStatus(final Path path, TrackingContext trackingContext) } } - public void access(final Path path, final FsAction mode, TrackingContext trackingContext) + public void access(final Path path, final FsAction mode, TracingContext tracingContext) throws AzureBlobFileSystemException { LOG.debug("access for filesystem: {}, path: {}, mode: {}", this.client.getFileSystem(), path, mode); if (!this.abfsConfiguration.isCheckAccessEnabled() - || !getIsNamespaceEnabled(trackingContext)) { + || !getIsNamespaceEnabled(tracingContext)) { LOG.debug("Returning; either check access is not enabled or the account" + " used is not namespace enabled"); return; } - trackingContext.setClientCorrelationID(abfsConfiguration.getClientCorrelationID()); try (AbfsPerfInfo perfInfo = startTracking("access", "checkAccess")) { final AbfsRestOperation op = this.client - .checkAccess(getRelativePath(path), mode.SYMBOL, trackingContext); + .checkAccess(getRelativePath(path), mode.SYMBOL, tracingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 9810553ef31d3..98ade13233e61 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -47,7 +47,7 @@ import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.hadoop.fs.azurebfs.utils.DateTimeUtils; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.io.IOUtils; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*; @@ -165,7 +165,7 @@ AbfsUriQueryBuilder createDefaultUriQueryBuilder() { return abfsUriQueryBuilder; } - public AbfsRestOperation createFilesystem(TrackingContext trackingContext) throws AzureBlobFileSystemException { + public AbfsRestOperation createFilesystem(TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); @@ -178,11 +178,11 @@ public AbfsRestOperation createFilesystem(TrackingContext trackingContext) throw HTTP_METHOD_PUT, url, requestHeaders); - op.execute(trackingContext); + op.execute(tracingContext); return op; } - public AbfsRestOperation setFilesystemProperties(final String properties, TrackingContext trackingContext) throws AzureBlobFileSystemException { + public AbfsRestOperation setFilesystemProperties(final String properties, TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. @@ -202,12 +202,12 @@ public AbfsRestOperation setFilesystemProperties(final String properties, Tracki HTTP_METHOD_PUT, url, requestHeaders); - op.execute(trackingContext); + op.execute(tracingContext); return op; } public AbfsRestOperation listPath(final String relativePath, final boolean recursive, final int listMaxResults, - final String continuation, TrackingContext trackingContext) + final String continuation, TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); @@ -227,11 +227,11 @@ public AbfsRestOperation listPath(final String relativePath, final boolean recur HTTP_METHOD_GET, url, requestHeaders); - op.execute(trackingContext); + op.execute(tracingContext); return op; } - public AbfsRestOperation getFilesystemProperties(TrackingContext trackingContext) throws AzureBlobFileSystemException { + public AbfsRestOperation getFilesystemProperties(TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); @@ -244,11 +244,11 @@ public AbfsRestOperation getFilesystemProperties(TrackingContext trackingContext HTTP_METHOD_HEAD, url, requestHeaders); - op.execute(trackingContext); + op.execute(tracingContext); return op; } - public AbfsRestOperation deleteFilesystem(TrackingContext trackingContext) throws AzureBlobFileSystemException { + public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); @@ -261,14 +261,14 @@ public AbfsRestOperation deleteFilesystem(TrackingContext trackingContext) throw HTTP_METHOD_DELETE, url, requestHeaders); - op.execute(trackingContext); + op.execute(tracingContext); return op; } public AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite, final String permission, final String umask, final boolean isAppendBlob, final String eTag, - TrackingContext trackingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); if (!overwrite) { requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, AbfsHttpConstants.STAR)); @@ -304,12 +304,12 @@ public AbfsRestOperation createPath(final String path, final boolean isFile, fin HTTP_METHOD_PUT, url, requestHeaders); - op.execute(trackingContext); + op.execute(tracingContext); return op; } public AbfsRestOperation renamePath(String source, final String destination, final String continuation, - TrackingContext trackingContext) + TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); @@ -337,10 +337,10 @@ public AbfsRestOperation renamePath(String source, final String destination, fin requestHeaders); Instant renameRequestStartTime = Instant.now(); try { - op.execute(trackingContext); + op.execute(tracingContext); } catch (AzureBlobFileSystemException e) { final AbfsRestOperation idempotencyOp = renameIdempotencyCheckOp( - renameRequestStartTime, op, destination, trackingContext); + renameRequestStartTime, op, destination, tracingContext); if (idempotencyOp.getResult().getStatusCode() == op.getResult().getStatusCode()) { // idempotency did not return different result @@ -372,7 +372,7 @@ public AbfsRestOperation renameIdempotencyCheckOp( final Instant renameRequestStartTime, final AbfsRestOperation op, final String destination, - TrackingContext trackingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) throws AzureBlobFileSystemException { if ((op.isARetriedRequest()) && (op.getResult().getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND)) { // Server has returned HTTP 404, which means rename source no longer @@ -381,7 +381,7 @@ public AbfsRestOperation renameIdempotencyCheckOp( try { final AbfsRestOperation destStatusOp = getPathStatus(destination, - false , trackingContext); + false , tracingContext); if (destStatusOp.getResult().getStatusCode() == HttpURLConnection.HTTP_OK) { String lmt = destStatusOp.getResult().getResponseHeader( @@ -403,12 +403,12 @@ public AbfsRestOperation renameIdempotencyCheckOp( public AbfsRestOperation append(final String path, final long position, final byte[] buffer, final int offset, final int length, final String cachedSasToken, final boolean isAppendBlob) throws AzureBlobFileSystemException { return append(path, position, buffer, offset, length, cachedSasToken, - isAppendBlob, new TrackingContext("test-filesystem-id", "AP")); + isAppendBlob, new TracingContext("test-filesystem-id", "AP")); } public AbfsRestOperation append(final String path, final long position, final byte[] buffer, final int offset, final int length, final String cachedSasToken, final boolean isAppendBlob, - TrackingContext trackingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. @@ -430,9 +430,9 @@ public AbfsRestOperation append(final String path, final long position, final by url, requestHeaders, buffer, offset, length, sasTokenForReuse); try { - op.execute(trackingContext); + op.execute(tracingContext); } catch (AzureBlobFileSystemException e) { - if (isAppendBlob && appendSuccessCheckOp(op, path, (position + length), trackingContext)) { + if (isAppendBlob && appendSuccessCheckOp(op, path, (position + length), tracingContext)) { final AbfsRestOperation successOp = new AbfsRestOperation( AbfsRestOperationType.Append, this, @@ -454,10 +454,10 @@ public AbfsRestOperation append(final String path, final long position, final by // Hence, we pass/succeed the appendblob append call // in case we are doing a retry after checking the length of the file public boolean appendSuccessCheckOp(AbfsRestOperation op, final String path, - final long length, TrackingContext trackingContext) throws AzureBlobFileSystemException { + final long length, TracingContext tracingContext) throws AzureBlobFileSystemException { if ((op.isARetriedRequest()) && (op.getResult().getStatusCode() == HttpURLConnection.HTTP_BAD_REQUEST)) { - final AbfsRestOperation destStatusOp = getPathStatus(path, false, trackingContext); + final AbfsRestOperation destStatusOp = getPathStatus(path, false, tracingContext); if (destStatusOp.getResult().getStatusCode() == HttpURLConnection.HTTP_OK) { String fileLength = destStatusOp.getResult().getResponseHeader( HttpHeaderConfigurations.CONTENT_LENGTH); @@ -471,7 +471,7 @@ public boolean appendSuccessCheckOp(AbfsRestOperation op, final String path, public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData, boolean isClose, final String cachedSasToken, - TrackingContext trackingContext) + TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use @@ -495,12 +495,12 @@ public AbfsRestOperation flush(final String path, final long position, boolean r HTTP_METHOD_PUT, url, requestHeaders, sasTokenForReuse); - op.execute(trackingContext); + op.execute(tracingContext); return op; } public AbfsRestOperation setPathProperties(final String path, final String properties, - TrackingContext trackingContext) + TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use @@ -521,12 +521,12 @@ public AbfsRestOperation setPathProperties(final String path, final String prope HTTP_METHOD_PUT, url, requestHeaders); - op.execute(trackingContext); + op.execute(tracingContext); return op; } public AbfsRestOperation getPathStatus(final String path, final boolean includeProperties, - TrackingContext trackingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); @@ -548,14 +548,13 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP HTTP_METHOD_HEAD, url, requestHeaders); - op.execute(trackingContext); + op.execute(tracingContext); return op; } public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset, final int bufferLength, final String eTag, String cachedSasToken, - TrackingContext trackingContext) throws AzureBlobFileSystemException { - System.out.println("in read"); + TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); requestHeaders.add(new AbfsHttpHeader(RANGE, String.format("bytes=%d-%d", position, position + bufferLength - 1))); @@ -567,7 +566,6 @@ public AbfsRestOperation read(final String path, final long position, final byte abfsUriQueryBuilder, cachedSasToken); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - System.out.println("in the read method of client"); final AbfsRestOperation op = new AbfsRestOperation( AbfsRestOperationType.ReadFile, this, @@ -577,15 +575,13 @@ public AbfsRestOperation read(final String path, final long position, final byte buffer, bufferOffset, bufferLength, sasTokenForReuse); - System.out.println("op for read"); - op.execute(trackingContext); - System.out.println("executed read"); + op.execute(tracingContext); return op; } public AbfsRestOperation deletePath(final String path, final boolean recursive, final String continuation, - TrackingContext trackingContext) + TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); @@ -603,7 +599,7 @@ public AbfsRestOperation deletePath(final String path, final boolean recursive, url, requestHeaders); try { - op.execute(trackingContext); + op.execute(tracingContext); } catch (AzureBlobFileSystemException e) { final AbfsRestOperation idempotencyOp = deleteIdempotencyCheckOp(op); if (idempotencyOp.getResult().getStatusCode() @@ -654,7 +650,7 @@ public AbfsRestOperation deleteIdempotencyCheckOp(final AbfsRestOperation op) { } public AbfsRestOperation setOwner(final String path, final String owner, final String group, - TrackingContext trackingContext) + TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use @@ -680,12 +676,12 @@ public AbfsRestOperation setOwner(final String path, final String owner, final S AbfsHttpConstants.HTTP_METHOD_PUT, url, requestHeaders); - op.execute(trackingContext); + op.execute(tracingContext); return op; } public AbfsRestOperation setPermission(final String path, final String permission, - TrackingContext trackingContext) + TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use @@ -706,17 +702,17 @@ public AbfsRestOperation setPermission(final String path, final String permissio AbfsHttpConstants.HTTP_METHOD_PUT, url, requestHeaders); - op.execute(trackingContext); + op.execute(tracingContext); return op; } public AbfsRestOperation setAcl(final String path, final String aclSpecString, - TrackingContext trackingContext) throws AzureBlobFileSystemException { - return setAcl(path, aclSpecString, AbfsHttpConstants.EMPTY_STRING, trackingContext); + TracingContext tracingContext) throws AzureBlobFileSystemException { + return setAcl(path, aclSpecString, AbfsHttpConstants.EMPTY_STRING, tracingContext); } public AbfsRestOperation setAcl(final String path, final String aclSpecString, final String eTag, - TrackingContext trackingContext) + TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use @@ -741,17 +737,17 @@ public AbfsRestOperation setAcl(final String path, final String aclSpecString, f AbfsHttpConstants.HTTP_METHOD_PUT, url, requestHeaders); - op.execute(trackingContext); + op.execute(tracingContext); return op; } - public AbfsRestOperation getAclStatus(final String path, TrackingContext trackingContext) + public AbfsRestOperation getAclStatus(final String path, TracingContext tracingContext) throws AzureBlobFileSystemException { - return getAclStatus(path, abfsConfiguration.isUpnUsed(), trackingContext); + return getAclStatus(path, abfsConfiguration.isUpnUsed(), tracingContext); } public AbfsRestOperation getAclStatus(final String path, final boolean useUPN, - TrackingContext trackingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); @@ -766,7 +762,7 @@ public AbfsRestOperation getAclStatus(final String path, final boolean useUPN, AbfsHttpConstants.HTTP_METHOD_HEAD, url, requestHeaders); - op.execute(trackingContext); + op.execute(tracingContext); return op; } @@ -779,7 +775,7 @@ public AbfsRestOperation getAclStatus(final String path, final boolean useUPN, * @return The {@link AbfsRestOperation} object for the operation * @throws AzureBlobFileSystemException in case of bad requests */ - public AbfsRestOperation checkAccess(String path, String rwx, TrackingContext trackingContext) + public AbfsRestOperation checkAccess(String path, String rwx, TracingContext tracingContext) throws AzureBlobFileSystemException { AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, CHECK_ACCESS); @@ -789,7 +785,7 @@ public AbfsRestOperation checkAccess(String path, String rwx, TrackingContext tr AbfsRestOperation op = new AbfsRestOperation( AbfsRestOperationType.CheckAccess, this, AbfsHttpConstants.HTTP_METHOD_HEAD, url, createDefaultHeaders()); - op.execute(trackingContext); + op.execute(tracingContext); return op; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index 3e7b67f6ced61..dcf9b7c681db9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -43,7 +43,7 @@ import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsPerfLoggable; import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; /** * Represents an HTTP operation. @@ -240,7 +240,7 @@ public String getLogString() { * @throws IOException if an error occurs. */ // public AbfsHttpOperation(final URL url, final String method, final List requestHeaders) throws IOException { -// this(url, method, requestHeaders, new TrackingContext("", "OP")); +// this(url, method, requestHeaders, new TracingContext("", "OP")); // } public AbfsHttpOperation(final URL url, final String method, final List requestHeaders) @@ -248,7 +248,7 @@ public AbfsHttpOperation(final URL url, final String method, final List 0 && nextOffset < contentLength) { nextSize = Math.min((long) bufferSize, contentLength - nextOffset); LOG.debug("issuing read ahead requestedOffset = {} requested size {}", nextOffset, nextSize); - trackingContext.firstRequest = numReadAheads == readAheadQueueDepth; ReadBufferManager.getBufferManager().queueReadAhead(this, nextOffset, (int) nextSize, - trackingContext); + readAheadTracingContext); nextOffset = nextOffset + nextSize; numReadAheads--; - System.out.println("in the readahead loop " + Integer.toString(numReadAheads)); } // try reading from buffers first @@ -272,18 +259,17 @@ private int readInternal(final long position, final byte[] b, final int offset, } return receivedBytes; } - trackingContext.reset(); //remove to enable remote read to have same preq id // got nothing from read-ahead, do our own read now - receivedBytes = readRemote(position, b, offset, length, new TrackingContext(trackingContext)); + receivedBytes = readRemote(position, b, offset, length, new TracingContext(tracingContext)); return receivedBytes; } else { LOG.debug("read ahead disabled, reading remote"); - return readRemote(position, b, offset, length, new TrackingContext(trackingContext)); + return readRemote(position, b, offset, length, new TracingContext(tracingContext)); } } - int readRemote(long position, byte[] b, int offset, int length, TrackingContext trackingContext) throws IOException { + int readRemote(long position, byte[] b, int offset, int length, TracingContext tracingContext) throws IOException { if (position < 0) { throw new IllegalArgumentException("attempting to read from negative offset"); } @@ -306,9 +292,8 @@ int readRemote(long position, byte[] b, int offset, int length, TrackingContext AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "readRemote", "read")) { LOG.trace("Trigger client.read for path={} position={} offset={} length={}", path, position, offset, length); - System.out.println("going to client read"); op = client.read(path, position, b, offset, length, tolerateOobAppends ? "*" : eTag, cachedSasToken.get(), - trackingContext); + tracingContext); cachedSasToken.update(op.getSasToken()); if (streamStatistics != null) { streamStatistics.remoteReadOperation(); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index d2cfb291c70f4..dc97a3fbc5f5d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -36,7 +36,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -82,7 +82,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable, StreamCa // SAS tokens can be re-used until they expire private CachedSASToken cachedSasToken; private String outputStreamID; - private TrackingContext trackingContext; + private TracingContext tracingContext; /** * Queue storing buffers with the size of the Azure block ready for @@ -99,22 +99,13 @@ public class AbfsOutputStream extends OutputStream implements Syncable, StreamCa private static final Logger LOG = LoggerFactory.getLogger(AbfsOutputStream.class); - public AbfsOutputStream(final AbfsClient client, - final Statistics statistics, - final String path, - final long position, - AbfsOutputStreamContext abfsOutputStreamContext) { - this(client, statistics, path, position, abfsOutputStreamContext, - new TrackingContext("test-filesystem-id", "OP")); - } - public AbfsOutputStream( final AbfsClient client, final Statistics statistics, final String path, final long position, AbfsOutputStreamContext abfsOutputStreamContext, - TrackingContext trackingContext) { + TracingContext tracingContext) { this.client = client; this.statistics = statistics; this.path = path; @@ -150,8 +141,8 @@ public AbfsOutputStream( this.cachedSasToken = new CachedSASToken( abfsOutputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds()); this.outputStreamID = StringUtils.right(UUID.randomUUID().toString(), 12); - this.trackingContext = new TrackingContext(trackingContext); - this.trackingContext.setStreamID(outputStreamID); + this.tracingContext = new TracingContext(tracingContext); + this.tracingContext.setStreamID(outputStreamID); } /** @@ -354,7 +345,7 @@ private void writeAppendBlobCurrentBufferToService() throws IOException { "writeCurrentBufferToService", "append")) { AbfsRestOperation op = client.append(path, offset, bytes, 0, bytesLength, cachedSasToken.get(), this.isAppendBlob, - new TrackingContext(trackingContext)); + new TracingContext(tracingContext)); cachedSasToken.update(op.getSasToken()); outputStreamStatistics.uploadSuccessful(bytesLength); perfInfo.registerResult(op.getResult()); @@ -408,7 +399,7 @@ public Void call() throws Exception { "writeCurrentBufferToService", "append")) { AbfsRestOperation op = client.append(path, offset, bytes, 0, bytesLength, cachedSasToken.get(), false, - new TrackingContext(trackingContext)); + new TracingContext(tracingContext)); cachedSasToken.update(op.getSasToken()); perfInfo.registerResult(op.getResult()); byteBufferPool.putBuffer(ByteBuffer.wrap(bytes)); @@ -470,7 +461,7 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset, try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "flushWrittenBytesToServiceInternal", "flush")) { AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, isClose, cachedSasToken.get(), - new TrackingContext(trackingContext)); + new TracingContext(tracingContext)); cachedSasToken.update(op.getSasToken()); perfInfo.registerResult(op.getResult()).registerSuccess(true); } catch (AzureBlobFileSystemException ex) { 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 29f8524cba217..6a4dab822c214 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 @@ -35,7 +35,7 @@ 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; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; /** * The AbfsRestOperation for Rest AbfsClient. @@ -104,7 +104,7 @@ String getSasToken() { * @param url The full URL including query string parameters. * @param requestHeaders The HTTP request headers. */ - public AbfsRestOperation(final AbfsRestOperationType operationType, + AbfsRestOperation(final AbfsRestOperationType operationType, final AbfsClient client, final String method, final URL url, @@ -173,7 +173,7 @@ public AbfsRestOperation(final AbfsRestOperationType operationType, * HTTP operations. */ @VisibleForTesting - public void execute(TrackingContext trackingContext) throws AzureBlobFileSystemException { + public void execute(TracingContext tracingContext) throws AzureBlobFileSystemException { // see if we have latency reports from the previous requests String latencyHeader = this.client.getAbfsPerfTracker().getClientLatency(); if (latencyHeader != null && !latencyHeader.isEmpty()) { @@ -184,10 +184,10 @@ public void execute(TrackingContext trackingContext) throws AzureBlobFileSystemE retryCount = 0; LOG.debug("First execution of REST operation - {}", operationType); - while (!executeHttpOperation(retryCount, trackingContext)) { + while (!executeHttpOperation(retryCount, tracingContext)) { try { ++retryCount; - trackingContext.updateRetryCount(); + tracingContext.updateRetryCount(); LOG.debug("Retrying REST operation {}. RetryCount = {}", operationType, retryCount); Thread.sleep(client.getRetryPolicy().getRetryInterval(retryCount)); @@ -209,17 +209,17 @@ public void execute(TrackingContext trackingContext) throws AzureBlobFileSystemE * fails, there may be a retry. The retryCount is incremented with each * attempt. */ - public boolean executeHttpOperation(final int retryCount, TrackingContext trackingContext) + private boolean executeHttpOperation(final int retryCount, TracingContext tracingContext) throws AzureBlobFileSystemException { AbfsHttpOperation httpOperation = null; try { // initialize the HTTP request and open the connection - trackingContext.setClientRequestID(); + tracingContext.generateClientRequestID(); httpOperation = new AbfsHttpOperation(url, method, requestHeaders); incrementCounter(AbfsStatistic.CONNECTIONS_MADE, 1); httpOperation.getConnection() .setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, - trackingContext.toString()); + tracingContext.toString()); switch(client.getAuthType()) { case Custom: diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java index c79d4099612d6..b23367cfbed61 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java @@ -22,7 +22,7 @@ import java.util.concurrent.CountDownLatch; import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import static org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus.READ_FAILED; @@ -37,7 +37,7 @@ class ReadBuffer { private ReadBufferStatus status; // status of the buffer private CountDownLatch latch = null; // signaled when the buffer is done reading, so any client // waiting on this buffer gets unblocked - private TrackingContext trackingContext; + private TracingContext tracingContext; // fields to help with eviction logic private long timeStamp = 0; // tick at which buffer became available to read @@ -55,12 +55,12 @@ public void setStream(AbfsInputStream stream) { this.stream = stream; } - public void setTrackingContext(TrackingContext trackingContext) { - this.trackingContext = new TrackingContext(trackingContext); + public void setTracingContext(TracingContext tracingContext) { + this.tracingContext = new TracingContext(tracingContext); } - public TrackingContext getTrackingContext() { - return trackingContext; + public TracingContext getTracingContext() { + return tracingContext; } public long getOffset() { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java index 42307fbe4db0b..6353974d55021 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java @@ -18,7 +18,7 @@ package org.apache.hadoop.fs.azurebfs.services; import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; //import org.graalvm.compiler.core.common.alloc.Trace; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -98,7 +98,7 @@ private ReadBufferManager() { * @param requestedLength The length to read */ void queueReadAhead(final AbfsInputStream stream, final long requestedOffset, final int requestedLength, - TrackingContext trackingContext) { + TracingContext tracingContext) { if (LOGGER.isTraceEnabled()) { LOGGER.trace("Start Queueing readAhead for {} offset {} length {}", stream.getPath(), requestedOffset, requestedLength); @@ -119,7 +119,7 @@ void queueReadAhead(final AbfsInputStream stream, final long requestedOffset, fi buffer.setRequestedLength(requestedLength); buffer.setStatus(ReadBufferStatus.NOT_AVAILABLE); buffer.setLatch(new CountDownLatch(1)); - buffer.setTrackingContext(trackingContext); + buffer.setTracingContext(tracingContext); Integer bufferIndex = freeList.pop(); // will return a value, since we have checked size > 0 already diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java index 3bb5bb54005f2..21c9d1be76657 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java @@ -62,7 +62,6 @@ public void run() { if (buffer != null) { try { // do the actual read, from the file. - System.out.println(buffer.getTrackingContext().toString()); int bytesRead = buffer.getStream().readRemote( buffer.getOffset(), buffer.getBuffer(), @@ -71,7 +70,7 @@ public void run() { // read-ahead buffer size, make sure a valid length is passed // for remote read Math.min(buffer.getRequestedLength(), buffer.getBuffer().length), - buffer.getTrackingContext()); + buffer.getTracingContext()); bufferManager.doneReading(buffer, ReadBufferStatus.AVAILABLE, bytesRead); // post result back to ReadBufferManager } catch (Exception ex) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java new file mode 100644 index 0000000000000..13b40e26a457f --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -0,0 +1,107 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs.utils; + +import java.util.UUID; + +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; + +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TracingContext { + private String clientCorrelationID; + private final String fileSystemID; + private String clientRequestID = ""; + private String primaryRequestID; + private String streamID = ""; + private int retryCount; + private String hadoopOpName = ""; + + private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); + public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; + public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*"; + + // for non-continuation ops (no primary request id necessary) + public TracingContext(String clientCorrelationID, String fileSystemID, String hadoopOpName) { + this.fileSystemID = fileSystemID; + this.hadoopOpName = hadoopOpName; + validateClientCorrelationID(clientCorrelationID); + streamID = EMPTY_STRING; + retryCount = 0; + primaryRequestID = ""; + } + + // for ops with primary request + public TracingContext(String clientCorrelationID, String fileSystemID, String hadoopOpName, + String primaryRequestID) { + this(clientCorrelationID, fileSystemID, hadoopOpName); + this.primaryRequestID = primaryRequestID; + } + + public TracingContext(TracingContext originalTracingContext) { + this.fileSystemID = originalTracingContext.fileSystemID; + this.streamID = originalTracingContext.streamID; + this.clientCorrelationID = originalTracingContext.clientCorrelationID; + this.hadoopOpName = originalTracingContext.hadoopOpName; + this.retryCount = 0; + this.primaryRequestID = originalTracingContext.primaryRequestID; + } + + public void validateClientCorrelationID(String clientCorrelationID) { + if ((clientCorrelationID.length() > MAX_CLIENT_CORRELATION_ID_LENGTH) + || (!clientCorrelationID.matches(CLIENT_CORRELATION_ID_PATTERN))) { + this.clientCorrelationID = EMPTY_STRING; + LOG.debug( + "Invalid config provided; correlation id not included in header."); + } else if (clientCorrelationID.length() > 0) { + this.clientCorrelationID = clientCorrelationID + ":"; + } else { + this.clientCorrelationID = EMPTY_STRING; + } + } + + public void generateClientRequestID() { + clientRequestID = UUID.randomUUID().toString(); + } + + public void setPrimaryRequestID() { + primaryRequestID = UUID.randomUUID().toString(); + } + + public void setStreamID(String stream) { + streamID = stream; + } + + public void updateRetryCount() { + retryCount++; + } + + public void reset() { + primaryRequestID = EMPTY_STRING; + retryCount = 0; + } + + public String toString() { + return clientCorrelationID + clientRequestID + ":" + fileSystemID + ":" + primaryRequestID + + ":" + streamID + ":" + hadoopOpName + ":" + retryCount; + } + +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java deleted file mode 100644 index 5d8c0d894963a..0000000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TrackingContext.java +++ /dev/null @@ -1,146 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.utils; - -import java.util.UUID; - -import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.commons.lang3.StringUtils; - -public class TrackingContext { - private String clientCorrelationID; - private String fileSystemID; - private String clientRequestID = ""; //UUID.fromString("00000000-0000-0000-0000-000000000000"); - private String primaryRequestID; - private String streamID = ""; - private int retryCount; - private String hadoopOpName = ""; - public boolean firstRequest = false; - - private static final Logger LOG = LoggerFactory.getLogger( - org.apache.hadoop.fs.azurebfs.services.AbfsClient.class); - public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; - public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*"; - - public TrackingContext(String fileSystemID, String hadoopOpName) { - this.fileSystemID = fileSystemID; - this.hadoopOpName = hadoopOpName; - streamID = EMPTY_STRING; - retryCount = 0; - primaryRequestID = ""; - firstRequest = false; - } - - public TrackingContext(String fileSystemID, String streamID, String hadoopOpName) { - this(fileSystemID, hadoopOpName); - this.streamID = streamID; - } - - public TrackingContext(TrackingContext originalTrackingContext) { - this.fileSystemID = originalTrackingContext.fileSystemID; - this.streamID = originalTrackingContext.streamID; - this.clientCorrelationID = originalTrackingContext.clientCorrelationID; - this.hadoopOpName = originalTrackingContext.hadoopOpName; - this.retryCount = 0; - this.firstRequest = originalTrackingContext.firstRequest; - - if (originalTrackingContext.firstRequest) { - this.clientRequestID = originalTrackingContext.primaryRequestID; - this.primaryRequestID = EMPTY_STRING; - } - else { - this.primaryRequestID = originalTrackingContext.primaryRequestID; - this.clientRequestID = UUID.randomUUID().toString(); - } - } - - public void setClientCorrelationID(String clientCorrelationID) { - //validation - if ((clientCorrelationID.length() > MAX_CLIENT_CORRELATION_ID_LENGTH) - || (!clientCorrelationID.matches(CLIENT_CORRELATION_ID_PATTERN))) { - this.clientCorrelationID = EMPTY_STRING; - LOG.debug( - "Invalid config provided; correlation id not included in header."); - } else if (clientCorrelationID.length() > 0) { - this.clientCorrelationID = clientCorrelationID + ":"; - } else { - this.clientCorrelationID = EMPTY_STRING; - } - } - - public void setFileSystemID(String fileSystemID) { - this.fileSystemID = fileSystemID; - } - - public void updateRetryCount() { -// retryCount = count; - retryCount++; - firstRequest = false; //this is necessary for retry in case of first request of continuation ops - } - -// public void setClientRequestID(boolean forceUpdate) { -// if(forceUpdate || clientRequestID.isEmpty()) { -// clientRequestID = UUID.randomUUID().toString(); -// } -// } - - public void setClientRequestID() { - if (clientRequestID.isEmpty() || !firstRequest) { - clientRequestID = UUID.randomUUID().toString(); - } - } - - public String getPrimaryRequestID() { - return primaryRequestID; - } - - public void reset() { - primaryRequestID = EMPTY_STRING; - retryCount = 0; - firstRequest = false; - } - - public void setStreamID(String stream) { - streamID = stream; - } - -// public void setOperation(org.apache.hadoop.fs.azurebfs.services.AbfsRestOperationType op) { -// opName = op.name();//convert -// } -// public void updateIteration() { -// iteration++; -// primaryRequestID = clientRequestID + ":"; -// } - - public void setPrimaryRequestID() { -// this.primaryRequestID = clientRequestID; - primaryRequestID = UUID.randomUUID().toString(); - } - - public String toString() { -// if (iteration) - String operation = hadoopOpName; // + (hadoopOpName == "LS"? Integer.toString(iteration).toString() : ""); - return clientCorrelationID + clientRequestID + ":" + fileSystemID + ":" + primaryRequestID - + ":" + streamID + ":" + operation + ":" + retryCount; - } - -} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index 9ac2b8bee0f7d..71d41993e79ed 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -25,7 +25,7 @@ import java.util.UUID; import java.util.concurrent.Callable; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -145,11 +145,13 @@ protected AbstractAbfsIntegrationTest() throws Exception { public void setup() throws Exception { //Create filesystem first to make sure getWasbFileSystem() can return an existing filesystem. createFileSystem(); + TracingContext tracingContext = new TracingContext(abfsConfig.getClientCorrelationID(), + getFileSystem().getFileSystemID(), "TS"); // Only live account without namespace support can run ABFS&WASB compatibility tests if (!isIPAddress && (abfsConfig.getAuthType(accountName) != AuthType.SAS) - && !abfs.getIsNamespaceEnabled()) { + && !abfs.getIsNamespaceEnabled(tracingContext)) { final URI wasbUri = new URI(abfsUrlToWasbUrl(getTestUrl())); final AzureNativeFileSystemStore azureNativeFileSystemStore = new AzureNativeFileSystemStore(); @@ -181,23 +183,23 @@ public void teardown() throws Exception { if (abfs == null) { return; } - TrackingContext trackingContext = new TrackingContext(abfs.getFileSystemID(), "DL"); + TracingContext tracingContext = new TracingContext(abfs.getFileSystemID(), "DL"); if (usingFilesystemForSASTests) { abfsConfig.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey.name()); AzureBlobFileSystem tempFs = (AzureBlobFileSystem) FileSystem.newInstance(rawConfig); - tempFs.getAbfsStore().deleteFilesystem(trackingContext); + tempFs.getAbfsStore().deleteFilesystem(tracingContext); } else if (!useConfiguredFileSystem) { // Delete all uniquely created filesystem from the account final AzureBlobFileSystemStore abfsStore = abfs.getAbfsStore(); - abfsStore.deleteFilesystem(trackingContext); + abfsStore.deleteFilesystem(tracingContext); AbfsRestOperationException ex = intercept(AbfsRestOperationException.class, new Callable>() { @Override public Hashtable call() throws Exception { - return abfsStore.getFilesystemProperties(trackingContext); + return abfsStore.getFilesystemProperties(tracingContext); } }); if (FILE_SYSTEM_NOT_FOUND.getStatusCode() != ex.getStatusCode()) { @@ -432,7 +434,7 @@ protected AbfsOutputStream createAbfsOutputStreamWithFlushEnabled( return (AbfsOutputStream) abfss.createFile(path, fs.getFsStatistics(), true, FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()), - new TrackingContext(fs.getFileSystemID(), "OP")); + new TracingContext(fs.getFileSystemID(), "OP")); } /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java index 51229d69e3a37..8d7be9c9ac2a1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java @@ -27,7 +27,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Ignore; @@ -74,7 +74,7 @@ public void testContinuationTokenHavingEqualSign() throws Exception { try { AbfsRestOperation op = abfsClient.listPath("/", true, LIST_MAX_RESULTS, "===========", - new TrackingContext(fs.getFileSystemID(), "PA")); + new TracingContext("test-corr-id", fs.getFileSystemID(), "PA")); Assert.assertTrue(false); } catch (AbfsRestOperationException ex) { Assert.assertEquals("InvalidQueryParameterValue", ex.getErrorCode().getErrorCode()); @@ -124,34 +124,41 @@ public void checkRequest(String clientCorrelationId, boolean includeInHeader) final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem .newInstance(this.getFileSystem().getUri(), config); AbfsClient client = fs.getAbfsClient(); -// TrackingContext +// TracingContext String path = getRelativePath(new Path("/testDir")); boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(); String permission = isNamespaceEnabled ? getOctalNotation(FsPermission.getDirDefault()) : null; String umask = isNamespaceEnabled ? getOctalNotation(FsPermission.getUMask(fs.getConf())) : null; AbfsRestOperation op = client.createPath(path, false, true, permission, umask, false, null, - new TrackingContext(fs.getFileSystemID(), "CR")); + new TracingContext(clientCorrelationId, + fs.getFileSystemID(), "CR")); int responseCode = op.getResult().getStatusCode(); - assertEquals("Status code", HTTP_CREATED, responseCode); + Assertions.assertThat(responseCode).describedAs("Status code").isEqualTo(HTTP_CREATED); + +// op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); + Assertions.assertThat(responseCode).describedAs("Status code").isEqualTo(HTTP_CREATED); - op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); String requestHeader = op.getResult().getRequestHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); List clientRequestIds = java.util.Arrays.asList( requestHeader.replace("[","") .replace("]", "") .split(":")); if (includeInHeader) { - assertEquals("There should be 7 items in the header when valid clientCorrelationId is set", - 7, clientRequestIds.size()); - assertTrue("clientCorrelation should be included in the header", - clientRequestIds.contains(clientCorrelationId)); + Assertions.assertThat(clientRequestIds) + .describedAs("There should be 7 items in the header when valid clientCorrelationId is set") + .hasSize(7); + Assertions.assertThat(clientRequestIds) + .describedAs("clientCorrelation should be included in the header") + .contains(clientCorrelationId); } else if (clientCorrelationId.length() > 0){ - assertEquals("There should be only 6 item in the header when invalid clientCorrelationId is set", - 6, clientRequestIds.size()); - assertFalse("Invalid or empty correlationId value should not be included in header", - clientRequestIds.contains(clientCorrelationId)); + Assertions.assertThat(clientRequestIds) + .describedAs("There should be only 6 item in the header when invalid clientCorrelationId is set") + .hasSize(6); + Assertions.assertThat(clientRequestIds) + .describedAs("Invalid or empty correlationId value should not be included in header") + .doesNotContain(clientCorrelationId); } } @@ -204,8 +211,8 @@ private List listPath(String directory) throws IOException { return getFileSystem().getAbfsClient() .listPath(directory, false, getListMaxResults(), null, - new TrackingContext(getFileSystem().getFileSystemID(), "PA")).getResult() - .getListResultSchema().paths(); + new TracingContext("test-corr-id", getFileSystem().getFileSystemID(), + "PA")).getResult().getListResultSchema().paths(); } private int getListMaxResults() throws IOException { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java index f8f5fb0aecf97..67500d98513a7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java @@ -20,7 +20,7 @@ import java.io.IOException; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.assertj.core.api.Assertions; import org.junit.Test; import org.slf4j.Logger; @@ -64,7 +64,7 @@ public void testInitValues() throws IOException { outputStream = createAbfsOutputStreamWithFlushEnabled(fs, initValuesPath); inputStream = abfss.openFileForRead(initValuesPath, fs.getFsStatistics(), - new TrackingContext("test-filesystem-id", "NS")); + new TracingContext("test-filesystem-id", "NS")); AbfsInputStreamStatisticsImpl stats = (AbfsInputStreamStatisticsImpl) inputStream.getStreamStatistics(); @@ -109,7 +109,7 @@ public void testSeekStatistics() throws IOException { out.write(defBuffer); out.hflush(); in = abfss.openFileForRead(seekStatPath, fs.getFsStatistics(), - new TrackingContext(fs.getFileSystemID(), "IN")); + new TracingContext(fs.getFileSystemID(), "IN")); /* * Writing 1MB buffer to the file, this would make the fCursor(Current @@ -201,7 +201,7 @@ public void testReadStatistics() throws IOException { out.write(defBuffer); out.hflush(); in = abfss.openFileForRead(readStatPath, fs.getFsStatistics(), - new TrackingContext("test-filesystem-id", "NS")); + new TracingContext("test-filesystem-id", "NS")); /* * Doing file read 10 times. @@ -271,16 +271,20 @@ public void testWithNullStreamStatistics() throws IOException { out.write(oneKbBuff); out.hflush(); + TracingContext tracingContext = new TracingContext("testCorrId", + fs.getFileSystemID(), "PA"); + // AbfsRestOperation Instance required for eTag. AbfsRestOperation abfsRestOperation = fs.getAbfsClient().getPathStatus(nullStatFilePath.toUri().getPath(), false, - new TrackingContext(fs.getFileSystemID(), "PA")); + tracingContext); // AbfsInputStream with no StreamStatistics. in = new AbfsInputStream(fs.getAbfsClient(), null, nullStatFilePath.toUri().getPath(), ONE_KB, abfsInputStreamContext, - abfsRestOperation.getResult().getResponseHeader("ETag")); + abfsRestOperation.getResult().getResponseHeader("ETag"), + tracingContext); // Verifying that AbfsInputStream Operations works with null statistics. assertNotEquals("AbfsInputStream read() with null statistics should " @@ -324,7 +328,7 @@ public void testReadAheadCounters() throws IOException { out.close(); in = abfss.openFileForRead(readAheadCountersPath, fs.getFsStatistics(), - new TrackingContext("test-filesystem-id", "NS")); + new TracingContext("test-filesystem-id", "NS")); /* * Reading 1KB after each i * KB positions. Hence the reads are from 0 diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java index 393e3d4346b39..4189d666e7a70 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java @@ -73,7 +73,6 @@ private void setTestUserFs() throws Exception { return; } String orgClientId = getConfiguration().get(FS_AZURE_BLOB_FS_CLIENT_ID); - System.out.println(orgClientId); String orgClientSecret = getConfiguration() .get(FS_AZURE_BLOB_FS_CLIENT_SECRET); Boolean orgCreateFileSystemDurungInit = getConfiguration() diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 8ad2ce2316f95..e63aef513022e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -25,7 +25,7 @@ import java.util.EnumSet; import java.util.UUID; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Test; import org.apache.hadoop.conf.Configuration; @@ -344,11 +344,11 @@ public void testNegativeScenariosForCreateOverwriteDisabled() = TestAbfsClient.getMockAbfsClient( fs.getAbfsStore().getClient(), fs.getAbfsStore().getAbfsConfiguration()); - TrackingContext trackingContext = new TrackingContext(fs.getFileSystemID(), "CR"); + TracingContext tracingContext = new TracingContext(fs.getFileSystemID(), "CR"); AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); abfsStore = setAzureBlobSystemStoreField(abfsStore, "client", mockClient); - boolean isNamespaceEnabled = abfsStore.getIsNamespaceEnabled(trackingContext); + boolean isNamespaceEnabled = abfsStore.getIsNamespaceEnabled(tracingContext); AbfsRestOperation successOp = mock( AbfsRestOperation.class); @@ -379,14 +379,14 @@ public void testNegativeScenariosForCreateOverwriteDisabled() .createPath(any(String.class), eq(true), eq(false), isNamespaceEnabled ? any(String.class) : eq(null), isNamespaceEnabled ? any(String.class) : eq(null), - any(boolean.class), eq(null), any(TrackingContext.class)); + any(boolean.class), eq(null), any(TracingContext.class)); doThrow(fileNotFoundResponseEx) // Scn1: GFS fails with Http404 .doThrow(serverErrorResponseEx) // Scn2: GFS fails with Http500 .doReturn(successOp) // Scn3: create overwrite=true fails with Http412 .doReturn(successOp) // Scn4: create overwrite=true fails with Http500 .when(mockClient) - .getPathStatus(any(String.class), eq(false), any(TrackingContext.class)); + .getPathStatus(any(String.class), eq(false), any(TracingContext.class)); // mock for overwrite=true doThrow( @@ -397,7 +397,7 @@ public void testNegativeScenariosForCreateOverwriteDisabled() .createPath(any(String.class), eq(true), eq(true), isNamespaceEnabled ? any(String.class) : eq(null), isNamespaceEnabled ? any(String.class) : eq(null), - any(boolean.class), eq(null), any(TrackingContext.class)); + any(boolean.class), eq(null), any(TracingContext.class)); // Scn1: GFS fails with Http404 // Sequence of events expected: @@ -464,7 +464,7 @@ private void validateCreateFileException(final Class ex intercept( exceptionClass, () -> abfsStore.createFile(testPath, null, true, permission, umask, - new TrackingContext("test-filesystem-id", "CR"))); + new TracingContext("test-filesystem-id", "CR"))); } private AbfsRestOperationException getMockAbfsRestOperationException(int status) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index ad03fc82ec01b..af6af31ee09a9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -26,7 +26,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.Test; @@ -213,7 +213,7 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { fs.getAbfsStore().getClient(), this.getConfiguration()); - TrackingContext trackingContext = new TrackingContext(fs.getFileSystemID(), "DL"); + TracingContext tracingContext = new TracingContext(fs.getFileSystemID(), "DL"); // Case 1: Not a retried case should throw error back intercept(AbfsRestOperationException.class, @@ -221,7 +221,7 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { "/NonExistingPath", false, null, - trackingContext)); + tracingContext)); // mock idempotency check to mimic retried case AbfsClient mockClient = TestAbfsClient.getMockAbfsClient( @@ -237,13 +237,13 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { doReturn(idempotencyRetOp).when(mockClient).deleteIdempotencyCheckOp(any()); when(mockClient.deletePath("/NonExistingPath", false, - null, trackingContext)).thenCallRealMethod(); + null, tracingContext)).thenCallRealMethod(); Assertions.assertThat(mockClient.deletePath( "/NonExistingPath", false, null, - trackingContext) + tracingContext) .getResult() .getStatusCode()) .describedAs("Idempotency check reports successful " diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java index 997a681f081ae..ab3bb0d0b14eb 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java @@ -22,7 +22,7 @@ import java.io.InputStream; import java.util.Map; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Assume; import org.junit.Test; import org.slf4j.Logger; @@ -124,7 +124,7 @@ public void testBlobDataReader() throws Exception { Assume.assumeTrue("Reader client id not provided", clientId != null); String secret = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET); Assume.assumeTrue("Reader client secret not provided", secret != null); - TrackingContext trackingContext = new TrackingContext("test-fileSystemID", "AU"); + TracingContext tracingContext = new TracingContext("test-fileSystemID", "AU"); prepareFiles(); final AzureBlobFileSystem fs = getBlobReader(); @@ -132,27 +132,27 @@ public void testBlobDataReader() throws Exception { // Use abfsStore in this test to verify the ERROR code in AbfsRestOperationException AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); // TEST READ FS - Map properties = abfsStore.getFilesystemProperties(trackingContext); + Map properties = abfsStore.getFilesystemProperties(tracingContext); // TEST READ FOLDER assertTrue(fs.exists(EXISTED_FOLDER_PATH)); // TEST DELETE FOLDER try { - abfsStore.delete(EXISTED_FOLDER_PATH, true, trackingContext); + abfsStore.delete(EXISTED_FOLDER_PATH, true, tracingContext); } catch (AbfsRestOperationException e) { assertEquals(AzureServiceErrorCode.AUTHORIZATION_PERMISSION_MISS_MATCH, e.getErrorCode()); } // TEST READ FILE try (InputStream inputStream = abfsStore.openFileForRead(EXISTED_FILE_PATH, null, - trackingContext)) { + tracingContext)) { assertTrue(inputStream.read() != 0); } // TEST WRITE FILE try { abfsStore.openFileForWrite(EXISTED_FILE_PATH, fs.getFsStatistics(), true, - trackingContext); + tracingContext); } catch (AbfsRestOperationException e) { assertEquals(AzureServiceErrorCode.AUTHORIZATION_PERMISSION_MISS_MATCH, e.getErrorCode()); } finally { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java index b4ec055c8e424..f9301e802e12f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java @@ -26,7 +26,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.assertj.core.api.Assertions; import org.junit.Test; import org.junit.Assert; @@ -238,13 +238,13 @@ private void testRenameIdempotencyTriggerChecks( fs.getAbfsStore().getClient(), this.getConfiguration()); - TrackingContext trackingContext = new TrackingContext(fs.getFileSystemID(), "RN"); + TracingContext tracingContext = new TracingContext(fs.getFileSystemID(), "RN"); AbfsRestOperation idempotencyRetOp = mock(AbfsRestOperation.class); when(idempotencyRetOp.getResult()).thenReturn(idempotencyRetHttpOp); doReturn(idempotencyRetOp).when(client).renameIdempotencyCheckOp(any(), - any(), any(), trackingContext); - when(client.renamePath(any(), any(), any(), trackingContext)).thenCallRealMethod(); + any(), any(), tracingContext); + when(client.renamePath(any(), any(), any(), tracingContext)).thenCallRealMethod(); // rename on non-existing source file will trigger idempotency check if (idempotencyRetHttpOp.getStatusCode() == HTTP_OK) { @@ -253,7 +253,7 @@ private void testRenameIdempotencyTriggerChecks( "/NonExistingsourcepath", "/destpath", null, - trackingContext) + tracingContext) .getResult() .getStatusCode()) .describedAs("Idempotency check reports recent successful " @@ -266,7 +266,7 @@ private void testRenameIdempotencyTriggerChecks( "/NonExistingsourcepath", "/destpath", "", - trackingContext)); + tracingContext)); } } @@ -327,7 +327,7 @@ private void testRenameTimeout( renameRequestStartTime, op, destinationPath.toUri().getPath(), - new TrackingContext("test-fs-id", "RN")) + new TracingContext("test-fs-id", "RN")) .getResult() .getStatusCode()) .describedAs(assertMessage) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java index 07e9da1bfd90e..4500c121b7eb6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.Arrays; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -108,7 +109,8 @@ public ITestAzureBlobFileSystemStoreListStatusWithRange() throws Exception { public void testListWithRange() throws IOException { try { FileStatus[] listResult = store.listStatus(new Path(path), startFrom, - new org.apache.hadoop.fs.azurebfs.utils.TrackingContext("test-filesystem-id", "LS")); + new TracingContext("test-corr-id", + "test-filesystem-id", "LS")); if (!expectedResult) { Assert.fail("Excepting failure with IllegalArgumentException"); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java index d9e17ddb8100d..4b3805f058606 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java @@ -6,7 +6,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.http.HttpException; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; @@ -59,7 +59,7 @@ public void testRetryCount() throws IOException { // AbfsRestOperation op = PowerMock.spy(AbfsRestOperation.class, "executeHttpOperation"); // PowerMockito.doThrow(new AbfsRestOperationException(new HttpException())) // .when(op, "executeHttpOperation", any(int), -// any(TrackingContext.class)); +// any(TracingContext.class)); AbfsClient client = mock(AbfsClient.class); @@ -73,11 +73,11 @@ public void testRetryCount() throws IOException { // AbfsRestOperation op = spy(restop); - when(op.executeHttpOperation(any(int.class), any(TrackingContext.class))) - .thenReturn(false); +// when(op.executeHttpOperation(any(int.class), any(TracingContext.class))) +// .thenReturn(false); // .thenThrow(AzureBlobFileSystemException.class); - op.execute(new org.apache.hadoop.fs.azurebfs.utils.TrackingContext("fsid","op")); + op.execute(new TracingContext("test-corr-id", "fsid","op")); String path = getRelativePath(new Path("/testDir")); boolean isNamespaceEnabled = true;//fs.getIsNamespaceEnabled(); @@ -87,10 +87,10 @@ public void testRetryCount() throws IOException { // AbfsClient client = mock(AbfsClient.class); when(client.createPath(any(String.class), eq(true), eq(false), eq(null), eq(null), - any(boolean.class), eq(null), any(TrackingContext.class))).thenReturn(op); + any(boolean.class), eq(null), any(TracingContext.class))).thenReturn(op); AbfsRestOperation op1 = client.createPath(path, false, true, permission, umask, false, null, - new TrackingContext("fsid", "CR")); + new TracingContext("test-corr-id", "fsid", "CR")); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java index 7c487a906fb4b..e7e6b7bd0f924 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java @@ -26,7 +26,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; /** * Test FileSystemProperties. @@ -62,12 +62,14 @@ public void testWriteOneByteToFileAndEnsureThreadPoolCleanup() throws Exception @Test public void testBase64FileSystemProperties() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); + TracingContext tracingContext = new TracingContext(fs.getAbfsStore().getAbfsConfiguration() + .getClientCorrelationID(), fs.getFileSystemID(), "FS"); final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: value }"); - fs.getAbfsStore().setFilesystemProperties(properties); + fs.getAbfsStore().setFilesystemProperties(properties, tracingContext); Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties( - new TrackingContext(fs.getFileSystemID(), "FS")); + tracingContext); assertEquals(properties, fetchedProperties); } @@ -75,13 +77,14 @@ public void testBase64FileSystemProperties() throws Exception { @Test public void testBase64PathProperties() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); + TracingContext tracingContext = new TracingContext(fs.getAbfsStore().getAbfsConfiguration() + .getClientCorrelationID(), fs.getFileSystemID(), "FS"); final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueTest }"); touch(TEST_PATH); - TrackingContext trackingContext = new TrackingContext(fs.getFileSystemID(), "PA"); - fs.getAbfsStore().setPathProperties(TEST_PATH, properties, trackingContext); + fs.getAbfsStore().setPathProperties(TEST_PATH, properties, tracingContext); Hashtable fetchedProperties = - fs.getAbfsStore().getPathStatus(TEST_PATH, trackingContext); + fs.getAbfsStore().getPathStatus(TEST_PATH, tracingContext); assertEquals(properties, fetchedProperties); } @@ -89,12 +92,13 @@ public void testBase64PathProperties() throws Exception { @Test (expected = Exception.class) public void testBase64InvalidFileSystemProperties() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); - TrackingContext trackingContext = new TrackingContext(fs.getFileSystemID(), "PA"); + TracingContext tracingContext = new TracingContext(fs.getAbfsStore().getAbfsConfiguration() + .getClientCorrelationID(), fs.getFileSystemID(), "FS"); final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueæ­² }"); - fs.getAbfsStore().setFilesystemProperties(properties); + fs.getAbfsStore().setFilesystemProperties(properties, tracingContext); Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties( - trackingContext); + tracingContext); assertEquals(properties, fetchedProperties); } @@ -102,13 +106,14 @@ public void testBase64InvalidFileSystemProperties() throws Exception { @Test (expected = Exception.class) public void testBase64InvalidPathProperties() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); + TracingContext tracingContext = new TracingContext(fs.getAbfsStore().getAbfsConfiguration() + .getClientCorrelationID(), fs.getFileSystemID(), "FS"); final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueTestå…© }"); - TrackingContext trackingContext = new TrackingContext(fs.getFileSystemID(), "PA"); touch(TEST_PATH); - fs.getAbfsStore().setPathProperties(TEST_PATH, properties, trackingContext); + fs.getAbfsStore().setPathProperties(TEST_PATH, properties, tracingContext); Hashtable fetchedProperties = fs.getAbfsStore().getPathStatus(TEST_PATH, - trackingContext); + tracingContext); assertEquals(properties, fetchedProperties); } @@ -116,11 +121,13 @@ public void testBase64InvalidPathProperties() throws Exception { @Test public void testSetFileSystemProperties() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); + TracingContext tracingContext = new TracingContext(fs.getAbfsStore().getAbfsConfiguration() + .getClientCorrelationID(), fs.getFileSystemID(), "FS"); final Hashtable properties = new Hashtable<>(); properties.put("containerForDevTest", "true"); - fs.getAbfsStore().setFilesystemProperties(properties); + fs.getAbfsStore().setFilesystemProperties(properties, tracingContext); Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties( - new TrackingContext(fs.getFileSystemID(), "FS")); + tracingContext); assertEquals(properties, fetchedProperties); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java index 42257f9ef4137..3243574853e6a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java @@ -22,7 +22,7 @@ import java.util.UUID; import org.apache.hadoop.fs.azurebfs.enums.Trilean; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Assume; import org.junit.Test; import org.assertj.core.api.Assertions; @@ -84,7 +84,7 @@ public void testGetIsNamespaceEnabledWhenConfigIsTrue() throws Exception { Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs( "getIsNamespaceEnabled should return true when the " + "config is set as true").isTrue(); - fs.getAbfsStore().deleteFilesystem(new TrackingContext(fileSystemID, "DL")); + fs.getAbfsStore().deleteFilesystem(new TracingContext(fileSystemID, "DL")); unsetAndAssert(); } @@ -94,7 +94,7 @@ public void testGetIsNamespaceEnabledWhenConfigIsFalse() throws Exception { Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs( "getIsNamespaceEnabled should return false when the " + "config is set as false").isFalse(); - fs.getAbfsStore().deleteFilesystem(new TrackingContext(fileSystemID, "DL")); + fs.getAbfsStore().deleteFilesystem(new TracingContext(fileSystemID, "DL")); unsetAndAssert(); } @@ -107,7 +107,7 @@ private void unsetAndAssert() throws Exception { "getIsNamespaceEnabled should return the value " + "configured for fs.azure.test.namespace.enabled") .isEqualTo(expectedValue); - fs.getAbfsStore().deleteFilesystem(new TrackingContext(fileSystemID, "DL")); + fs.getAbfsStore().deleteFilesystem(new TracingContext(fileSystemID, "DL")); } private AzureBlobFileSystem getNewFSWithHnsConf( @@ -181,7 +181,7 @@ private void ensureGetAclCallIsMadeOnceForInvalidConf(String invalidConf) AbfsClient mockClient = callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient(); verify(mockClient, times(1)).getAclStatus(anyString(), - new TrackingContext("test-filesystem-id", "NS")); + new TracingContext("test-filesystem-id", "NS")); } private void ensureGetAclCallIsNeverMadeForValidConf(String validConf) @@ -191,7 +191,7 @@ private void ensureGetAclCallIsNeverMadeForValidConf(String validConf) AbfsClient mockClient = callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient(); verify(mockClient, never()).getAclStatus(anyString(), - new TrackingContext("test-filesystem-id", "NS")); + new TracingContext("test-filesystem-id", "NS")); } private void unsetConfAndEnsureGetAclCallIsMadeOnce() throws IOException { @@ -199,7 +199,7 @@ private void unsetConfAndEnsureGetAclCallIsMadeOnce() throws IOException { AbfsClient mockClient = callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient(); verify(mockClient, times(1)).getAclStatus(anyString(), - new TrackingContext("test-filesystem-id", "NS")); + new TracingContext("test-filesystem-id", "NS")); } private AbfsClient callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient() @@ -208,7 +208,7 @@ private AbfsClient callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient() final AzureBlobFileSystemStore abfsStore = abfs.getAbfsStore(); final AbfsClient mockClient = mock(AbfsClient.class); doReturn(mock(AbfsRestOperation.class)).when(mockClient) - .getAclStatus(anyString(), new TrackingContext("test-filesystem-id", "NS")); + .getAclStatus(anyString(), new TracingContext("test-filesystem-id", "NS")); abfsStore.setClient(mockClient); abfs.getIsNamespaceEnabled(); return mockClient; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java index 901bee4444f0c..ebc58e53d4af1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.fs.azurebfs; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Assume; import org.junit.Test; @@ -55,8 +55,8 @@ public void testWithWrongSharedKey() throws Exception { + "Authorization header is formed correctly including the " + "signature.\", 403", () -> { - abfsClient.getAclStatus("/", new TrackingContext("test-fileSystemID", - "AC")); + abfsClient.getAclStatus("/", new TracingContext("test-corr-id", + "test-fileSystemID", "AC")); }); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java index e6ae3ac84543b..80eec4dac0c39 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java @@ -37,7 +37,7 @@ import org.apache.hadoop.fs.azurebfs.utils.Base64; import org.apache.hadoop.fs.azurebfs.utils.DelegationSASGenerator; import org.apache.hadoop.fs.azurebfs.utils.SASGenerator; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.security.AccessControlException; /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index 9489a940f6bfd..6b190035e840b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -31,7 +31,7 @@ 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.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import static org.assertj.core.api.Assertions.assertThat; @@ -336,10 +336,10 @@ public static AbfsClient getMockAbfsClient(AbfsClient baseAbfsClientInstance, client = TestAbfsClient.setAbfsClientField(client, "userAgent", userAgent); // set tracking context -// TrackingContext trackingContext = new TrackingContext("clientCorrID", "TS"); -// client = TestAbfsClient.setAbfsClientField(client, "trackingContext", -// trackingContext); -// when(client.getTrackingContext()).thenReturn(trackingContext); +// TracingContext tracingContext = new TracingContext("clientCorrID", "TS"); +// client = TestAbfsClient.setAbfsClientField(client, "tracingContext", +// tracingContext); +// when(client.getTracingContext()).thenReturn(tracingContext); return client; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index 2095baa10dcb6..c9b547dc74c11 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -20,7 +20,7 @@ import java.io.IOException; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Assert; import org.junit.Test; @@ -72,7 +72,7 @@ private AbfsClient getMockAbfsClient() { return client; } - private AbfsInputStream getAbfsInputStream(AbfsClient mockAbfsClient, String fileName) { + private AbfsInputStream getAbfsInputStream(AbfsClient mockAbfsClient, String fileName) throws IOException { AbfsInputStreamContext inputStreamContext = new AbfsInputStreamContext(-1); // Create AbfsInputStream with the client instance AbfsInputStream inputStream = new AbfsInputStream( @@ -81,7 +81,9 @@ private AbfsInputStream getAbfsInputStream(AbfsClient mockAbfsClient, String fil FORWARD_SLASH + fileName, THREE_KB, inputStreamContext.withReadBufferSize(ONE_KB).withReadAheadQueueDepth(10), - "eTag"); + "eTag", + new TracingContext("testCorrId", + getFileSystem().getFileSystemID(), "IP")); inputStream.setCachedSasToken( TestCachedSASToken.getTestCachedSASTokenInstance()); @@ -92,11 +94,11 @@ private AbfsInputStream getAbfsInputStream(AbfsClient mockAbfsClient, String fil private void queueReadAheads(AbfsInputStream inputStream) { // Mimic AbfsInputStream readAhead queue requests ReadBufferManager.getBufferManager() - .queueReadAhead(inputStream, 0, ONE_KB, inputStream.trackingContext); + .queueReadAhead(inputStream, 0, ONE_KB, inputStream.tracingContext); ReadBufferManager.getBufferManager() - .queueReadAhead(inputStream, ONE_KB, ONE_KB, inputStream.trackingContext); + .queueReadAhead(inputStream, ONE_KB, ONE_KB, inputStream.tracingContext); ReadBufferManager.getBufferManager() - .queueReadAhead(inputStream, TWO_KB, TWO_KB, inputStream.trackingContext); + .queueReadAhead(inputStream, TWO_KB, TWO_KB, inputStream.tracingContext); } private void verifyReadCallCount(AbfsClient client, int count) throws @@ -107,7 +109,7 @@ private void verifyReadCallCount(AbfsClient client, int count) throws verify(client, times(count)).read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - any(TrackingContext.class)); + any(TracingContext.class)); } private void checkEvictedStatus(AbfsInputStream inputStream, int position, boolean expectedToThrowException) @@ -163,7 +165,7 @@ public void testFailedReadAhead() throws Exception { .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - any(TrackingContext.class)); + any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testFailedReadAhead.txt"); @@ -211,7 +213,7 @@ public void testOlderReadAheadFailure() throws Exception { .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - any(TrackingContext.class)); + any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testOlderReadAheadFailure.txt"); @@ -266,7 +268,7 @@ public void testSuccessfulReadAhead() throws Exception { .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - any(TrackingContext.class)); + any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testSuccessfulReadAhead.txt"); @@ -312,7 +314,7 @@ public void testReadAheadManagerForFailedReadAhead() throws Exception { .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - any(TrackingContext.class)); + any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testReadAheadManagerForFailedReadAhead.txt"); @@ -366,7 +368,7 @@ public void testReadAheadManagerForOlderReadAheadFailure() throws Exception { .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - any(TrackingContext.class)); + any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testReadAheadManagerForOlderReadAheadFailure.txt"); @@ -421,7 +423,7 @@ public void testReadAheadManagerForSuccessfulReadAhead() throws Exception { .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), any(String.class), - any(TrackingContext.class)); + any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testSuccessfulReadAhead.txt"); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index 5d5b98c927cfd..74023a74cbf83 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -23,7 +23,7 @@ import java.util.HashSet; import java.util.Random; -import org.apache.hadoop.fs.azurebfs.utils.TrackingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Test; import org.mockito.ArgumentCaptor; @@ -81,14 +81,18 @@ public void verifyShortWriteRequest() throws Exception { AbfsConfiguration abfsConf; final Configuration conf = new Configuration(); conf.set(accountKey1, accountValue1); + TracingContext tracingContext = new TracingContext("test-corr-id", + "test-fs-id", "WR"); abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), - new TrackingContext("test-filesystem-id", "OP"))).thenReturn(op); + any(TracingContext.class))).thenReturn(op); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, + populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, + false), tracingContext); final byte[] b = new byte[WRITE_SIZE]; new Random().nextBytes(b); out.write(b); @@ -132,13 +136,17 @@ public void verifyWriteRequest() throws Exception { conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); + TracingContext tracingContext = new TracingContext("test-corr-id", + "test-fs-id", "OP") when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), - new TrackingContext("test-filesystem-id", "OP"))).thenReturn(op); + any(TracingContext.class))).thenReturn(op); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, + populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, + false), tracingContext); final byte[] b = new byte[WRITE_SIZE]; new Random().nextBytes(b); @@ -171,8 +179,7 @@ public void verifyWriteRequest() throws Exception { ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); verify(client, times(1)).flush(acFlushString.capture(), acFlushLong.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture(), - new TrackingContext("test-filesystem-id", "OP")); + acFlushSASToken.capture(), tracingContext); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushString.getAllValues()); assertThat(Arrays.asList(Long.valueOf(5*WRITE_SIZE))).describedAs("position").isEqualTo(acFlushLong.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); @@ -193,15 +200,19 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); + TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), + "test-fs-id", "OP"); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), - new TrackingContext("test-filesystem-id", "OP"))).thenReturn(op); + any(TracingContext.class))).thenReturn(op); when(op.getSasToken()).thenReturn("testToken"); when(op.getResult()).thenReturn(httpOp); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, + populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, + false), tracingContext); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -233,8 +244,7 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); verify(client, times(1)).flush(acFlushString.capture(), acFlushLong.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture(), - new TrackingContext("test-filesystem-id", "OP")); + acFlushSASToken.capture(), tracingContext); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushString.getAllValues()); assertThat(Arrays.asList(Long.valueOf(2*BUFFER_SIZE))).describedAs("position").isEqualTo(acFlushLong.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); @@ -260,11 +270,13 @@ public void verifyWriteRequestOfBufferSize() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), - new TrackingContext("test-filesystem-id", "OP"))).thenReturn(op); + any(TracingContext.class))).thenReturn(op); when(op.getSasToken()).thenReturn("testToken"); when(op.getResult()).thenReturn(httpOp); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, + populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, + false), new TracingContext("corr-id", "fs-id", "WR")); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -308,9 +320,11 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), - new TrackingContext("test-filesystem-id", "OP"))).thenReturn(op); + any(TracingContext.class))).thenReturn(op); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, true)); + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, + populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, + true), new TracingContext("fs-id")); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -350,13 +364,17 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); + TracingContext tracingContext = new TracingContext("test-corr-id", + "test-fs-id", "PA"); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), - new TrackingContext("test-filesystem-id", "OP"))).thenReturn(op); + any(TracingContext.class))).thenReturn(op); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext( + BUFFER_SIZE, true, false, false), + tracingContext); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -388,7 +406,7 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); verify(client, times(1)).flush(acFlushString.capture(), acFlushLong.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture(), new TrackingContext("test-filesystem-id", "OP")); + acFlushSASToken.capture(), new TracingContext("test-filesystem-id", "OP")); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushString.getAllValues()); assertThat(Arrays.asList(Long.valueOf(2*BUFFER_SIZE))).describedAs("position").isEqualTo(acFlushLong.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); @@ -408,13 +426,17 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { final Configuration conf = new Configuration(); conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); + TracingContext tracingContext = new TracingContext("test-corr-id", + "test-fs-id", "WR"); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), - new TrackingContext("test-filesystem-id", "OP"))).thenReturn(op); + any(TracingContext.class))).thenReturn(op); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false)); + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, + populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, + false), tracingContext); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); diff --git a/pom.xml b/pom.xml index 5565562425e32..9388817a7f5a0 100644 --- a/pom.xml +++ b/pom.xml @@ -77,27 +77,10 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x Apache Software Foundation https://www.apache.org - - - - - org.powermock - powermock-module-junit4 - ${powermock.version} - test - - - org.powermock - powermock-api-mockito - ${powermock.version} - test - - 3.4.0-SNAPSHOT - 2.0.2 apache.snapshots.https Apache Development Snapshot Repository From 166f99cd6239cab03076e6dd02192eebd3f79165 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Sun, 18 Oct 2020 18:44:34 -0700 Subject: [PATCH 18/77] fix errors --- .../hadoop/fs/azurebfs/AzureBlobFileSystem.java | 2 +- .../hadoop/fs/azurebfs/services/AbfsClient.java | 2 +- .../azurebfs/AbstractAbfsIntegrationTest.java | 6 ++++-- .../ITestAbfsInputStreamStatistics.java | 8 ++++---- .../ITestAzureBlobFileSystemCreate.java | 6 ++++-- .../ITestAzureBlobFileSystemDelete.java | 3 ++- .../azurebfs/ITestAzureBlobFileSystemOauth.java | 3 ++- .../ITestAzureBlobFileSystemRename.java | 6 ++++-- .../azurebfs/ITestClientCorrelationHeader.java | 4 ++-- .../fs/azurebfs/ITestGetNameSpaceEnabled.java | 17 ++++++++++------- .../azurebfs/services/TestAbfsOutputStream.java | 8 +++++--- 11 files changed, 39 insertions(+), 26 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index cdbebd8e10a10..74e64a4ed6a82 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -110,7 +110,6 @@ public void initialize(URI uri, Configuration configuration) super.initialize(uri, configuration); setConf(configuration); fileSystemID = UUID.randomUUID().toString(); - clientCorrelationID = abfsStore.getAbfsConfiguration().getClientCorrelationID(); LOG.debug("Initializing AzureBlobFileSystem for {}", uri); @@ -121,6 +120,7 @@ public void initialize(URI uri, Configuration configuration) LOG.trace("AzureBlobFileSystemStore init complete"); final AbfsConfiguration abfsConfiguration = abfsStore.getAbfsConfiguration(); + clientCorrelationID = abfsConfiguration.getClientCorrelationID(); this.setWorkingDirectory(this.getHomeDirectory()); if (abfsConfiguration.getCreateRemoteFileSystemDuringInitialization()) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 98ade13233e61..345ac8ffdcc61 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -403,7 +403,7 @@ public AbfsRestOperation renameIdempotencyCheckOp( public AbfsRestOperation append(final String path, final long position, final byte[] buffer, final int offset, final int length, final String cachedSasToken, final boolean isAppendBlob) throws AzureBlobFileSystemException { return append(path, position, buffer, offset, length, cachedSasToken, - isAppendBlob, new TracingContext("test-filesystem-id", "AP")); + isAppendBlob, new TracingContext("test-corr-id", "test-filesystem-id", "AP")); } public AbfsRestOperation append(final String path, final long position, final byte[] buffer, final int offset, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index 71d41993e79ed..be65ff1f6263b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -183,7 +183,8 @@ public void teardown() throws Exception { if (abfs == null) { return; } - TracingContext tracingContext = new TracingContext(abfs.getFileSystemID(), "DL"); + TracingContext tracingContext = new TracingContext(abfs.getAbfsStore() + .getAbfsConfiguration().getClientCorrelationID(), abfs.getFileSystemID(), "DL"); if (usingFilesystemForSASTests) { abfsConfig.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey.name()); @@ -434,7 +435,8 @@ protected AbfsOutputStream createAbfsOutputStreamWithFlushEnabled( return (AbfsOutputStream) abfss.createFile(path, fs.getFsStatistics(), true, FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()), - new TracingContext(fs.getFileSystemID(), "OP")); + new TracingContext(fs.getAbfsStore().getAbfsConfiguration() + .getClientCorrelationID(), fs.getFileSystemID(), "OP")); } /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java index 67500d98513a7..71ef60c98b85d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java @@ -64,7 +64,7 @@ public void testInitValues() throws IOException { outputStream = createAbfsOutputStreamWithFlushEnabled(fs, initValuesPath); inputStream = abfss.openFileForRead(initValuesPath, fs.getFsStatistics(), - new TracingContext("test-filesystem-id", "NS")); + new TracingContext("test-corr-id", "test-filesystem-id", "NS")); AbfsInputStreamStatisticsImpl stats = (AbfsInputStreamStatisticsImpl) inputStream.getStreamStatistics(); @@ -109,7 +109,7 @@ public void testSeekStatistics() throws IOException { out.write(defBuffer); out.hflush(); in = abfss.openFileForRead(seekStatPath, fs.getFsStatistics(), - new TracingContext(fs.getFileSystemID(), "IN")); + new TracingContext("test-corr-id", fs.getFileSystemID(), "IN")); /* * Writing 1MB buffer to the file, this would make the fCursor(Current @@ -201,7 +201,7 @@ public void testReadStatistics() throws IOException { out.write(defBuffer); out.hflush(); in = abfss.openFileForRead(readStatPath, fs.getFsStatistics(), - new TracingContext("test-filesystem-id", "NS")); + new TracingContext("test-corr-id", "test-filesystem-id", "NS")); /* * Doing file read 10 times. @@ -328,7 +328,7 @@ public void testReadAheadCounters() throws IOException { out.close(); in = abfss.openFileForRead(readAheadCountersPath, fs.getFsStatistics(), - new TracingContext("test-filesystem-id", "NS")); + new TracingContext("test-corr-id", "test-filesystem-id", "NS")); /* * Reading 1KB after each i * KB positions. Hence the reads are from 0 diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index e63aef513022e..085cd72ebe0f6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -344,7 +344,9 @@ public void testNegativeScenariosForCreateOverwriteDisabled() = TestAbfsClient.getMockAbfsClient( fs.getAbfsStore().getClient(), fs.getAbfsStore().getAbfsConfiguration()); - TracingContext tracingContext = new TracingContext(fs.getFileSystemID(), "CR"); + TracingContext tracingContext = new TracingContext( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), + fs.getFileSystemID(), "CR"); AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); abfsStore = setAzureBlobSystemStoreField(abfsStore, "client", mockClient); @@ -464,7 +466,7 @@ private void validateCreateFileException(final Class ex intercept( exceptionClass, () -> abfsStore.createFile(testPath, null, true, permission, umask, - new TracingContext("test-filesystem-id", "CR"))); + new TracingContext("test-corr-id", "test-filesystem-id", "CR"))); } private AbfsRestOperationException getMockAbfsRestOperationException(int status) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index af6af31ee09a9..ffb10f0efd00d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -213,7 +213,8 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { fs.getAbfsStore().getClient(), this.getConfiguration()); - TracingContext tracingContext = new TracingContext(fs.getFileSystemID(), "DL"); + TracingContext tracingContext = new TracingContext(fs.getAbfsStore() + .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), "DL"); // Case 1: Not a retried case should throw error back intercept(AbfsRestOperationException.class, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java index ab3bb0d0b14eb..5bed7243ee984 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java @@ -124,7 +124,8 @@ public void testBlobDataReader() throws Exception { Assume.assumeTrue("Reader client id not provided", clientId != null); String secret = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET); Assume.assumeTrue("Reader client secret not provided", secret != null); - TracingContext tracingContext = new TracingContext("test-fileSystemID", "AU"); + TracingContext tracingContext = new TracingContext("test-corr-id", + "test-fileSystemID", "AU"); prepareFiles(); final AzureBlobFileSystem fs = getBlobReader(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java index f9301e802e12f..e5bc772d5c329 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java @@ -238,7 +238,9 @@ private void testRenameIdempotencyTriggerChecks( fs.getAbfsStore().getClient(), this.getConfiguration()); - TracingContext tracingContext = new TracingContext(fs.getFileSystemID(), "RN"); + TracingContext tracingContext = new TracingContext(fs.getAbfsStore() + .getAbfsConfiguration().getClientCorrelationID(), + fs.getFileSystemID(), "RN"); AbfsRestOperation idempotencyRetOp = mock(AbfsRestOperation.class); when(idempotencyRetOp.getResult()).thenReturn(idempotencyRetHttpOp); @@ -327,7 +329,7 @@ private void testRenameTimeout( renameRequestStartTime, op, destinationPath.toUri().getPath(), - new TracingContext("test-fs-id", "RN")) + new TracingContext("test-corr-id", "test-fs-id", "RN")) .getResult() .getStatusCode()) .describedAs(assertMessage) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java index 4b3805f058606..81a1bd7f93849 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java @@ -52,7 +52,7 @@ public void testFilesystemID() { // all requests using a given filesystem instance should have same fs id // in slot 0 } - +/* @Test // @PrepareForTest(AbfsRestOperation.class) public void testRetryCount() throws IOException { @@ -102,6 +102,6 @@ private String getOctalNotation(FsPermission fsPermission) { private String getRelativePath(final Path path) { Preconditions.checkNotNull(path, "path"); return path.toUri().getPath(); - } + }*/ } \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java index 3243574853e6a..3c6f216273cd1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java @@ -56,6 +56,7 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest { private static final String TRUE_STR = "true"; private static final String FALSE_STR = "false"; private final String fileSystemID = "test-filesystem-id"; + private final String clientCorrelationID = "test-corr-id"; private boolean isUsingXNSAccount; public ITestGetNameSpaceEnabled() throws Exception { @@ -84,7 +85,8 @@ public void testGetIsNamespaceEnabledWhenConfigIsTrue() throws Exception { Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs( "getIsNamespaceEnabled should return true when the " + "config is set as true").isTrue(); - fs.getAbfsStore().deleteFilesystem(new TracingContext(fileSystemID, "DL")); + fs.getAbfsStore().deleteFilesystem(new TracingContext(clientCorrelationID, + fileSystemID, "DL")); unsetAndAssert(); } @@ -94,7 +96,7 @@ public void testGetIsNamespaceEnabledWhenConfigIsFalse() throws Exception { Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs( "getIsNamespaceEnabled should return false when the " + "config is set as false").isFalse(); - fs.getAbfsStore().deleteFilesystem(new TracingContext(fileSystemID, "DL")); + fs.getAbfsStore().deleteFilesystem(new TracingContext(clientCorrelationID, fileSystemID, "DL")); unsetAndAssert(); } @@ -107,7 +109,8 @@ private void unsetAndAssert() throws Exception { "getIsNamespaceEnabled should return the value " + "configured for fs.azure.test.namespace.enabled") .isEqualTo(expectedValue); - fs.getAbfsStore().deleteFilesystem(new TracingContext(fileSystemID, "DL")); + fs.getAbfsStore().deleteFilesystem(new TracingContext(clientCorrelationID, + fileSystemID, "DL")); } private AzureBlobFileSystem getNewFSWithHnsConf( @@ -181,7 +184,7 @@ private void ensureGetAclCallIsMadeOnceForInvalidConf(String invalidConf) AbfsClient mockClient = callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient(); verify(mockClient, times(1)).getAclStatus(anyString(), - new TracingContext("test-filesystem-id", "NS")); + new TracingContext(clientCorrelationID, fileSystemID, "NS")); } private void ensureGetAclCallIsNeverMadeForValidConf(String validConf) @@ -191,7 +194,7 @@ private void ensureGetAclCallIsNeverMadeForValidConf(String validConf) AbfsClient mockClient = callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient(); verify(mockClient, never()).getAclStatus(anyString(), - new TracingContext("test-filesystem-id", "NS")); + new TracingContext(clientCorrelationID, fileSystemID, "NS")); } private void unsetConfAndEnsureGetAclCallIsMadeOnce() throws IOException { @@ -199,7 +202,7 @@ private void unsetConfAndEnsureGetAclCallIsMadeOnce() throws IOException { AbfsClient mockClient = callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient(); verify(mockClient, times(1)).getAclStatus(anyString(), - new TracingContext("test-filesystem-id", "NS")); + new TracingContext(clientCorrelationID, fileSystemID, "NS")); } private AbfsClient callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient() @@ -208,7 +211,7 @@ private AbfsClient callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient() final AzureBlobFileSystemStore abfsStore = abfs.getAbfsStore(); final AbfsClient mockClient = mock(AbfsClient.class); doReturn(mock(AbfsRestOperation.class)).when(mockClient) - .getAclStatus(anyString(), new TracingContext("test-filesystem-id", "NS")); + .getAclStatus(anyString(), new TracingContext(clientCorrelationID, fileSystemID, "NS")); abfsStore.setClient(mockClient); abfs.getIsNamespaceEnabled(); return mockClient; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index 74023a74cbf83..86fe6c91908e6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -137,7 +137,7 @@ public void verifyWriteRequest() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext("test-corr-id", - "test-fs-id", "OP") + "test-fs-id", "OP"); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); @@ -324,7 +324,8 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, - true), new TracingContext("fs-id")); + true), new TracingContext(abfsConf.getClientCorrelationID(), + "fs-id", "OP")); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -404,9 +405,10 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { ArgumentCaptor acFlushRetainUnCommittedData = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acFlushClose = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); + ArgumentCaptor acTracingContext = ArgumentCaptor.forClass(TracingContext.class); verify(client, times(1)).flush(acFlushString.capture(), acFlushLong.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture(), new TracingContext("test-filesystem-id", "OP")); + acFlushSASToken.capture(), acTracingContext.capture()); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushString.getAllValues()); assertThat(Arrays.asList(Long.valueOf(2*BUFFER_SIZE))).describedAs("position").isEqualTo(acFlushLong.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); From d762d864206f3b88af01d1558b4101ce8746dedd Mon Sep 17 00:00:00 2001 From: sumangala Date: Tue, 20 Oct 2020 15:33:27 +0530 Subject: [PATCH 19/77] test --- .../fs/azurebfs/AzureBlobFileSystem.java | 9 ++- .../fs/azurebfs/AzureBlobFileSystemStore.java | 4 ++ .../fs/azurebfs/utils/TracingContext.java | 12 ++++ .../ITestClientCorrelationHeader.java | 70 +++++++++++++++++-- 4 files changed, 86 insertions(+), 9 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 74e64a4ed6a82..caca43e406431 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -102,6 +102,8 @@ public class AzureBlobFileSystem extends FileSystem { private AbfsDelegationTokenManager delegationTokenManager; private AbfsCounters abfsCounters; private String clientCorrelationID; + public boolean testTracing = false; + public List headers; @Override public void initialize(URI uri, Configuration configuration) @@ -183,11 +185,14 @@ public String getFileSystemID() { public FSDataInputStream open(final Path path, final int bufferSize) throws IOException { LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize); statIncrement(CALL_OPEN); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "FS"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "OF"); Path qualifiedPath = makeQualified(path); try { InputStream inputStream = abfsStore.openFileForRead(qualifiedPath, statistics, tracingContext); + if(testTracing) { + headers = tracingContext.getRequestHeaders(); + } return new FSDataInputStream(inputStream); } catch(AzureBlobFileSystemException ex) { checkException(path, ex); @@ -203,7 +208,7 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi permission, overwrite, blockSize); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "LS"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "CR"); statIncrement(CALL_CREATE); trailingPeriodCheck(f); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 7520576279ae6..6d8b88b620ae5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -501,6 +501,7 @@ isAppendBlob, new TracingContext(tracingContext) isNamespaceEnabled ? getOctalNotation(umask) : null, isAppendBlob, null, tracingContext); + } perfInfo.registerResult(op.getResult()).registerSuccess(true); @@ -541,6 +542,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa op = client.createPath(relativePath, true, false, permission, umask, isAppendBlob, null, new TracingContext(tracingContext)); + } catch (AbfsRestOperationException e) { if (e.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { // File pre-exists, fetch eTag @@ -637,6 +639,8 @@ public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statist final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); final long contentLength = Long.parseLong(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + tracingContext.updateRequestHeader(op.getResult() + .getRequestHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID)); if (parseIsDirectory(resourceType)) { throw new AbfsRestOperationException( diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index 13b40e26a457f..b196aeb02d1b0 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -18,6 +18,9 @@ package org.apache.hadoop.fs.azurebfs.utils; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.UUID; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; @@ -34,6 +37,7 @@ public class TracingContext { private String streamID = ""; private int retryCount; private String hadoopOpName = ""; + private final List headers = null; private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; @@ -104,4 +108,12 @@ public String toString() { + ":" + streamID + ":" + hadoopOpName + ":" + retryCount; } + public void updateRequestHeader(String requestHeader) { + headers.add(requestHeader); + } + + public List getRequestHeaders() { + return headers; + } + } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java index 81a1bd7f93849..04bdbcdf921a9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java @@ -1,46 +1,102 @@ package org.apache.hadoop.fs.azurebfs; import com.google.common.base.Preconditions; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.http.HttpException; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; + +//import org.graalvm.compiler.debug.Assertions; import org.junit.Test; +import org.assertj.core.api.Assertions; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.Path; +import javax.security.auth.login.Configuration; import java.io.IOException; +import java.util.List; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.eq; //import static org.powermock.api.easymock.PowerMock.*; +import static org.mockito.ArgumentMatchers.*; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; public final class ITestClientCorrelationHeader extends AbstractAbfsIntegrationTest { - AzureBlobFileSystem fs = getFileSystem(); + AzureBlobFileSystem fs = getFileSystem(); //ensure new instance of fs is created here public ITestClientCorrelationHeader() throws Exception { super(); } + public void testAllOperations() throws IOException { + fs.testTracing = true; //store request headers + + // doing one op first. Replace with a loop over all ops or use other tests later + fs.open(new Path("testFolder")); + List headerList = fs.headers; + String fs_id = headerList.get(0).split(":")[2]; + String corr_id = headerList.get(0).split(":")[0]; + for (String header : headerList) { + testHeader(header, "OF"); + Assertions.assertThat(fs_id) + .describedAs("filesystem id should be same for requests with same filesystem") + .isEqualTo(header.split(":")[2]); + Assertions.assertThat(corr_id) + .describedAs("correlation id should be same for requests using a given config") + .isEqualTo(header.split(":")[0]); + } + fs.headers = null; + fs.testTracing = false; + } + + public void testHeader(String responseHeader, String operation) { + boolean isContinuationOp = false; + switch (operation) { + case 'LS': + case 'CR': + case 'RD': isContinuationOp = true; + } + String[] headerList = responseHeader.split(":"); + // check if all IDs included + Assertions.assertThat(headerList) + .describedAs("header should have 7 elements").hasSize(7); + + //check that necessary ids are not empty + Assertions.assertThat(headerList[1]).isNotEmpty(); //client request id + Assertions.assertThat(headerList[2]).isNotEmpty(); //filesystem id + + checkClientCorrelationId(); + + //check that primary req id is present when required + if (isContinuationOp) { + Assertions.assertThat(headerList[3]).describedAs("Continuation ops should have a primary request id") + .isNotEmpty(); + } + + } + + void checkClientCorrelationId() { + //check if non-empty corr-id satisfies the constraints (copy old test) + } + public void testPrimaryRequestID() { // readahead // liststatus // createoverwrite // rename - // check if placed in the correct slot and matches client req id - // how to verify for readahead - concurrent requests? - - //check client correlation id - response vs request header? + //check client correlation id } public void testStreamID() { From d905c93c79a3db9d55505bbd022e9d8ae62f8050 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 20 Oct 2020 06:47:23 -0700 Subject: [PATCH 20/77] 1 test draft --- hadoop-tools/hadoop-azure/pom.xml | 1 - .../org/apache/hadoop/fs/azurebfs/utils/TracingContext.java | 4 ++-- .../hadoop/fs/azurebfs/ITestClientCorrelationHeader.java | 6 +++--- 3 files changed, 5 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml index 450feb5afba1e..6f23986c37ca0 100644 --- a/hadoop-tools/hadoop-azure/pom.xml +++ b/hadoop-tools/hadoop-azure/pom.xml @@ -45,7 +45,6 @@ 7200 10 1000 - diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index b196aeb02d1b0..16460b72d4aa5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -18,7 +18,7 @@ package org.apache.hadoop.fs.azurebfs.utils; -import java.util.HashMap; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.UUID; @@ -37,7 +37,7 @@ public class TracingContext { private String streamID = ""; private int retryCount; private String hadoopOpName = ""; - private final List headers = null; + private final java.util.ArrayList headers = new java.util.ArrayList(); private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java index 04bdbcdf921a9..03a70fcc5693e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java @@ -63,9 +63,9 @@ public void testAllOperations() throws IOException { public void testHeader(String responseHeader, String operation) { boolean isContinuationOp = false; switch (operation) { - case 'LS': - case 'CR': - case 'RD': isContinuationOp = true; + case "LS": + case "CR": + case "RD": isContinuationOp = true; } String[] headerList = responseHeader.split(":"); // check if all IDs included From 56acf8059ff53d9a833df3e45951d2d6e64182db Mon Sep 17 00:00:00 2001 From: Sumangala Date: Fri, 23 Oct 2020 11:17:35 +0530 Subject: [PATCH 21/77] test IDs --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 41 +++- .../azurebfs/services/AbfsOutputStream.java | 4 + .../azurebfs/services/AbfsRestOperation.java | 4 + .../fs/azurebfs/utils/TracingContext.java | 9 +- .../hadoop/fs/azurebfs/ITestAbfsClient.java | 65 ------ .../azurebfs/ITestAbfsReadWriteAndSeek.java | 1 + .../ITestClientCorrelationHeader.java | 200 +++++++++++++----- 7 files changed, 194 insertions(+), 130 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 6d8b88b620ae5..83668e82bd1e5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -487,11 +487,17 @@ public OutputStream createFile(final Path path, AbfsRestOperation op; if (triggerConditionalCreateOverwrite) { +// op = conditionalCreateOverwriteFile(relativePath, +// statistics, +// isNamespaceEnabled ? getOctalNotation(permission) : null, +// isNamespaceEnabled ? getOctalNotation(umask) : null, +// isAppendBlob, new TracingContext(tracingContext) +// ); op = conditionalCreateOverwriteFile(relativePath, - statistics, - isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null, - isAppendBlob, new TracingContext(tracingContext) + statistics, + isNamespaceEnabled ? getOctalNotation(permission) : null, + isNamespaceEnabled ? getOctalNotation(umask) : null, + isAppendBlob, tracingContext ); } else { @@ -501,6 +507,7 @@ isAppendBlob, new TracingContext(tracingContext) isNamespaceEnabled ? getOctalNotation(umask) : null, isAppendBlob, null, tracingContext); + tracingContext.headers.add(tracingContext.toString()); } perfInfo.registerResult(op.getResult()).registerSuccess(true); @@ -540,15 +547,23 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa // avoided for cases when no pre-existing file is present (major portion // of create file traffic falls into the case of no pre-existing file). - op = client.createPath(relativePath, true, - false, permission, umask, isAppendBlob, null, new TracingContext(tracingContext)); +// op = client.createPath(relativePath, true, false, +// permission, umask, isAppendBlob, null, new TracingContext(tracingContext)); + op = client.createPath(relativePath, true, false, + permission, umask, isAppendBlob, null, tracingContext); + tracingContext.headers.add(tracingContext.toString()); } catch (AbfsRestOperationException e) { if (e.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { // File pre-exists, fetch eTag try { - op = client.getPathStatus(relativePath, false, new TracingContext(tracingContext)); + tracingContext.headers.add(tracingContext.toString()); + System.out.println("added"); +// op = client.getPathStatus(relativePath, false, new TracingContext(tracingContext)); + op = client.getPathStatus(relativePath, false, tracingContext); + tracingContext.headers.add(tracingContext.toString()); } catch (AbfsRestOperationException ex) { + tracingContext.headers.add(tracingContext.toString()); if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { // Is a parallel access case, as file which was found to be // present went missing by this request. @@ -565,9 +580,13 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa try { // overwrite only if eTag matches with the file properties fetched befpre +// op = client.createPath(relativePath, true, +// true, permission, umask, isAppendBlob, eTag, new TracingContext(tracingContext)); op = client.createPath(relativePath, true, - true, permission, umask, isAppendBlob, eTag, new TracingContext(tracingContext)); + true, permission, umask, isAppendBlob, eTag, tracingContext); + tracingContext.headers.add(tracingContext.toString()); } catch (AbfsRestOperationException ex) { + tracingContext.headers.add(tracingContext.toString()); if (ex.getStatusCode() == HttpURLConnection.HTTP_PRECON_FAILED) { // Is a parallel access case, as file with eTag was just queried // and precondition failure can happen only when another file with @@ -639,8 +658,6 @@ public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statist final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); final long contentLength = Long.parseLong(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); - tracingContext.updateRequestHeader(op.getResult() - .getRequestHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID)); if (parseIsDirectory(resourceType)) { throw new AbfsRestOperationException( @@ -907,6 +924,10 @@ public FileStatus[] listStatus(final Path path, final String startFrom, perfInfo.registerResult(op.getResult()); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); ListResultSchema retrievedSchema = op.getResult().getListResultSchema(); +// originalTracingContext.updateRequestHeader(op.getResult().getRequestHeader( +// HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID)); +// originalTracingContext.headers.add(op.tracingContext.toString()); + originalTracingContext.headers.add(op.requestHeader); if (retrievedSchema == null) { throw new AbfsRestOperationException( AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index dc97a3fbc5f5d..d335df51afefd 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -277,6 +277,10 @@ public void hflush() throws IOException { } } + public String getStreamID() { + return outputStreamID; + } + /** * Force all data in the output stream to be written to Azure storage. * Wait to return until this is complete. Close the access to the stream and 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 6a4dab822c214..30b8cbd332137 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 @@ -70,6 +70,8 @@ public class AbfsRestOperation { private AbfsHttpOperation result; private AbfsCounters abfsCounters; +// public TracingContext tracingContext; + public String requestHeader = ""; public AbfsHttpOperation getResult() { return result; @@ -195,6 +197,8 @@ public void execute(TracingContext tracingContext) throws AzureBlobFileSystemExc Thread.currentThread().interrupt(); } } +// this.tracingContext = tracingContext; //if retried, will reflect final header + this.requestHeader = tracingContext.toString(); if (result.getStatusCode() >= HttpURLConnection.HTTP_BAD_REQUEST) { throw new AbfsRestOperationException(result.getStatusCode(), result.getStorageErrorCode(), diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index 16460b72d4aa5..7a6738443c5be 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -37,7 +37,7 @@ public class TracingContext { private String streamID = ""; private int retryCount; private String hadoopOpName = ""; - private final java.util.ArrayList headers = new java.util.ArrayList(); + public java.util.ArrayList headers = new java.util.ArrayList(); private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; @@ -108,12 +108,7 @@ public String toString() { + ":" + streamID + ":" + hadoopOpName + ":" + retryCount; } - public void updateRequestHeader(String requestHeader) { - headers.add(requestHeader); - } - - public List getRequestHeaders() { + public ArrayList getRequestHeaders() { return headers; } - } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java index 8d7be9c9ac2a1..7e08f5e26c602 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java @@ -57,9 +57,6 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest { private static final int LIST_MAX_RESULTS = 500; private static final int LIST_MAX_RESULTS_SERVER = 5000; - private static final int HTTP_CREATED = 201; - private static final String[] CLIENT_CORRELATIONID_LIST = { - "valid-corr-id-123", "inval!d", ""}; public ITestAbfsClient() throws Exception { super(); @@ -99,68 +96,6 @@ public void testUnknownHost() throws Exception { () -> FileSystem.get(conf.getRawConfiguration())); } - @Test - public void testClientCorrelation() throws IOException { - checkRequest(CLIENT_CORRELATIONID_LIST[0], true); - checkRequest(CLIENT_CORRELATIONID_LIST[1], false); - checkRequest(CLIENT_CORRELATIONID_LIST[2], false); - } - - private String getOctalNotation(FsPermission fsPermission) { - Preconditions.checkNotNull(fsPermission, "fsPermission"); - return String.format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal()); - } - - private String getRelativePath(final Path path) { - Preconditions.checkNotNull(path, "path"); - return path.toUri().getPath(); - } - - public void checkRequest(String clientCorrelationId, boolean includeInHeader) - throws IOException { - Configuration config = new Configuration(this.getRawConfiguration()); - config.set(FS_AZURE_CLIENT_CORRELATIONID, clientCorrelationId); - - final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem - .newInstance(this.getFileSystem().getUri(), config); - AbfsClient client = fs.getAbfsClient(); -// TracingContext - String path = getRelativePath(new Path("/testDir")); - boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(); - String permission = isNamespaceEnabled ? getOctalNotation(FsPermission.getDirDefault()) : null; - String umask = isNamespaceEnabled ? getOctalNotation(FsPermission.getUMask(fs.getConf())) : null; - AbfsRestOperation op = client.createPath(path, false, true, - permission, umask, false, null, - new TracingContext(clientCorrelationId, - fs.getFileSystemID(), "CR")); - - int responseCode = op.getResult().getStatusCode(); - Assertions.assertThat(responseCode).describedAs("Status code").isEqualTo(HTTP_CREATED); - -// op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); - Assertions.assertThat(responseCode).describedAs("Status code").isEqualTo(HTTP_CREATED); - - String requestHeader = op.getResult().getRequestHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); - List clientRequestIds = java.util.Arrays.asList( - requestHeader.replace("[","") - .replace("]", "") - .split(":")); - if (includeInHeader) { - Assertions.assertThat(clientRequestIds) - .describedAs("There should be 7 items in the header when valid clientCorrelationId is set") - .hasSize(7); - Assertions.assertThat(clientRequestIds) - .describedAs("clientCorrelation should be included in the header") - .contains(clientCorrelationId); - } else if (clientCorrelationId.length() > 0){ - Assertions.assertThat(clientRequestIds) - .describedAs("There should be only 6 item in the header when invalid clientCorrelationId is set") - .hasSize(6); - Assertions.assertThat(clientRequestIds) - .describedAs("Invalid or empty correlationId value should not be included in header") - .doesNotContain(clientCorrelationId); - } - } @Test public void testListPathWithValidListMaxResultsValues() diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index 2e26b7680f88a..c4c39e36bf46d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -1,3 +1,4 @@ + /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java index 03a70fcc5693e..891cb2a054b8d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java @@ -1,15 +1,14 @@ package org.apache.hadoop.fs.azurebfs; import com.google.common.base.Preconditions; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; -import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader; -import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; -import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.*; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.http.HttpException; @@ -21,93 +20,198 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.Path; -import javax.security.auth.login.Configuration; import java.io.IOException; +import java.util.ArrayList; import java.util.List; //import static org.powermock.api.easymock.PowerMock.*; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_CORRELATIONID; import static org.mockito.ArgumentMatchers.*; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; public final class ITestClientCorrelationHeader extends AbstractAbfsIntegrationTest { - AzureBlobFileSystem fs = getFileSystem(); //ensure new instance of fs is created here +// AzureBlobFileSystem fs = getFileSystem(); //ensure new instance of fs is created here + private static final int HTTP_CREATED = 201; + private static final String[] CLIENT_CORRELATIONID_LIST = { + "valid-corr-id-123", "inval!d", ""}; public ITestClientCorrelationHeader() throws Exception { super(); } - public void testAllOperations() throws IOException { - fs.testTracing = true; //store request headers + @Test + public void testCreateOverwrite() throws IOException { + AzureBlobFileSystem fs = getFileSystem(); + AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); + TracingContext tracingContext = new TracingContext("corr-id", + fs.getFileSystemID(), "CR"); + AbfsOutputStream out = (AbfsOutputStream) abfsStore.createFile( + new Path("/testFolder"), fs.getFsStatistics(),true, + FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()), + tracingContext); + tracingContext.headers.clear(); + System.out.println("now second call"); + out = (AbfsOutputStream) abfsStore.createFile( + new Path("/testFolder"), fs.getFsStatistics(),true, + FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()), + tracingContext); - // doing one op first. Replace with a loop over all ops or use other tests later - fs.open(new Path("testFolder")); - List headerList = fs.headers; - String fs_id = headerList.get(0).split(":")[2]; - String corr_id = headerList.get(0).split(":")[0]; - for (String header : headerList) { - testHeader(header, "OF"); - Assertions.assertThat(fs_id) - .describedAs("filesystem id should be same for requests with same filesystem") - .isEqualTo(header.split(":")[2]); - Assertions.assertThat(corr_id) - .describedAs("correlation id should be same for requests using a given config") - .isEqualTo(header.split(":")[0]); + ArrayList headers = tracingContext.getRequestHeaders(); + testHeaders(headers); + testPrimaryRequestID(headers); + } + + @Test + public void testListStatus() throws IOException { + AzureBlobFileSystem fs = getFileSystem(); + AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); + TracingContext tracingContext = new TracingContext("corr-id", + fs.getFileSystemID(), "LS", "preqid"); + abfsStore.listStatus(new Path("/"), tracingContext); + ArrayList headers = tracingContext.getRequestHeaders(); + for (String header : headers) { + System.out.println(header); } - fs.headers = null; - fs.testTracing = false; + testHeaders(headers); + testPrimaryRequestID(headers); } - public void testHeader(String responseHeader, String operation) { +// @Test +// public void testStream() throws IOException { +// } + + public void testHeaders(ArrayList headers) { + String operation = headers.get(0).split(":")[5]; boolean isContinuationOp = false; + boolean isStreamOp = false; switch (operation) { case "LS": case "CR": case "RD": isContinuationOp = true; } - String[] headerList = responseHeader.split(":"); - // check if all IDs included - Assertions.assertThat(headerList) - .describedAs("header should have 7 elements").hasSize(7); - - //check that necessary ids are not empty - Assertions.assertThat(headerList[1]).isNotEmpty(); //client request id - Assertions.assertThat(headerList[2]).isNotEmpty(); //filesystem id + for (String headerList : headers) { + String[] id_list = headerList.split(":"); + System.out.println(headerList); + // check format for parsing + Assertions.assertThat(id_list) + .describedAs("header should have 7 elements").hasSize(7); - checkClientCorrelationId(); + //check that necessary ids are not empty + Assertions.assertThat(id_list[1]).isNotEmpty(); //client request id + Assertions.assertThat(id_list[2]).isNotEmpty(); //filesystem id - //check that primary req id is present when required - if (isContinuationOp) { - Assertions.assertThat(headerList[3]).describedAs("Continuation ops should have a primary request id") - .isNotEmpty(); + //check that primary req id is present when required + if (isContinuationOp) { + Assertions.assertThat(id_list[3]).describedAs("Continuation ops should have a primary request id") + .isNotEmpty(); + } } + //test values + String fs_id = headers.get(0).split(":")[2]; + String corr_id = headers.get(0).split(":")[0]; + for (String headerList : headers) { + String[] id_list = headerList.split(":"); + Assertions.assertThat(fs_id) + .describedAs("filesystem id should be same for requests with same filesystem") + .isEqualTo(id_list[2]); + Assertions.assertThat(corr_id) + .describedAs("correlation id should be same for requests using a given config") + .isEqualTo(id_list[0]); + Assertions.assertThat(Integer.parseInt(id_list[6])) + .describedAs("Max retries allowed = 30") + .isLessThan(30); + } } - void checkClientCorrelationId() { - //check if non-empty corr-id satisfies the constraints (copy old test) - } - - public void testPrimaryRequestID() { + public void testPrimaryRequestID(ArrayList headerList) { // readahead // liststatus // createoverwrite // rename + String preq = headerList.get(0).split(":")[3]; + for (String header : headerList) { + Assertions.assertThat (header.split(":")[3]) + .describedAs("preq id should be same for given set of requests") + .isEqualTo(preq); + } + } - //check client correlation id + public void checkStreamID(String streamID, ArrayList headers) { + for (String header : headers) { + Assertions.assertThat(header.split(":")[4]) + .describedAs("stream id should be same for a given stream") + .isEqualTo(streamID); + } + } + + @Test + public void testClientCorrelationId() throws IOException { + checkRequest(CLIENT_CORRELATIONID_LIST[0], true); + checkRequest(CLIENT_CORRELATIONID_LIST[1], false); + checkRequest(CLIENT_CORRELATIONID_LIST[2], false); + } + + private String getOctalNotation(FsPermission fsPermission) { + Preconditions.checkNotNull(fsPermission, "fsPermission"); + return String.format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal()); } - public void testStreamID() { - // create one input and one output stream. All req on a given stream should have same stream id throughout - // placed correctly? should be in slot 4 while parsing ( + private String getRelativePath(final Path path) { + Preconditions.checkNotNull(path, "path"); + return path.toUri().getPath(); } - public void testFilesystemID() { - // all requests using a given filesystem instance should have same fs id - // in slot 0 + public void checkRequest(String clientCorrelationId, boolean includeInHeader) + throws IOException { + org.apache.hadoop.conf.Configuration config = new Configuration(this.getRawConfiguration()); + config.set(FS_AZURE_CLIENT_CORRELATIONID, clientCorrelationId); + + final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem + .newInstance(this.getFileSystem().getUri(), config); + AbfsClient client = fs.getAbfsClient(); +// TracingContext + String path = getRelativePath(new Path("/testDir")); + boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(); + String permission = isNamespaceEnabled ? getOctalNotation(FsPermission.getDirDefault()) : null; + String umask = isNamespaceEnabled ? getOctalNotation(FsPermission.getUMask(fs.getConf())) : null; + AbfsRestOperation op = client.createPath(path, false, true, + permission, umask, false, null, + new TracingContext(clientCorrelationId, + fs.getFileSystemID(), "CR")); + + int responseCode = op.getResult().getStatusCode(); + Assertions.assertThat(responseCode).describedAs("Status code").isEqualTo(HTTP_CREATED); + +// op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); + Assertions.assertThat(responseCode).describedAs("Status code").isEqualTo(HTTP_CREATED); + + String requestHeader = op.getResult().getRequestHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); + List clientRequestIds = java.util.Arrays.asList( + requestHeader.replace("[","") + .replace("]", "") + .split(":")); + if (includeInHeader) { + Assertions.assertThat(clientRequestIds) + .describedAs("There should be 7 items in the header when valid clientCorrelationId is set") + .hasSize(7); + Assertions.assertThat(clientRequestIds) + .describedAs("clientCorrelation should be included in the header") + .contains(clientCorrelationId); + } else if (clientCorrelationId.length() > 0){ + Assertions.assertThat(clientRequestIds) + .describedAs("There should be only 6 item in the header when invalid clientCorrelationId is set") + .hasSize(6); + Assertions.assertThat(clientRequestIds) + .describedAs("Invalid or empty correlationId value should not be included in header") + .doesNotContain(clientCorrelationId); + } } + + /* @Test // @PrepareForTest(AbfsRestOperation.class) From 0a96fbe57d115c80277de7c11510bcbad9de3d55 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Fri, 23 Oct 2020 12:32:41 +0530 Subject: [PATCH 22/77] clear() --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 16 ++--- .../ITestClientCorrelationHeader.java | 59 +------------------ 2 files changed, 10 insertions(+), 65 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 83668e82bd1e5..b702a539e195e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -507,7 +507,7 @@ public OutputStream createFile(final Path path, isNamespaceEnabled ? getOctalNotation(umask) : null, isAppendBlob, null, tracingContext); - tracingContext.headers.add(tracingContext.toString()); +// tracingContext.headers.add(tracingContext.toString()); } perfInfo.registerResult(op.getResult()).registerSuccess(true); @@ -551,19 +551,19 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa // permission, umask, isAppendBlob, null, new TracingContext(tracingContext)); op = client.createPath(relativePath, true, false, permission, umask, isAppendBlob, null, tracingContext); - tracingContext.headers.add(tracingContext.toString()); +// tracingContext.headers.add(tracingContext.toString()); } catch (AbfsRestOperationException e) { if (e.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { // File pre-exists, fetch eTag try { - tracingContext.headers.add(tracingContext.toString()); +// tracingContext.headers.add(tracingContext.toString()); System.out.println("added"); // op = client.getPathStatus(relativePath, false, new TracingContext(tracingContext)); op = client.getPathStatus(relativePath, false, tracingContext); - tracingContext.headers.add(tracingContext.toString()); +// tracingContext.headers.add(tracingContext.toString()); } catch (AbfsRestOperationException ex) { - tracingContext.headers.add(tracingContext.toString()); +// tracingContext.headers.add(tracingContext.toString()); if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { // Is a parallel access case, as file which was found to be // present went missing by this request. @@ -584,9 +584,9 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa // true, permission, umask, isAppendBlob, eTag, new TracingContext(tracingContext)); op = client.createPath(relativePath, true, true, permission, umask, isAppendBlob, eTag, tracingContext); - tracingContext.headers.add(tracingContext.toString()); +// tracingContext.headers.add(tracingContext.toString()); } catch (AbfsRestOperationException ex) { - tracingContext.headers.add(tracingContext.toString()); +// tracingContext.headers.add(tracingContext.toString()); if (ex.getStatusCode() == HttpURLConnection.HTTP_PRECON_FAILED) { // Is a parallel access case, as file with eTag was just queried // and precondition failure can happen only when another file with @@ -927,7 +927,7 @@ public FileStatus[] listStatus(final Path path, final String startFrom, // originalTracingContext.updateRequestHeader(op.getResult().getRequestHeader( // HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID)); // originalTracingContext.headers.add(op.tracingContext.toString()); - originalTracingContext.headers.add(op.requestHeader); +// originalTracingContext.headers.add(op.requestHeader); if (retrievedSchema == null) { throw new AbfsRestOperationException( AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java index 891cb2a054b8d..8f17042a22202 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java @@ -52,8 +52,8 @@ public void testCreateOverwrite() throws IOException { new Path("/testFolder"), fs.getFsStatistics(),true, FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()), tracingContext); - tracingContext.headers.clear(); - System.out.println("now second call"); +// tracingContext.headers.clear(); +// System.out.println("now second call"); out = (AbfsOutputStream) abfsStore.createFile( new Path("/testFolder"), fs.getFsStatistics(),true, FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()), @@ -186,7 +186,6 @@ public void checkRequest(String clientCorrelationId, boolean includeInHeader) int responseCode = op.getResult().getStatusCode(); Assertions.assertThat(responseCode).describedAs("Status code").isEqualTo(HTTP_CREATED); -// op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); Assertions.assertThat(responseCode).describedAs("Status code").isEqualTo(HTTP_CREATED); String requestHeader = op.getResult().getRequestHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); @@ -210,58 +209,4 @@ public void checkRequest(String clientCorrelationId, boolean includeInHeader) .doesNotContain(clientCorrelationId); } } - - -/* - @Test -// @PrepareForTest(AbfsRestOperation.class) - public void testRetryCount() throws IOException { -// AbfsRestOperation op = PowerMock.spy(AbfsRestOperation.class, "executeHttpOperation"); -// PowerMockito.doThrow(new AbfsRestOperationException(new HttpException())) -// .when(op, "executeHttpOperation", any(int), -// any(TracingContext.class)); - - AbfsClient client = mock(AbfsClient.class); - - AbfsRestOperation restop = new AbfsRestOperation( - org.apache.hadoop.fs.azurebfs.services.AbfsRestOperationType.CreatePath, - client, - HTTP_METHOD_PUT, - new java.net.URL("url"), - new java.util.ArrayList()); - AbfsRestOperation op = spy(restop); - - -// AbfsRestOperation op = spy(restop); -// when(op.executeHttpOperation(any(int.class), any(TracingContext.class))) -// .thenReturn(false); -// .thenThrow(AzureBlobFileSystemException.class); - - op.execute(new TracingContext("test-corr-id", "fsid","op")); - - String path = getRelativePath(new Path("/testDir")); - boolean isNamespaceEnabled = true;//fs.getIsNamespaceEnabled(); - String permission = null;// isNamespaceEnabled ? getOctalNotation(FsPermission.getDirDefault()) : null; - String umask = null; //isNamespaceEnabled ? getOctalNotation(FsPermission.getUMask(fs.getConf())) : null; - -// AbfsClient client = mock(AbfsClient.class); - when(client.createPath(any(String.class), eq(true), eq(false), - eq(null), eq(null), - any(boolean.class), eq(null), any(TracingContext.class))).thenReturn(op); - AbfsRestOperation op1 = client.createPath(path, false, true, - permission, umask, false, null, - new TracingContext("test-corr-id", "fsid", "CR")); - - } - - private String getOctalNotation(FsPermission fsPermission) { - Preconditions.checkNotNull(fsPermission, "fsPermission"); - return String.format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal()); - } - - private String getRelativePath(final Path path) { - Preconditions.checkNotNull(path, "path"); - return path.toUri().getPath(); - }*/ - } \ No newline at end of file From ba8d98837dc0c5d291a9d1bcde096c09327a024d Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 27 Oct 2020 23:22:22 +0530 Subject: [PATCH 23/77] minor edits --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 21 +- .../fs/azurebfs/AzureBlobFileSystem.java | 5 - .../fs/azurebfs/AzureBlobFileSystemStore.java | 18 -- .../azurebfs/constants/ConfigurationKeys.java | 1 + .../fs/azurebfs/services/AbfsClient.java | 6 + .../azurebfs/services/AbfsHttpOperation.java | 3 - .../azurebfs/services/AbfsRestOperation.java | 20 +- .../fs/azurebfs/utils/TracingContext.java | 33 +-- .../azurebfs/AbstractAbfsIntegrationTest.java | 9 +- .../hadoop/fs/azurebfs/ITestAbfsClient.java | 6 +- .../ITestAbfsInputStreamStatistics.java | 12 +- .../ITestAzureBlobFileSystemCreate.java | 5 +- .../ITestAzureBlobFileSystemDelete.java | 5 +- .../ITestAzureBlobFileSystemOauth.java | 3 - .../ITestAzureBlobFileSystemRename.java | 12 +- ...lobFileSystemStoreListStatusWithRange.java | 4 +- .../ITestClientCorrelationHeader.java | 212 ------------------ .../azurebfs/ITestFileSystemProperties.java | 11 - .../fs/azurebfs/ITestGetNameSpaceEnabled.java | 22 +- .../fs/azurebfs/ITestSharedKeyAuth.java | 4 +- .../fs/azurebfs/services/TestAbfsClient.java | 7 - .../services/TestAbfsInputStream.java | 3 +- .../services/TestAbfsOutputStream.java | 2 +- 23 files changed, 69 insertions(+), 355 deletions(-) delete mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 3977a396dd1cb..df974575c9fdb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -218,10 +218,6 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_FS_AZURE_USER_AGENT_PREFIX) private String userAgentId; - @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_CLIENT_CORRELATIONID, - DefaultValue = EMPTY_STRING) - private String clientCorrelationID; - @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_CLUSTER_NAME, DefaultValue = DEFAULT_VALUE_UNKNOWN) private String clusterName; @@ -230,6 +226,14 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_VALUE_UNKNOWN) private String clusterType; + @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_CLIENT_CORRELATIONID, + DefaultValue = EMPTY_STRING) + private String clientCorrelationID; + + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ENABLE_CORRELATION_HEADER, + DefaultValue = true) + private boolean enableCorrelationHeader; + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ENABLE_DELEGATION_TOKEN, DefaultValue = DEFAULT_ENABLE_DELEGATION_TOKEN) private boolean enableDelegationToken; @@ -300,6 +304,15 @@ public String getClientCorrelationID() { return clientCorrelationID; } + /** + * Config to allow user to enable list of identifiers for correlation. + * Default value is true. Only clientRequestId appears in logs when disabled + * @return enableCorrelationHeader config + */ + public boolean isCorrelationHeaderEnabled() { + return enableCorrelationHeader; + } + /** * Appends an account name to a configuration key yielding the * account-specific form. diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index caca43e406431..b2c4770c40e4f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -102,8 +102,6 @@ public class AzureBlobFileSystem extends FileSystem { private AbfsDelegationTokenManager delegationTokenManager; private AbfsCounters abfsCounters; private String clientCorrelationID; - public boolean testTracing = false; - public List headers; @Override public void initialize(URI uri, Configuration configuration) @@ -190,9 +188,6 @@ public FSDataInputStream open(final Path path, final int bufferSize) throws IOEx try { InputStream inputStream = abfsStore.openFileForRead(qualifiedPath, statistics, tracingContext); - if(testTracing) { - headers = tracingContext.getRequestHeaders(); - } return new FSDataInputStream(inputStream); } catch(AzureBlobFileSystemException ex) { checkException(path, ex); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index b702a539e195e..f9c9237ae6694 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -487,12 +487,6 @@ public OutputStream createFile(final Path path, AbfsRestOperation op; if (triggerConditionalCreateOverwrite) { -// op = conditionalCreateOverwriteFile(relativePath, -// statistics, -// isNamespaceEnabled ? getOctalNotation(permission) : null, -// isNamespaceEnabled ? getOctalNotation(umask) : null, -// isAppendBlob, new TracingContext(tracingContext) -// ); op = conditionalCreateOverwriteFile(relativePath, statistics, isNamespaceEnabled ? getOctalNotation(permission) : null, @@ -547,23 +541,15 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa // avoided for cases when no pre-existing file is present (major portion // of create file traffic falls into the case of no pre-existing file). -// op = client.createPath(relativePath, true, false, -// permission, umask, isAppendBlob, null, new TracingContext(tracingContext)); op = client.createPath(relativePath, true, false, permission, umask, isAppendBlob, null, tracingContext); -// tracingContext.headers.add(tracingContext.toString()); } catch (AbfsRestOperationException e) { if (e.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { // File pre-exists, fetch eTag try { -// tracingContext.headers.add(tracingContext.toString()); - System.out.println("added"); -// op = client.getPathStatus(relativePath, false, new TracingContext(tracingContext)); op = client.getPathStatus(relativePath, false, tracingContext); -// tracingContext.headers.add(tracingContext.toString()); } catch (AbfsRestOperationException ex) { -// tracingContext.headers.add(tracingContext.toString()); if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { // Is a parallel access case, as file which was found to be // present went missing by this request. @@ -580,13 +566,9 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa try { // overwrite only if eTag matches with the file properties fetched befpre -// op = client.createPath(relativePath, true, -// true, permission, umask, isAppendBlob, eTag, new TracingContext(tracingContext)); op = client.createPath(relativePath, true, true, permission, umask, isAppendBlob, eTag, tracingContext); -// tracingContext.headers.add(tracingContext.toString()); } catch (AbfsRestOperationException ex) { -// tracingContext.headers.add(tracingContext.toString()); if (ex.getStatusCode() == HttpURLConnection.HTTP_PRECON_FAILED) { // Is a parallel access case, as file with eTag was just queried // and precondition failure can happen only when another file with diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index b97b9daf846cf..8082b01fced91 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -90,6 +90,7 @@ public final class ConfigurationKeys { * x-ms-client-request-Id header. Defaults to empty string if the length and * character constraints are not satisfied. **/ public static final String FS_AZURE_CLIENT_CORRELATIONID = "fs.azure.client.correlationid"; + public static final String FS_AZURE_ENABLE_CORRELATION_HEADER = "fs.azure.enable.correlation.header"; public static final String FS_AZURE_CLUSTER_NAME = "fs.azure.cluster.name"; public static final String FS_AZURE_CLUSTER_TYPE = "fs.azure.cluster.type"; public static final String FS_AZURE_SSL_CHANNEL_MODE_KEY = "fs.azure.ssl.channel.mode"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 345ac8ffdcc61..1c8056ecf3f20 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -75,6 +75,7 @@ public class AbfsClient implements Closeable { private AccessTokenProvider tokenProvider; private SASTokenProvider sasTokenProvider; private final AbfsCounters abfsCounters; + private final boolean enableCorrelationHeader; private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, final AbfsConfiguration abfsConfiguration, @@ -106,6 +107,7 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden this.userAgent = initializeUserAgent(abfsConfiguration, sslProviderName); this.abfsPerfTracker = abfsClientContext.getAbfsPerfTracker(); this.abfsCounters = abfsClientContext.getAbfsCounters(); + this.enableCorrelationHeader = abfsConfiguration.isCorrelationHeaderEnabled(); } public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, @@ -135,6 +137,10 @@ public String getFileSystem() { return filesystem; } + public boolean isCorrelationHeaderEnabled() { + return enableCorrelationHeader; + } + protected AbfsPerfTracker getAbfsPerfTracker() { return abfsPerfTracker; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index dcf9b7c681db9..c93bf47dfeec8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -239,9 +239,6 @@ public String getLogString() { * * @throws IOException if an error occurs. */ -// public AbfsHttpOperation(final URL url, final String method, final List requestHeaders) throws IOException { -// this(url, method, requestHeaders, new TracingContext("", "OP")); -// } public AbfsHttpOperation(final URL url, final String method, final List requestHeaders) throws IOException { 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 30b8cbd332137..7322335b88cd3 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 @@ -23,6 +23,7 @@ import java.net.URL; import java.net.UnknownHostException; import java.util.List; +import java.util.UUID; import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; @@ -70,8 +71,6 @@ public class AbfsRestOperation { private AbfsHttpOperation result; private AbfsCounters abfsCounters; -// public TracingContext tracingContext; - public String requestHeader = ""; public AbfsHttpOperation getResult() { return result; @@ -197,8 +196,6 @@ public void execute(TracingContext tracingContext) throws AzureBlobFileSystemExc Thread.currentThread().interrupt(); } } -// this.tracingContext = tracingContext; //if retried, will reflect final header - this.requestHeader = tracingContext.toString(); if (result.getStatusCode() >= HttpURLConnection.HTTP_BAD_REQUEST) { throw new AbfsRestOperationException(result.getStatusCode(), result.getStorageErrorCode(), @@ -218,12 +215,19 @@ private boolean executeHttpOperation(final int retryCount, TracingContext tracin AbfsHttpOperation httpOperation = null; try { // initialize the HTTP request and open the connection - tracingContext.generateClientRequestID(); httpOperation = new AbfsHttpOperation(url, method, requestHeaders); incrementCounter(AbfsStatistic.CONNECTIONS_MADE, 1); - httpOperation.getConnection() - .setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, - tracingContext.toString()); + if (client.isCorrelationHeaderEnabled()) { + tracingContext.generateClientRequestID(); + httpOperation.getConnection() + .setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, + tracingContext.toString()); + } + else { + httpOperation.getConnection() + .setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, + UUID.randomUUID().toString()); + } switch(client.getAuthType()) { case Custom: diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index 7a6738443c5be..ca239f122d9c4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -30,36 +30,27 @@ import org.slf4j.LoggerFactory; public class TracingContext { - private String clientCorrelationID; + private final String clientCorrelationID; private final String fileSystemID; private String clientRequestID = ""; private String primaryRequestID; private String streamID = ""; private int retryCount; private String hadoopOpName = ""; - public java.util.ArrayList headers = new java.util.ArrayList(); private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*"; - // for non-continuation ops (no primary request id necessary) public TracingContext(String clientCorrelationID, String fileSystemID, String hadoopOpName) { this.fileSystemID = fileSystemID; this.hadoopOpName = hadoopOpName; - validateClientCorrelationID(clientCorrelationID); + this.clientCorrelationID = validateClientCorrelationID(clientCorrelationID); streamID = EMPTY_STRING; retryCount = 0; primaryRequestID = ""; } - // for ops with primary request - public TracingContext(String clientCorrelationID, String fileSystemID, String hadoopOpName, - String primaryRequestID) { - this(clientCorrelationID, fileSystemID, hadoopOpName); - this.primaryRequestID = primaryRequestID; - } - public TracingContext(TracingContext originalTracingContext) { this.fileSystemID = originalTracingContext.fileSystemID; this.streamID = originalTracingContext.streamID; @@ -69,17 +60,14 @@ public TracingContext(TracingContext originalTracingContext) { this.primaryRequestID = originalTracingContext.primaryRequestID; } - public void validateClientCorrelationID(String clientCorrelationID) { + public String validateClientCorrelationID(String clientCorrelationID) { if ((clientCorrelationID.length() > MAX_CLIENT_CORRELATION_ID_LENGTH) || (!clientCorrelationID.matches(CLIENT_CORRELATION_ID_PATTERN))) { - this.clientCorrelationID = EMPTY_STRING; LOG.debug( "Invalid config provided; correlation id not included in header."); - } else if (clientCorrelationID.length() > 0) { - this.clientCorrelationID = clientCorrelationID + ":"; - } else { - this.clientCorrelationID = EMPTY_STRING; + return EMPTY_STRING; } + return clientCorrelationID; } public void generateClientRequestID() { @@ -98,17 +86,8 @@ public void updateRetryCount() { retryCount++; } - public void reset() { - primaryRequestID = EMPTY_STRING; - retryCount = 0; - } - public String toString() { - return clientCorrelationID + clientRequestID + ":" + fileSystemID + ":" + primaryRequestID + return clientCorrelationID + ":" + clientRequestID + ":" + fileSystemID + ":" + primaryRequestID + ":" + streamID + ":" + hadoopOpName + ":" + retryCount; } - - public ArrayList getRequestHeaders() { - return headers; - } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index be65ff1f6263b..a319fe51b76a1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -82,6 +82,7 @@ public abstract class AbstractAbfsIntegrationTest extends private AuthType authType; private boolean useConfiguredFileSystem = false; private boolean usingFilesystemForSASTests = false; + public TracingContext tracingContext; protected AbstractAbfsIntegrationTest() throws Exception { fileSystemName = TEST_CONTAINER_PREFIX + UUID.randomUUID().toString(); @@ -145,8 +146,8 @@ protected AbstractAbfsIntegrationTest() throws Exception { public void setup() throws Exception { //Create filesystem first to make sure getWasbFileSystem() can return an existing filesystem. createFileSystem(); - TracingContext tracingContext = new TracingContext(abfsConfig.getClientCorrelationID(), - getFileSystem().getFileSystemID(), "TS"); + tracingContext = new TracingContext(abfsConfig.getClientCorrelationID(), + abfs.getFileSystemID(), "TS"); // Only live account without namespace support can run ABFS&WASB compatibility tests if (!isIPAddress @@ -434,9 +435,7 @@ protected AbfsOutputStream createAbfsOutputStreamWithFlushEnabled( abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false); return (AbfsOutputStream) abfss.createFile(path, fs.getFsStatistics(), - true, FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()), - new TracingContext(fs.getAbfsStore().getAbfsConfiguration() - .getClientCorrelationID(), fs.getFileSystemID(), "OP")); + true, FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()), tracingContext); } /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java index 7e08f5e26c602..22bc3dd50e45f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java @@ -27,7 +27,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Ignore; @@ -71,7 +70,7 @@ public void testContinuationTokenHavingEqualSign() throws Exception { try { AbfsRestOperation op = abfsClient.listPath("/", true, LIST_MAX_RESULTS, "===========", - new TracingContext("test-corr-id", fs.getFileSystemID(), "PA")); + tracingContext); Assert.assertTrue(false); } catch (AbfsRestOperationException ex) { Assert.assertEquals("InvalidQueryParameterValue", ex.getErrorCode().getErrorCode()); @@ -146,8 +145,7 @@ private List listPath(String directory) throws IOException { return getFileSystem().getAbfsClient() .listPath(directory, false, getListMaxResults(), null, - new TracingContext("test-corr-id", getFileSystem().getFileSystemID(), - "PA")).getResult().getListResultSchema().paths(); + tracingContext).getResult().getListResultSchema().paths(); } private int getListMaxResults() throws IOException { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java index 71ef60c98b85d..9950e7995e9f0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java @@ -64,7 +64,7 @@ public void testInitValues() throws IOException { outputStream = createAbfsOutputStreamWithFlushEnabled(fs, initValuesPath); inputStream = abfss.openFileForRead(initValuesPath, fs.getFsStatistics(), - new TracingContext("test-corr-id", "test-filesystem-id", "NS")); + tracingContext); AbfsInputStreamStatisticsImpl stats = (AbfsInputStreamStatisticsImpl) inputStream.getStreamStatistics(); @@ -109,7 +109,7 @@ public void testSeekStatistics() throws IOException { out.write(defBuffer); out.hflush(); in = abfss.openFileForRead(seekStatPath, fs.getFsStatistics(), - new TracingContext("test-corr-id", fs.getFileSystemID(), "IN")); + tracingContext); /* * Writing 1MB buffer to the file, this would make the fCursor(Current @@ -201,7 +201,7 @@ public void testReadStatistics() throws IOException { out.write(defBuffer); out.hflush(); in = abfss.openFileForRead(readStatPath, fs.getFsStatistics(), - new TracingContext("test-corr-id", "test-filesystem-id", "NS")); + tracingContext); /* * Doing file read 10 times. @@ -271,9 +271,6 @@ public void testWithNullStreamStatistics() throws IOException { out.write(oneKbBuff); out.hflush(); - TracingContext tracingContext = new TracingContext("testCorrId", - fs.getFileSystemID(), "PA"); - // AbfsRestOperation Instance required for eTag. AbfsRestOperation abfsRestOperation = fs.getAbfsClient().getPathStatus(nullStatFilePath.toUri().getPath(), false, @@ -327,8 +324,7 @@ public void testReadAheadCounters() throws IOException { out.write(defBuffer); out.close(); - in = abfss.openFileForRead(readAheadCountersPath, fs.getFsStatistics(), - new TracingContext("test-corr-id", "test-filesystem-id", "NS")); + in = abfss.openFileForRead(readAheadCountersPath, fs.getFsStatistics(), tracingContext); /* * Reading 1KB after each i * KB positions. Hence the reads are from 0 diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 085cd72ebe0f6..b53286ba7d561 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -344,9 +344,6 @@ public void testNegativeScenariosForCreateOverwriteDisabled() = TestAbfsClient.getMockAbfsClient( fs.getAbfsStore().getClient(), fs.getAbfsStore().getAbfsConfiguration()); - TracingContext tracingContext = new TracingContext( - fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), "CR"); AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); abfsStore = setAzureBlobSystemStoreField(abfsStore, "client", mockClient); @@ -466,7 +463,7 @@ private void validateCreateFileException(final Class ex intercept( exceptionClass, () -> abfsStore.createFile(testPath, null, true, permission, umask, - new TracingContext("test-corr-id", "test-filesystem-id", "CR"))); + tracingContext)); } private AbfsRestOperationException getMockAbfsRestOperationException(int status) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index ffb10f0efd00d..492ff75442b5d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -213,9 +213,6 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { fs.getAbfsStore().getClient(), this.getConfiguration()); - TracingContext tracingContext = new TracingContext(fs.getAbfsStore() - .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), "DL"); - // Case 1: Not a retried case should throw error back intercept(AbfsRestOperationException.class, () -> client.deletePath( @@ -238,7 +235,7 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { doReturn(idempotencyRetOp).when(mockClient).deleteIdempotencyCheckOp(any()); when(mockClient.deletePath("/NonExistingPath", false, - null, tracingContext)).thenCallRealMethod(); + null, any(TracingContext.class))).thenCallRealMethod(); Assertions.assertThat(mockClient.deletePath( "/NonExistingPath", diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java index 5bed7243ee984..a761e923a0241 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java @@ -22,7 +22,6 @@ import java.io.InputStream; import java.util.Map; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Assume; import org.junit.Test; import org.slf4j.Logger; @@ -124,8 +123,6 @@ public void testBlobDataReader() throws Exception { Assume.assumeTrue("Reader client id not provided", clientId != null); String secret = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET); Assume.assumeTrue("Reader client secret not provided", secret != null); - TracingContext tracingContext = new TracingContext("test-corr-id", - "test-fileSystemID", "AU"); prepareFiles(); final AzureBlobFileSystem fs = getBlobReader(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java index e5bc772d5c329..5e4cc45fedaa6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java @@ -26,7 +26,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.assertj.core.api.Assertions; import org.junit.Test; import org.junit.Assert; @@ -238,15 +237,11 @@ private void testRenameIdempotencyTriggerChecks( fs.getAbfsStore().getClient(), this.getConfiguration()); - TracingContext tracingContext = new TracingContext(fs.getAbfsStore() - .getAbfsConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), "RN"); - AbfsRestOperation idempotencyRetOp = mock(AbfsRestOperation.class); when(idempotencyRetOp.getResult()).thenReturn(idempotencyRetHttpOp); doReturn(idempotencyRetOp).when(client).renameIdempotencyCheckOp(any(), - any(), any(), tracingContext); - when(client.renamePath(any(), any(), any(), tracingContext)).thenCallRealMethod(); + any(), any(), any()); + when(client.renamePath(any(), any(), any(), any())).thenCallRealMethod(); // rename on non-existing source file will trigger idempotency check if (idempotencyRetHttpOp.getStatusCode() == HTTP_OK) { @@ -328,8 +323,7 @@ private void testRenameTimeout( Assertions.assertThat(testClient.renameIdempotencyCheckOp( renameRequestStartTime, op, - destinationPath.toUri().getPath(), - new TracingContext("test-corr-id", "test-fs-id", "RN")) + destinationPath.toUri().getPath(), tracingContext) .getResult() .getStatusCode()) .describedAs(assertMessage) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java index 4500c121b7eb6..2cc1298a01943 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.Arrays; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -109,8 +108,7 @@ public ITestAzureBlobFileSystemStoreListStatusWithRange() throws Exception { public void testListWithRange() throws IOException { try { FileStatus[] listResult = store.listStatus(new Path(path), startFrom, - new TracingContext("test-corr-id", - "test-filesystem-id", "LS")); + tracingContext); if (!expectedResult) { Assert.fail("Excepting failure with IllegalArgumentException"); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java deleted file mode 100644 index 8f17042a22202..0000000000000 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestClientCorrelationHeader.java +++ /dev/null @@ -1,212 +0,0 @@ -package org.apache.hadoop.fs.azurebfs; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; -import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; -import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; -import org.apache.hadoop.fs.azurebfs.services.*; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.http.HttpException; - -import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; - -//import org.graalvm.compiler.debug.Assertions; -import org.junit.Test; -import org.assertj.core.api.Assertions; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.fs.Path; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -//import static org.powermock.api.easymock.PowerMock.*; - -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_CORRELATIONID; -import static org.mockito.ArgumentMatchers.*; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.when; - -public final class ITestClientCorrelationHeader extends AbstractAbfsIntegrationTest { -// AzureBlobFileSystem fs = getFileSystem(); //ensure new instance of fs is created here - private static final int HTTP_CREATED = 201; - private static final String[] CLIENT_CORRELATIONID_LIST = { - "valid-corr-id-123", "inval!d", ""}; - - public ITestClientCorrelationHeader() throws Exception { - super(); - } - - @Test - public void testCreateOverwrite() throws IOException { - AzureBlobFileSystem fs = getFileSystem(); - AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); - TracingContext tracingContext = new TracingContext("corr-id", - fs.getFileSystemID(), "CR"); - AbfsOutputStream out = (AbfsOutputStream) abfsStore.createFile( - new Path("/testFolder"), fs.getFsStatistics(),true, - FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()), - tracingContext); -// tracingContext.headers.clear(); -// System.out.println("now second call"); - out = (AbfsOutputStream) abfsStore.createFile( - new Path("/testFolder"), fs.getFsStatistics(),true, - FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()), - tracingContext); - - ArrayList headers = tracingContext.getRequestHeaders(); - testHeaders(headers); - testPrimaryRequestID(headers); - } - - @Test - public void testListStatus() throws IOException { - AzureBlobFileSystem fs = getFileSystem(); - AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); - TracingContext tracingContext = new TracingContext("corr-id", - fs.getFileSystemID(), "LS", "preqid"); - abfsStore.listStatus(new Path("/"), tracingContext); - ArrayList headers = tracingContext.getRequestHeaders(); - for (String header : headers) { - System.out.println(header); - } - testHeaders(headers); - testPrimaryRequestID(headers); - } - -// @Test -// public void testStream() throws IOException { -// } - - public void testHeaders(ArrayList headers) { - String operation = headers.get(0).split(":")[5]; - boolean isContinuationOp = false; - boolean isStreamOp = false; - switch (operation) { - case "LS": - case "CR": - case "RD": isContinuationOp = true; - } - for (String headerList : headers) { - String[] id_list = headerList.split(":"); - System.out.println(headerList); - // check format for parsing - Assertions.assertThat(id_list) - .describedAs("header should have 7 elements").hasSize(7); - - //check that necessary ids are not empty - Assertions.assertThat(id_list[1]).isNotEmpty(); //client request id - Assertions.assertThat(id_list[2]).isNotEmpty(); //filesystem id - - //check that primary req id is present when required - if (isContinuationOp) { - Assertions.assertThat(id_list[3]).describedAs("Continuation ops should have a primary request id") - .isNotEmpty(); - } - } - - //test values - String fs_id = headers.get(0).split(":")[2]; - String corr_id = headers.get(0).split(":")[0]; - for (String headerList : headers) { - String[] id_list = headerList.split(":"); - Assertions.assertThat(fs_id) - .describedAs("filesystem id should be same for requests with same filesystem") - .isEqualTo(id_list[2]); - Assertions.assertThat(corr_id) - .describedAs("correlation id should be same for requests using a given config") - .isEqualTo(id_list[0]); - Assertions.assertThat(Integer.parseInt(id_list[6])) - .describedAs("Max retries allowed = 30") - .isLessThan(30); - } - } - - public void testPrimaryRequestID(ArrayList headerList) { - // readahead - // liststatus - // createoverwrite - // rename - String preq = headerList.get(0).split(":")[3]; - for (String header : headerList) { - Assertions.assertThat (header.split(":")[3]) - .describedAs("preq id should be same for given set of requests") - .isEqualTo(preq); - } - } - - public void checkStreamID(String streamID, ArrayList headers) { - for (String header : headers) { - Assertions.assertThat(header.split(":")[4]) - .describedAs("stream id should be same for a given stream") - .isEqualTo(streamID); - } - } - - @Test - public void testClientCorrelationId() throws IOException { - checkRequest(CLIENT_CORRELATIONID_LIST[0], true); - checkRequest(CLIENT_CORRELATIONID_LIST[1], false); - checkRequest(CLIENT_CORRELATIONID_LIST[2], false); - } - - private String getOctalNotation(FsPermission fsPermission) { - Preconditions.checkNotNull(fsPermission, "fsPermission"); - return String.format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal()); - } - - private String getRelativePath(final Path path) { - Preconditions.checkNotNull(path, "path"); - return path.toUri().getPath(); - } - - public void checkRequest(String clientCorrelationId, boolean includeInHeader) - throws IOException { - org.apache.hadoop.conf.Configuration config = new Configuration(this.getRawConfiguration()); - config.set(FS_AZURE_CLIENT_CORRELATIONID, clientCorrelationId); - - final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem - .newInstance(this.getFileSystem().getUri(), config); - AbfsClient client = fs.getAbfsClient(); -// TracingContext - String path = getRelativePath(new Path("/testDir")); - boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(); - String permission = isNamespaceEnabled ? getOctalNotation(FsPermission.getDirDefault()) : null; - String umask = isNamespaceEnabled ? getOctalNotation(FsPermission.getUMask(fs.getConf())) : null; - AbfsRestOperation op = client.createPath(path, false, true, - permission, umask, false, null, - new TracingContext(clientCorrelationId, - fs.getFileSystemID(), "CR")); - - int responseCode = op.getResult().getStatusCode(); - Assertions.assertThat(responseCode).describedAs("Status code").isEqualTo(HTTP_CREATED); - - Assertions.assertThat(responseCode).describedAs("Status code").isEqualTo(HTTP_CREATED); - - String requestHeader = op.getResult().getRequestHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); - List clientRequestIds = java.util.Arrays.asList( - requestHeader.replace("[","") - .replace("]", "") - .split(":")); - if (includeInHeader) { - Assertions.assertThat(clientRequestIds) - .describedAs("There should be 7 items in the header when valid clientCorrelationId is set") - .hasSize(7); - Assertions.assertThat(clientRequestIds) - .describedAs("clientCorrelation should be included in the header") - .contains(clientCorrelationId); - } else if (clientCorrelationId.length() > 0){ - Assertions.assertThat(clientRequestIds) - .describedAs("There should be only 6 item in the header when invalid clientCorrelationId is set") - .hasSize(6); - Assertions.assertThat(clientRequestIds) - .describedAs("Invalid or empty correlationId value should not be included in header") - .doesNotContain(clientCorrelationId); - } - } -} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java index e7e6b7bd0f924..5727c55538993 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java @@ -26,7 +26,6 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; /** * Test FileSystemProperties. @@ -62,8 +61,6 @@ public void testWriteOneByteToFileAndEnsureThreadPoolCleanup() throws Exception @Test public void testBase64FileSystemProperties() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); - TracingContext tracingContext = new TracingContext(fs.getAbfsStore().getAbfsConfiguration() - .getClientCorrelationID(), fs.getFileSystemID(), "FS"); final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: value }"); @@ -77,8 +74,6 @@ public void testBase64FileSystemProperties() throws Exception { @Test public void testBase64PathProperties() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); - TracingContext tracingContext = new TracingContext(fs.getAbfsStore().getAbfsConfiguration() - .getClientCorrelationID(), fs.getFileSystemID(), "FS"); final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueTest }"); touch(TEST_PATH); @@ -92,8 +87,6 @@ public void testBase64PathProperties() throws Exception { @Test (expected = Exception.class) public void testBase64InvalidFileSystemProperties() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); - TracingContext tracingContext = new TracingContext(fs.getAbfsStore().getAbfsConfiguration() - .getClientCorrelationID(), fs.getFileSystemID(), "FS"); final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueæ­² }"); fs.getAbfsStore().setFilesystemProperties(properties, tracingContext); @@ -106,8 +99,6 @@ public void testBase64InvalidFileSystemProperties() throws Exception { @Test (expected = Exception.class) public void testBase64InvalidPathProperties() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); - TracingContext tracingContext = new TracingContext(fs.getAbfsStore().getAbfsConfiguration() - .getClientCorrelationID(), fs.getFileSystemID(), "FS"); final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueTestå…© }"); touch(TEST_PATH); @@ -121,8 +112,6 @@ public void testBase64InvalidPathProperties() throws Exception { @Test public void testSetFileSystemProperties() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); - TracingContext tracingContext = new TracingContext(fs.getAbfsStore().getAbfsConfiguration() - .getClientCorrelationID(), fs.getFileSystemID(), "FS"); final Hashtable properties = new Hashtable<>(); properties.put("containerForDevTest", "true"); fs.getAbfsStore().setFilesystemProperties(properties, tracingContext); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java index 3c6f216273cd1..22abf4e43941d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java @@ -34,6 +34,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -55,8 +56,6 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest { private static final String TRUE_STR = "true"; private static final String FALSE_STR = "false"; - private final String fileSystemID = "test-filesystem-id"; - private final String clientCorrelationID = "test-corr-id"; private boolean isUsingXNSAccount; public ITestGetNameSpaceEnabled() throws Exception { @@ -85,8 +84,7 @@ public void testGetIsNamespaceEnabledWhenConfigIsTrue() throws Exception { Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs( "getIsNamespaceEnabled should return true when the " + "config is set as true").isTrue(); - fs.getAbfsStore().deleteFilesystem(new TracingContext(clientCorrelationID, - fileSystemID, "DL")); + fs.getAbfsStore().deleteFilesystem(tracingContext); unsetAndAssert(); } @@ -96,7 +94,7 @@ public void testGetIsNamespaceEnabledWhenConfigIsFalse() throws Exception { Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs( "getIsNamespaceEnabled should return false when the " + "config is set as false").isFalse(); - fs.getAbfsStore().deleteFilesystem(new TracingContext(clientCorrelationID, fileSystemID, "DL")); + fs.getAbfsStore().deleteFilesystem(tracingContext); unsetAndAssert(); } @@ -109,8 +107,7 @@ private void unsetAndAssert() throws Exception { "getIsNamespaceEnabled should return the value " + "configured for fs.azure.test.namespace.enabled") .isEqualTo(expectedValue); - fs.getAbfsStore().deleteFilesystem(new TracingContext(clientCorrelationID, - fileSystemID, "DL")); + fs.getAbfsStore().deleteFilesystem(tracingContext); } private AzureBlobFileSystem getNewFSWithHnsConf( @@ -183,8 +180,7 @@ private void ensureGetAclCallIsMadeOnceForInvalidConf(String invalidConf) .setNamespaceEnabled(Trilean.getTrilean(invalidConf)); AbfsClient mockClient = callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient(); - verify(mockClient, times(1)).getAclStatus(anyString(), - new TracingContext(clientCorrelationID, fileSystemID, "NS")); + verify(mockClient, times(1)).getAclStatus(anyString(), any(TracingContext.class)); } private void ensureGetAclCallIsNeverMadeForValidConf(String validConf) @@ -193,16 +189,14 @@ private void ensureGetAclCallIsNeverMadeForValidConf(String validConf) .setNamespaceEnabled(Trilean.getTrilean(validConf)); AbfsClient mockClient = callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient(); - verify(mockClient, never()).getAclStatus(anyString(), - new TracingContext(clientCorrelationID, fileSystemID, "NS")); + verify(mockClient, never()).getAclStatus(anyString(), any(TracingContext.class)); } private void unsetConfAndEnsureGetAclCallIsMadeOnce() throws IOException { this.getFileSystem().getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN); AbfsClient mockClient = callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient(); - verify(mockClient, times(1)).getAclStatus(anyString(), - new TracingContext(clientCorrelationID, fileSystemID, "NS")); + verify(mockClient, times(1)).getAclStatus(anyString(), any(TracingContext.class)); } private AbfsClient callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient() @@ -211,7 +205,7 @@ private AbfsClient callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient() final AzureBlobFileSystemStore abfsStore = abfs.getAbfsStore(); final AbfsClient mockClient = mock(AbfsClient.class); doReturn(mock(AbfsRestOperation.class)).when(mockClient) - .getAclStatus(anyString(), new TracingContext(clientCorrelationID, fileSystemID, "NS")); + .getAclStatus(anyString(), any(TracingContext.class)); abfsStore.setClient(mockClient); abfs.getIsNamespaceEnabled(); return mockClient; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java index ebc58e53d4af1..36d9886e0d6b3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.fs.azurebfs; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Assume; import org.junit.Test; @@ -55,8 +54,7 @@ public void testWithWrongSharedKey() throws Exception { + "Authorization header is formed correctly including the " + "signature.\", 403", () -> { - abfsClient.getAclStatus("/", new TracingContext("test-corr-id", - "test-fileSystemID", "AC")); + abfsClient.getAclStatus("/", tracingContext); }); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index 6b190035e840b..7a7992d9bb475 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -31,7 +31,6 @@ 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.security.ssl.DelegatingSSLSocketFactory; import static org.assertj.core.api.Assertions.assertThat; @@ -335,12 +334,6 @@ public static AbfsClient getMockAbfsClient(AbfsClient baseAbfsClientInstance, + "UNKNOWN/UNKNOWN) MSFT"; client = TestAbfsClient.setAbfsClientField(client, "userAgent", userAgent); - // set tracking context -// TracingContext tracingContext = new TracingContext("clientCorrID", "TS"); -// client = TestAbfsClient.setAbfsClientField(client, "tracingContext", -// tracingContext); -// when(client.getTracingContext()).thenReturn(tracingContext); - return client; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index c9b547dc74c11..e266f7a64a855 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -82,8 +82,7 @@ private AbfsInputStream getAbfsInputStream(AbfsClient mockAbfsClient, String fil THREE_KB, inputStreamContext.withReadBufferSize(ONE_KB).withReadAheadQueueDepth(10), "eTag", - new TracingContext("testCorrId", - getFileSystem().getFileSystemID(), "IP")); + tracingContext); inputStream.setCachedSasToken( TestCachedSASToken.getTestCachedSASTokenInstance()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index 86fe6c91908e6..5cfdb48c9578b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -325,7 +325,7 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, true), new TracingContext(abfsConf.getClientCorrelationID(), - "fs-id", "OP")); + "test-fs-id", "OP")); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); From 6be948e986a26c754ba9306a5f6a740e42f25bdd Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 27 Oct 2020 23:57:35 +0530 Subject: [PATCH 24/77] minor edits --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 31 ++++++------------- .../azurebfs/services/AbfsHttpOperation.java | 1 - .../src/test/resources/log4j.properties | 1 - pom.xml | 4 +-- 4 files changed, 10 insertions(+), 27 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index f9c9237ae6694..46b98671bead5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -280,11 +280,6 @@ private String[] authorityParts(URI uri) throws InvalidUriAuthorityException, In return authorityParts; } - public boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException { - return getIsNamespaceEnabled(new TracingContext(abfsConfiguration.getClientCorrelationID(), - "test-filesystem-id", "NS")); - } - public boolean getIsNamespaceEnabled(TracingContext tracingContext) throws AzureBlobFileSystemException { try { return this.isNamespaceEnabled.toBoolean(); @@ -488,20 +483,19 @@ public OutputStream createFile(final Path path, AbfsRestOperation op; if (triggerConditionalCreateOverwrite) { op = conditionalCreateOverwriteFile(relativePath, - statistics, - isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null, - isAppendBlob, tracingContext + statistics, + isNamespaceEnabled ? getOctalNotation(permission) : null, + isNamespaceEnabled ? getOctalNotation(umask) : null, + isAppendBlob, tracingContext ); } else { op = client.createPath(relativePath, true, - overwrite, - isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null, - isAppendBlob, - null, tracingContext); -// tracingContext.headers.add(tracingContext.toString()); + overwrite, + isNamespaceEnabled ? getOctalNotation(permission) : null, + isNamespaceEnabled ? getOctalNotation(umask) : null, + isAppendBlob, + null, tracingContext); } perfInfo.registerResult(op.getResult()).registerSuccess(true); @@ -656,7 +650,6 @@ public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statist relativePath, contentLength, populateAbfsInputStreamContext(), eTag, tracingContext); -// tracingContext.setStreamID(in.getInputStreamID()); return in; } } @@ -710,7 +703,6 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic offset, populateAbfsOutputStreamContext(isAppendBlob), tracingContext); -// tracingContext.setStreamID(out.getOutputStreamID()); return out; } @@ -880,7 +872,6 @@ public FileStatus[] listStatus(final Path path, final String startFrom, final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue = true; -// tracingContext.updateIteration(); LOG.debug("listStatus filesystem: {} path: {}, startFrom: {}", client.getFileSystem(), @@ -906,10 +897,6 @@ public FileStatus[] listStatus(final Path path, final String startFrom, perfInfo.registerResult(op.getResult()); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); ListResultSchema retrievedSchema = op.getResult().getListResultSchema(); -// originalTracingContext.updateRequestHeader(op.getResult().getRequestHeader( -// HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID)); -// originalTracingContext.headers.add(op.tracingContext.toString()); -// originalTracingContext.headers.add(op.requestHeader); if (retrievedSchema == null) { throw new AbfsRestOperationException( AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index c93bf47dfeec8..d7704b211cb42 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -264,7 +264,6 @@ public AbfsHttpOperation(final URL url, final String method, final Listbash org.fusesource.leveldbjni - + hadoop-project From 077b5bd26503e998071c3b4289d6ddbeab79c63c Mon Sep 17 00:00:00 2001 From: Sumangala Date: Wed, 28 Oct 2020 00:10:31 +0530 Subject: [PATCH 26/77] minor edits --- .../apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index d7704b211cb42..c30e7449fe143 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -83,7 +83,7 @@ public class AbfsHttpOperation implements AbfsPerfLoggable { public static AbfsHttpOperation getAbfsHttpOperationWithFixedResult(final URL url, final String method, final int httpStatus) { - return new AbfsHttpOperation(url, method, httpStatus); + return new AbfsHttpOperation(url, method, httpStatus); } private AbfsHttpOperation(final URL url, final String method, @@ -245,7 +245,6 @@ public AbfsHttpOperation(final URL url, final String method, final List Date: Wed, 28 Oct 2020 00:42:35 +0530 Subject: [PATCH 27/77] minor edits/whitespc --- .../azurebfs/services/AbfsHttpOperation.java | 2 +- .../fs/azurebfs/services/AbfsInputStream.java | 3 +-- .../azurebfs/services/AbfsRestOperation.java | 1 - .../azurebfs/ITestAbfsReadWriteAndSeek.java | 1 - .../MockDelegationSASTokenProvider.java | 1 - .../services/TestAbfsOutputStream.java | 19 +++++++++++-------- 6 files changed, 13 insertions(+), 14 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index c30e7449fe143..4315f5f4f9ced 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -265,7 +265,7 @@ public AbfsHttpOperation(final URL url, final String method, final List acFlushRetainUnCommittedData = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acFlushClose = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); + ArgumentCaptor acTracingContext = ArgumentCaptor.forClass(TracingContext.class); verify(client, times(1)).flush(acFlushString.capture(), acFlushLong.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture(), tracingContext); + acFlushSASToken.capture(), acTracingContext.capture()); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushString.getAllValues()); assertThat(Arrays.asList(Long.valueOf(5*WRITE_SIZE))).describedAs("position").isEqualTo(acFlushLong.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); @@ -242,9 +244,10 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { ArgumentCaptor acFlushRetainUnCommittedData = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acFlushClose = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); + ArgumentCaptor acTracingContext = ArgumentCaptor.forClass(TracingContext.class); verify(client, times(1)).flush(acFlushString.capture(), acFlushLong.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture(), tracingContext); + acFlushSASToken.capture(), acTracingContext.capture()); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushString.getAllValues()); assertThat(Arrays.asList(Long.valueOf(2*BUFFER_SIZE))).describedAs("position").isEqualTo(acFlushLong.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); @@ -276,7 +279,7 @@ public void verifyWriteRequestOfBufferSize() throws Exception { AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, - false), new TracingContext("corr-id", "fs-id", "WR")); + false), new TracingContext(abfsConf.getClientCorrelationID(), "fs-id", "WR")); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -365,7 +368,7 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); - TracingContext tracingContext = new TracingContext("test-corr-id", + TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), "test-fs-id", "PA"); when(client.getAbfsPerfTracker()).thenReturn(tracker); @@ -428,7 +431,7 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { final Configuration conf = new Configuration(); conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); - TracingContext tracingContext = new TracingContext("test-corr-id", + TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), "test-fs-id", "WR"); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); From 9457a04dbcac6da42253c4f9bdc0c98758e608a2 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Mon, 2 Nov 2020 16:18:06 +0530 Subject: [PATCH 28/77] merge conflict --- .../java/org/apache/hadoop/fs/azurebfs/utils/Listener.java | 4 ++++ .../java/org/apache/hadoop/fs/azurebfs/utils/testHeader.java | 2 ++ .../java/org/apache/hadoop/fs/azurebfs/ITestCorrelation.java | 2 ++ 3 files changed, 8 insertions(+) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/testHeader.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCorrelation.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java new file mode 100644 index 0000000000000..d1b275eb90c6f --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java @@ -0,0 +1,4 @@ +package org.apache.hadoop.fs.azurebfs.utils; + +public interface Listener { +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/testHeader.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/testHeader.java new file mode 100644 index 0000000000000..113eac93e4b6c --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/testHeader.java @@ -0,0 +1,2 @@ +package org.apache.hadoop.fs.azurebfs.utils;public class testHeader { +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCorrelation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCorrelation.java new file mode 100644 index 0000000000000..7a9187c705ba7 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCorrelation.java @@ -0,0 +1,2 @@ +package org.apache.hadoop.fs.azurebfs;public class ITestCorrelation { +} From 16811384244eeb61895c7d04037000b646e5d717 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 3 Nov 2020 16:02:00 +0530 Subject: [PATCH 29/77] pr changes + dummyTC --- .../hadoop-azure/src/config/codestyle.xml | 61 +++++++++++++ .../hadoop/fs/azurebfs/AbfsConfiguration.java | 11 +-- .../fs/azurebfs/AzureBlobFileSystem.java | 90 +++++++++++-------- .../fs/azurebfs/AzureBlobFileSystemStore.java | 28 +++--- .../fs/azurebfs/constants/AbfsOperations.java | 20 +++++ .../azurebfs/constants/ConfigurationKeys.java | 3 +- .../fs/azurebfs/services/AbfsClient.java | 12 --- .../fs/azurebfs/services/AbfsInputStream.java | 9 +- .../azurebfs/services/AbfsOutputStream.java | 10 ++- .../azurebfs/services/AbfsRestOperation.java | 24 +++-- .../azurebfs/services/ReadBufferManager.java | 1 - .../hadoop/fs/azurebfs/utils/Listener.java | 4 - .../fs/azurebfs/utils/TracingContext.java | 33 ++++--- .../hadoop/fs/azurebfs/utils/testHeader.java | 2 - .../azurebfs/AbstractAbfsIntegrationTest.java | 26 ++++-- .../hadoop/fs/azurebfs/ITestAbfsClient.java | 6 +- .../ITestAbfsInputStreamStatistics.java | 13 +-- .../ITestAzureBlobFileSystemCreate.java | 5 +- .../ITestAzureBlobFileSystemDelete.java | 4 +- .../ITestAzureBlobFileSystemOauth.java | 2 + .../ITestAzureBlobFileSystemRename.java | 7 +- ...lobFileSystemStoreListStatusWithRange.java | 2 +- .../hadoop/fs/azurebfs/ITestCorrelation.java | 2 - .../azurebfs/ITestFileSystemProperties.java | 6 ++ .../fs/azurebfs/ITestGetNameSpaceEnabled.java | 6 +- .../fs/azurebfs/ITestSharedKeyAuth.java | 3 +- .../services/TestAbfsInputStream.java | 2 +- .../services/TestAbfsOutputStream.java | 76 +++++++++++----- 28 files changed, 307 insertions(+), 161 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/config/codestyle.xml create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperations.java delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/testHeader.java delete mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCorrelation.java diff --git a/hadoop-tools/hadoop-azure/src/config/codestyle.xml b/hadoop-tools/hadoop-azure/src/config/codestyle.xml new file mode 100644 index 0000000000000..28dde371bbb87 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/config/codestyle.xml @@ -0,0 +1,61 @@ + + + diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index df974575c9fdb..7a96f15ca56ed 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -230,9 +230,10 @@ public class AbfsConfiguration{ DefaultValue = EMPTY_STRING) private String clientCorrelationID; - @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ENABLE_CORRELATION_HEADER, - DefaultValue = true) - private boolean enableCorrelationHeader; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = + FS_AZURE_TRACINGCONTEXT_FORMAT, + DefaultValue = 1) + private int tracingContextFormat; @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ENABLE_DELEGATION_TOKEN, DefaultValue = DEFAULT_ENABLE_DELEGATION_TOKEN) @@ -309,8 +310,8 @@ public String getClientCorrelationID() { * Default value is true. Only clientRequestId appears in logs when disabled * @return enableCorrelationHeader config */ - public boolean isCorrelationHeaderEnabled() { - return enableCorrelationHeader; + public int getTracingContextFormat() { + return tracingContextFormat; } /** diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index b2c4770c40e4f..e565339c09435 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -60,6 +60,7 @@ import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperations; import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; @@ -96,12 +97,13 @@ public class AzureBlobFileSystem extends FileSystem { private Path workingDir; private AzureBlobFileSystemStore abfsStore; private boolean isClosed; - private String fileSystemID; + private final String fileSystemID = UUID.randomUUID().toString(); private boolean delegationTokenEnabled = false; private AbfsDelegationTokenManager delegationTokenManager; private AbfsCounters abfsCounters; private String clientCorrelationID; + private int tracingContextFormat; @Override public void initialize(URI uri, Configuration configuration) @@ -109,7 +111,6 @@ public void initialize(URI uri, Configuration configuration) uri = ensureAuthority(uri, configuration); super.initialize(uri, configuration); setConf(configuration); - fileSystemID = UUID.randomUUID().toString(); LOG.debug("Initializing AzureBlobFileSystem for {}", uri); @@ -121,6 +122,7 @@ public void initialize(URI uri, Configuration configuration) final AbfsConfiguration abfsConfiguration = abfsStore.getAbfsConfiguration(); clientCorrelationID = abfsConfiguration.getClientCorrelationID(); + tracingContextFormat = abfsConfiguration.getTracingContextFormat(); this.setWorkingDirectory(this.getHomeDirectory()); if (abfsConfiguration.getCreateRemoteFileSystemDuringInitialization()) { @@ -175,7 +177,8 @@ public URI getUri() { return this.uri; } - public String getFileSystemID() { + @VisibleForTesting + String getFileSystemID() { return fileSystemID; } @@ -183,7 +186,8 @@ public String getFileSystemID() { public FSDataInputStream open(final Path path, final int bufferSize) throws IOException { LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize); statIncrement(CALL_OPEN); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "OF"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.OPEN, tracingContextFormat); Path qualifiedPath = makeQualified(path); try { @@ -203,7 +207,8 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi permission, overwrite, blockSize); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "CR"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.CREATE, tracingContextFormat); statIncrement(CALL_CREATE); trailingPeriodCheck(f); @@ -273,7 +278,8 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr f.toString(), bufferSize); statIncrement(CALL_APPEND); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AP"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.APPEND, tracingContextFormat); Path qualifiedPath = makeQualified(f); try { @@ -289,8 +295,8 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr public boolean rename(final Path src, final Path dst) throws IOException { LOG.debug("AzureBlobFileSystem.rename src: {} dst: {}", src, dst); statIncrement(CALL_RENAME); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "RN"); - tracingContext.setPrimaryRequestID(); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.RENAME, true, tracingContextFormat); trailingPeriodCheck(dst); @@ -361,7 +367,8 @@ public boolean delete(final Path f, final boolean recursive) throws IOException "AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), recursive); statIncrement(CALL_DELETE); Path qualifiedPath = makeQualified(f); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "DL"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.DELETE, tracingContextFormat); if (f.isRoot()) { if (!recursive) { @@ -387,8 +394,8 @@ public FileStatus[] listStatus(final Path f) throws IOException { "AzureBlobFileSystem.listStatus path: {}", f.toString()); statIncrement(CALL_LIST_STATUS); Path qualifiedPath = makeQualified(f); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "LS"); - tracingContext.setPrimaryRequestID(); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.LISTSTATUS, true, tracingContextFormat); try { FileStatus[] result = abfsStore.listStatus(qualifiedPath, tracingContext); @@ -448,8 +455,8 @@ public boolean mkdirs(final Path f, final FsPermission permission) throws IOExce "AzureBlobFileSystem.mkdirs path: {} permissions: {}", f, permission); statIncrement(CALL_MKDIRS); trailingPeriodCheck(f); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "MK"); - tracingContext.setPrimaryRequestID(); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.MKDIR, true, tracingContextFormat); final Path parentFolder = f.getParent(); if (parentFolder == null) { @@ -488,7 +495,8 @@ public FileStatus getFileStatus(final Path f) throws IOException { LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f); statIncrement(CALL_GET_FILE_STATUS); Path qualifiedPath = makeQualified(f); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "FS"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.FILESYSTEM_OPS, tracingContextFormat); try { return abfsStore.getFileStatus(qualifiedPath, tracingContext); @@ -665,8 +673,8 @@ public void setOwner(final Path path, final String owner, final String group) throws IOException { LOG.debug( "AzureBlobFileSystem.setOwner path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "PA"); - tracingContext.setPrimaryRequestID(); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.PATH_OPS, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { super.setOwner(path, owner, group); @@ -703,8 +711,8 @@ public void setOwner(final Path path, final String owner, final String group) public void setXAttr(final Path path, final String name, final byte[] value, final EnumSet flag) throws IOException { LOG.debug("AzureBlobFileSystem.setXAttr path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AT"); - tracingContext.setPrimaryRequestID(); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.ATTR_OPS, true, tracingContextFormat); if (name == null || name.isEmpty() || value == null) { throw new IllegalArgumentException("A valid name and value must be specified."); @@ -744,8 +752,8 @@ public byte[] getXAttr(final Path path, final String name) if (name == null || name.isEmpty()) { throw new IllegalArgumentException("A valid name must be specified."); } - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AT"); - tracingContext.setPrimaryRequestID(); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.ATTR_OPS, true, tracingContextFormat); Path qualifiedPath = makeQualified(path); @@ -778,8 +786,8 @@ private static String ensureValidAttributeName(String attribute) { public void setPermission(final Path path, final FsPermission permission) throws IOException { LOG.debug("AzureBlobFileSystem.setPermission path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "PR"); - tracingContext.setPrimaryRequestID(); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.PERMISSION, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { super.setPermission(path, permission); @@ -814,8 +822,8 @@ public void setPermission(final Path path, final FsPermission permission) public void modifyAclEntries(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.modifyAclEntries path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AC"); - tracingContext.setPrimaryRequestID(); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.ACL_OPS,true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -849,8 +857,8 @@ public void modifyAclEntries(final Path path, final List aclSpec) public void removeAclEntries(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.removeAclEntries path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AC"); - tracingContext.setPrimaryRequestID(); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.ACL_OPS, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -880,7 +888,8 @@ public void removeAclEntries(final Path path, final List aclSpec) @Override public void removeDefaultAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeDefaultAcl path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AC"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.ACL_OPS, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -908,8 +917,8 @@ public void removeDefaultAcl(final Path path) throws IOException { @Override public void removeAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeAcl path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AC"); - tracingContext.setPrimaryRequestID(); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.ACL_OPS, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -940,8 +949,8 @@ public void removeAcl(final Path path) throws IOException { public void setAcl(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.setAcl path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AC"); - tracingContext.setPrimaryRequestID(); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.ACL_OPS, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -972,8 +981,8 @@ public void setAcl(final Path path, final List aclSpec) @Override public AclStatus getAclStatus(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.getAclStatus path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "AC"); - tracingContext.setPrimaryRequestID(); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.ACL_OPS, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -1006,7 +1015,8 @@ public AclStatus getAclStatus(final Path path) throws IOException { @Override public void access(final Path path, final FsAction mode) throws IOException { LOG.debug("AzureBlobFileSystem.access path : {}, mode : {}", path, mode); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "LS"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.PATH_OPS, tracingContextFormat); Path qualifiedPath = makeQualified(path); try { this.abfsStore.access(qualifiedPath, mode, tracingContext); @@ -1041,7 +1051,8 @@ private FileStatus tryGetFileStatus(final Path f) { private boolean fileSystemExists() throws IOException { LOG.debug( "AzureBlobFileSystem.fileSystemExists uri: {}", uri); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "FS"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.FILESYSTEM_OPS, tracingContextFormat); try { abfsStore.getFilesystemProperties(tracingContext); } catch (AzureBlobFileSystemException ex) { @@ -1061,7 +1072,8 @@ private boolean fileSystemExists() throws IOException { private void createFileSystem() throws IOException { LOG.debug( "AzureBlobFileSystem.createFileSystem uri: {}", uri); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "FS"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.FILESYSTEM_OPS, tracingContextFormat); try { abfsStore.createFilesystem(tracingContext); } catch (AzureBlobFileSystemException ex) { @@ -1291,7 +1303,8 @@ boolean getIsNamespaceEnabled(TracingContext tracingContext) throws AzureBlobFil @VisibleForTesting boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException { return abfsStore.getIsNamespaceEnabled( - new TracingContext(clientCorrelationID, fileSystemID, "NS")); + new TracingContext(clientCorrelationID, fileSystemID, + AbfsOperations.GET_NAMESPACE, tracingContextFormat)); } @VisibleForTesting @@ -1304,7 +1317,8 @@ public boolean hasPathCapability(final Path path, final String capability) throws IOException { // qualify the path to make sure that it refers to the current FS. final Path p = makeQualified(path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, "PA"); + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.PATH_OPS, tracingContextFormat); switch (validatePathCapabilityArgs(p, capability)) { case CommonPathCapabilities.FS_PERMISSIONS: case CommonPathCapabilities.FS_APPEND: diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 46b98671bead5..5174fa5a70417 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -280,7 +280,7 @@ private String[] authorityParts(URI uri) throws InvalidUriAuthorityException, In return authorityParts; } - public boolean getIsNamespaceEnabled(TracingContext tracingContext) throws AzureBlobFileSystemException { + public boolean getIsNamespaceEnabled(TracingContext tracingContext) throws AzureBlobFileSystemException { try { return this.isNamespaceEnabled.toBoolean(); } catch (TrileanConversionException e) { @@ -486,7 +486,8 @@ public OutputStream createFile(final Path path, statistics, isNamespaceEnabled ? getOctalNotation(permission) : null, isNamespaceEnabled ? getOctalNotation(umask) : null, - isAppendBlob, tracingContext + isAppendBlob, + tracingContext ); } else { @@ -495,7 +496,8 @@ public OutputStream createFile(final Path path, isNamespaceEnabled ? getOctalNotation(permission) : null, isNamespaceEnabled ? getOctalNotation(umask) : null, isAppendBlob, - null, tracingContext); + null, + tracingContext); } perfInfo.registerResult(op.getResult()).registerSuccess(true); @@ -560,8 +562,8 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa try { // overwrite only if eTag matches with the file properties fetched befpre - op = client.createPath(relativePath, true, - true, permission, umask, isAppendBlob, eTag, tracingContext); + op = client.createPath(relativePath, true, true, permission, umask, + isAppendBlob, eTag, tracingContext); } catch (AbfsRestOperationException ex) { if (ex.getStatusCode() == HttpURLConnection.HTTP_PRECON_FAILED) { // Is a parallel access case, as file with eTag was just queried @@ -646,11 +648,10 @@ public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statist perfInfo.registerSuccess(true); // Add statistics for InputStream - AbfsInputStream in = new AbfsInputStream(client, statistics, + return new AbfsInputStream(client, statistics, relativePath, contentLength, populateAbfsInputStreamContext(), eTag, tracingContext); - return in; } } @@ -696,15 +697,13 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic if (isAppendBlobKey(path.toString())) { isAppendBlob = true; } - AbfsOutputStream out = new AbfsOutputStream( + return new AbfsOutputStream( client, statistics, relativePath, offset, populateAbfsOutputStreamContext(isAppendBlob), tracingContext); - - return out; } } @@ -732,7 +731,7 @@ public void rename(final Path source, final Path destination, TracingContext tra do { try (AbfsPerfInfo perfInfo = startTracking("rename", "renamePath")) { AbfsRestOperation op = client.renamePath(sourceRelativePath, - destinationRelativePath, continuation, new TracingContext(tracingContext)); + destinationRelativePath, continuation, tracingContext); perfInfo.registerResult(op.getResult()); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); perfInfo.registerSuccess(true); @@ -868,7 +867,7 @@ public FileStatus[] listStatus(final Path path, TracingContext tracingContext) t * */ @InterfaceStability.Unstable public FileStatus[] listStatus(final Path path, final String startFrom, - TracingContext originalTracingContext) throws IOException { + TracingContext tracingContext) throws IOException { final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue = true; @@ -883,17 +882,16 @@ public FileStatus[] listStatus(final Path path, final String startFrom, // generate continuation token if a valid startFrom is provided. if (startFrom != null && !startFrom.isEmpty()) { - continuation = getIsNamespaceEnabled(originalTracingContext) + continuation = getIsNamespaceEnabled(tracingContext) ? generateContinuationTokenForXns(startFrom) : generateContinuationTokenForNonXns(relativePath, startFrom); } - TracingContext tracingContext = new TracingContext(originalTracingContext); ArrayList fileStatuses = new ArrayList<>(); do { try (AbfsPerfInfo perfInfo = startTracking("listStatus", "listPath")) { AbfsRestOperation op = client.listPath(relativePath, false, - abfsConfiguration.getListMaxResults(), continuation, new TracingContext(tracingContext)); + abfsConfiguration.getListMaxResults(), continuation, tracingContext); perfInfo.registerResult(op.getResult()); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); ListResultSchema retrievedSchema = op.getResult().getListResultSchema(); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperations.java new file mode 100644 index 0000000000000..e9ed78054a342 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperations.java @@ -0,0 +1,20 @@ +package org.apache.hadoop.fs.azurebfs.constants; + +public final class AbfsOperations { + public static final String OPEN = "OP"; + public static final String READ = "RD"; + public static final String WRITE = "WR"; + public static final String CREATE = "CR"; + public static final String LISTSTATUS = "LS"; + public static final String RENAME = "RN"; + public static final String DELETE = "DL"; + public static final String ACL_OPS = "AC"; + public static final String FILESYSTEM_OPS = "FS"; + public static final String MKDIR = "MK"; + public static final String APPEND = "AP"; + public static final String ATTR_OPS = "AT"; + public static final String PERMISSION = "PR"; + public static final String PATH_OPS = "PA"; + public static final String GET_NAMESPACE = "NS"; + public static final String TEST_OP = "TS"; +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index 8082b01fced91..828d609ba93d6 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -90,7 +90,8 @@ public final class ConfigurationKeys { * x-ms-client-request-Id header. Defaults to empty string if the length and * character constraints are not satisfied. **/ public static final String FS_AZURE_CLIENT_CORRELATIONID = "fs.azure.client.correlationid"; - public static final String FS_AZURE_ENABLE_CORRELATION_HEADER = "fs.azure.enable.correlation.header"; + public static final String FS_AZURE_TRACINGCONTEXT_FORMAT = "fs.azure" + + ".tracingcontext.format"; public static final String FS_AZURE_CLUSTER_NAME = "fs.azure.cluster.name"; public static final String FS_AZURE_CLUSTER_TYPE = "fs.azure.cluster.type"; public static final String FS_AZURE_SSL_CHANNEL_MODE_KEY = "fs.azure.ssl.channel.mode"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 1c8056ecf3f20..f7341657a8070 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -75,7 +75,6 @@ public class AbfsClient implements Closeable { private AccessTokenProvider tokenProvider; private SASTokenProvider sasTokenProvider; private final AbfsCounters abfsCounters; - private final boolean enableCorrelationHeader; private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, final AbfsConfiguration abfsConfiguration, @@ -107,7 +106,6 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden this.userAgent = initializeUserAgent(abfsConfiguration, sslProviderName); this.abfsPerfTracker = abfsClientContext.getAbfsPerfTracker(); this.abfsCounters = abfsClientContext.getAbfsCounters(); - this.enableCorrelationHeader = abfsConfiguration.isCorrelationHeaderEnabled(); } public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, @@ -137,10 +135,6 @@ public String getFileSystem() { return filesystem; } - public boolean isCorrelationHeaderEnabled() { - return enableCorrelationHeader; - } - protected AbfsPerfTracker getAbfsPerfTracker() { return abfsPerfTracker; } @@ -406,12 +400,6 @@ public AbfsRestOperation renameIdempotencyCheckOp( return op; } - public AbfsRestOperation append(final String path, final long position, final byte[] buffer, final int offset, - final int length, final String cachedSasToken, final boolean isAppendBlob) throws AzureBlobFileSystemException { - return append(path, position, buffer, offset, length, cachedSasToken, - isAppendBlob, new TracingContext("test-corr-id", "test-filesystem-id", "AP")); - } - public AbfsRestOperation append(final String path, final long position, final byte[] buffer, final int offset, final int length, final String cachedSasToken, final boolean isAppendBlob, TracingContext tracingContext) throws AzureBlobFileSystemException { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index bf792079b98fb..51b24a6601ef2 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -98,7 +98,7 @@ public AbfsInputStream( this.cachedSasToken = new CachedSASToken( abfsInputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds()); this.streamStatistics = abfsInputStreamContext.getStreamStatistics(); - this.inputStreamID = StringUtils.right(UUID.randomUUID().toString(), 12); + this.inputStreamID = getInputStreamID(); this.tracingContext = new TracingContext(tracingContext); this.tracingContext.setStreamID(inputStreamID); } @@ -107,7 +107,12 @@ public String getPath() { return path; } + private String getInputStreamID() { + return StringUtils.right(UUID.randomUUID().toString(), 12); + } + @Override + public int read() throws IOException { byte[] b = new byte[1]; int numberOfBytesRead = read(b, 0, 1); @@ -242,7 +247,7 @@ private int readInternal(final long position, final byte[] b, final int offset, LOG.debug("issuing read ahead requestedOffset = {} requested size {}", nextOffset, nextSize); ReadBufferManager.getBufferManager().queueReadAhead(this, nextOffset, (int) nextSize, - readAheadTracingContext); + new TracingContext(readAheadTracingContext)); nextOffset = nextOffset + nextSize; numReadAheads--; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index d335df51afefd..ae10a2185d6b5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -36,7 +36,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,6 +43,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.utils.CachedSASToken; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.io.ElasticByteBufferPool; import org.apache.hadoop.fs.FileSystem.Statistics; import org.apache.hadoop.fs.FSExceptionMessages; @@ -81,7 +81,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable, StreamCa // SAS tokens can be re-used until they expire private CachedSASToken cachedSasToken; - private String outputStreamID; + private final String outputStreamID; private TracingContext tracingContext; /** @@ -140,11 +140,15 @@ public AbfsOutputStream( this.completionService = new ExecutorCompletionService<>(this.threadExecutor); this.cachedSasToken = new CachedSASToken( abfsOutputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds()); - this.outputStreamID = StringUtils.right(UUID.randomUUID().toString(), 12); + this.outputStreamID = getOutputStreamID(); this.tracingContext = new TracingContext(tracingContext); this.tracingContext.setStreamID(outputStreamID); } + private String getOutputStreamID() { + return StringUtils.right(UUID.randomUUID().toString(), 12); + } + /** * Query the stream for a specific capability. * 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 10523e7a1cfc5..84005eab90c34 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 @@ -188,7 +188,7 @@ public void execute(TracingContext tracingContext) throws AzureBlobFileSystemExc while (!executeHttpOperation(retryCount, tracingContext)) { try { ++retryCount; - tracingContext.updateRetryCount(); + tracingContext.setRetryCount(retryCount); LOG.debug("Retrying REST operation {}. RetryCount = {}", operationType, retryCount); Thread.sleep(client.getRetryPolicy().getRetryInterval(retryCount)); @@ -205,6 +205,14 @@ public void execute(TracingContext tracingContext) throws AzureBlobFileSystemExc LOG.trace("{} REST operation complete", operationType); } + private void updateClientRequestHeader(AbfsHttpOperation httpOperation, + TracingContext tracingContext) { + tracingContext.generateClientRequestID(); + httpOperation.getConnection() + .setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, + tracingContext.toString()); + } + /** * Executes a single HTTP operation to complete the REST operation. If it * fails, there may be a retry. The retryCount is incremented with each @@ -217,17 +225,8 @@ private boolean executeHttpOperation(final int retryCount, TracingContext tracin // initialize the HTTP request and open the connection httpOperation = new AbfsHttpOperation(url, method, requestHeaders); incrementCounter(AbfsStatistic.CONNECTIONS_MADE, 1); - if (client.isCorrelationHeaderEnabled()) { - tracingContext.generateClientRequestID(); - httpOperation.getConnection() - .setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, - tracingContext.toString()); - } - else { - httpOperation.getConnection() - .setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, - UUID.randomUUID().toString()); - } + + updateClientRequestHeader(httpOperation, tracingContext); switch(client.getAuthType()) { case Custom: @@ -269,7 +268,6 @@ private boolean executeHttpOperation(final int retryCount, TracingContext tracin incrementCounter(AbfsStatistic.BYTES_RECEIVED, httpOperation.getBytesReceived()); } - } catch (IOException ex) { if (ex instanceof UnknownHostException) { LOG.warn(String.format("Unknown host name: %s. Retrying to resolve the host name...", httpOperation.getUrl().getHost())); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java index 6353974d55021..332a623521331 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java @@ -19,7 +19,6 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -//import org.graalvm.compiler.core.common.alloc.Trace; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java deleted file mode 100644 index d1b275eb90c6f..0000000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java +++ /dev/null @@ -1,4 +0,0 @@ -package org.apache.hadoop.fs.azurebfs.utils; - -public interface Listener { -} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index ca239f122d9c4..3bc91d9a93d5e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -18,9 +18,6 @@ package org.apache.hadoop.fs.azurebfs.utils; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; import java.util.UUID; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; @@ -34,21 +31,31 @@ public class TracingContext { private final String fileSystemID; private String clientRequestID = ""; private String primaryRequestID; - private String streamID = ""; + private String streamID; private int retryCount; - private String hadoopOpName = ""; + private String hadoopOpName; + private final int format; private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*"; - public TracingContext(String clientCorrelationID, String fileSystemID, String hadoopOpName) { + public TracingContext(String clientCorrelationID, String fileSystemID, + String hadoopOpName, int tracingContextFormat) { this.fileSystemID = fileSystemID; this.hadoopOpName = hadoopOpName; this.clientCorrelationID = validateClientCorrelationID(clientCorrelationID); streamID = EMPTY_STRING; retryCount = 0; primaryRequestID = ""; + format = tracingContextFormat; + } + + public TracingContext(String clientCorrelationID, String fileSystemID, + String hadoopOpName, boolean needsPrimaryReqId, + int tracingContextFormat) { + this(clientCorrelationID, fileSystemID, hadoopOpName, tracingContextFormat); + primaryRequestID = needsPrimaryReqId? UUID.randomUUID().toString() : ""; } public TracingContext(TracingContext originalTracingContext) { @@ -58,6 +65,7 @@ public TracingContext(TracingContext originalTracingContext) { this.hadoopOpName = originalTracingContext.hadoopOpName; this.retryCount = 0; this.primaryRequestID = originalTracingContext.primaryRequestID; + this.format = originalTracingContext.format; } public String validateClientCorrelationID(String clientCorrelationID) { @@ -82,12 +90,17 @@ public void setStreamID(String stream) { streamID = stream; } - public void updateRetryCount() { - retryCount++; + public void setRetryCount(int retryCount) { + this.retryCount = retryCount; } public String toString() { - return clientCorrelationID + ":" + clientRequestID + ":" + fileSystemID + ":" + primaryRequestID - + ":" + streamID + ":" + hadoopOpName + ":" + retryCount; + String header = clientRequestID; //case 0, no IDs for correlation + switch (format) { + case 1: header = header + ":" + fileSystemID + ":" + primaryRequestID + + ":" + streamID + ":" + hadoopOpName + ":" + retryCount; //all IDs + case 2: header = clientCorrelationID + ":" + header; // 2-ID format + } + return header; } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/testHeader.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/testHeader.java deleted file mode 100644 index 113eac93e4b6c..0000000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/testHeader.java +++ /dev/null @@ -1,2 +0,0 @@ -package org.apache.hadoop.fs.azurebfs.utils;public class testHeader { -} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index a319fe51b76a1..1d959ef559b38 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -25,6 +25,8 @@ import java.util.UUID; import java.util.concurrent.Callable; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperations; +import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.After; import org.junit.Assert; @@ -82,7 +84,6 @@ public abstract class AbstractAbfsIntegrationTest extends private AuthType authType; private boolean useConfiguredFileSystem = false; private boolean usingFilesystemForSASTests = false; - public TracingContext tracingContext; protected AbstractAbfsIntegrationTest() throws Exception { fileSystemName = TEST_CONTAINER_PREFIX + UUID.randomUUID().toString(); @@ -141,18 +142,29 @@ protected AbstractAbfsIntegrationTest() throws Exception { } } + public TracingContext getTestTracingContext(AzureBlobFileSystem fs, boolean isCont) { + if (fs == null) { + return new TracingContext("test-corr-id", + "test-fs-id", "TS", false, 1); + } + String fsID = fs.getFileSystemID(); + AbfsConfiguration abfsConf = fs.getAbfsStore().getAbfsConfiguration(); + String corrID = abfsConf == null? "test-corr-id" : + abfsConf.getClientCorrelationID(); + int format = abfsConf == null? 1 : abfsConf.getTracingContextFormat(); + return new TracingContext(corrID, fsID, "TS", isCont, format); + } + @Before public void setup() throws Exception { //Create filesystem first to make sure getWasbFileSystem() can return an existing filesystem. createFileSystem(); - tracingContext = new TracingContext(abfsConfig.getClientCorrelationID(), - abfs.getFileSystemID(), "TS"); // Only live account without namespace support can run ABFS&WASB compatibility tests if (!isIPAddress && (abfsConfig.getAuthType(accountName) != AuthType.SAS) - && !abfs.getIsNamespaceEnabled(tracingContext)) { + && !abfs.getIsNamespaceEnabled(getTestTracingContext(getFileSystem(), false))) { final URI wasbUri = new URI(abfsUrlToWasbUrl(getTestUrl())); final AzureNativeFileSystemStore azureNativeFileSystemStore = new AzureNativeFileSystemStore(); @@ -184,8 +196,7 @@ public void teardown() throws Exception { if (abfs == null) { return; } - TracingContext tracingContext = new TracingContext(abfs.getAbfsStore() - .getAbfsConfiguration().getClientCorrelationID(), abfs.getFileSystemID(), "DL"); + TracingContext tracingContext = getTestTracingContext(getFileSystem(), false); if (usingFilesystemForSASTests) { abfsConfig.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey.name()); @@ -435,7 +446,8 @@ protected AbfsOutputStream createAbfsOutputStreamWithFlushEnabled( abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false); return (AbfsOutputStream) abfss.createFile(path, fs.getFsStatistics(), - true, FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()), tracingContext); + true, FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()), + getTestTracingContext(fs, false)); } /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java index 22bc3dd50e45f..d47719cd7baa7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java @@ -69,8 +69,8 @@ public void testContinuationTokenHavingEqualSign() throws Exception { AbfsClient abfsClient = fs.getAbfsClient(); try { - AbfsRestOperation op = abfsClient.listPath("/", true, LIST_MAX_RESULTS, "===========", - tracingContext); + AbfsRestOperation op = abfsClient.listPath("/", true, LIST_MAX_RESULTS, + "===========", getTestTracingContext(fs, true)); Assert.assertTrue(false); } catch (AbfsRestOperationException ex) { Assert.assertEquals("InvalidQueryParameterValue", ex.getErrorCode().getErrorCode()); @@ -145,7 +145,7 @@ private List listPath(String directory) throws IOException { return getFileSystem().getAbfsClient() .listPath(directory, false, getListMaxResults(), null, - tracingContext).getResult().getListResultSchema().paths(); + getTestTracingContext(getFileSystem(), true)).getResult().getListResultSchema().paths(); } private int getListMaxResults() throws IOException { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java index 9950e7995e9f0..338a1e0f9a02f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java @@ -64,7 +64,7 @@ public void testInitValues() throws IOException { outputStream = createAbfsOutputStreamWithFlushEnabled(fs, initValuesPath); inputStream = abfss.openFileForRead(initValuesPath, fs.getFsStatistics(), - tracingContext); + getTestTracingContext(fs, false)); AbfsInputStreamStatisticsImpl stats = (AbfsInputStreamStatisticsImpl) inputStream.getStreamStatistics(); @@ -109,7 +109,7 @@ public void testSeekStatistics() throws IOException { out.write(defBuffer); out.hflush(); in = abfss.openFileForRead(seekStatPath, fs.getFsStatistics(), - tracingContext); + getTestTracingContext(fs, false)); /* * Writing 1MB buffer to the file, this would make the fCursor(Current @@ -201,7 +201,7 @@ public void testReadStatistics() throws IOException { out.write(defBuffer); out.hflush(); in = abfss.openFileForRead(readStatPath, fs.getFsStatistics(), - tracingContext); + getTestTracingContext(fs, false)); /* * Doing file read 10 times. @@ -274,14 +274,14 @@ public void testWithNullStreamStatistics() throws IOException { // AbfsRestOperation Instance required for eTag. AbfsRestOperation abfsRestOperation = fs.getAbfsClient().getPathStatus(nullStatFilePath.toUri().getPath(), false, - tracingContext); + getTestTracingContext(fs, false)); // AbfsInputStream with no StreamStatistics. in = new AbfsInputStream(fs.getAbfsClient(), null, nullStatFilePath.toUri().getPath(), ONE_KB, abfsInputStreamContext, abfsRestOperation.getResult().getResponseHeader("ETag"), - tracingContext); + getTestTracingContext(fs, false)); // Verifying that AbfsInputStream Operations works with null statistics. assertNotEquals("AbfsInputStream read() with null statistics should " @@ -324,7 +324,8 @@ public void testReadAheadCounters() throws IOException { out.write(defBuffer); out.close(); - in = abfss.openFileForRead(readAheadCountersPath, fs.getFsStatistics(), tracingContext); + in = abfss.openFileForRead(readAheadCountersPath, fs.getFsStatistics(), + getTestTracingContext(fs, false)); /* * Reading 1KB after each i * KB positions. Hence the reads are from 0 diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index b53286ba7d561..976f3c46491e8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -347,7 +347,8 @@ public void testNegativeScenariosForCreateOverwriteDisabled() AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); abfsStore = setAzureBlobSystemStoreField(abfsStore, "client", mockClient); - boolean isNamespaceEnabled = abfsStore.getIsNamespaceEnabled(tracingContext); + boolean isNamespaceEnabled = + abfsStore.getIsNamespaceEnabled(getTestTracingContext(fs, false)); AbfsRestOperation successOp = mock( AbfsRestOperation.class); @@ -463,7 +464,7 @@ private void validateCreateFileException(final Class ex intercept( exceptionClass, () -> abfsStore.createFile(testPath, null, true, permission, umask, - tracingContext)); + getTestTracingContext(getFileSystem(), true))); } private AbfsRestOperationException getMockAbfsRestOperationException(int status) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index 492ff75442b5d..66b1c93a00602 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -219,7 +219,7 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { "/NonExistingPath", false, null, - tracingContext)); + getTestTracingContext(fs, true))); // mock idempotency check to mimic retried case AbfsClient mockClient = TestAbfsClient.getMockAbfsClient( @@ -241,7 +241,7 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { "/NonExistingPath", false, null, - tracingContext) + getTestTracingContext(fs, true)) .getResult() .getStatusCode()) .describedAs("Idempotency check reports successful " diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java index a761e923a0241..707633a992695 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java @@ -22,6 +22,7 @@ import java.io.InputStream; import java.util.Map; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Assume; import org.junit.Test; import org.slf4j.Logger; @@ -129,6 +130,7 @@ public void testBlobDataReader() throws Exception { // Use abfsStore in this test to verify the ERROR code in AbfsRestOperationException AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); + TracingContext tracingContext = getTestTracingContext(fs, true); // TEST READ FS Map properties = abfsStore.getFilesystemProperties(tracingContext); // TEST READ FOLDER diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java index 5e4cc45fedaa6..ac1ec25d6d359 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java @@ -250,7 +250,7 @@ private void testRenameIdempotencyTriggerChecks( "/NonExistingsourcepath", "/destpath", null, - tracingContext) + getTestTracingContext(fs, true)) .getResult() .getStatusCode()) .describedAs("Idempotency check reports recent successful " @@ -263,7 +263,7 @@ private void testRenameIdempotencyTriggerChecks( "/NonExistingsourcepath", "/destpath", "", - tracingContext)); + getTestTracingContext(fs, true))); } } @@ -323,7 +323,8 @@ private void testRenameTimeout( Assertions.assertThat(testClient.renameIdempotencyCheckOp( renameRequestStartTime, op, - destinationPath.toUri().getPath(), tracingContext) + destinationPath.toUri().getPath(), + getTestTracingContext(fs, true)) .getResult() .getStatusCode()) .describedAs(assertMessage) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java index 2cc1298a01943..ef7f1565df73f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemStoreListStatusWithRange.java @@ -108,7 +108,7 @@ public ITestAzureBlobFileSystemStoreListStatusWithRange() throws Exception { public void testListWithRange() throws IOException { try { FileStatus[] listResult = store.listStatus(new Path(path), startFrom, - tracingContext); + getTestTracingContext(fs, true)); if (!expectedResult) { Assert.fail("Excepting failure with IllegalArgumentException"); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCorrelation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCorrelation.java deleted file mode 100644 index 7a9187c705ba7..0000000000000 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCorrelation.java +++ /dev/null @@ -1,2 +0,0 @@ -package org.apache.hadoop.fs.azurebfs;public class ITestCorrelation { -} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java index 5727c55538993..22cbd38f3165b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java @@ -20,6 +20,7 @@ import java.util.Hashtable; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Test; import org.apache.hadoop.fs.FSDataInputStream; @@ -64,6 +65,7 @@ public void testBase64FileSystemProperties() throws Exception { final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: value }"); + TracingContext tracingContext = getTestTracingContext(fs, true); fs.getAbfsStore().setFilesystemProperties(properties, tracingContext); Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties( tracingContext); @@ -77,6 +79,7 @@ public void testBase64PathProperties() throws Exception { final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueTest }"); touch(TEST_PATH); + TracingContext tracingContext = getTestTracingContext(fs, true); fs.getAbfsStore().setPathProperties(TEST_PATH, properties, tracingContext); Hashtable fetchedProperties = fs.getAbfsStore().getPathStatus(TEST_PATH, tracingContext); @@ -89,6 +92,7 @@ public void testBase64InvalidFileSystemProperties() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueæ­² }"); + TracingContext tracingContext = getTestTracingContext(fs, true); fs.getAbfsStore().setFilesystemProperties(properties, tracingContext); Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties( tracingContext); @@ -102,6 +106,7 @@ public void testBase64InvalidPathProperties() throws Exception { final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueTestå…© }"); touch(TEST_PATH); + TracingContext tracingContext = getTestTracingContext(fs, true); fs.getAbfsStore().setPathProperties(TEST_PATH, properties, tracingContext); Hashtable fetchedProperties = fs.getAbfsStore().getPathStatus(TEST_PATH, tracingContext); @@ -114,6 +119,7 @@ public void testSetFileSystemProperties() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); final Hashtable properties = new Hashtable<>(); properties.put("containerForDevTest", "true"); + TracingContext tracingContext = getTestTracingContext(fs, true); fs.getAbfsStore().setFilesystemProperties(properties, tracingContext); Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties( tracingContext); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java index 22abf4e43941d..3d336ae3870a3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java @@ -84,7 +84,7 @@ public void testGetIsNamespaceEnabledWhenConfigIsTrue() throws Exception { Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs( "getIsNamespaceEnabled should return true when the " + "config is set as true").isTrue(); - fs.getAbfsStore().deleteFilesystem(tracingContext); + fs.getAbfsStore().deleteFilesystem(getTestTracingContext(fs, false)); unsetAndAssert(); } @@ -94,7 +94,7 @@ public void testGetIsNamespaceEnabledWhenConfigIsFalse() throws Exception { Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs( "getIsNamespaceEnabled should return false when the " + "config is set as false").isFalse(); - fs.getAbfsStore().deleteFilesystem(tracingContext); + fs.getAbfsStore().deleteFilesystem(getTestTracingContext(fs, false)); unsetAndAssert(); } @@ -107,7 +107,7 @@ private void unsetAndAssert() throws Exception { "getIsNamespaceEnabled should return the value " + "configured for fs.azure.test.namespace.enabled") .isEqualTo(expectedValue); - fs.getAbfsStore().deleteFilesystem(tracingContext); + fs.getAbfsStore().deleteFilesystem(getTestTracingContext(fs, false)); } private AzureBlobFileSystem getNewFSWithHnsConf( diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java index 36d9886e0d6b3..3798b52c760c5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java @@ -54,7 +54,8 @@ public void testWithWrongSharedKey() throws Exception { + "Authorization header is formed correctly including the " + "signature.\", 403", () -> { - abfsClient.getAclStatus("/", tracingContext); + abfsClient.getAclStatus("/", getTestTracingContext(getFileSystem(), + false)); }); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index e266f7a64a855..77222cad4bab4 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -82,7 +82,7 @@ private AbfsInputStream getAbfsInputStream(AbfsClient mockAbfsClient, String fil THREE_KB, inputStreamContext.withReadBufferSize(ONE_KB).withReadAheadQueueDepth(10), "eTag", - tracingContext); + getTestTracingContext(getFileSystem(), false)); inputStream.setCachedSasToken( TestCachedSASToken.getTestCachedSASTokenInstance()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index 7780c1126b72a..464c0a42fdfd6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -23,6 +23,7 @@ import java.util.HashSet; import java.util.Random; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperations; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.bouncycastle.jcajce.provider.symmetric.TEA; import org.junit.Test; @@ -84,10 +85,11 @@ public void verifyShortWriteRequest() throws Exception { conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", "WR"); + "test-fs-id", AbfsOperations.WRITE, abfsConf.getTracingContextFormat()); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); - when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); + when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), + anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), any(TracingContext.class))).thenReturn(op); @@ -105,6 +107,8 @@ public void verifyShortWriteRequest() throws Exception { ArgumentCaptor acByteArray = ArgumentCaptor.forClass(byte[].class); ArgumentCaptor acAppendBlobAppend = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acSASToken = ArgumentCaptor.forClass(String.class); + ArgumentCaptor acTracingContext = + ArgumentCaptor.forClass(TracingContext.class); final byte[] b1 = new byte[2*WRITE_SIZE]; @@ -116,7 +120,8 @@ public void verifyShortWriteRequest() throws Exception { out.hsync(); verify(client, times(2)).append(acString.capture(), acLong.capture(), acByteArray.capture(), acBufferOffset.capture(), acBufferLength.capture(), - acSASToken.capture(), acAppendBlobAppend.capture()); + acSASToken.capture(), + acAppendBlobAppend.capture(), acTracingContext.capture()); assertThat(Arrays.asList(PATH, PATH)).describedAs("Path of the requests").isEqualTo(acString.getAllValues()); assertThat(Arrays.asList(Long.valueOf(0), Long.valueOf(WRITE_SIZE))).describedAs("Write Position").isEqualTo(acLong.getAllValues()); assertThat(Arrays.asList(0, 0)).describedAs("Buffer Offset").isEqualTo(acBufferOffset.getAllValues()); @@ -138,10 +143,11 @@ public void verifyWriteRequest() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", "OP"); + "test-fs-id", AbfsOperations.WRITE, abfsConf.getTracingContextFormat()); when(client.getAbfsPerfTracker()).thenReturn(tracker); - when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); + when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), + anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), any(TracingContext.class))).thenReturn(op); @@ -163,9 +169,12 @@ public void verifyWriteRequest() throws Exception { ArgumentCaptor acByteArray = ArgumentCaptor.forClass(byte[].class); ArgumentCaptor acAppendBlobAppend = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acSASToken = ArgumentCaptor.forClass(String.class); + ArgumentCaptor acTracingContext = + ArgumentCaptor.forClass(TracingContext.class); verify(client, times(2)).append(acString.capture(), acLong.capture(), acByteArray.capture(), acBufferOffset.capture(), acBufferLength.capture(), - acSASToken.capture(), acAppendBlobAppend.capture()); + acSASToken.capture(), + acAppendBlobAppend.capture(), acTracingContext.capture()); assertThat(Arrays.asList(PATH, PATH)).describedAs("Path").isEqualTo(acString.getAllValues()); assertThat(new HashSet(Arrays.asList(Long.valueOf(0), Long.valueOf(BUFFER_SIZE)))).describedAs("Position").isEqualTo(new HashSet( acLong.getAllValues())); @@ -178,7 +187,6 @@ public void verifyWriteRequest() throws Exception { ArgumentCaptor acFlushRetainUnCommittedData = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acFlushClose = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); - ArgumentCaptor acTracingContext = ArgumentCaptor.forClass(TracingContext.class); verify(client, times(1)).flush(acFlushString.capture(), acFlushLong.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), acFlushSASToken.capture(), acTracingContext.capture()); @@ -203,10 +211,11 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", "OP"); + "test-fs-id", AbfsOperations.WRITE, abfsConf.getTracingContextFormat()); when(client.getAbfsPerfTracker()).thenReturn(tracker); - when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); + when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), + anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), any(TracingContext.class))).thenReturn(op); when(op.getSasToken()).thenReturn("testToken"); @@ -230,9 +239,12 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { ArgumentCaptor acByteArray = ArgumentCaptor.forClass(byte[].class); ArgumentCaptor acAppendBlobAppend = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acSASToken = ArgumentCaptor.forClass(String.class); + ArgumentCaptor acTracingContext = + ArgumentCaptor.forClass(TracingContext.class); verify(client, times(2)).append(acString.capture(), acLong.capture(), acByteArray.capture(), acBufferOffset.capture(), acBufferLength.capture(), - acSASToken.capture(), acAppendBlobAppend.capture()); + acSASToken.capture(), + acAppendBlobAppend.capture(), acTracingContext.capture()); assertThat(Arrays.asList(PATH, PATH)).describedAs("path").isEqualTo(acString.getAllValues()); assertThat(new HashSet(Arrays.asList(Long.valueOf(0), Long.valueOf(BUFFER_SIZE)))).describedAs("Position").isEqualTo(new HashSet( acLong.getAllValues())); @@ -244,7 +256,6 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { ArgumentCaptor acFlushRetainUnCommittedData = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acFlushClose = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); - ArgumentCaptor acTracingContext = ArgumentCaptor.forClass(TracingContext.class); verify(client, times(1)).flush(acFlushString.capture(), acFlushLong.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), acFlushSASToken.capture(), acTracingContext.capture()); @@ -271,7 +282,8 @@ public void verifyWriteRequestOfBufferSize() throws Exception { AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); - when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); + when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), + anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), any(TracingContext.class))).thenReturn(op); when(op.getSasToken()).thenReturn("testToken"); @@ -279,7 +291,9 @@ public void verifyWriteRequestOfBufferSize() throws Exception { AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, - false), new TracingContext(abfsConf.getClientCorrelationID(), "fs-id", "WR")); + false), + new TracingContext(abfsConf.getClientCorrelationID(), "test-fs-id", + AbfsOperations.WRITE, abfsConf.getTracingContextFormat())); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -295,9 +309,12 @@ public void verifyWriteRequestOfBufferSize() throws Exception { ArgumentCaptor acByteArray = ArgumentCaptor.forClass(byte[].class); ArgumentCaptor acAppendBlobAppend = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acSASToken = ArgumentCaptor.forClass(String.class); + ArgumentCaptor acTracingContext = + ArgumentCaptor.forClass(TracingContext.class); verify(client, times(2)).append(acString.capture(), acLong.capture(), acByteArray.capture(), acBufferOffset.capture(), acBufferLength.capture(), - acSASToken.capture(), acAppendBlobAppend.capture()); + acSASToken.capture(), + acAppendBlobAppend.capture(), acTracingContext.capture()); assertThat(Arrays.asList(PATH, PATH)).describedAs("File Path").isEqualTo(acString.getAllValues()); assertThat(new HashSet(Arrays.asList(Long.valueOf(0), Long.valueOf(BUFFER_SIZE)))).describedAs("Position in file").isEqualTo( new HashSet(acLong.getAllValues())); @@ -321,14 +338,15 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); - when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); + when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), + anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), any(TracingContext.class))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, true), new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", "OP")); + "test-fs-id", AbfsOperations.OPEN, abfsConf.getTracingContextFormat())); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -344,9 +362,12 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { ArgumentCaptor acByteArray = ArgumentCaptor.forClass(byte[].class); ArgumentCaptor acAppendBlobAppend = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acSASToken = ArgumentCaptor.forClass(String.class); + ArgumentCaptor acTracingContext = + ArgumentCaptor.forClass(TracingContext.class); verify(client, times(2)).append(acString.capture(), acLong.capture(), acByteArray.capture(), acBufferOffset.capture(), acBufferLength.capture(), - acSASToken.capture(), acAppendBlobAppend.capture()); + acSASToken.capture(), + acAppendBlobAppend.capture(), acTracingContext.capture()); assertThat(Arrays.asList(PATH, PATH)).describedAs("File Path").isEqualTo(acString.getAllValues()); assertThat(Arrays.asList(Long.valueOf(0), Long.valueOf(BUFFER_SIZE))).describedAs("File Position").isEqualTo(acLong.getAllValues()); assertThat(Arrays.asList(0, 0)).describedAs("Buffer Offset").isEqualTo(acBufferOffset.getAllValues()); @@ -369,10 +390,11 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", "PA"); + "test-fs-id", AbfsOperations.WRITE, abfsConf.getTracingContextFormat()); when(client.getAbfsPerfTracker()).thenReturn(tracker); - when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); + when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), + anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), any(TracingContext.class))).thenReturn(op); @@ -394,9 +416,12 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { ArgumentCaptor acByteArray = ArgumentCaptor.forClass(byte[].class); ArgumentCaptor acAppendBlobAppend = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acSASToken = ArgumentCaptor.forClass(String.class); + ArgumentCaptor acTracingContext = + ArgumentCaptor.forClass(TracingContext.class); verify(client, times(2)).append(acString.capture(), acLong.capture(), acByteArray.capture(), acBufferOffset.capture(), acBufferLength.capture(), - acSASToken.capture(), acAppendBlobAppend.capture()); + acSASToken.capture(), + acAppendBlobAppend.capture(), acTracingContext.capture()); assertThat(Arrays.asList(PATH, PATH)).describedAs("File Path").isEqualTo(acString.getAllValues()); assertThat(new HashSet(Arrays.asList(Long.valueOf(0), Long.valueOf(BUFFER_SIZE)))).describedAs("File Position").isEqualTo( new HashSet(acLong.getAllValues())); @@ -408,7 +433,6 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { ArgumentCaptor acFlushRetainUnCommittedData = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acFlushClose = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); - ArgumentCaptor acTracingContext = ArgumentCaptor.forClass(TracingContext.class); verify(client, times(1)).flush(acFlushString.capture(), acFlushLong.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), acFlushSASToken.capture(), acTracingContext.capture()); @@ -432,10 +456,11 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", "WR"); + "test-fs-id", AbfsOperations.WRITE, abfsConf.getTracingContextFormat()); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); - when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean())).thenReturn(op); + when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), + anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), any(TracingContext.class))).thenReturn(op); @@ -459,9 +484,12 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { ArgumentCaptor acByteArray = ArgumentCaptor.forClass(byte[].class); ArgumentCaptor acAppendBlobAppend = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acSASToken = ArgumentCaptor.forClass(String.class); + ArgumentCaptor acTracingContext = + ArgumentCaptor.forClass(TracingContext.class); verify(client, times(2)).append(acString.capture(), acLong.capture(), acByteArray.capture(), acBufferOffset.capture(), acBufferLength.capture(), - acSASToken.capture(), acAppendBlobAppend.capture()); + acSASToken.capture(), + acAppendBlobAppend.capture(), acTracingContext.capture()); assertThat(Arrays.asList(PATH, PATH)).describedAs("path").isEqualTo(acString.getAllValues()); assertThat(new HashSet(Arrays.asList(Long.valueOf(0), Long.valueOf(BUFFER_SIZE)))).describedAs("Position").isEqualTo( new HashSet(acLong.getAllValues())); From a4755553881a1c8d39bdff737bf4a3d273238328 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 3 Nov 2020 19:36:56 +0530 Subject: [PATCH 30/77] test ns() + remove extra changes --- .../hadoop-azure/src/config/codestyle.xml | 61 -------- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 8 +- .../fs/azurebfs/AzureBlobFileSystem.java | 7 - .../fs/azurebfs/AzureBlobFileSystemStore.java | 46 +++--- .../azurebfs/services/AbfsHttpOperation.java | 99 ++++++------- .../fs/azurebfs/services/AbfsInputStream.java | 6 +- .../azurebfs/services/AbfsRestOperation.java | 1 - .../azurebfs/AbstractAbfsIntegrationTest.java | 8 ++ .../hadoop/fs/azurebfs/ITestAbfsClient.java | 1 - .../ITestAzureBlobFileSystemAttributes.java | 6 +- ...ITestAzureBlobFileSystemAuthorization.java | 18 +-- .../ITestAzureBlobFileSystemBackCompat.java | 2 +- .../ITestAzureBlobFileSystemDelete.java | 2 +- .../ITestAzureBlobFileSystemFileStatus.java | 2 +- .../ITestAzureBlobFileSystemPermission.java | 4 +- .../ITestAzureBlobFileSystemRandomRead.java | 4 +- .../azurebfs/ITestAzureBlobFilesystemAcl.java | 132 +++++++++--------- .../fs/azurebfs/ITestGetNameSpaceEnabled.java | 12 +- .../azurebfs/ITestWasbAbfsCompatibility.java | 8 +- 19 files changed, 174 insertions(+), 253 deletions(-) delete mode 100644 hadoop-tools/hadoop-azure/src/config/codestyle.xml diff --git a/hadoop-tools/hadoop-azure/src/config/codestyle.xml b/hadoop-tools/hadoop-azure/src/config/codestyle.xml deleted file mode 100644 index 28dde371bbb87..0000000000000 --- a/hadoop-tools/hadoop-azure/src/config/codestyle.xml +++ /dev/null @@ -1,61 +0,0 @@ - - - diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 7a96f15ca56ed..478faa7db1e8b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -306,9 +306,11 @@ public String getClientCorrelationID() { } /** - * Config to allow user to enable list of identifiers for correlation. - * Default value is true. Only clientRequestId appears in logs when disabled - * @return enableCorrelationHeader config + * Config to allow user to pick format of x-ms-client-request-id header + * 0 -> client-req-id + * 1 -> all IDs (default) + * 2 -> client-corr-id : client-req-id + * @return tracingContextFormat config */ public int getTracingContextFormat() { return tracingContextFormat; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index e565339c09435..c0f40311114b0 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -1300,13 +1300,6 @@ boolean getIsNamespaceEnabled(TracingContext tracingContext) throws AzureBlobFil return abfsStore.getIsNamespaceEnabled(tracingContext); } - @VisibleForTesting - boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException { - return abfsStore.getIsNamespaceEnabled( - new TracingContext(clientCorrelationID, fileSystemID, - AbfsOperations.GET_NAMESPACE, tracingContextFormat)); - } - @VisibleForTesting Map getInstrumentationMap() { return abfsCounters.toMap(); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 5174fa5a70417..0d576fa5e4a47 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -483,32 +483,32 @@ public OutputStream createFile(final Path path, AbfsRestOperation op; if (triggerConditionalCreateOverwrite) { op = conditionalCreateOverwriteFile(relativePath, - statistics, - isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null, - isAppendBlob, - tracingContext + statistics, + isNamespaceEnabled ? getOctalNotation(permission) : null, + isNamespaceEnabled ? getOctalNotation(umask) : null, + isAppendBlob, + tracingContext ); } else { op = client.createPath(relativePath, true, - overwrite, - isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null, - isAppendBlob, - null, - tracingContext); + overwrite, + isNamespaceEnabled ? getOctalNotation(permission) : null, + isNamespaceEnabled ? getOctalNotation(umask) : null, + isAppendBlob, + null, + tracingContext); } perfInfo.registerResult(op.getResult()).registerSuccess(true); return new AbfsOutputStream( - client, - statistics, - relativePath, - 0, - populateAbfsOutputStreamContext(isAppendBlob), - tracingContext); + client, + statistics, + relativePath, + 0, + populateAbfsOutputStreamContext(isAppendBlob), + tracingContext); } } @@ -698,12 +698,12 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic isAppendBlob = true; } return new AbfsOutputStream( - client, - statistics, - relativePath, - offset, - populateAbfsOutputStreamContext(isAppendBlob), - tracingContext); + client, + statistics, + relativePath, + offset, + populateAbfsOutputStreamContext(isAppendBlob), + tracingContext); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index 4315f5f4f9ced..37d0535c28681 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -66,8 +66,8 @@ public class AbfsHttpOperation implements AbfsPerfLoggable { private int statusCode; private String statusDescription; private String storageErrorCode = ""; - private String storageErrorMessage = ""; - private String requestId = ""; + private String storageErrorMessage = ""; + private String requestId = ""; private String expectedAppendPos = ""; private ListResultSchema listResultSchema = null; @@ -83,7 +83,7 @@ public class AbfsHttpOperation implements AbfsPerfLoggable { public static AbfsHttpOperation getAbfsHttpOperationWithFixedResult(final URL url, final String method, final int httpStatus) { - return new AbfsHttpOperation(url, method, httpStatus); + return new AbfsHttpOperation(url, method, httpStatus); } private AbfsHttpOperation(final URL url, final String method, @@ -94,7 +94,7 @@ private AbfsHttpOperation(final URL url, final String method, this.statusCode = httpStatus; } - protected HttpURLConnection getConnection() { + protected HttpURLConnection getConnection() { return connection; } @@ -195,37 +195,19 @@ public String getLogString() { try { urlStr = URLEncoder.encode(url.toString(), "UTF-8"); - } catch(UnsupportedEncodingException e) { + } catch (UnsupportedEncodingException e) { urlStr = "https%3A%2F%2Ffailed%2Fto%2Fencode%2Furl"; } final StringBuilder sb = new StringBuilder(); - sb.append("s=") - .append(statusCode) - .append(" e=") - .append(storageErrorCode) - .append(" ci=") - .append(getClientRequestId()) - .append(" ri=") - .append(requestId); + sb.append("s=").append(statusCode).append(" e=").append(storageErrorCode).append(" ci=").append(getClientRequestId()).append(" ri=") + .append(requestId); if (isTraceEnabled) { - sb.append(" ct=") - .append(connectionTimeMs) - .append(" st=") - .append(sendRequestTimeMs) - .append(" rt=") - .append(recvResponseTimeMs); + sb.append(" ct=").append(connectionTimeMs).append(" st=").append(sendRequestTimeMs).append(" rt=").append(recvResponseTimeMs); } - sb.append(" bs=") - .append(bytesSent) - .append(" br=") - .append(bytesReceived) - .append(" m=") - .append(method) - .append(" u=") - .append(urlStr); + sb.append(" bs=").append(bytesSent).append(" br=").append(bytesReceived).append(" m=").append(method).append(" u=").append(urlStr); return sb.toString(); } @@ -233,13 +215,12 @@ public String getLogString() { /** * Initializes a new HTTP request and opens the connection. * - * @param url The full URL including query string parameters. - * @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE). + * @param url The full URL including query string parameters. + * @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or + * DELETE). * @param requestHeaders The HTTP request headers.READ_TIMEOUT - * * @throws IOException if an error occurs. */ - public AbfsHttpOperation(final URL url, final String method, final List requestHeaders) throws IOException { this.isTraceEnabled = LOG.isTraceEnabled(); @@ -265,7 +246,7 @@ public AbfsHttpOperation(final URL url, final String method, final List * For storage errors, the response body *usually* has the following format: - * + *

* { - * "error": - * { - * "code": "string", - * "message": "string" - * } + * "error": + * { + * "code": "string", + * "message": "string" + * } * } - * */ private void processStorageErrorResponse() { try (InputStream stream = connection.getErrorStream()) { @@ -453,17 +433,17 @@ private void processStorageErrorResponse() { jp.nextToken(); fieldValue = jp.getText(); switch (fieldName) { - case "code": - storageErrorCode = fieldValue; - break; - case "message": - storageErrorMessage = fieldValue; - break; - case "ExpectedAppendPos": - expectedAppendPos = fieldValue; - break; - default: - break; + case "code": + storageErrorCode = fieldValue; + break; + case "message": + storageErrorMessage = fieldValue; + break; + case "ExpectedAppendPos": + expectedAppendPos = fieldValue; + break; + default: + break; } } jp.nextToken(); @@ -511,9 +491,10 @@ private void parseListFilesResponse(final InputStream stream) throws IOException /** * Check null stream, this is to pass findbugs's redundant check for NULL + * * @param stream InputStream */ private boolean isNullInputStream(InputStream stream) { return stream == null ? true : false; } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 51b24a6601ef2..6f137a119c715 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -112,7 +112,6 @@ private String getInputStreamID() { } @Override - public int read() throws IOException { byte[] b = new byte[1]; int numberOfBytesRead = read(b, 0, 1); @@ -296,8 +295,9 @@ int readRemote(long position, byte[] b, int offset, int length, TracingContext t AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "readRemote", "read")) { LOG.trace("Trigger client.read for path={} position={} offset={} length={}", path, position, offset, length); - op = client.read(path, position, b, offset, length, tolerateOobAppends ? "*" : eTag, cachedSasToken.get(), - tracingContext); + op = client.read(path, position, b, offset, length, + tolerateOobAppends ? "*" : eTag, cachedSasToken.get(), + tracingContext); cachedSasToken.update(op.getSasToken()); if (streamStatistics != null) { streamStatistics.remoteReadOperation(); 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 84005eab90c34..9ac754cd806bc 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 @@ -225,7 +225,6 @@ private boolean executeHttpOperation(final int retryCount, TracingContext tracin // initialize the HTTP request and open the connection httpOperation = new AbfsHttpOperation(url, method, requestHeaders); incrementCounter(AbfsStatistic.CONNECTIONS_MADE, 1); - updateClientRequestHeader(httpOperation, tracingContext); switch(client.getAuthType()) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index 1d959ef559b38..6279bbd44bfcb 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -142,6 +142,14 @@ protected AbstractAbfsIntegrationTest() throws Exception { } } + protected boolean getIsNamespaceEnabled(AzureBlobFileSystem fs) throws IOException { + AbfsConfiguration abfsConf = fs.getAbfsStore().getAbfsConfiguration(); + return fs.getAbfsStore().getIsNamespaceEnabled( + new TracingContext(abfsConf.getClientCorrelationID(), + fs.getFileSystemID(), AbfsOperations.GET_NAMESPACE, + abfsConf.getTracingContextFormat())); + } + public TracingContext getTestTracingContext(AzureBlobFileSystem fs, boolean isCont) { if (fs == null) { return new TracingContext("test-corr-id", diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java index d47719cd7baa7..a83c9bace44a0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java @@ -95,7 +95,6 @@ public void testUnknownHost() throws Exception { () -> FileSystem.get(conf.getRawConfiguration())); } - @Test public void testListPathWithValidListMaxResultsValues() throws IOException, ExecutionException, InterruptedException { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java index cc86923357aa5..d7373e5f8a8bf 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java @@ -42,7 +42,7 @@ public ITestAzureBlobFileSystemAttributes() throws Exception { @Test public void testSetGetXAttr() throws Exception { AzureBlobFileSystem fs = getFileSystem(); - Assume.assumeTrue(fs.getIsNamespaceEnabled()); + Assume.assumeTrue(getIsNamespaceEnabled(fs)); byte[] attributeValue1 = fs.getAbfsStore().encodeAttribute("hi"); byte[] attributeValue2 = fs.getAbfsStore().encodeAttribute("你好"); @@ -67,7 +67,7 @@ public void testSetGetXAttr() throws Exception { @Test public void testSetGetXAttrCreateReplace() throws Exception { AzureBlobFileSystem fs = getFileSystem(); - Assume.assumeTrue(fs.getIsNamespaceEnabled()); + Assume.assumeTrue(getIsNamespaceEnabled(fs)); byte[] attributeValue = fs.getAbfsStore().encodeAttribute("one"); String attributeName = "user.someAttribute"; Path testFile = path("createReplaceXAttr"); @@ -84,7 +84,7 @@ public void testSetGetXAttrCreateReplace() throws Exception { @Test public void testSetGetXAttrReplace() throws Exception { AzureBlobFileSystem fs = getFileSystem(); - Assume.assumeTrue(fs.getIsNamespaceEnabled()); + Assume.assumeTrue(getIsNamespaceEnabled(fs)); byte[] attributeValue1 = fs.getAbfsStore().encodeAttribute("one"); byte[] attributeValue2 = fs.getAbfsStore().encodeAttribute("two"); String attributeName = "user.someAttribute"; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAuthorization.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAuthorization.java index 1278e652b3bbb..589c3a285f48f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAuthorization.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAuthorization.java @@ -209,55 +209,55 @@ public void testGetFileStatusUnauthorized() throws Exception { @Test public void testSetOwnerUnauthorized() throws Exception { - Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled()); + Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.SetOwner, true); } @Test public void testSetPermissionUnauthorized() throws Exception { - Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled()); + Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.SetPermissions, true); } @Test public void testModifyAclEntriesUnauthorized() throws Exception { - Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled()); + Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.ModifyAclEntries, true); } @Test public void testRemoveAclEntriesUnauthorized() throws Exception { - Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled()); + Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.RemoveAclEntries, true); } @Test public void testRemoveDefaultAclUnauthorized() throws Exception { - Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled()); + Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.RemoveDefaultAcl, true); } @Test public void testRemoveAclUnauthorized() throws Exception { - Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled()); + Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.RemoveAcl, true); } @Test public void testSetAclUnauthorized() throws Exception { - Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled()); + Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.SetAcl, true); } @Test public void testGetAclStatusAuthorized() throws Exception { - Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled()); + Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.GetAcl, false); } @Test public void testGetAclStatusUnauthorized() throws Exception { - Assume.assumeTrue(this.getFileSystem().getIsNamespaceEnabled()); + Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); runTest(FileSystemOperations.GetAcl, true); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java index 5ac16b45a4572..e47a37e22691e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java @@ -43,7 +43,7 @@ public ITestAzureBlobFileSystemBackCompat() throws Exception { public void testBlobBackCompat() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); Assume.assumeFalse("This test does not support namespace enabled account", - this.getFileSystem().getIsNamespaceEnabled()); + getIsNamespaceEnabled(getFileSystem())); String storageConnectionString = getBlobConnectionString(); CloudStorageAccount storageAccount = CloudStorageAccount.parse(storageConnectionString); CloudBlobClient blobClient = storageAccount.createCloudBlobClient(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index 66b1c93a00602..fc804d42c746a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -219,7 +219,7 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { "/NonExistingPath", false, null, - getTestTracingContext(fs, true))); + getTestTracingContext(fs, true))); // mock idempotency check to mimic retried case AbfsClient mockClient = TestAbfsClient.getMockAbfsClient( diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java index 421fa9a65cc05..b9498be89a1ff 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java @@ -67,7 +67,7 @@ private FileStatus validateStatus(final AzureBlobFileSystem fs, final Path name, String errorInStatus = "error in " + fileStatus + " from " + fs; - if (!fs.getIsNamespaceEnabled()) { + if (!getIsNamespaceEnabled(fs)) { assertEquals(errorInStatus + ": owner", fs.getOwnerUser(), fileStatus.getOwner()); assertEquals(errorInStatus + ": group", diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java index 257fb4fdbd2ab..138e2023786bc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java @@ -76,7 +76,7 @@ public static Collection abfsCreateNonRecursiveTestData() public void testFilePermission() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(fs.getIsNamespaceEnabled()); + Assume.assumeTrue(getIsNamespaceEnabled(fs)); fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, DEFAULT_UMASK_VALUE); path = new Path(testRoot, UUID.randomUUID().toString()); @@ -92,7 +92,7 @@ public void testFilePermission() throws Exception { @Test public void testFolderPermission() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(fs.getIsNamespaceEnabled()); + Assume.assumeTrue(getIsNamespaceEnabled(fs)); fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "027"); path = new Path(testRoot, UUID.randomUUID().toString()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java index f58276312831a..f5938c00fc798 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java @@ -98,7 +98,7 @@ public void testBasicRead() throws Exception { @Test public void testRandomRead() throws Exception { Assume.assumeFalse("This test does not support namespace enabled account", - this.getFileSystem().getIsNamespaceEnabled()); + getIsNamespaceEnabled(getFileSystem())); assumeHugeFileExists(); try ( FSDataInputStream inputStreamV1 @@ -416,7 +416,7 @@ public void testSequentialReadAfterReverseSeekPerformance() @Ignore("HADOOP-16915") public void testRandomReadPerformance() throws Exception { Assume.assumeFalse("This test does not support namespace enabled account", - this.getFileSystem().getIsNamespaceEnabled()); + getIsNamespaceEnabled(getFileSystem())); createTestFile(); assumeHugeFileExists(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java index 397d1a6d9a19b..78426f0331dd6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java @@ -88,7 +88,7 @@ public ITestAzureBlobFilesystemAcl() throws Exception { @Test public void testModifyAclEntries() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); fs.mkdirs(path, FsPermission.createImmutable((short) RWX_RX)); @@ -121,7 +121,7 @@ public void testModifyAclEntries() throws Exception { @Test public void testModifyAclEntriesOnlyAccess() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); @@ -145,7 +145,7 @@ public void testModifyAclEntriesOnlyAccess() throws Exception { @Test public void testModifyAclEntriesOnlyDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -168,7 +168,7 @@ public void testModifyAclEntriesOnlyDefault() throws Exception { @Test public void testModifyAclEntriesMinimal() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); @@ -186,7 +186,7 @@ public void testModifyAclEntriesMinimal() throws Exception { @Test public void testModifyAclEntriesMinimalDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -206,7 +206,7 @@ public void testModifyAclEntriesMinimalDefault() throws Exception { @Test public void testModifyAclEntriesCustomMask() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); @@ -225,7 +225,7 @@ public void testModifyAclEntriesCustomMask() throws Exception { @Test public void testModifyAclEntriesStickyBit() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750)); List aclSpec = Lists.newArrayList( @@ -255,7 +255,7 @@ public void testModifyAclEntriesStickyBit() throws Exception { @Test(expected=FileNotFoundException.class) public void testModifyAclEntriesPathNotFound() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); // Path has not been created. List aclSpec = Lists.newArrayList( @@ -269,7 +269,7 @@ public void testModifyAclEntriesPathNotFound() throws Exception { @Test (expected=Exception.class) public void testModifyAclEntriesDefaultOnFile() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); @@ -281,7 +281,7 @@ public void testModifyAclEntriesDefaultOnFile() throws Exception { @Test public void testModifyAclEntriesWithDefaultMask() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -305,7 +305,7 @@ public void testModifyAclEntriesWithDefaultMask() throws Exception { @Test public void testModifyAclEntriesWithAccessMask() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -326,7 +326,7 @@ public void testModifyAclEntriesWithAccessMask() throws Exception { @Test(expected=PathIOException.class) public void testModifyAclEntriesWithDuplicateEntries() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -342,7 +342,7 @@ public void testModifyAclEntriesWithDuplicateEntries() throws Exception { @Test public void testRemoveAclEntries() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -370,7 +370,7 @@ public void testRemoveAclEntries() throws Exception { @Test public void testRemoveAclEntriesOnlyAccess() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); @@ -395,7 +395,7 @@ public void testRemoveAclEntriesOnlyAccess() throws Exception { @Test public void testRemoveAclEntriesOnlyDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -422,7 +422,7 @@ public void testRemoveAclEntriesOnlyDefault() throws Exception { @Test public void testRemoveAclEntriesMinimal() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) RWX_RW)); @@ -445,7 +445,7 @@ public void testRemoveAclEntriesMinimal() throws Exception { @Test public void testRemoveAclEntriesMinimalDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -473,7 +473,7 @@ public void testRemoveAclEntriesMinimalDefault() throws Exception { @Test public void testRemoveAclEntriesStickyBit() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750)); List aclSpec = Lists.newArrayList( @@ -501,7 +501,7 @@ public void testRemoveAclEntriesStickyBit() throws Exception { @Test(expected=FileNotFoundException.class) public void testRemoveAclEntriesPathNotFound() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); // Path has not been created. List aclSpec = Lists.newArrayList( @@ -512,7 +512,7 @@ public void testRemoveAclEntriesPathNotFound() throws Exception { @Test(expected=PathIOException.class) public void testRemoveAclEntriesAccessMask() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -526,7 +526,7 @@ public void testRemoveAclEntriesAccessMask() throws Exception { @Test(expected=PathIOException.class) public void testRemoveAclEntriesDefaultMask() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -540,7 +540,7 @@ public void testRemoveAclEntriesDefaultMask() throws Exception { @Test(expected=PathIOException.class) public void testRemoveAclEntriesWithDuplicateEntries() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -556,7 +556,7 @@ public void testRemoveAclEntriesWithDuplicateEntries() throws Exception { @Test public void testRemoveDefaultAcl() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -578,7 +578,7 @@ public void testRemoveDefaultAcl() throws Exception { @Test public void testRemoveDefaultAclOnlyAccess() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); @@ -600,7 +600,7 @@ public void testRemoveDefaultAclOnlyAccess() throws Exception { @Test public void testRemoveDefaultAclOnlyDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -616,7 +616,7 @@ public void testRemoveDefaultAclOnlyDefault() throws Exception { @Test public void testRemoveDefaultAclMinimal() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); fs.removeDefaultAcl(path); @@ -629,7 +629,7 @@ public void testRemoveDefaultAclMinimal() throws Exception { @Test public void testRemoveDefaultAclStickyBit() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750)); List aclSpec = Lists.newArrayList( @@ -651,7 +651,7 @@ public void testRemoveDefaultAclStickyBit() throws Exception { @Test(expected=FileNotFoundException.class) public void testRemoveDefaultAclPathNotFound() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); // Path has not been created. fs.removeDefaultAcl(path); @@ -660,7 +660,7 @@ public void testRemoveDefaultAclPathNotFound() throws Exception { @Test public void testRemoveAcl() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -682,7 +682,7 @@ public void testRemoveAcl() throws Exception { @Test public void testRemoveAclMinimalAcl() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); @@ -696,7 +696,7 @@ public void testRemoveAclMinimalAcl() throws Exception { @Test public void testRemoveAclStickyBit() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750)); List aclSpec = Lists.newArrayList( @@ -716,7 +716,7 @@ public void testRemoveAclStickyBit() throws Exception { @Test public void testRemoveAclOnlyDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -735,7 +735,7 @@ public void testRemoveAclOnlyDefault() throws Exception { @Test(expected=FileNotFoundException.class) public void testRemoveAclPathNotFound() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); // Path has not been created. fs.removeAcl(path); @@ -744,7 +744,7 @@ public void testRemoveAclPathNotFound() throws Exception { @Test public void testSetAcl() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -770,7 +770,7 @@ public void testSetAcl() throws Exception { @Test public void testSetAclOnlyAccess() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); @@ -791,7 +791,7 @@ public void testSetAclOnlyAccess() throws Exception { @Test public void testSetAclOnlyDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -811,7 +811,7 @@ public void testSetAclOnlyDefault() throws Exception { @Test public void testSetAclMinimal() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) RW_R_R)); @@ -835,7 +835,7 @@ public void testSetAclMinimal() throws Exception { @Test public void testSetAclMinimalDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -855,7 +855,7 @@ public void testSetAclMinimalDefault() throws Exception { @Test public void testSetAclCustomMask() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); @@ -877,7 +877,7 @@ public void testSetAclCustomMask() throws Exception { @Test public void testSetAclStickyBit() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750)); List aclSpec = Lists.newArrayList( @@ -903,7 +903,7 @@ public void testSetAclStickyBit() throws Exception { @Test(expected=FileNotFoundException.class) public void testSetAclPathNotFound() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); // Path has not been created. List aclSpec = Lists.newArrayList( @@ -917,7 +917,7 @@ public void testSetAclPathNotFound() throws Exception { @Test(expected=Exception.class) public void testSetAclDefaultOnFile() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); @@ -929,7 +929,7 @@ public void testSetAclDefaultOnFile() throws Exception { @Test public void testSetAclDoesNotChangeDefaultMask() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -953,7 +953,7 @@ public void testSetAclDoesNotChangeDefaultMask() throws Exception { @Test(expected=PathIOException.class) public void testSetAclWithDuplicateEntries() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -965,7 +965,7 @@ public void testSetAclWithDuplicateEntries() throws Exception { @Test public void testSetPermission() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -992,7 +992,7 @@ public void testSetPermission() throws Exception { @Test public void testSetPermissionOnlyAccess() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); @@ -1014,7 +1014,7 @@ public void testSetPermissionOnlyAccess() throws Exception { @Test public void testSetPermissionOnlyDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -1038,7 +1038,7 @@ public void testSetPermissionOnlyDefault() throws Exception { @Test public void testDefaultAclNewFile() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -1058,7 +1058,7 @@ public void testDefaultAclNewFile() throws Exception { @Ignore // wait umask fix to be deployed public void testOnlyAccessAclNewFile() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -1075,7 +1075,7 @@ public void testOnlyAccessAclNewFile() throws Exception { @Test public void testDefaultMinimalAclNewFile() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -1094,7 +1094,7 @@ public void testDefaultMinimalAclNewFile() throws Exception { @Test public void testDefaultAclNewDir() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -1120,7 +1120,7 @@ public void testDefaultAclNewDir() throws Exception { @Test public void testOnlyAccessAclNewDir() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -1137,7 +1137,7 @@ public void testOnlyAccessAclNewDir() throws Exception { @Test public void testDefaultMinimalAclNewDir() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( @@ -1159,7 +1159,7 @@ public void testDefaultMinimalAclNewDir() throws Exception { @Test public void testDefaultAclNewFileWithMode() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX_RX)); List aclSpec = Lists.newArrayList( @@ -1181,7 +1181,7 @@ public void testDefaultAclNewFileWithMode() throws Exception { @Test public void testDefaultAclNewDirWithMode() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX_RX)); List aclSpec = Lists.newArrayList( @@ -1205,7 +1205,7 @@ public void testDefaultAclNewDirWithMode() throws Exception { @Test public void testDefaultAclRenamedFile() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); Path dirPath = new Path(path, "dir"); FileSystem.mkdirs(fs, dirPath, FsPermission.createImmutable((short) RWX_RX)); @@ -1227,7 +1227,7 @@ public void testDefaultAclRenamedFile() throws Exception { @Test public void testDefaultAclRenamedDir() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); Path dirPath = new Path(path, "dir"); FileSystem.mkdirs(fs, dirPath, FsPermission.createImmutable((short) RWX_RX)); @@ -1248,7 +1248,7 @@ public void testDefaultAclRenamedDir() throws Exception { @Test public void testEnsureAclOperationWorksForRoot() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - assumeTrue(fs.getIsNamespaceEnabled()); + assumeTrue(getIsNamespaceEnabled(fs)); Path rootPath = new Path("/"); @@ -1273,7 +1273,7 @@ public void testEnsureAclOperationWorksForRoot() throws Exception { @Test public void testSetOwnerForNonNamespaceEnabledAccount() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(!fs.getIsNamespaceEnabled()); + Assume.assumeTrue(!getIsNamespaceEnabled(fs)); final Path filePath = new Path(methodName.getMethodName()); fs.create(filePath); @@ -1290,7 +1290,7 @@ public void testSetOwnerForNonNamespaceEnabledAccount() throws Exception { @Test public void testSetPermissionForNonNamespaceEnabledAccount() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(!fs.getIsNamespaceEnabled()); + Assume.assumeTrue(!getIsNamespaceEnabled(fs)); final Path filePath = new Path(methodName.getMethodName()); fs.create(filePath); @@ -1309,7 +1309,7 @@ public void testSetPermissionForNonNamespaceEnabledAccount() throws Exception { @Test public void testModifyAclEntriesForNonNamespaceEnabledAccount() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(!fs.getIsNamespaceEnabled()); + Assume.assumeTrue(!getIsNamespaceEnabled(fs)); final Path filePath = new Path(methodName.getMethodName()); fs.create(filePath); try { @@ -1326,7 +1326,7 @@ public void testModifyAclEntriesForNonNamespaceEnabledAccount() throws Exception @Test public void testRemoveAclEntriesEntriesForNonNamespaceEnabledAccount() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(!fs.getIsNamespaceEnabled()); + Assume.assumeTrue(!getIsNamespaceEnabled(fs)); final Path filePath = new Path(methodName.getMethodName()); fs.create(filePath); try { @@ -1343,7 +1343,7 @@ public void testRemoveAclEntriesEntriesForNonNamespaceEnabledAccount() throws Ex @Test public void testRemoveDefaultAclForNonNamespaceEnabledAccount() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(!fs.getIsNamespaceEnabled()); + Assume.assumeTrue(!getIsNamespaceEnabled(fs)); final Path filePath = new Path(methodName.getMethodName()); fs.create(filePath); try { @@ -1357,7 +1357,7 @@ public void testRemoveDefaultAclForNonNamespaceEnabledAccount() throws Exception @Test public void testRemoveAclForNonNamespaceEnabledAccount() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(!fs.getIsNamespaceEnabled()); + Assume.assumeTrue(!getIsNamespaceEnabled(fs)); final Path filePath = new Path(methodName.getMethodName()); fs.create(filePath); try { @@ -1371,7 +1371,7 @@ public void testRemoveAclForNonNamespaceEnabledAccount() throws Exception { @Test public void testSetAclForNonNamespaceEnabledAccount() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(!fs.getIsNamespaceEnabled()); + Assume.assumeTrue(!getIsNamespaceEnabled(fs)); final Path filePath = new Path(methodName.getMethodName()); fs.create(filePath); try { @@ -1388,7 +1388,7 @@ public void testSetAclForNonNamespaceEnabledAccount() throws Exception { @Test public void testGetAclStatusForNonNamespaceEnabledAccount() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - Assume.assumeTrue(!fs.getIsNamespaceEnabled()); + Assume.assumeTrue(!getIsNamespaceEnabled(fs)); final Path filePath = new Path(methodName.getMethodName()); fs.create(filePath); try { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java index 3d336ae3870a3..bb2e330e7b00e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java @@ -67,7 +67,7 @@ public void testXNSAccount() throws IOException { Assume.assumeTrue("Skip this test because the account being used for test is a non XNS account", isUsingXNSAccount); assertTrue("Expecting getIsNamespaceEnabled() return true", - getFileSystem().getIsNamespaceEnabled()); + getIsNamespaceEnabled(getFileSystem())); } @Test @@ -75,13 +75,13 @@ public void testNonXNSAccount() throws IOException { Assume.assumeFalse("Skip this test because the account being used for test is a XNS account", isUsingXNSAccount); assertFalse("Expecting getIsNamespaceEnabled() return false", - getFileSystem().getIsNamespaceEnabled()); + getIsNamespaceEnabled(getFileSystem())); } @Test public void testGetIsNamespaceEnabledWhenConfigIsTrue() throws Exception { AzureBlobFileSystem fs = getNewFSWithHnsConf(TRUE_STR); - Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs( + Assertions.assertThat(getIsNamespaceEnabled(fs)).describedAs( "getIsNamespaceEnabled should return true when the " + "config is set as true").isTrue(); fs.getAbfsStore().deleteFilesystem(getTestTracingContext(fs, false)); @@ -91,7 +91,7 @@ public void testGetIsNamespaceEnabledWhenConfigIsTrue() throws Exception { @Test public void testGetIsNamespaceEnabledWhenConfigIsFalse() throws Exception { AzureBlobFileSystem fs = getNewFSWithHnsConf(FALSE_STR); - Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs( + Assertions.assertThat(getIsNamespaceEnabled(fs)).describedAs( "getIsNamespaceEnabled should return false when the " + "config is set as false").isFalse(); fs.getAbfsStore().deleteFilesystem(getTestTracingContext(fs, false)); @@ -103,7 +103,7 @@ private void unsetAndAssert() throws Exception { DEFAULT_FS_AZURE_ACCOUNT_IS_HNS_ENABLED); boolean expectedValue = this.getConfiguration() .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false); - Assertions.assertThat(fs.getIsNamespaceEnabled()).describedAs( + Assertions.assertThat(getIsNamespaceEnabled(fs)).describedAs( "getIsNamespaceEnabled should return the value " + "configured for fs.azure.test.namespace.enabled") .isEqualTo(expectedValue); @@ -207,7 +207,7 @@ private AbfsClient callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient() doReturn(mock(AbfsRestOperation.class)).when(mockClient) .getAclStatus(anyString(), any(TracingContext.class)); abfsStore.setClient(mockClient); - abfs.getIsNamespaceEnabled(); + getIsNamespaceEnabled(abfs); return mockClient; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java index e8f845c9a39bc..94d64b050105c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java @@ -58,7 +58,7 @@ public void testListFileStatus() throws Exception { AzureBlobFileSystem fs = getFileSystem(); // test only valid for non-namespace enabled account Assume.assumeFalse("Namespace enabled account does not support this test,", - fs.getIsNamespaceEnabled()); + getIsNamespaceEnabled(fs)); NativeAzureFileSystem wasb = getWasbFileSystem(); @@ -93,7 +93,7 @@ public void testReadFile() throws Exception { AzureBlobFileSystem abfs = getFileSystem(); // test only valid for non-namespace enabled account Assume.assumeFalse("Namespace enabled account does not support this test", - abfs.getIsNamespaceEnabled()); + getIsNamespaceEnabled(abfs)); NativeAzureFileSystem wasb = getWasbFileSystem(); @@ -133,7 +133,7 @@ public void testDir() throws Exception { AzureBlobFileSystem abfs = getFileSystem(); // test only valid for non-namespace enabled account Assume.assumeFalse("Namespace enabled account does not support this test", - abfs.getIsNamespaceEnabled()); + getIsNamespaceEnabled(abfs)); NativeAzureFileSystem wasb = getWasbFileSystem(); @@ -168,7 +168,7 @@ public void testSetWorkingDirectory() throws Exception { AzureBlobFileSystem abfs = getFileSystem(); // test only valid for non-namespace enabled account Assume.assumeFalse("Namespace enabled account does not support this test", - abfs.getIsNamespaceEnabled()); + getIsNamespaceEnabled(abfs)); NativeAzureFileSystem wasb = getWasbFileSystem(); From 644293231d6eb34245742c1fde75fc022d7dae66 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 3 Nov 2020 19:55:18 +0530 Subject: [PATCH 31/77] revert httpop formatting --- .../azurebfs/services/AbfsHttpOperation.java | 80 +++++++++++-------- 1 file changed, 46 insertions(+), 34 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index 37d0535c28681..6fc3cb488bd65 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -26,6 +26,7 @@ import java.net.URL; import java.net.URLEncoder; import java.util.List; +import java.util.UUID; import javax.net.ssl.HttpsURLConnection; import javax.net.ssl.SSLSocketFactory; @@ -43,7 +44,6 @@ import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsPerfLoggable; import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; /** * Represents an HTTP operation. @@ -66,8 +66,8 @@ public class AbfsHttpOperation implements AbfsPerfLoggable { private int statusCode; private String statusDescription; private String storageErrorCode = ""; - private String storageErrorMessage = ""; - private String requestId = ""; + private String storageErrorMessage = ""; + private String requestId = ""; private String expectedAppendPos = ""; private ListResultSchema listResultSchema = null; @@ -94,7 +94,7 @@ private AbfsHttpOperation(final URL url, final String method, this.statusCode = httpStatus; } - protected HttpURLConnection getConnection() { + protected HttpURLConnection getConnection() { return connection; } @@ -123,8 +123,7 @@ public String getStorageErrorMessage() { } public String getClientRequestId() { - return this.connection - .getRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); + return this.connection.getRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); } public String getExpectedAppendPos() { @@ -151,11 +150,6 @@ public String getResponseHeader(String httpHeader) { return connection.getHeaderField(httpHeader); } - @com.google.common.annotations.VisibleForTesting - public String getRequestHeader(String httpHeader) { - return connection.getRequestProperties().get(httpHeader).toString(); - } - // Returns a trace message for the request @Override public String toString() { @@ -195,19 +189,37 @@ public String getLogString() { try { urlStr = URLEncoder.encode(url.toString(), "UTF-8"); - } catch (UnsupportedEncodingException e) { + } catch(UnsupportedEncodingException e) { urlStr = "https%3A%2F%2Ffailed%2Fto%2Fencode%2Furl"; } final StringBuilder sb = new StringBuilder(); - sb.append("s=").append(statusCode).append(" e=").append(storageErrorCode).append(" ci=").append(getClientRequestId()).append(" ri=") + sb.append("s=") + .append(statusCode) + .append(" e=") + .append(storageErrorCode) + .append(" ci=") + .append(getClientRequestId()) + .append(" ri=") .append(requestId); if (isTraceEnabled) { - sb.append(" ct=").append(connectionTimeMs).append(" st=").append(sendRequestTimeMs).append(" rt=").append(recvResponseTimeMs); + sb.append(" ct=") + .append(connectionTimeMs) + .append(" st=") + .append(sendRequestTimeMs) + .append(" rt=") + .append(recvResponseTimeMs); } - sb.append(" bs=").append(bytesSent).append(" br=").append(bytesReceived).append(" m=").append(method).append(" u=").append(urlStr); + sb.append(" bs=") + .append(bytesSent) + .append(" br=") + .append(bytesReceived) + .append(" m=") + .append(method) + .append(" u=") + .append(urlStr); return sb.toString(); } @@ -215,10 +227,10 @@ public String getLogString() { /** * Initializes a new HTTP request and opens the connection. * - * @param url The full URL including query string parameters. - * @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or - * DELETE). + * @param url The full URL including query string parameters. + * @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE). * @param requestHeaders The HTTP request headers.READ_TIMEOUT + * * @throws IOException if an error occurs. */ public AbfsHttpOperation(final URL url, final String method, final List requestHeaders) @@ -254,16 +266,16 @@ public AbfsHttpOperation(final URL url, final String method, final List + * * For storage errors, the response body *usually* has the following format: - *

- * { - * "error": + * * { - * "code": "string", - * "message": "string" - * } + * "error": + * { + * "code": "string", + * "message": "string" + * } * } + * */ private void processStorageErrorResponse() { try (InputStream stream = connection.getErrorStream()) { @@ -491,10 +504,9 @@ private void parseListFilesResponse(final InputStream stream) throws IOException /** * Check null stream, this is to pass findbugs's redundant check for NULL - * * @param stream InputStream */ private boolean isNullInputStream(InputStream stream) { return stream == null ? true : false; } -} +} \ No newline at end of file From ec92f2576698dcb702898f1af61cfb71fb9d54b0 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 3 Nov 2020 20:02:59 +0530 Subject: [PATCH 32/77] revert httpop formatting --- .../azurebfs/services/AbfsHttpOperation.java | 69 ++++++++++--------- 1 file changed, 35 insertions(+), 34 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index 6fc3cb488bd65..1bfbe5a4b58f4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -83,7 +83,7 @@ public class AbfsHttpOperation implements AbfsPerfLoggable { public static AbfsHttpOperation getAbfsHttpOperationWithFixedResult(final URL url, final String method, final int httpStatus) { - return new AbfsHttpOperation(url, method, httpStatus); + return new AbfsHttpOperation(url, method, httpStatus); } private AbfsHttpOperation(final URL url, final String method, @@ -123,7 +123,8 @@ public String getStorageErrorMessage() { } public String getClientRequestId() { - return this.connection.getRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); + return this.connection + .getRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); } public String getExpectedAppendPos() { @@ -195,31 +196,31 @@ public String getLogString() { final StringBuilder sb = new StringBuilder(); sb.append("s=") - .append(statusCode) - .append(" e=") - .append(storageErrorCode) - .append(" ci=") - .append(getClientRequestId()) - .append(" ri=") - .append(requestId); + .append(statusCode) + .append(" e=") + .append(storageErrorCode) + .append(" ci=") + .append(getClientRequestId()) + .append(" ri=") + .append(requestId); if (isTraceEnabled) { sb.append(" ct=") - .append(connectionTimeMs) - .append(" st=") - .append(sendRequestTimeMs) - .append(" rt=") - .append(recvResponseTimeMs); + .append(connectionTimeMs) + .append(" st=") + .append(sendRequestTimeMs) + .append(" rt=") + .append(recvResponseTimeMs); } sb.append(" bs=") - .append(bytesSent) - .append(" br=") - .append(bytesReceived) - .append(" m=") - .append(method) - .append(" u=") - .append(urlStr); + .append(bytesSent) + .append(" br=") + .append(bytesReceived) + .append(" m=") + .append(method) + .append(" u=") + .append(urlStr); return sb.toString(); } @@ -258,7 +259,7 @@ public AbfsHttpOperation(final URL url, final String method, final List Date: Tue, 3 Nov 2020 20:38:44 +0530 Subject: [PATCH 33/77] move tc init near usage --- .../fs/azurebfs/AzureBlobFileSystem.java | 56 +++++++++---------- .../fs/azurebfs/AzureBlobFileSystemStore.java | 1 - 2 files changed, 28 insertions(+), 29 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index c0f40311114b0..11ee0c5201904 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -186,11 +186,11 @@ String getFileSystemID() { public FSDataInputStream open(final Path path, final int bufferSize) throws IOException { LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize); statIncrement(CALL_OPEN); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.OPEN, tracingContextFormat); Path qualifiedPath = makeQualified(path); try { + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.OPEN, tracingContextFormat); InputStream inputStream = abfsStore.openFileForRead(qualifiedPath, statistics, tracingContext); return new FSDataInputStream(inputStream); } catch(AzureBlobFileSystemException ex) { @@ -207,8 +207,6 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi permission, overwrite, blockSize); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.CREATE, tracingContextFormat); statIncrement(CALL_CREATE); trailingPeriodCheck(f); @@ -216,6 +214,8 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi Path qualifiedPath = makeQualified(f); try { + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.CREATE, tracingContextFormat); OutputStream outputStream = abfsStore.createFile(qualifiedPath, statistics, overwrite, permission == null ? FsPermission.getFileDefault() : permission, FsPermission.getUMask(getConf()), tracingContext); @@ -278,11 +278,11 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr f.toString(), bufferSize); statIncrement(CALL_APPEND); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.APPEND, tracingContextFormat); Path qualifiedPath = makeQualified(f); try { + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.APPEND, tracingContextFormat); OutputStream outputStream = abfsStore.openFileForWrite(qualifiedPath, statistics, false, tracingContext); return new FSDataOutputStream(outputStream, statistics); @@ -295,8 +295,6 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr public boolean rename(final Path src, final Path dst) throws IOException { LOG.debug("AzureBlobFileSystem.rename src: {} dst: {}", src, dst); statIncrement(CALL_RENAME); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.RENAME, true, tracingContextFormat); trailingPeriodCheck(dst); @@ -326,6 +324,8 @@ public boolean rename(final Path src, final Path dst) throws IOException { } // Non-HNS account need to check dst status on driver side. + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.RENAME, true, tracingContextFormat); if (!abfsStore.getIsNamespaceEnabled(tracingContext) && dstFileStatus == null) { dstFileStatus = tryGetFileStatus(qualifiedDstPath); } @@ -367,8 +367,6 @@ public boolean delete(final Path f, final boolean recursive) throws IOException "AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), recursive); statIncrement(CALL_DELETE); Path qualifiedPath = makeQualified(f); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.DELETE, tracingContextFormat); if (f.isRoot()) { if (!recursive) { @@ -379,6 +377,8 @@ public boolean delete(final Path f, final boolean recursive) throws IOException } try { + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.DELETE, tracingContextFormat); abfsStore.delete(qualifiedPath, recursive, tracingContext); return true; } catch (AzureBlobFileSystemException ex) { @@ -394,10 +394,10 @@ public FileStatus[] listStatus(final Path f) throws IOException { "AzureBlobFileSystem.listStatus path: {}", f.toString()); statIncrement(CALL_LIST_STATUS); Path qualifiedPath = makeQualified(f); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.LISTSTATUS, true, tracingContextFormat); try { + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.LISTSTATUS, true, tracingContextFormat); FileStatus[] result = abfsStore.listStatus(qualifiedPath, tracingContext); return result; } catch (AzureBlobFileSystemException ex) { @@ -455,8 +455,6 @@ public boolean mkdirs(final Path f, final FsPermission permission) throws IOExce "AzureBlobFileSystem.mkdirs path: {} permissions: {}", f, permission); statIncrement(CALL_MKDIRS); trailingPeriodCheck(f); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.MKDIR, true, tracingContextFormat); final Path parentFolder = f.getParent(); if (parentFolder == null) { @@ -467,6 +465,8 @@ public boolean mkdirs(final Path f, final FsPermission permission) throws IOExce Path qualifiedPath = makeQualified(f); try { + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.MKDIR, true, tracingContextFormat); abfsStore.createDirectory(qualifiedPath, permission == null ? FsPermission.getDirDefault() : permission, FsPermission.getUMask(getConf()), tracingContext); statIncrement(DIRECTORIES_CREATED); @@ -495,10 +495,10 @@ public FileStatus getFileStatus(final Path f) throws IOException { LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f); statIncrement(CALL_GET_FILE_STATUS); Path qualifiedPath = makeQualified(f); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.FILESYSTEM_OPS, tracingContextFormat); try { + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.FILESYSTEM_OPS, tracingContextFormat); return abfsStore.getFileStatus(qualifiedPath, tracingContext); } catch(AzureBlobFileSystemException ex) { checkException(f, ex); @@ -711,8 +711,6 @@ public void setOwner(final Path path, final String owner, final String group) public void setXAttr(final Path path, final String name, final byte[] value, final EnumSet flag) throws IOException { LOG.debug("AzureBlobFileSystem.setXAttr path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.ATTR_OPS, true, tracingContextFormat); if (name == null || name.isEmpty() || value == null) { throw new IllegalArgumentException("A valid name and value must be specified."); @@ -721,6 +719,8 @@ public void setXAttr(final Path path, final String name, final byte[] value, fin Path qualifiedPath = makeQualified(path); try { + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.ATTR_OPS, true, tracingContextFormat); Hashtable properties = abfsStore.getPathStatus(qualifiedPath, tracingContext); String xAttrName = ensureValidAttributeName(name); boolean xAttrExists = properties.containsKey(xAttrName); @@ -752,13 +752,13 @@ public byte[] getXAttr(final Path path, final String name) if (name == null || name.isEmpty()) { throw new IllegalArgumentException("A valid name must be specified."); } - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.ATTR_OPS, true, tracingContextFormat); Path qualifiedPath = makeQualified(path); byte[] value = null; try { + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.ATTR_OPS, true, tracingContextFormat); Hashtable properties = abfsStore.getPathStatus(qualifiedPath, tracingContext); String xAttrName = ensureValidAttributeName(name); if (properties.containsKey(xAttrName)) { @@ -1015,10 +1015,10 @@ public AclStatus getAclStatus(final Path path) throws IOException { @Override public void access(final Path path, final FsAction mode) throws IOException { LOG.debug("AzureBlobFileSystem.access path : {}, mode : {}", path, mode); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.PATH_OPS, tracingContextFormat); Path qualifiedPath = makeQualified(path); try { + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.PATH_OPS, tracingContextFormat); this.abfsStore.access(qualifiedPath, mode, tracingContext); } catch (AzureBlobFileSystemException ex) { checkCheckAccessException(path, ex); @@ -1051,9 +1051,9 @@ private FileStatus tryGetFileStatus(final Path f) { private boolean fileSystemExists() throws IOException { LOG.debug( "AzureBlobFileSystem.fileSystemExists uri: {}", uri); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.FILESYSTEM_OPS, tracingContextFormat); try { + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.FILESYSTEM_OPS, tracingContextFormat); abfsStore.getFilesystemProperties(tracingContext); } catch (AzureBlobFileSystemException ex) { try { @@ -1072,9 +1072,9 @@ private boolean fileSystemExists() throws IOException { private void createFileSystem() throws IOException { LOG.debug( "AzureBlobFileSystem.createFileSystem uri: {}", uri); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.FILESYSTEM_OPS, tracingContextFormat); try { + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.FILESYSTEM_OPS, tracingContextFormat); abfsStore.createFilesystem(tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(null, ex); @@ -1310,13 +1310,13 @@ public boolean hasPathCapability(final Path path, final String capability) throws IOException { // qualify the path to make sure that it refers to the current FS. final Path p = makeQualified(path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.PATH_OPS, tracingContextFormat); switch (validatePathCapabilityArgs(p, capability)) { case CommonPathCapabilities.FS_PERMISSIONS: case CommonPathCapabilities.FS_APPEND: return true; case CommonPathCapabilities.FS_ACLS: + TracingContext tracingContext = new TracingContext(clientCorrelationID, + fileSystemID, AbfsOperations.PATH_OPS, tracingContextFormat); return getIsNamespaceEnabled(tracingContext); default: return super.hasPathCapability(p, capability); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 0d576fa5e4a47..8bbbe4b650d32 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -531,7 +531,6 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa AbfsRestOperation op; tracingContext.setPrimaryRequestID(); - try { // Trigger a create with overwrite=false first so that eTag fetch can be // avoided for cases when no pre-existing file is present (major portion From 1cead5325d825d9076063fef209b07e2f2ae9ca3 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Wed, 4 Nov 2020 11:22:05 +0530 Subject: [PATCH 34/77] minor change --- .../hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index 6279bbd44bfcb..09df218ed2052 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -143,11 +143,8 @@ protected AbstractAbfsIntegrationTest() throws Exception { } protected boolean getIsNamespaceEnabled(AzureBlobFileSystem fs) throws IOException { - AbfsConfiguration abfsConf = fs.getAbfsStore().getAbfsConfiguration(); return fs.getAbfsStore().getIsNamespaceEnabled( - new TracingContext(abfsConf.getClientCorrelationID(), - fs.getFileSystemID(), AbfsOperations.GET_NAMESPACE, - abfsConf.getTracingContextFormat())); + getTestTracingContext(fs, false)); } public TracingContext getTestTracingContext(AzureBlobFileSystem fs, boolean isCont) { From 5dbc7830221092d91562723320e271901ba4eb82 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Thu, 5 Nov 2020 08:10:38 +0530 Subject: [PATCH 35/77] pr changes --- .../fs/azurebfs/AzureBlobFileSystem.java | 49 ++++++++++--------- .../fs/azurebfs/AzureBlobFileSystemStore.java | 12 ++--- ...tions.java => AbfsOperationConstants.java} | 25 +++++----- .../azurebfs/services/AbfsOutputStream.java | 4 -- .../fs/azurebfs/services/ReadBuffer.java | 2 +- .../azurebfs/services/ReadBufferManager.java | 1 + .../fs/azurebfs/utils/TracingContext.java | 9 ++++ .../azurebfs/AbstractAbfsIntegrationTest.java | 2 - .../services/TestAbfsOutputStream.java | 17 +++---- 9 files changed, 64 insertions(+), 57 deletions(-) rename hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/{AbfsOperations.java => AbfsOperationConstants.java} (72%) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 11ee0c5201904..b5bedc140b2a8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -60,7 +60,7 @@ import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperations; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; @@ -190,7 +190,7 @@ public FSDataInputStream open(final Path path, final int bufferSize) throws IOEx try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.OPEN, tracingContextFormat); + fileSystemID, AbfsOperationConstants.OPEN, tracingContextFormat); InputStream inputStream = abfsStore.openFileForRead(qualifiedPath, statistics, tracingContext); return new FSDataInputStream(inputStream); } catch(AzureBlobFileSystemException ex) { @@ -215,7 +215,8 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.CREATE, tracingContextFormat); + fileSystemID, AbfsOperationConstants.CREATE, overwrite, + tracingContextFormat); OutputStream outputStream = abfsStore.createFile(qualifiedPath, statistics, overwrite, permission == null ? FsPermission.getFileDefault() : permission, FsPermission.getUMask(getConf()), tracingContext); @@ -282,7 +283,7 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.APPEND, tracingContextFormat); + fileSystemID, AbfsOperationConstants.APPEND, tracingContextFormat); OutputStream outputStream = abfsStore.openFileForWrite(qualifiedPath, statistics, false, tracingContext); return new FSDataOutputStream(outputStream, statistics); @@ -325,7 +326,7 @@ public boolean rename(final Path src, final Path dst) throws IOException { // Non-HNS account need to check dst status on driver side. TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.RENAME, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.RENAME, true, tracingContextFormat); if (!abfsStore.getIsNamespaceEnabled(tracingContext) && dstFileStatus == null) { dstFileStatus = tryGetFileStatus(qualifiedDstPath); } @@ -378,7 +379,7 @@ public boolean delete(final Path f, final boolean recursive) throws IOException try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.DELETE, tracingContextFormat); + fileSystemID, AbfsOperationConstants.DELETE, tracingContextFormat); abfsStore.delete(qualifiedPath, recursive, tracingContext); return true; } catch (AzureBlobFileSystemException ex) { @@ -397,7 +398,7 @@ public FileStatus[] listStatus(final Path f) throws IOException { try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.LISTSTATUS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.LISTSTATUS, true, tracingContextFormat); FileStatus[] result = abfsStore.listStatus(qualifiedPath, tracingContext); return result; } catch (AzureBlobFileSystemException ex) { @@ -466,7 +467,8 @@ public boolean mkdirs(final Path f, final FsPermission permission) throws IOExce try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.MKDIR, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.MKDIR, false, + tracingContextFormat); abfsStore.createDirectory(qualifiedPath, permission == null ? FsPermission.getDirDefault() : permission, FsPermission.getUMask(getConf()), tracingContext); statIncrement(DIRECTORIES_CREATED); @@ -498,7 +500,7 @@ public FileStatus getFileStatus(final Path f) throws IOException { try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.FILESYSTEM_OPS, tracingContextFormat); + fileSystemID, AbfsOperationConstants.GETFILESTATUS, tracingContextFormat); return abfsStore.getFileStatus(qualifiedPath, tracingContext); } catch(AzureBlobFileSystemException ex) { checkException(f, ex); @@ -674,7 +676,7 @@ public void setOwner(final Path path, final String owner, final String group) LOG.debug( "AzureBlobFileSystem.setOwner path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.PATH_OPS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.SET_OWNER, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { super.setOwner(path, owner, group); @@ -720,7 +722,7 @@ public void setXAttr(final Path path, final String name, final byte[] value, fin try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.ATTR_OPS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.ATTR_OPS, true, tracingContextFormat); Hashtable properties = abfsStore.getPathStatus(qualifiedPath, tracingContext); String xAttrName = ensureValidAttributeName(name); boolean xAttrExists = properties.containsKey(xAttrName); @@ -758,7 +760,7 @@ public byte[] getXAttr(final Path path, final String name) byte[] value = null; try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.ATTR_OPS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.ATTR_OPS, true, tracingContextFormat); Hashtable properties = abfsStore.getPathStatus(qualifiedPath, tracingContext); String xAttrName = ensureValidAttributeName(name); if (properties.containsKey(xAttrName)) { @@ -787,7 +789,7 @@ public void setPermission(final Path path, final FsPermission permission) throws IOException { LOG.debug("AzureBlobFileSystem.setPermission path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.PERMISSION, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.PERMISSION, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { super.setPermission(path, permission); @@ -823,7 +825,7 @@ public void modifyAclEntries(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.modifyAclEntries path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.ACL_OPS,true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.ACL_OPS,true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -858,7 +860,7 @@ public void removeAclEntries(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.removeAclEntries path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.ACL_OPS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.ACL_OPS, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -889,7 +891,7 @@ public void removeAclEntries(final Path path, final List aclSpec) public void removeDefaultAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeDefaultAcl path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.ACL_OPS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.ACL_OPS, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -918,7 +920,7 @@ public void removeDefaultAcl(final Path path) throws IOException { public void removeAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeAcl path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.ACL_OPS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.ACL_OPS, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -950,7 +952,7 @@ public void setAcl(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.setAcl path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.ACL_OPS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.ACL_OPS, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -982,7 +984,7 @@ public void setAcl(final Path path, final List aclSpec) public AclStatus getAclStatus(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.getAclStatus path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.ACL_OPS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.ACL_OPS, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -1018,7 +1020,7 @@ public void access(final Path path, final FsAction mode) throws IOException { Path qualifiedPath = makeQualified(path); try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.PATH_OPS, tracingContextFormat); + fileSystemID, AbfsOperationConstants.PATH_OPS, tracingContextFormat); this.abfsStore.access(qualifiedPath, mode, tracingContext); } catch (AzureBlobFileSystemException ex) { checkCheckAccessException(path, ex); @@ -1053,7 +1055,7 @@ private boolean fileSystemExists() throws IOException { "AzureBlobFileSystem.fileSystemExists uri: {}", uri); try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.FILESYSTEM_OPS, tracingContextFormat); + fileSystemID, AbfsOperationConstants.GETFILESTATUS, tracingContextFormat); abfsStore.getFilesystemProperties(tracingContext); } catch (AzureBlobFileSystemException ex) { try { @@ -1074,7 +1076,7 @@ private void createFileSystem() throws IOException { "AzureBlobFileSystem.createFileSystem uri: {}", uri); try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.FILESYSTEM_OPS, tracingContextFormat); + fileSystemID, AbfsOperationConstants.CREATEFILESYSTEM, tracingContextFormat); abfsStore.createFilesystem(tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(null, ex); @@ -1296,6 +1298,7 @@ AbfsDelegationTokenManager getDelegationTokenManager() { return delegationTokenManager; } + @VisibleForTesting boolean getIsNamespaceEnabled(TracingContext tracingContext) throws AzureBlobFileSystemException { return abfsStore.getIsNamespaceEnabled(tracingContext); } @@ -1316,7 +1319,7 @@ public boolean hasPathCapability(final Path path, final String capability) return true; case CommonPathCapabilities.FS_ACLS: TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperations.PATH_OPS, tracingContextFormat); + fileSystemID, AbfsOperationConstants.PATH_OPS, tracingContextFormat); return getIsNamespaceEnabled(tracingContext); default: return super.hasPathCapability(p, capability); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 8bbbe4b650d32..b4df573911f8a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -483,11 +483,11 @@ public OutputStream createFile(final Path path, AbfsRestOperation op; if (triggerConditionalCreateOverwrite) { op = conditionalCreateOverwriteFile(relativePath, - statistics, - isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null, - isAppendBlob, - tracingContext + statistics, + isNamespaceEnabled ? getOctalNotation(permission) : null, + isNamespaceEnabled ? getOctalNotation(umask) : null, + isAppendBlob, + tracingContext ); } else { @@ -530,7 +530,6 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa final boolean isAppendBlob, TracingContext tracingContext) throws AzureBlobFileSystemException { AbfsRestOperation op; - tracingContext.setPrimaryRequestID(); try { // Trigger a create with overwrite=false first so that eTag fetch can be // avoided for cases when no pre-existing file is present (major portion @@ -696,6 +695,7 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic if (isAppendBlobKey(path.toString())) { isAppendBlob = true; } + return new AbfsOutputStream( client, statistics, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperationConstants.java similarity index 72% rename from hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperations.java rename to hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperationConstants.java index e9ed78054a342..44f8c119c8212 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperationConstants.java @@ -1,20 +1,21 @@ package org.apache.hadoop.fs.azurebfs.constants; -public final class AbfsOperations { - public static final String OPEN = "OP"; - public static final String READ = "RD"; - public static final String WRITE = "WR"; - public static final String CREATE = "CR"; - public static final String LISTSTATUS = "LS"; - public static final String RENAME = "RN"; - public static final String DELETE = "DL"; +public final class AbfsOperationConstants { public static final String ACL_OPS = "AC"; - public static final String FILESYSTEM_OPS = "FS"; - public static final String MKDIR = "MK"; public static final String APPEND = "AP"; public static final String ATTR_OPS = "AT"; + public static final String CREATE = "CR"; + public static final String CREATEFILESYSTEM = "CF"; + public static final String DELETE = "DL"; + public static final String GETFILESTATUS = "FS"; + public static final String GETNAMESPACE = "NS"; + public static final String LISTSTATUS = "LS"; + public static final String MKDIR = "MK"; + public static final String OPEN = "OP"; public static final String PERMISSION = "PR"; - public static final String PATH_OPS = "PA"; - public static final String GET_NAMESPACE = "NS"; + public static final String READ = "RD"; + public static final String RENAME = "RN"; + public static final String SET_OWNER = "SO"; public static final String TEST_OP = "TS"; + public static final String WRITE = "WR"; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index ae10a2185d6b5..d8230e242a8ed 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -281,10 +281,6 @@ public void hflush() throws IOException { } } - public String getStreamID() { - return outputStreamID; - } - /** * Force all data in the output stream to be written to Azure storage. * Wait to return until this is complete. Close the access to the stream and diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java index b23367cfbed61..9ce926d841c84 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java @@ -56,7 +56,7 @@ public void setStream(AbfsInputStream stream) { } public void setTracingContext(TracingContext tracingContext) { - this.tracingContext = new TracingContext(tracingContext); + this.tracingContext = tracingContext; } public TracingContext getTracingContext() { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java index 332a623521331..61707ef0c2bcd 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java @@ -268,6 +268,7 @@ private boolean evict(final ReadBuffer buf) { } completedReadList.remove(buf); + buf.setTracingContext(null); if (LOGGER.isTraceEnabled()) { LOGGER.trace("Evicting buffer idx {}; was used for file {} offset {} length {}", buf.getBufferindex(), buf.getStream().getPath(), buf.getOffset(), buf.getLength()); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index 3bc91d9a93d5e..bf1babbf25c3c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -40,6 +40,15 @@ public class TracingContext { public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*"; +// public static enum TracingContextFormat { +// SINGLE_ID_FORMAT, +// ALL_IDS, +// TWO_ID_FORMAT; +// +// private TracingContextFormat() { +// } +// } + public TracingContext(String clientCorrelationID, String fileSystemID, String hadoopOpName, int tracingContextFormat) { this.fileSystemID = fileSystemID; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index 09df218ed2052..bf39c3555fefe 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -25,8 +25,6 @@ import java.util.UUID; import java.util.concurrent.Callable; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperations; -import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.After; import org.junit.Assert; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index 464c0a42fdfd6..05384e808d6b3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -23,9 +23,8 @@ import java.util.HashSet; import java.util.Random; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperations; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.bouncycastle.jcajce.provider.symmetric.TEA; import org.junit.Test; import org.mockito.ArgumentCaptor; @@ -85,7 +84,7 @@ public void verifyShortWriteRequest() throws Exception { conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperations.WRITE, abfsConf.getTracingContextFormat()); + "test-fs-id", AbfsOperationConstants.WRITE, abfsConf.getTracingContextFormat()); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), @@ -143,7 +142,7 @@ public void verifyWriteRequest() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperations.WRITE, abfsConf.getTracingContextFormat()); + "test-fs-id", AbfsOperationConstants.WRITE, abfsConf.getTracingContextFormat()); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), @@ -211,7 +210,7 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperations.WRITE, abfsConf.getTracingContextFormat()); + "test-fs-id", AbfsOperationConstants.WRITE, abfsConf.getTracingContextFormat()); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), @@ -293,7 +292,7 @@ public void verifyWriteRequestOfBufferSize() throws Exception { populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), new TracingContext(abfsConf.getClientCorrelationID(), "test-fs-id", - AbfsOperations.WRITE, abfsConf.getTracingContextFormat())); + AbfsOperationConstants.WRITE, abfsConf.getTracingContextFormat())); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -346,7 +345,7 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, true), new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperations.OPEN, abfsConf.getTracingContextFormat())); + "test-fs-id", AbfsOperationConstants.OPEN, abfsConf.getTracingContextFormat())); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -390,7 +389,7 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperations.WRITE, abfsConf.getTracingContextFormat()); + "test-fs-id", AbfsOperationConstants.WRITE, abfsConf.getTracingContextFormat()); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), @@ -456,7 +455,7 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperations.WRITE, abfsConf.getTracingContextFormat()); + "test-fs-id", AbfsOperationConstants.WRITE, abfsConf.getTracingContextFormat()); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), From 6409e3fde14ea4167d6d6eeacaba2430d9ce15f1 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Thu, 5 Nov 2020 11:53:28 +0530 Subject: [PATCH 36/77] enum, opnames --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 8 +++-- .../fs/azurebfs/AzureBlobFileSystem.java | 29 +++++++++++-------- .../constants/AbfsOperationConstants.java | 17 +++++++---- .../constants/FileSystemConfigurations.java | 3 ++ .../fs/azurebfs/utils/TracingContext.java | 29 +++++++++---------- .../azurebfs/AbstractAbfsIntegrationTest.java | 12 +++++--- 6 files changed, 60 insertions(+), 38 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 478faa7db1e8b..45fe858b77f33 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -60,6 +60,7 @@ import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy; import org.apache.hadoop.fs.azurebfs.services.KeyProvider; import org.apache.hadoop.fs.azurebfs.services.SimpleKeyProvider; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext.TracingContextFormat; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.apache.hadoop.security.ProviderUtils; import org.apache.hadoop.util.ReflectionUtils; @@ -312,9 +313,12 @@ public String getClientCorrelationID() { * 2 -> client-corr-id : client-req-id * @return tracingContextFormat config */ - public int getTracingContextFormat() { - return tracingContextFormat; + public TracingContextFormat getTracingContextFormat() { + return getEnum(FS_AZURE_TRACINGCONTEXT_FORMAT, DEFAULT_TRACINGCONTEXT_FORMAT); } +// public int getTracingContextFormat() { +// return tracingContextFormat; +// } /** * Appends an account name to a configuration key yielding the diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index b5bedc140b2a8..7dcc27afe9102 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -40,6 +40,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext.TracingContextFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -103,7 +104,7 @@ public class AzureBlobFileSystem extends FileSystem { private AbfsDelegationTokenManager delegationTokenManager; private AbfsCounters abfsCounters; private String clientCorrelationID; - private int tracingContextFormat; + private TracingContextFormat tracingContextFormat; @Override public void initialize(URI uri, Configuration configuration) @@ -676,7 +677,7 @@ public void setOwner(final Path path, final String owner, final String group) LOG.debug( "AzureBlobFileSystem.setOwner path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.SET_OWNER, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.SETOWNER, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { super.setOwner(path, owner, group); @@ -722,7 +723,8 @@ public void setXAttr(final Path path, final String name, final byte[] value, fin try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.ATTR_OPS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.SETATTR, true, + tracingContextFormat); Hashtable properties = abfsStore.getPathStatus(qualifiedPath, tracingContext); String xAttrName = ensureValidAttributeName(name); boolean xAttrExists = properties.containsKey(xAttrName); @@ -760,7 +762,8 @@ public byte[] getXAttr(final Path path, final String name) byte[] value = null; try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.ATTR_OPS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.GETATTR, true, + tracingContextFormat); Hashtable properties = abfsStore.getPathStatus(qualifiedPath, tracingContext); String xAttrName = ensureValidAttributeName(name); if (properties.containsKey(xAttrName)) { @@ -825,7 +828,7 @@ public void modifyAclEntries(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.modifyAclEntries path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.ACL_OPS,true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.MODIFYACL,true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -860,7 +863,8 @@ public void removeAclEntries(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.removeAclEntries path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.ACL_OPS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.REMOVEACLENTRIES, true, + tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -891,7 +895,8 @@ public void removeAclEntries(final Path path, final List aclSpec) public void removeDefaultAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeDefaultAcl path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.ACL_OPS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.REMOVEDEFAULTACL, true, + tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -920,7 +925,7 @@ public void removeDefaultAcl(final Path path) throws IOException { public void removeAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeAcl path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.ACL_OPS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.REMOVEACL, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -952,7 +957,7 @@ public void setAcl(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.setAcl path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.ACL_OPS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.SETACL, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -984,7 +989,7 @@ public void setAcl(final Path path, final List aclSpec) public AclStatus getAclStatus(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.getAclStatus path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.ACL_OPS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.GETACLSTATUS, true, tracingContextFormat); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -1020,7 +1025,7 @@ public void access(final Path path, final FsAction mode) throws IOException { Path qualifiedPath = makeQualified(path); try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.PATH_OPS, tracingContextFormat); + fileSystemID, AbfsOperationConstants.ACCESS, tracingContextFormat); this.abfsStore.access(qualifiedPath, mode, tracingContext); } catch (AzureBlobFileSystemException ex) { checkCheckAccessException(path, ex); @@ -1319,7 +1324,7 @@ public boolean hasPathCapability(final Path path, final String capability) return true; case CommonPathCapabilities.FS_ACLS: TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.PATH_OPS, tracingContextFormat); + fileSystemID, AbfsOperationConstants.PATH, tracingContextFormat); return getIsNamespaceEnabled(tracingContext); default: return super.hasPathCapability(p, capability); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperationConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperationConstants.java index 44f8c119c8212..460706b71ddbc 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperationConstants.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperationConstants.java @@ -1,21 +1,28 @@ package org.apache.hadoop.fs.azurebfs.constants; public final class AbfsOperationConstants { - public static final String ACL_OPS = "AC"; + public static final String ACCESS = "AS"; public static final String APPEND = "AP"; - public static final String ATTR_OPS = "AT"; public static final String CREATE = "CR"; public static final String CREATEFILESYSTEM = "CF"; public static final String DELETE = "DL"; + public static final String GETACLSTATUS = "GS"; + public static final String GETATTR = "GA"; public static final String GETFILESTATUS = "FS"; - public static final String GETNAMESPACE = "NS"; public static final String LISTSTATUS = "LS"; public static final String MKDIR = "MK"; + public static final String MODIFYACL = "MA"; public static final String OPEN = "OP"; + public static final String PATH = "PA"; public static final String PERMISSION = "PR"; public static final String READ = "RD"; + public static final String REMOVEACL = "RA"; + public static final String REMOVEACLENTRIES = "RE"; + public static final String REMOVEDEFAULTACL = "DA"; public static final String RENAME = "RN"; - public static final String SET_OWNER = "SO"; - public static final String TEST_OP = "TS"; + public static final String SETOWNER = "SO"; + public static final String SETATTR = "ST"; + public static final String SETACL = "SA"; + public static final String TESTOP = "TS"; public static final String WRITE = "WR"; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index 2b50a5ddfd40d..d49ee4ef0352b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -20,6 +20,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext.TracingContextFormat; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; @@ -80,6 +81,8 @@ public final class FileSystemConfigurations { public static final DelegatingSSLSocketFactory.SSLChannelMode DEFAULT_FS_AZURE_SSL_CHANNEL_MODE = DelegatingSSLSocketFactory.SSLChannelMode.Default; + public static final TracingContextFormat DEFAULT_TRACINGCONTEXT_FORMAT = + TracingContextFormat.ALL_ID_FORMAT; public static final boolean DEFAULT_ENABLE_DELEGATION_TOKEN = false; public static final boolean DEFAULT_ENABLE_HTTPS = true; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index bf1babbf25c3c..0a99b8fcb712c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -33,24 +33,21 @@ public class TracingContext { private String primaryRequestID; private String streamID; private int retryCount; - private String hadoopOpName; - private final int format; + private final String hadoopOpName; + private final TracingContextFormat format; private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*"; -// public static enum TracingContextFormat { -// SINGLE_ID_FORMAT, -// ALL_IDS, -// TWO_ID_FORMAT; -// -// private TracingContextFormat() { -// } -// } + public static enum TracingContextFormat { + SINGLE_ID_FORMAT, + ALL_ID_FORMAT, + TWO_ID_FORMAT; + } public TracingContext(String clientCorrelationID, String fileSystemID, - String hadoopOpName, int tracingContextFormat) { + String hadoopOpName, TracingContextFormat tracingContextFormat) { this.fileSystemID = fileSystemID; this.hadoopOpName = hadoopOpName; this.clientCorrelationID = validateClientCorrelationID(clientCorrelationID); @@ -62,7 +59,7 @@ public TracingContext(String clientCorrelationID, String fileSystemID, public TracingContext(String clientCorrelationID, String fileSystemID, String hadoopOpName, boolean needsPrimaryReqId, - int tracingContextFormat) { + TracingContextFormat tracingContextFormat) { this(clientCorrelationID, fileSystemID, hadoopOpName, tracingContextFormat); primaryRequestID = needsPrimaryReqId? UUID.randomUUID().toString() : ""; } @@ -106,9 +103,11 @@ public void setRetryCount(int retryCount) { public String toString() { String header = clientRequestID; //case 0, no IDs for correlation switch (format) { - case 1: header = header + ":" + fileSystemID + ":" + primaryRequestID - + ":" + streamID + ":" + hadoopOpName + ":" + retryCount; //all IDs - case 2: header = clientCorrelationID + ":" + header; // 2-ID format + case ALL_ID_FORMAT: header = + clientCorrelationID + ":" + header + ":" + fileSystemID + ":" + primaryRequestID + + ":" + streamID + ":" + hadoopOpName + ":" + retryCount; + break; + case TWO_ID_FORMAT: header = clientCorrelationID + ":" + header; } return header; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index bf39c3555fefe..bc0cab6a05ec7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -25,7 +25,9 @@ import java.util.UUID; import java.util.concurrent.Callable; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext.TracingContextFormat; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -147,15 +149,17 @@ protected boolean getIsNamespaceEnabled(AzureBlobFileSystem fs) throws IOExcepti public TracingContext getTestTracingContext(AzureBlobFileSystem fs, boolean isCont) { if (fs == null) { - return new TracingContext("test-corr-id", - "test-fs-id", "TS", false, 1); + return new TracingContext("test-corr-id", "test-fs-id", "TS", false, + TracingContextFormat.ALL_ID_FORMAT); } String fsID = fs.getFileSystemID(); AbfsConfiguration abfsConf = fs.getAbfsStore().getAbfsConfiguration(); String corrID = abfsConf == null? "test-corr-id" : abfsConf.getClientCorrelationID(); - int format = abfsConf == null? 1 : abfsConf.getTracingContextFormat(); - return new TracingContext(corrID, fsID, "TS", isCont, format); + TracingContextFormat format = abfsConf == null? TracingContextFormat.ALL_ID_FORMAT : + abfsConf.getTracingContextFormat(); + return new TracingContext(corrID, fsID, AbfsOperationConstants.TESTOP, isCont, + format); } From d304123654e1cbe18c6617b317727c7e76a900a5 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Sun, 8 Nov 2020 14:16:42 +0530 Subject: [PATCH 37/77] enum --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 20 +++++++++---------- .../fs/azurebfs/AzureBlobFileSystem.java | 2 +- .../constants/FileSystemConfigurations.java | 7 ++++--- .../fs/azurebfs/utils/TracingContext.java | 6 ------ .../azurebfs/utils/TracingContextFormat.java | 20 +++++++++++++++++++ .../azurebfs/AbstractAbfsIntegrationTest.java | 2 +- 6 files changed, 36 insertions(+), 21 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 45fe858b77f33..c7841a4c500be 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -60,7 +60,7 @@ import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy; import org.apache.hadoop.fs.azurebfs.services.KeyProvider; import org.apache.hadoop.fs.azurebfs.services.SimpleKeyProvider; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext.TracingContextFormat; +import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.apache.hadoop.security.ProviderUtils; import org.apache.hadoop.util.ReflectionUtils; @@ -231,16 +231,16 @@ public class AbfsConfiguration{ DefaultValue = EMPTY_STRING) private String clientCorrelationID; - @IntegerConfigurationValidatorAnnotation(ConfigurationKey = - FS_AZURE_TRACINGCONTEXT_FORMAT, - DefaultValue = 1) - private int tracingContextFormat; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_TRACINGCONTEXT_FORMAT, + DefaultValue = 1)// DEFAULT_TRACINGCONTEXT_FORMATSIZE) + private int inputTracingContextFormat; + + private final TracingContextFormat tracingContextFormat; @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ENABLE_DELEGATION_TOKEN, DefaultValue = DEFAULT_ENABLE_DELEGATION_TOKEN) private boolean enableDelegationToken; - @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ALWAYS_USE_HTTPS, DefaultValue = DEFAULT_ENABLE_HTTPS) private boolean alwaysUseHttps; @@ -284,6 +284,9 @@ public AbfsConfiguration(final Configuration rawConfig, String accountName) field.set(this, validateBoolean(field)); } } + System.out.println(inputTracingContextFormat + "abfsconf"); + tracingContextFormat = + TracingContextFormat.valueOf(inputTracingContextFormat); } public Trilean getIsNamespaceEnabledAccount() { @@ -314,11 +317,8 @@ public String getClientCorrelationID() { * @return tracingContextFormat config */ public TracingContextFormat getTracingContextFormat() { - return getEnum(FS_AZURE_TRACINGCONTEXT_FORMAT, DEFAULT_TRACINGCONTEXT_FORMAT); + return tracingContextFormat; } -// public int getTracingContextFormat() { -// return tracingContextFormat; -// } /** * Appends an account name to a configuration key yielding the diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 7dcc27afe9102..256b722448375 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -40,7 +40,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext.TracingContextFormat; +import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index d49ee4ef0352b..2e5932391b24d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -20,7 +20,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext.TracingContextFormat; +import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; @@ -81,8 +81,9 @@ public final class FileSystemConfigurations { public static final DelegatingSSLSocketFactory.SSLChannelMode DEFAULT_FS_AZURE_SSL_CHANNEL_MODE = DelegatingSSLSocketFactory.SSLChannelMode.Default; - public static final TracingContextFormat DEFAULT_TRACINGCONTEXT_FORMAT = - TracingContextFormat.ALL_ID_FORMAT; + public static final int DEFAULT_TRACINGCONTEXT_FORMAT = + TracingContextFormat.ALL_ID_FORMAT.ordinal(); + //ordinal has associated risks - check public static final boolean DEFAULT_ENABLE_DELEGATION_TOKEN = false; public static final boolean DEFAULT_ENABLE_HTTPS = true; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index 0a99b8fcb712c..c0fa6b38a8af4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -40,12 +40,6 @@ public class TracingContext { public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*"; - public static enum TracingContextFormat { - SINGLE_ID_FORMAT, - ALL_ID_FORMAT, - TWO_ID_FORMAT; - } - public TracingContext(String clientCorrelationID, String fileSystemID, String hadoopOpName, TracingContextFormat tracingContextFormat) { this.fileSystemID = fileSystemID; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java new file mode 100644 index 0000000000000..f897be9ea7500 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java @@ -0,0 +1,20 @@ +package org.apache.hadoop.fs.azurebfs.utils; + +public enum TracingContextFormat { + SINGLE_ID_FORMAT, + ALL_ID_FORMAT, + TWO_ID_FORMAT; + + private static final TracingContextFormat[] formatValues = + TracingContextFormat.values(); + private static final int FORMAT_COUNT = formatValues.length; + +// public static int getFormatCount() { +// return FORMAT_COUNT; +// } + public static TracingContextFormat valueOf(int number) { + return number < FORMAT_COUNT? formatValues[number] : + TracingContextFormat.ALL_ID_FORMAT; //because values() + // is expensive + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index bc0cab6a05ec7..b2ec2abdbcd8c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -27,7 +27,7 @@ import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext.TracingContextFormat; +import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import org.junit.After; import org.junit.Assert; import org.junit.Before; From 1e4f46db77273da9ec37770aa2923eb4c8350df2 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Thu, 12 Nov 2020 10:27:37 +0530 Subject: [PATCH 38/77] format changes --- .../apache/hadoop/fs/azurebfs/AbfsConfiguration.java | 1 - .../apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java | 10 +++++----- .../fs/azurebfs/constants/AbfsOperationConstants.java | 2 +- .../hadoop/fs/azurebfs/utils/TracingContextFormat.java | 7 ++----- 4 files changed, 8 insertions(+), 12 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index c7841a4c500be..a1763b69c0496 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -284,7 +284,6 @@ public AbfsConfiguration(final Configuration rawConfig, String accountName) field.set(this, validateBoolean(field)); } } - System.out.println(inputTracingContextFormat + "abfsconf"); tracingContextFormat = TracingContextFormat.valueOf(inputTracingContextFormat); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 256b722448375..61ad55de4e67e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -178,11 +178,6 @@ public URI getUri() { return this.uri; } - @VisibleForTesting - String getFileSystemID() { - return fileSystemID; - } - @Override public FSDataInputStream open(final Path path, final int bufferSize) throws IOException { LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize); @@ -1313,6 +1308,11 @@ Map getInstrumentationMap() { return abfsCounters.toMap(); } + @VisibleForTesting + String getFileSystemID() { + return fileSystemID; + } + @Override public boolean hasPathCapability(final Path path, final String capability) throws IOException { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperationConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperationConstants.java index 460706b71ddbc..4587f287d1e84 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperationConstants.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperationConstants.java @@ -20,8 +20,8 @@ public final class AbfsOperationConstants { public static final String REMOVEACLENTRIES = "RE"; public static final String REMOVEDEFAULTACL = "DA"; public static final String RENAME = "RN"; - public static final String SETOWNER = "SO"; public static final String SETATTR = "ST"; + public static final String SETOWNER = "SO"; public static final String SETACL = "SA"; public static final String TESTOP = "TS"; public static final String WRITE = "WR"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java index f897be9ea7500..22c91bdf86aa7 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java @@ -5,16 +5,13 @@ public enum TracingContextFormat { ALL_ID_FORMAT, TWO_ID_FORMAT; + //save list as calling values() is expensive private static final TracingContextFormat[] formatValues = TracingContextFormat.values(); private static final int FORMAT_COUNT = formatValues.length; -// public static int getFormatCount() { -// return FORMAT_COUNT; -// } public static TracingContextFormat valueOf(int number) { return number < FORMAT_COUNT? formatValues[number] : - TracingContextFormat.ALL_ID_FORMAT; //because values() - // is expensive + TracingContextFormat.ALL_ID_FORMAT; } } From 15a22cd4a3f9e57ec35b31d5b786ee19a1213ce8 Mon Sep 17 00:00:00 2001 From: sumangala-patki <70206833+sumangala-patki@users.noreply.github.com> Date: Thu, 12 Nov 2020 13:05:27 +0530 Subject: [PATCH 39/77] test code (#3) * adding callback structure * testListPath correlation header * validate IDs; readahead/streamid * add common tests + other changes * remove stream/extra stuff * handle parallel requests * clear * testTC, retryNum * rebase on HADOOP-17290 --- .../fs/azurebfs/AzureBlobFileSystem.java | 51 ++++---- .../azurebfs/services/AbfsHttpOperation.java | 5 + .../fs/azurebfs/services/AbfsInputStream.java | 13 ++ .../azurebfs/services/AbfsOutputStream.java | 14 +- .../azurebfs/services/AbfsRestOperation.java | 9 +- .../hadoop/fs/azurebfs/utils/Listener.java | 7 + .../fs/azurebfs/utils/TracingContext.java | 35 ++++- .../azurebfs/AbstractAbfsIntegrationTest.java | 14 +- .../azurebfs/ITestAbfsReadWriteAndSeek.java | 8 ++ .../ITestAzureBlobFileSystemCreate.java | 10 ++ .../ITestAzureBlobFileSystemListStatus.java | 16 ++- .../fs/azurebfs/TestTracingContext.java | 121 ++++++++++++++++++ .../services/TestAbfsOutputStream.java | 23 +++- .../utils/TracingHeaderValidator.java | 97 ++++++++++++++ 14 files changed, 377 insertions(+), 46 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 61ad55de4e67e..187cefca19371 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -41,6 +41,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; +import org.apache.hadoop.fs.azurebfs.utils.Listener; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -105,6 +106,7 @@ public class AzureBlobFileSystem extends FileSystem { private AbfsCounters abfsCounters; private String clientCorrelationID; private TracingContextFormat tracingContextFormat; + private Listener listener; @Override public void initialize(URI uri, Configuration configuration) @@ -177,6 +179,10 @@ public boolean isSecureScheme() { public URI getUri() { return this.uri; } + + public void registerListener(Listener listener1) { + listener = listener1; + } @Override public FSDataInputStream open(final Path path, final int bufferSize) throws IOException { @@ -186,7 +192,8 @@ public FSDataInputStream open(final Path path, final int bufferSize) throws IOEx try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.OPEN, tracingContextFormat); + fileSystemID, AbfsOperationConstants.OPEN, tracingContextFormat, + listener); InputStream inputStream = abfsStore.openFileForRead(qualifiedPath, statistics, tracingContext); return new FSDataInputStream(inputStream); } catch(AzureBlobFileSystemException ex) { @@ -212,7 +219,7 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi try { TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, AbfsOperationConstants.CREATE, overwrite, - tracingContextFormat); + tracingContextFormat, listener); OutputStream outputStream = abfsStore.createFile(qualifiedPath, statistics, overwrite, permission == null ? FsPermission.getFileDefault() : permission, FsPermission.getUMask(getConf()), tracingContext); @@ -279,7 +286,7 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.APPEND, tracingContextFormat); + fileSystemID, AbfsOperationConstants.APPEND, tracingContextFormat, listener); OutputStream outputStream = abfsStore.openFileForWrite(qualifiedPath, statistics, false, tracingContext); return new FSDataOutputStream(outputStream, statistics); @@ -322,7 +329,7 @@ public boolean rename(final Path src, final Path dst) throws IOException { // Non-HNS account need to check dst status on driver side. TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.RENAME, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.RENAME, true, tracingContextFormat, listener); if (!abfsStore.getIsNamespaceEnabled(tracingContext) && dstFileStatus == null) { dstFileStatus = tryGetFileStatus(qualifiedDstPath); } @@ -375,7 +382,7 @@ public boolean delete(final Path f, final boolean recursive) throws IOException try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.DELETE, tracingContextFormat); + fileSystemID, AbfsOperationConstants.DELETE, tracingContextFormat, listener); abfsStore.delete(qualifiedPath, recursive, tracingContext); return true; } catch (AzureBlobFileSystemException ex) { @@ -394,7 +401,7 @@ public FileStatus[] listStatus(final Path f) throws IOException { try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.LISTSTATUS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.LISTSTATUS, true, tracingContextFormat, listener); FileStatus[] result = abfsStore.listStatus(qualifiedPath, tracingContext); return result; } catch (AzureBlobFileSystemException ex) { @@ -464,7 +471,7 @@ public boolean mkdirs(final Path f, final FsPermission permission) throws IOExce try { TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, AbfsOperationConstants.MKDIR, false, - tracingContextFormat); + tracingContextFormat, listener); abfsStore.createDirectory(qualifiedPath, permission == null ? FsPermission.getDirDefault() : permission, FsPermission.getUMask(getConf()), tracingContext); statIncrement(DIRECTORIES_CREATED); @@ -496,7 +503,7 @@ public FileStatus getFileStatus(final Path f) throws IOException { try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.GETFILESTATUS, tracingContextFormat); + fileSystemID, AbfsOperationConstants.GETFILESTATUS, tracingContextFormat, listener); return abfsStore.getFileStatus(qualifiedPath, tracingContext); } catch(AzureBlobFileSystemException ex) { checkException(f, ex); @@ -672,7 +679,7 @@ public void setOwner(final Path path, final String owner, final String group) LOG.debug( "AzureBlobFileSystem.setOwner path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.SETOWNER, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.SETOWNER, true, tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { super.setOwner(path, owner, group); @@ -719,7 +726,7 @@ public void setXAttr(final Path path, final String name, final byte[] value, fin try { TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, AbfsOperationConstants.SETATTR, true, - tracingContextFormat); + tracingContextFormat, listener); Hashtable properties = abfsStore.getPathStatus(qualifiedPath, tracingContext); String xAttrName = ensureValidAttributeName(name); boolean xAttrExists = properties.containsKey(xAttrName); @@ -758,7 +765,7 @@ public byte[] getXAttr(final Path path, final String name) try { TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, AbfsOperationConstants.GETATTR, true, - tracingContextFormat); + tracingContextFormat, listener); Hashtable properties = abfsStore.getPathStatus(qualifiedPath, tracingContext); String xAttrName = ensureValidAttributeName(name); if (properties.containsKey(xAttrName)) { @@ -787,7 +794,7 @@ public void setPermission(final Path path, final FsPermission permission) throws IOException { LOG.debug("AzureBlobFileSystem.setPermission path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.PERMISSION, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.PERMISSION, true, tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { super.setPermission(path, permission); @@ -823,7 +830,7 @@ public void modifyAclEntries(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.modifyAclEntries path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.MODIFYACL,true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.MODIFYACL,true, tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -859,7 +866,7 @@ public void removeAclEntries(final Path path, final List aclSpec) LOG.debug("AzureBlobFileSystem.removeAclEntries path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, AbfsOperationConstants.REMOVEACLENTRIES, true, - tracingContextFormat); + tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -891,7 +898,7 @@ public void removeDefaultAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeDefaultAcl path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, AbfsOperationConstants.REMOVEDEFAULTACL, true, - tracingContextFormat); + tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -920,7 +927,7 @@ public void removeDefaultAcl(final Path path) throws IOException { public void removeAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeAcl path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.REMOVEACL, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.REMOVEACL, true, tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -952,7 +959,7 @@ public void setAcl(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.setAcl path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.SETACL, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.SETACL, true, tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -984,7 +991,7 @@ public void setAcl(final Path path, final List aclSpec) public AclStatus getAclStatus(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.getAclStatus path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.GETACLSTATUS, true, tracingContextFormat); + fileSystemID, AbfsOperationConstants.GETACLSTATUS, true, tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -1020,7 +1027,7 @@ public void access(final Path path, final FsAction mode) throws IOException { Path qualifiedPath = makeQualified(path); try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.ACCESS, tracingContextFormat); + fileSystemID, AbfsOperationConstants.ACCESS, tracingContextFormat, listener); this.abfsStore.access(qualifiedPath, mode, tracingContext); } catch (AzureBlobFileSystemException ex) { checkCheckAccessException(path, ex); @@ -1055,7 +1062,7 @@ private boolean fileSystemExists() throws IOException { "AzureBlobFileSystem.fileSystemExists uri: {}", uri); try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.GETFILESTATUS, tracingContextFormat); + fileSystemID, AbfsOperationConstants.GETFILESTATUS, tracingContextFormat, listener); abfsStore.getFilesystemProperties(tracingContext); } catch (AzureBlobFileSystemException ex) { try { @@ -1076,7 +1083,7 @@ private void createFileSystem() throws IOException { "AzureBlobFileSystem.createFileSystem uri: {}", uri); try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.CREATEFILESYSTEM, tracingContextFormat); + fileSystemID, AbfsOperationConstants.CREATEFILESYSTEM, tracingContextFormat, listener); abfsStore.createFilesystem(tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(null, ex); @@ -1324,7 +1331,7 @@ public boolean hasPathCapability(final Path path, final String capability) return true; case CommonPathCapabilities.FS_ACLS: TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.PATH, tracingContextFormat); + fileSystemID, AbfsOperationConstants.PATH, tracingContextFormat, listener); return getIsNamespaceEnabled(tracingContext); default: return super.hasPathCapability(p, capability); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index 1bfbe5a4b58f4..0f69139a1cef0 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -151,6 +151,11 @@ public String getResponseHeader(String httpHeader) { return connection.getHeaderField(httpHeader); } + @com.google.common.annotations.VisibleForTesting + public String getRequestHeader(String httpHeader) { + return connection.getRequestProperties().get(httpHeader).toString(); + } + // Returns a trace message for the request @Override public String toString() { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 6f137a119c715..91f7089c45008 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -28,6 +28,8 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.utils.Listener; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -77,6 +79,7 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer, private final AbfsInputStreamStatistics streamStatistics; private long bytesFromReadAhead; // bytes read from readAhead; for testing private long bytesFromRemoteRead; // bytes read remotely; for testing + private Listener listener; public AbfsInputStream( final AbfsClient client, @@ -100,6 +103,7 @@ public AbfsInputStream( this.streamStatistics = abfsInputStreamContext.getStreamStatistics(); this.inputStreamID = getInputStreamID(); this.tracingContext = new TracingContext(tracingContext); + this.tracingContext.setOperation(AbfsOperationConstants.READ); this.tracingContext.setStreamID(inputStreamID); } @@ -111,6 +115,15 @@ private String getInputStreamID() { return StringUtils.right(UUID.randomUUID().toString(), 12); } + public void registerListener(Listener listener1) { + listener = listener1; + tracingContext.setListener(listener); + } + + public String getStreamID() { + return inputStreamID; + } + @Override public int read() throws IOException { byte[] b = new byte[1]; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index d8230e242a8ed..46d624d0f18cf 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -36,6 +36,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import org.apache.hadoop.fs.azurebfs.utils.Listener; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -82,7 +83,8 @@ public class AbfsOutputStream extends OutputStream implements Syncable, StreamCa // SAS tokens can be re-used until they expire private CachedSASToken cachedSasToken; private final String outputStreamID; - private TracingContext tracingContext; + private final TracingContext tracingContext; + private Listener listener; /** * Queue storing buffers with the size of the Azure block ready for @@ -280,6 +282,15 @@ public void hflush() throws IOException { flushInternal(false); } } + + public String getStreamID() { + return outputStreamID; + } + + public void registerListener(Listener listener1) { + listener = listener1; + tracingContext.setListener(listener); + } /** * Force all data in the output stream to be written to Azure storage. @@ -464,6 +475,7 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset, AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "flushWrittenBytesToServiceInternal", "flush")) { + AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, isClose, cachedSasToken.get(), new TracingContext(tracingContext)); cachedSasToken.update(op.getSasToken()); 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 9ac754cd806bc..268e97bf473eb 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 @@ -122,7 +122,8 @@ String getSasToken() { * @param requestHeaders The HTTP request headers. * @param sasToken A sasToken for optional re-use by AbfsInputStream/AbfsOutputStream. */ - AbfsRestOperation(final AbfsRestOperationType operationType, + @VisibleForTesting + protected AbfsRestOperation(final AbfsRestOperationType operationType, final AbfsClient client, final String method, final URL url, @@ -205,7 +206,7 @@ public void execute(TracingContext tracingContext) throws AzureBlobFileSystemExc LOG.trace("{} REST operation complete", operationType); } - private void updateClientRequestHeader(AbfsHttpOperation httpOperation, + protected void updateClientRequestHeader(AbfsHttpOperation httpOperation, TracingContext tracingContext) { tracingContext.generateClientRequestID(); httpOperation.getConnection() @@ -218,8 +219,8 @@ private void updateClientRequestHeader(AbfsHttpOperation httpOperation, * fails, there may be a retry. The retryCount is incremented with each * attempt. */ - private boolean executeHttpOperation(final int retryCount, TracingContext tracingContext) - throws AzureBlobFileSystemException { + protected boolean executeHttpOperation(final int retryCount, + TracingContext tracingContext) throws AzureBlobFileSystemException { AbfsHttpOperation httpOperation = null; try { // initialize the HTTP request and open the connection diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java new file mode 100644 index 0000000000000..8d01a2e8c3bdb --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java @@ -0,0 +1,7 @@ +package org.apache.hadoop.fs.azurebfs.utils; + +public interface Listener { + void afterOp(String header); + void updatePrimaryRequestID(String primaryRequestID); + Listener getClone(); +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index c0fa6b38a8af4..a4593f6c124a9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.azurebfs.utils; +import java.io.IOException; import java.util.UUID; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; @@ -33,15 +34,16 @@ public class TracingContext { private String primaryRequestID; private String streamID; private int retryCount; - private final String hadoopOpName; + private String hadoopOpName; private final TracingContextFormat format; + private Listener listener = null; private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*"; public TracingContext(String clientCorrelationID, String fileSystemID, - String hadoopOpName, TracingContextFormat tracingContextFormat) { + String hadoopOpName, TracingContextFormat tracingContextFormat, Listener listener) { this.fileSystemID = fileSystemID; this.hadoopOpName = hadoopOpName; this.clientCorrelationID = validateClientCorrelationID(clientCorrelationID); @@ -49,13 +51,17 @@ public TracingContext(String clientCorrelationID, String fileSystemID, retryCount = 0; primaryRequestID = ""; format = tracingContextFormat; + this.listener = listener; } public TracingContext(String clientCorrelationID, String fileSystemID, String hadoopOpName, boolean needsPrimaryReqId, - TracingContextFormat tracingContextFormat) { - this(clientCorrelationID, fileSystemID, hadoopOpName, tracingContextFormat); + TracingContextFormat tracingContextFormat, Listener listener) { + this(clientCorrelationID, fileSystemID, hadoopOpName, + tracingContextFormat, listener); primaryRequestID = needsPrimaryReqId? UUID.randomUUID().toString() : ""; + if (listener != null) + listener.updatePrimaryRequestID(primaryRequestID); } public TracingContext(TracingContext originalTracingContext) { @@ -66,6 +72,9 @@ public TracingContext(TracingContext originalTracingContext) { this.retryCount = 0; this.primaryRequestID = originalTracingContext.primaryRequestID; this.format = originalTracingContext.format; + if(originalTracingContext.listener != null) { + this.listener = originalTracingContext.listener.getClone(); + } } public String validateClientCorrelationID(String clientCorrelationID) { @@ -84,16 +93,26 @@ public void generateClientRequestID() { public void setPrimaryRequestID() { primaryRequestID = UUID.randomUUID().toString(); + if(listener != null) + listener.updatePrimaryRequestID(primaryRequestID); } public void setStreamID(String stream) { streamID = stream; } + public void setOperation(String operation) { + this.hadoopOpName = operation; + } + public void setRetryCount(int retryCount) { this.retryCount = retryCount; } + public void setListener(Listener listener) { + this.listener = listener; + } + public String toString() { String header = clientRequestID; //case 0, no IDs for correlation switch (format) { @@ -103,6 +122,14 @@ public String toString() { break; case TWO_ID_FORMAT: header = clientCorrelationID + ":" + header; } + if (listener != null) { + listener.afterOp(header); + } return header; } + +// public void validateTracingHeader() { +// if (listener != null) +// listener.afterOp(toString()); +// } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index b2ec2abdbcd8c..1dc0f60fbe3a2 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -25,7 +25,6 @@ import java.util.UUID; import java.util.concurrent.Callable; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import org.junit.After; @@ -143,14 +142,15 @@ protected AbstractAbfsIntegrationTest() throws Exception { } protected boolean getIsNamespaceEnabled(AzureBlobFileSystem fs) throws IOException { - return fs.getAbfsStore().getIsNamespaceEnabled( - getTestTracingContext(fs, false)); + return fs.getAbfsStore().getIsNamespaceEnabled(getTestTracingContext(fs, + false)); } - public TracingContext getTestTracingContext(AzureBlobFileSystem fs, boolean isCont) { + public TracingContext getTestTracingContext(AzureBlobFileSystem fs, + boolean isCont) { if (fs == null) { return new TracingContext("test-corr-id", "test-fs-id", "TS", false, - TracingContextFormat.ALL_ID_FORMAT); + TracingContextFormat.ALL_ID_FORMAT, null); } String fsID = fs.getFileSystemID(); AbfsConfiguration abfsConf = fs.getAbfsStore().getAbfsConfiguration(); @@ -158,8 +158,8 @@ public TracingContext getTestTracingContext(AzureBlobFileSystem fs, boolean isCo abfsConf.getClientCorrelationID(); TracingContextFormat format = abfsConf == null? TracingContextFormat.ALL_ID_FORMAT : abfsConf.getTracingContextFormat(); - return new TracingContext(corrID, fsID, AbfsOperationConstants.TESTOP, isCont, - format); + + return new TracingContext(corrID, fsID, "TS", isCont, format, null); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index 2e26b7680f88a..b45f1c733280a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -21,6 +21,9 @@ import java.util.Arrays; import java.util.Random; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -98,6 +101,11 @@ private void testReadWriteAndSeek(int bufferSize) throws Exception { final byte[] readBuffer = new byte[2 * bufferSize]; int result; try (FSDataInputStream inputStream = fs.open(TEST_PATH)) { + ((AbfsInputStream) inputStream.getWrappedStream()).registerListener(new + TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), + fs.getFileSystemID(), AbfsOperationConstants.READ, true, + 0, + ((AbfsInputStream) inputStream.getWrappedStream()).getStreamID())); inputStream.seek(bufferSize); result = inputStream.read(readBuffer, bufferSize, bufferSize); assertNotEquals(-1, result); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 976f3c46491e8..8624837f661a0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -25,7 +25,9 @@ import java.util.EnumSet; import java.util.UUID; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; import org.apache.hadoop.conf.Configuration; @@ -242,6 +244,7 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(currentFs.getUri(), config); + AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); long totalConnectionMadeBeforeTest = fs.getInstrumentationMap() .get(CONNECTIONS_MADE.getStatName()); @@ -263,8 +266,11 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) fs.getInstrumentationMap()); // Case 2: Not Overwrite - File pre-exists + fs.registerListener(new TracingHeaderValidator(conf.getClientCorrelationID(), + fs.getFileSystemID(), AbfsOperationConstants.CREATE, false, 0)); intercept(FileAlreadyExistsException.class, () -> fs.create(nonOverwriteFile, false)); + fs.registerListener(null); // One request to server to create path should be issued createRequestCount++; @@ -290,7 +296,11 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) fs.getInstrumentationMap()); // Case 4: Overwrite - File pre-exists + fs.registerListener(new TracingHeaderValidator(conf.getClientCorrelationID(), + fs.getFileSystemID(), AbfsOperationConstants.CREATE, true, + 0)); fs.create(overwriteFilePath, true); + fs.registerListener(null); if (enableConditionalCreateOverwrite) { // Three requests will be sent to server to create path, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java index 25a15679263b3..cad6e4be58441 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java @@ -27,6 +27,10 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; import org.apache.hadoop.fs.FSDataOutputStream; @@ -35,6 +39,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_CORRELATIONID; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists; @@ -77,7 +82,16 @@ public Void call() throws Exception { } es.shutdownNow(); - FileStatus[] files = fs.listStatus(new Path("/")); + Configuration config = new Configuration(this.getRawConfiguration()); + config.set(FS_AZURE_CLIENT_CORRELATIONID, "validconfig"); + AzureBlobFileSystem fs1 = + (AzureBlobFileSystem) FileSystem.newInstance(config); + AbfsConfiguration conf = fs1.getAbfsStore().getAbfsConfiguration(); + fs1.registerListener(new TracingHeaderValidator( + conf.getClientCorrelationID(), + fs1.getFileSystemID(), AbfsOperationConstants.LISTSTATUS, true, + 0)); + FileStatus[] files = fs1.listStatus(new Path("/")); assertEquals(TEST_FILES_NUMBER, files.length /* user directory */); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java new file mode 100644 index 0000000000000..2f552638f8c92 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -0,0 +1,121 @@ +package org.apache.hadoop.fs.azurebfs; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.enums.Trilean; +import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; +import org.apache.hadoop.fs.permission.FsPermission; +import org.assertj.core.api.Assertions; +import org.junit.Ignore; +import org.junit.Test; + +import java.io.IOException; +import java.lang.reflect.Method; +import java.util.HashMap; +import java.util.Map; + +public class TestTracingContext extends AbstractAbfsIntegrationTest { + private static final String[] CLIENT_CORRELATIONID_LIST = { + "valid-corr-id-123", "inval!d", ""}; + private static final int HTTP_CREATED = 201; + private final String EMPTY_STRING = ""; + String GUID_PATTERN = "[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}"; + String prevClientRequestID = ""; + + protected TestTracingContext() throws Exception { + super(); + } + + @Test + public void testClientCorrelationID() throws IOException { + checkCorrelationConfigValidation(CLIENT_CORRELATIONID_LIST[0], true); + checkCorrelationConfigValidation(CLIENT_CORRELATIONID_LIST[1], false); + checkCorrelationConfigValidation(CLIENT_CORRELATIONID_LIST[2], false); + } + + private String getOctalNotation(FsPermission fsPermission) { + Preconditions.checkNotNull(fsPermission, "fsPermission"); + return String.format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal()); + } + + public void checkCorrelationConfigValidation(String clientCorrelationId, + boolean includeInHeader) throws IOException { + AzureBlobFileSystem fs = getFileSystem(); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fs.getFileSystemID(), AbfsOperationConstants.TESTOP, + TracingContextFormat.ALL_ID_FORMAT,null); + String correlationID = tracingContext.toString().split(":")[0]; + if (includeInHeader) { + Assertions.assertThat(correlationID) + .describedAs("Correlation ID should match config when valid") + .isEqualTo(clientCorrelationId); + } else { + Assertions.assertThat(correlationID) + .describedAs("Invalid ID should be replaced with empty string") + .isEqualTo(EMPTY_STRING); + } + + //request should not fail for invalid clientCorrelationID + fs.getAbfsStore().setNamespaceEnabled(Trilean.getTrilean(true)); + AbfsRestOperation op = fs.getAbfsStore().getClient().createPath("/testDir", + false, true, getOctalNotation(FsPermission.getDefault()), + getOctalNotation(FsPermission.getUMask(getRawConfiguration())), + false, null, tracingContext); + + int statusCode = op.getResult().getStatusCode(); + Assertions.assertThat(statusCode).describedAs("Request should not fail") + .isEqualTo(HTTP_CREATED); + + String requestHeader = op.getResult().getRequestHeader( + HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID) + .replace("[", "").replace("]", ""); + Assertions.assertThat(requestHeader) + .describedAs("Client Request Header should match TracingContext") + .isEqualTo(tracingContext.toString()); + + // use fn below or pass listener to run all TracingHeaderValidator checks + checkRequiredIDs(requestHeader); + } + + private void checkRequiredIDs(String requestHeader) { + String[] id_list = requestHeader.split(":"); + + Assertions.assertThat(id_list[1]) + .describedAs("client-req-id should be a guid") + .matches(GUID_PATTERN); + Assertions.assertThat(id_list[2]) + .describedAs("filesystem-id should not be empty") + .isNotEmpty(); + Assertions.assertThat(id_list[1]) + .describedAs("client-request-id should be unique") + .isNotEqualTo(prevClientRequestID); + } + + @Ignore + @Test + //call test methods from the respective test classes + //can be ignored when running all tests as these get covered + public void runCorrelationTestForAllMethods() throws Exception { + //map to avoid creating new instance and calling setup() for each test + Map testClasses = new HashMap<>(); + + testClasses.put(new ITestAzureBlobFileSystemListStatus(), + ITestAzureBlobFileSystemListStatus.class.getMethod("testListPath")); + testClasses.put(new ITestAzureBlobFileSystemCreate(), + ITestAzureBlobFileSystemCreate.class.getMethod( + "testDefaultCreateOverwriteFileTest")); + //add other ops' testClasses and testMethods that have listener registered + + for (AbstractAbfsIntegrationTest testClass : testClasses.keySet()) { + testClass.setup(); + testClasses.get(testClass).invoke(testClass); + testClass.teardown(); + } + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index 05384e808d6b3..2f530971ecceb 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -24,6 +24,7 @@ import java.util.Random; import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Test; @@ -84,7 +85,8 @@ public void verifyShortWriteRequest() throws Exception { conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperationConstants.WRITE, abfsConf.getTracingContextFormat()); + "test-fs-id", AbfsOperationConstants.WRITE, + abfsConf.getTracingContextFormat(), null); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), @@ -142,7 +144,8 @@ public void verifyWriteRequest() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperationConstants.WRITE, abfsConf.getTracingContextFormat()); + "test-fs-id", AbfsOperationConstants.WRITE, + abfsConf.getTracingContextFormat(), null); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), @@ -210,7 +213,8 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperationConstants.WRITE, abfsConf.getTracingContextFormat()); + "test-fs-id", AbfsOperationConstants.WRITE, + abfsConf.getTracingContextFormat(), null); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), @@ -292,7 +296,9 @@ public void verifyWriteRequestOfBufferSize() throws Exception { populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), new TracingContext(abfsConf.getClientCorrelationID(), "test-fs-id", - AbfsOperationConstants.WRITE, abfsConf.getTracingContextFormat())); + AbfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), + null)); + final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -345,7 +351,8 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, true), new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperationConstants.OPEN, abfsConf.getTracingContextFormat())); + "test-fs-id", AbfsOperationConstants.OPEN, + abfsConf.getTracingContextFormat(), null)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -389,7 +396,8 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperationConstants.WRITE, abfsConf.getTracingContextFormat()); + "test-fs-id", AbfsOperationConstants.WRITE, + abfsConf.getTracingContextFormat(), null); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), @@ -455,7 +463,8 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperationConstants.WRITE, abfsConf.getTracingContextFormat()); + "test-fs-id", AbfsOperationConstants.WRITE, + abfsConf.getTracingContextFormat(), null); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java new file mode 100644 index 0000000000000..f154b4b98c5be --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java @@ -0,0 +1,97 @@ +package org.apache.hadoop.fs.azurebfs.utils; + +import org.assertj.core.api.Assertions; + +public class TracingHeaderValidator implements Listener { + String clientCorrelationID; + String fileSystemID; + String primaryRequestID = ""; + boolean needsPrimaryRequestID; + String streamID = ""; + String operation; + int retryNum; + String GUID_PATTERN = "[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}"; + // client-req-id as per docs: ^[{(]?[0-9a-f]{8}[-]?([0-9a-f]{4}[-]?) + // {3}[0-9a-f]{12}[)}]?$ + + @Override + public void afterOp(String tracingContextHeader) { + validateTracingHeader(tracingContextHeader); + } + + @Override + public void updatePrimaryRequestID(String primaryRequestID) { + this.primaryRequestID = primaryRequestID; + } + + @Override + public TracingHeaderValidator getClone() { + TracingHeaderValidator tracingHeaderValidator = + new TracingHeaderValidator(clientCorrelationID, fileSystemID, + operation, needsPrimaryRequestID, retryNum, streamID); + tracingHeaderValidator.primaryRequestID = primaryRequestID; + return tracingHeaderValidator; + } + + public TracingHeaderValidator(String clientCorrelationID, + String fileSystemID, String operation, + boolean needsPrimaryRequestID, int retryNum) { + this.clientCorrelationID = clientCorrelationID; + this.fileSystemID = fileSystemID; + this.operation = operation; + this.retryNum = retryNum; + this.needsPrimaryRequestID = needsPrimaryRequestID; + } + + public TracingHeaderValidator(String clientCorrelationID, + String fileSystemID, String operation, boolean needsPrimaryRequestID, + int retryNum, String streamID) { + this(clientCorrelationID, fileSystemID, operation, needsPrimaryRequestID, + retryNum); + this.streamID = streamID; + } + + private void validateTracingHeader(String tracingContextHeader) { + String[] id_list = tracingContextHeader.split(":"); + validateBasicFormat(id_list); + if (needsPrimaryRequestID) { + Assertions.assertThat(id_list[3]) + .describedAs("PrimaryReqID should be common for these requests") + .isEqualTo(primaryRequestID); + Assertions.assertThat(id_list[2]).describedAs( + "FilesystemID should be same for requests with same filesystem") + .isEqualTo(fileSystemID); + } + if (!streamID.isEmpty()) { + System.out.println("check stream" + streamID); + Assertions.assertThat(id_list[4]) + .describedAs("Stream id should be common for these requests") + .isEqualTo(streamID); + } + } + + private void validateBasicFormat(String[] id_list) { + Assertions.assertThat(id_list) + .describedAs("header should have 7 elements").hasSize(7); + + if(clientCorrelationID.matches("[a-zA-Z0-9-]*")) { + Assertions.assertThat(id_list[0]).describedAs("Correlation ID should match config") + .isEqualTo(clientCorrelationID); + } else { + Assertions.assertThat(id_list[0]) + .describedAs("Invalid config should be replaced with empty string") + .isEmpty(); + } + Assertions.assertThat(id_list[1]) + .describedAs("Client request ID is a guid") + .matches(GUID_PATTERN); + Assertions.assertThat(id_list[2]).describedAs("Filesystem ID incorrect") + .isEqualTo(fileSystemID); + Assertions.assertThat(id_list[5]).describedAs("Operation name incorrect") + .isEqualTo(operation); + int retryCount = Integer.parseInt(id_list[6]); + Assertions.assertThat(retryCount) + .describedAs("Retry count incorrect") + .isEqualTo(retryNum); + } +} From e44c64c720485a058d8acd60844848980396ac97 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Thu, 12 Nov 2020 17:45:40 +0530 Subject: [PATCH 40/77] other tests (main) --- .../fs/azurebfs/utils/TracingContext.java | 1 - .../azurebfs/ITestAbfsNetworkStatistics.java | 7 +++++ .../fs/azurebfs/ITestAbfsStatistics.java | 7 +++++ .../ITestAzureBlobFileSystemCreate.java | 6 +++++ .../ITestAzureBlobFileSystemDelete.java | 7 +++++ .../azurebfs/ITestAzureBlobFilesystemAcl.java | 27 +++++++++++++++++++ .../fs/azurebfs/TestTracingContext.java | 17 ++++++++++++ .../utils/TracingHeaderValidator.java | 15 +++++++++++ 8 files changed, 86 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index a4593f6c124a9..3d7824fa0b888 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -18,7 +18,6 @@ package org.apache.hadoop.fs.azurebfs.utils; -import java.io.IOException; import java.util.UUID; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java index c2dbe937b812b..93a1e14d0153b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java @@ -21,6 +21,8 @@ import java.io.IOException; import java.util.Map; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.junit.Test; @@ -198,7 +200,12 @@ public void testAbfsHttpResponseStatistics() throws IOException { * (Writing data in Data store). * */ + AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); + fs.registerListener(new TracingHeaderValidator(conf.getClientCorrelationID(), + fs.getFileSystemID(), AbfsOperationConstants.CREATE, + true, 0)); out = fs.create(getResponsePath); + fs.registerListener(null); out.write(testResponseString.getBytes()); out.hflush(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java index 42205807c1b3e..5826cbeb07f63 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java @@ -21,6 +21,8 @@ import java.io.IOException; import java.util.Map; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; import org.apache.hadoop.fs.Path; @@ -176,12 +178,17 @@ public void testOpenAppendRenameExists() throws IOException { + "exists methods on Abfs"); AzureBlobFileSystem fs = getFileSystem(); + AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); Path createFilePath = path(getMethodName()); Path destCreateFilePath = path(getMethodName() + "New"); fs.create(createFilePath); fs.open(createFilePath); + fs.registerListener(new TracingHeaderValidator(conf.getClientCorrelationID(), + fs.getFileSystemID(), AbfsOperationConstants.APPEND, + false, 0)); fs.append(createFilePath); + fs.registerListener(null); assertTrue(fs.rename(createFilePath, destCreateFilePath)); Map metricMap = fs.getInstrumentationMap(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 8624837f661a0..504a052749644 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -92,13 +92,19 @@ public void testEnsureFileCreatedImmediately() throws Exception { @SuppressWarnings("deprecation") public void testCreateNonRecursive() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); + AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE); try { fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null); fail("Should've thrown"); } catch (FileNotFoundException expected) { } + fs.registerListener(new TracingHeaderValidator(conf.getClientCorrelationID(), + fs.getFileSystemID(), AbfsOperationConstants.MKDIR, + false, 0)); fs.mkdirs(TEST_FOLDER_PATH); + fs.registerListener(null); + fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null) .close(); assertIsFile(fs, testFile); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index fc804d42c746a..50788ec5f4c67 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -26,7 +26,9 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.Test; @@ -71,6 +73,7 @@ public ITestAzureBlobFileSystemDelete() throws Exception { @Test public void testDeleteRoot() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); + AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); fs.mkdirs(new Path("/testFolder0")); fs.mkdirs(new Path("/testFolder1")); @@ -145,9 +148,13 @@ public Void call() throws Exception { es.shutdownNow(); Path dir = new Path("/test"); + fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() + .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), + AbfsOperationConstants.DELETE, false, 0)); // first try a non-recursive delete, expect failure intercept(FileAlreadyExistsException.class, () -> fs.delete(dir, false)); + fs.registerListener(null); assertDeleted(fs, dir, true); assertPathDoesNotExist(fs, "deleted", dir); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java index 78426f0331dd6..05ea2f1357532 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java @@ -24,6 +24,8 @@ import java.util.List; import java.util.UUID; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Assume; import org.junit.Ignore; import org.junit.Test; @@ -1205,20 +1207,38 @@ public void testDefaultAclNewDirWithMode() throws Exception { @Test public void testDefaultAclRenamedFile() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); + TracingHeaderValidator tracingHeaderValidator = + new TracingHeaderValidator(conf.getClientCorrelationID(), + fs.getFileSystemID(), AbfsOperationConstants.TESTOP, + true, 0); assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); Path dirPath = new Path(path, "dir"); FileSystem.mkdirs(fs, dirPath, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, FOO, ALL)); + fs.registerListener(tracingHeaderValidator.getClone(AbfsOperationConstants.SETACL)); fs.setAcl(dirPath, aclSpec); Path filePath = new Path(path, "file1"); + fs.registerListener(null); fs.create(filePath).close(); + + fs.registerListener(tracingHeaderValidator + .getClone(AbfsOperationConstants.PERMISSION)); fs.setPermission(filePath, FsPermission.createImmutable((short) RW_R)); Path renamedFilePath = new Path(dirPath, "file1"); + + fs.registerListener(tracingHeaderValidator + .getClone(AbfsOperationConstants.RENAME)); fs.rename(filePath, renamedFilePath); AclEntry[] expected = new AclEntry[] { }; + + fs.registerListener(tracingHeaderValidator + .getClone(AbfsOperationConstants.GETACLSTATUS)); AclStatus s = fs.getAclStatus(renamedFilePath); + fs.registerListener(null); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(expected, returned); assertPermission(fs, renamedFilePath, (short) RW_R); @@ -1273,14 +1293,21 @@ public void testEnsureAclOperationWorksForRoot() throws Exception { @Test public void testSetOwnerForNonNamespaceEnabledAccount() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); Assume.assumeTrue(!getIsNamespaceEnabled(fs)); final Path filePath = new Path(methodName.getMethodName()); fs.create(filePath); assertTrue(fs.exists(filePath)); + TracingHeaderValidator tracingHeaderValidator = new TracingHeaderValidator( + conf.getClientCorrelationID(), fs.getFileSystemID(), + AbfsOperationConstants.GETFILESTATUS, false, 1); + fs.registerListener(tracingHeaderValidator); FileStatus oldFileStatus = fs.getFileStatus(filePath); + tracingHeaderValidator.setOperation(AbfsOperationConstants.SETOWNER); fs.setOwner(filePath, TEST_OWNER, TEST_GROUP); + fs.registerListener(null); FileStatus newFileStatus = fs.getFileStatus(filePath); assertEquals(oldFileStatus.getOwner(), newFileStatus.getOwner()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 2f552638f8c92..20cb15c4c802a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -110,6 +110,23 @@ public void runCorrelationTestForAllMethods() throws Exception { testClasses.put(new ITestAzureBlobFileSystemCreate(), ITestAzureBlobFileSystemCreate.class.getMethod( "testDefaultCreateOverwriteFileTest")); + testClasses.put(new ITestAbfsNetworkStatistics(), + ITestAbfsNetworkStatistics.class.getMethod( + "testAbfsHttpResponseStatistics")); //fs.open + testClasses.put(new ITestAbfsStatistics(), + ITestAbfsStatistics.class.getMethod("testOpenAppendRenameExists")); + //setacl, setpermission, rename, getaclstatus + testClasses.put(new ITestAzureBlobFilesystemAcl(), + ITestAzureBlobFilesystemAcl.class.getMethod( + "testDefaultAclRenamedFile")); + testClasses.put(new ITestAzureBlobFileSystemDelete(), + ITestAzureBlobFileSystemDelete.class.getMethod( + "testDeleteFirstLevelDirectory")); //delete + testClasses.put(new ITestAzureBlobFileSystemCreate(), + ITestAzureBlobFileSystemCreate.class.getMethod( + "testCreateNonRecursive")); //mkdir + + //add other ops' testClasses and testMethods that have listener registered for (AbstractAbfsIntegrationTest testClass : testClasses.keySet()) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java index f154b4b98c5be..721ddc20b292b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java @@ -1,5 +1,6 @@ package org.apache.hadoop.fs.azurebfs.utils; +import com.google.common.annotations.VisibleForTesting; import org.assertj.core.api.Assertions; public class TracingHeaderValidator implements Listener { @@ -33,6 +34,12 @@ public TracingHeaderValidator getClone() { return tracingHeaderValidator; } + public TracingHeaderValidator getClone(String operation) { + TracingHeaderValidator tracingHeaderValidator = getClone(); + tracingHeaderValidator.operation = operation; + return tracingHeaderValidator; + } + public TracingHeaderValidator(String clientCorrelationID, String fileSystemID, String operation, boolean needsPrimaryRequestID, int retryNum) { @@ -61,16 +68,24 @@ private void validateTracingHeader(String tracingContextHeader) { Assertions.assertThat(id_list[2]).describedAs( "FilesystemID should be same for requests with same filesystem") .isEqualTo(fileSystemID); + System.out.println("primaryreq " + primaryRequestID); } if (!streamID.isEmpty()) { System.out.println("check stream" + streamID); Assertions.assertThat(id_list[4]) .describedAs("Stream id should be common for these requests") .isEqualTo(streamID); + System.out.println("streamid " + streamID); } } + @VisibleForTesting + public void setOperation(String operation) { + this.operation = operation; + } + private void validateBasicFormat(String[] id_list) { + System.out.println("basic test"); Assertions.assertThat(id_list) .describedAs("header should have 7 elements").hasSize(7); From 725c98415030f856ce333b605cbf3de82bb2b028 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Mon, 16 Nov 2020 10:32:22 +0530 Subject: [PATCH 41/77] all tests --- .../fs/azurebfs/AzureBlobFileSystem.java | 4 ++ .../fs/azurebfs/services/AbfsInputStream.java | 8 +++ .../azurebfs/services/AbfsOutputStream.java | 7 +++ .../hadoop/fs/azurebfs/utils/Listener.java | 1 + .../fs/azurebfs/utils/TracingContext.java | 5 ++ .../azurebfs/ITestAbfsNetworkStatistics.java | 5 -- .../azurebfs/ITestAbfsReadWriteAndSeek.java | 23 +++++++ .../fs/azurebfs/ITestAbfsStatistics.java | 4 -- .../ITestAzureBlobFileSystemAppend.java | 23 +++++++ .../ITestAzureBlobFileSystemAttributes.java | 8 +++ .../ITestAzureBlobFileSystemCheckAccess.java | 7 +++ .../azurebfs/ITestAzureBlobFileSystemE2E.java | 10 +++ .../ITestAzureBlobFileSystemFlush.java | 1 + .../azurebfs/ITestAzureBlobFilesystemAcl.java | 32 +++++----- .../fs/azurebfs/TestTracingContext.java | 63 ++++++++++++++----- .../utils/TracingHeaderValidator.java | 19 ++++-- 16 files changed, 176 insertions(+), 44 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 187cefca19371..11edf66deffaf 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -1271,6 +1271,10 @@ FileSystem.Statistics getFsStatistics() { return this.statistics; } + public void setListenerOperation(String operation) { + listener.setOperation(operation); + } + @VisibleForTesting static class FileSystemOperation { private final T result; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 91f7089c45008..7bb4e9bac003c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -206,6 +206,8 @@ private int readOneBlock(final byte[] b, final int off, final int len) throws IO // Enable readAhead when reading sequentially if (-1 == fCursorAfterLastRead || fCursorAfterLastRead == fCursor || b.length >= bufferSize) { + System.out.println(fCursorAfterLastRead + " " + fCursor + " b.l " + b.length + + " " + bufferSize); bytesRead = readInternal(fCursor, buffer, 0, bufferSize, false); } else { bytesRead = readInternal(fCursor, buffer, 0, b.length, true); @@ -260,6 +262,7 @@ private int readInternal(final long position, final byte[] b, final int offset, nextOffset, nextSize); ReadBufferManager.getBufferManager().queueReadAhead(this, nextOffset, (int) nextSize, new TracingContext(readAheadTracingContext)); + System.out.println("----------------1-------------------"); nextOffset = nextOffset + nextSize; numReadAheads--; } @@ -269,6 +272,7 @@ private int readInternal(final long position, final byte[] b, final int offset, bytesFromReadAhead += receivedBytes; if (receivedBytes > 0) { incrementReadOps(); + System.out.println("Received data from read ahead"); LOG.debug("Received data from read ahead, not doing remote read"); if (streamStatistics != null) { streamStatistics.readAheadBytesRead(receivedBytes); @@ -278,9 +282,12 @@ private int readInternal(final long position, final byte[] b, final int offset, // got nothing from read-ahead, do our own read now receivedBytes = readRemote(position, b, offset, length, new TracingContext(tracingContext)); + System.out.println("----------------2-----------------"); return receivedBytes; } else { LOG.debug("read ahead disabled, reading remote"); + System.out.println("----------------3-----------------"); + tracingContext.printTC(); return readRemote(position, b, offset, length, new TracingContext(tracingContext)); } } @@ -311,6 +318,7 @@ int readRemote(long position, byte[] b, int offset, int length, TracingContext t op = client.read(path, position, b, offset, length, tolerateOobAppends ? "*" : eTag, cachedSasToken.get(), tracingContext); + System.out.println("----------------4-----------------"); cachedSasToken.update(op.getSasToken()); if (streamStatistics != null) { streamStatistics.remoteReadOperation(); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 46d624d0f18cf..efb47a4a5f969 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -145,6 +145,7 @@ public AbfsOutputStream( this.outputStreamID = getOutputStreamID(); this.tracingContext = new TracingContext(tracingContext); this.tracingContext.setStreamID(outputStreamID); + System.out.println("----------------5----------------"); } private String getOutputStreamID() { @@ -344,8 +345,10 @@ private synchronized void flushInternalAsync() throws IOException { private void writeAppendBlobCurrentBufferToService() throws IOException { if (bufferIndex == 0) { + System.out.println("bufferIndex is 0"); return; } + System.out.println("bufferIndex not 0"); outputStreamStatistics.writeCurrentBuffer(); final byte[] bytes = buffer; @@ -358,6 +361,7 @@ private void writeAppendBlobCurrentBufferToService() throws IOException { AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "writeCurrentBufferToService", "append")) { + System.out.println("----------------6----------------"); AbfsRestOperation op = client.append(path, offset, bytes, 0, bytesLength, cachedSasToken.get(), this.isAppendBlob, new TracingContext(tracingContext)); @@ -383,6 +387,7 @@ private void writeAppendBlobCurrentBufferToService() throws IOException { private synchronized void writeCurrentBufferToService() throws IOException { if (this.isAppendBlob) { + System.out.println("append blob true"); writeAppendBlobCurrentBufferToService(); return; } @@ -412,6 +417,7 @@ public Void call() throws Exception { AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "writeCurrentBufferToService", "append")) { + System.out.println("----------------7----------------"); AbfsRestOperation op = client.append(path, offset, bytes, 0, bytesLength, cachedSasToken.get(), false, new TracingContext(tracingContext)); @@ -476,6 +482,7 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset, try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "flushWrittenBytesToServiceInternal", "flush")) { + System.out.println("----------------8----------------"); AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, isClose, cachedSasToken.get(), new TracingContext(tracingContext)); cachedSasToken.update(op.getSasToken()); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java index 8d01a2e8c3bdb..5e44adf31a64e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java @@ -4,4 +4,5 @@ public interface Listener { void afterOp(String header); void updatePrimaryRequestID(String primaryRequestID); Listener getClone(); + void setOperation(String operation); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index 3d7824fa0b888..a2ff11614c7d6 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -76,6 +76,11 @@ public TracingContext(TracingContext originalTracingContext) { } } + public void printTC() { + System.out.println(clientCorrelationID + ":" + clientRequestID + + ":" + primaryRequestID); + } + public String validateClientCorrelationID(String clientCorrelationID) { if ((clientCorrelationID.length() > MAX_CLIENT_CORRELATION_ID_LENGTH) || (!clientCorrelationID.matches(CLIENT_CORRELATION_ID_PATTERN))) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java index 93a1e14d0153b..82af8fbd56d70 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java @@ -200,12 +200,7 @@ public void testAbfsHttpResponseStatistics() throws IOException { * (Writing data in Data store). * */ - AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); - fs.registerListener(new TracingHeaderValidator(conf.getClientCorrelationID(), - fs.getFileSystemID(), AbfsOperationConstants.CREATE, - true, 0)); out = fs.create(getResponsePath); - fs.registerListener(null); out.write(testResponseString.getBytes()); out.hflush(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index b45f1c733280a..d14ae05ee5a86 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; +import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; import org.junit.runner.RunWith; @@ -75,14 +76,27 @@ public void testReadAheadRequestID() throws java.io.IOException { final byte[] b = new byte[bufferSize * 10]; new Random().nextBytes(b); try (FSDataOutputStream stream = fs.create(TEST_PATH)) { + ((AbfsOutputStream)stream.getWrappedStream()).registerListener(new + TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), + fs.getFileSystemID(), AbfsOperationConstants.CREATE, + false, 0)); stream.write(b); } final byte[] readBuffer = new byte[4 * bufferSize]; int result; + TracingHeaderValidator tracingHeaderValidator = new TracingHeaderValidator( + abfsConfiguration.getClientCorrelationID(), fs.getFileSystemID(), + AbfsOperationConstants.OPEN, false, 0); + fs.registerListener(tracingHeaderValidator); try (FSDataInputStream inputStream = fs.open(TEST_PATH)) { + ((AbfsInputStream)inputStream.getWrappedStream()).registerListener(new + TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), + fs.getFileSystemID(), AbfsOperationConstants.READ, + false, 0)); result = inputStream.read(readBuffer, 0, bufferSize*4); } + fs.registerListener(null); } private void testReadWriteAndSeek(int bufferSize) throws Exception { @@ -109,8 +123,17 @@ private void testReadWriteAndSeek(int bufferSize) throws Exception { inputStream.seek(bufferSize); result = inputStream.read(readBuffer, bufferSize, bufferSize); assertNotEquals(-1, result); + + //to test tracingHeader for case with bypassReadAhead == true + inputStream.seek(0); + byte[] temp = new byte[5]; + int t = inputStream.read(temp, 0, 1); + inputStream.seek(0); result = inputStream.read(readBuffer, 0, bufferSize); +// inputStream.seek(bufferSize - 1); +// result += inputStream.read(readBuffer, bufferSize-1, 1); + } assertNotEquals("data read in final read()", -1, result); assertArrayEquals(readBuffer, b); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java index 5826cbeb07f63..21cb584498a9b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java @@ -184,11 +184,7 @@ public void testOpenAppendRenameExists() throws IOException { fs.create(createFilePath); fs.open(createFilePath); - fs.registerListener(new TracingHeaderValidator(conf.getClientCorrelationID(), - fs.getFileSystemID(), AbfsOperationConstants.APPEND, - false, 0)); fs.append(createFilePath); - fs.registerListener(null); assertTrue(fs.rename(createFilePath, destCreateFilePath)); Map metricMap = fs.getInstrumentationMap(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java index cbe19396d1277..78a76f1017d26 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java @@ -19,8 +19,13 @@ package org.apache.hadoop.fs.azurebfs; import java.io.FileNotFoundException; +import java.io.IOException; import java.util.Random; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; import org.apache.hadoop.fs.FSDataOutputStream; @@ -53,7 +58,9 @@ public void testAppendWithLength0() throws Exception { try(FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { final byte[] b = new byte[1024]; new Random().nextBytes(b); + System.out.println("going to write"); stream.write(b, 1000, 0); + System.out.println("done write"); assertEquals(0, stream.getPos()); } } @@ -66,6 +73,9 @@ public void testAppendFileAfterDelete() throws Exception { ContractTestUtils.touch(fs, filePath); fs.delete(filePath, false); + fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() + .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), + AbfsOperationConstants.APPEND, false, 0)); fs.append(filePath); } @@ -76,4 +86,17 @@ public void testAppendDirectory() throws Exception { fs.mkdirs(folderPath); fs.append(folderPath); } + + @Test + public void testTracingForAppend() throws IOException { + AzureBlobFileSystem fs = getFileSystem(); + fs.create(TEST_FILE_PATH); + fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() + .getAbfsConfiguration().getClientCorrelationID(), + fs.getFileSystemID(), AbfsOperationConstants.APPEND, + false, 0)); + FSDataOutputStream in = fs.append(TEST_FILE_PATH, 10); + byte[] buf = new byte[100]; + in.write(buf, 0, 5); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java index d7373e5f8a8bf..1763f87c7e6a5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java @@ -23,6 +23,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.XAttrSetFlag; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Assume; import org.junit.Test; @@ -42,6 +44,7 @@ public ITestAzureBlobFileSystemAttributes() throws Exception { @Test public void testSetGetXAttr() throws Exception { AzureBlobFileSystem fs = getFileSystem(); + AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); Assume.assumeTrue(getIsNamespaceEnabled(fs)); byte[] attributeValue1 = fs.getAbfsStore().encodeAttribute("hi"); @@ -55,8 +58,13 @@ public void testSetGetXAttr() throws Exception { assertNull(fs.getXAttr(testFile, attributeName1)); // after setting the xAttr on the file, the value should be retrievable + fs.registerListener(new TracingHeaderValidator(conf.getClientCorrelationID(), + fs.getFileSystemID(), AbfsOperationConstants.SETATTR, + true, 0)); fs.setXAttr(testFile, attributeName1, attributeValue1); + fs.setListenerOperation(AbfsOperationConstants.GETATTR); assertArrayEquals(attributeValue1, fs.getXAttr(testFile, attributeName1)); + fs.registerListener(null); // after setting a second xAttr on the file, the first xAttr values should not be overwritten fs.setXAttr(testFile, attributeName2, attributeValue2); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java index 4189d666e7a70..0b425cc06e133 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java @@ -23,6 +23,8 @@ import java.io.IOException; import java.util.List; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Assume; import org.junit.Test; @@ -179,7 +181,12 @@ public void testFsActionEXECUTE() throws Exception { setTestUserFs(); Path testFilePath = setupTestDirectoryAndUserAccess("/test3.txt", FsAction.EXECUTE); + AzureBlobFileSystem fs = (AzureBlobFileSystem)testUserFs; + fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() + .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), + AbfsOperationConstants.ACCESS, false, 0)); assertAccessible(testFilePath, FsAction.EXECUTE); + fs.registerListener(null); assertInaccessible(testFilePath, FsAction.READ); assertInaccessible(testFilePath, FsAction.WRITE); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java index 05c3855f5c89d..399ea50935453 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java @@ -23,6 +23,9 @@ import java.util.Arrays; import java.util.Random; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; import org.apache.hadoop.conf.Configuration; @@ -139,7 +142,14 @@ public void testWriteWithBufferOffset() throws Exception { } final byte[] r = new byte[TEST_DEFAULT_READ_BUFFER_SIZE]; + AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); + TracingHeaderValidator tracingHeaderValidator = new TracingHeaderValidator + (conf.getClientCorrelationID(), fs.getFileSystemID(), AbfsOperationConstants.OPEN, + false, 0); + fs.registerListener(tracingHeaderValidator); FSDataInputStream inputStream = fs.open(testFilePath, TEST_DEFAULT_BUFFER_SIZE); + tracingHeaderValidator.setOperation(AbfsOperationConstants.READ); + ((AbfsInputStream)inputStream.getWrappedStream()).registerListener(tracingHeaderValidator); int result = inputStream.read(r); assertNotEquals(-1, result); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index 92aa5520ee4fd..8d15bb21849dc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -235,6 +235,7 @@ private void testFlush(boolean disableOutputStreamFlush) throws Exception { boolean isAppendBlob = true; if (!fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(testFilePath).toString())) { isAppendBlob = false; + System.out.println("append false"); } try (FSDataOutputStream stream = fs.create(testFilePath)) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java index 05ea2f1357532..b738b7aebb998 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java @@ -1207,37 +1207,25 @@ public void testDefaultAclNewDirWithMode() throws Exception { @Test public void testDefaultAclRenamedFile() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); - TracingHeaderValidator tracingHeaderValidator = - new TracingHeaderValidator(conf.getClientCorrelationID(), - fs.getFileSystemID(), AbfsOperationConstants.TESTOP, - true, 0); assumeTrue(getIsNamespaceEnabled(fs)); path = new Path(testRoot, UUID.randomUUID().toString()); Path dirPath = new Path(path, "dir"); FileSystem.mkdirs(fs, dirPath, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, FOO, ALL)); - fs.registerListener(tracingHeaderValidator.getClone(AbfsOperationConstants.SETACL)); fs.setAcl(dirPath, aclSpec); Path filePath = new Path(path, "file1"); - fs.registerListener(null); fs.create(filePath).close(); - - fs.registerListener(tracingHeaderValidator - .getClone(AbfsOperationConstants.PERMISSION)); fs.setPermission(filePath, FsPermission.createImmutable((short) RW_R)); Path renamedFilePath = new Path(dirPath, "file1"); - fs.registerListener(tracingHeaderValidator - .getClone(AbfsOperationConstants.RENAME)); + fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() + .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), + AbfsOperationConstants.RENAME, true, 0)); fs.rename(filePath, renamedFilePath); + fs.registerListener(null); AclEntry[] expected = new AclEntry[] { }; - - fs.registerListener(tracingHeaderValidator - .getClone(AbfsOperationConstants.GETACLSTATUS)); AclStatus s = fs.getAclStatus(renamedFilePath); - fs.registerListener(null); AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(expected, returned); @@ -1275,18 +1263,30 @@ public void testEnsureAclOperationWorksForRoot() throws Exception { List aclSpec1 = Lists.newArrayList( aclEntry(DEFAULT, GROUP, FOO, ALL), aclEntry(ACCESS, GROUP, BAR, ALL)); + + fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() + .getAbfsConfiguration().getClientCorrelationID(), + fs.getFileSystemID(), AbfsOperationConstants.SETACL, true, 0)); fs.setAcl(rootPath, aclSpec1); + + fs.setListenerOperation(AbfsOperationConstants.GETACLSTATUS); fs.getAclStatus(rootPath); + fs.setListenerOperation(AbfsOperationConstants.SETOWNER); fs.setOwner(rootPath, TEST_OWNER, TEST_GROUP); + fs.setListenerOperation(AbfsOperationConstants.PERMISSION); fs.setPermission(rootPath, new FsPermission("777")); List aclSpec2 = Lists.newArrayList( aclEntry(DEFAULT, USER, FOO, ALL), aclEntry(ACCESS, USER, BAR, ALL)); + fs.setListenerOperation(AbfsOperationConstants.MODIFYACL); fs.modifyAclEntries(rootPath, aclSpec2); + fs.setListenerOperation(AbfsOperationConstants.REMOVEACLENTRIES); fs.removeAclEntries(rootPath, aclSpec2); + fs.setListenerOperation(AbfsOperationConstants.REMOVEDEFAULTACL); fs.removeDefaultAcl(rootPath); + fs.setListenerOperation(AbfsOperationConstants.REMOVEACL); fs.removeAcl(rootPath); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 20cb15c4c802a..353e9cbf84ac5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -2,7 +2,10 @@ import com.google.common.base.Preconditions; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonPathCapabilities; +import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; @@ -10,6 +13,8 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; +import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.assertj.core.api.Assertions; import org.junit.Ignore; @@ -28,7 +33,7 @@ public class TestTracingContext extends AbstractAbfsIntegrationTest { String GUID_PATTERN = "[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}"; String prevClientRequestID = ""; - protected TestTracingContext() throws Exception { + public TestTracingContext() throws Exception { super(); } @@ -105,34 +110,62 @@ public void runCorrelationTestForAllMethods() throws Exception { //map to avoid creating new instance and calling setup() for each test Map testClasses = new HashMap<>(); - testClasses.put(new ITestAzureBlobFileSystemListStatus(), - ITestAzureBlobFileSystemListStatus.class.getMethod("testListPath")); +// testClasses.put(new ITestAzureBlobFileSystemListStatus(), //liststatus +// ITestAzureBlobFileSystemListStatus.class.getMethod("testListPath")); + testClasses.put(new ITestAbfsReadWriteAndSeek(32), //open, read, write + ITestAbfsReadWriteAndSeek.class.getMethod("testReadAheadRequestID")); + testClasses.put(new ITestAbfsReadWriteAndSeek(32), //read + ITestAbfsReadWriteAndSeek.class.getMethod("testReadAndWriteWithDifferentBufferSizesAndSeek")); + testClasses.put(new ITestAzureBlobFileSystemAppend(), //append + ITestAzureBlobFileSystemAppend.class.getMethod("testTracingForAppend")); testClasses.put(new ITestAzureBlobFileSystemCreate(), ITestAzureBlobFileSystemCreate.class.getMethod( - "testDefaultCreateOverwriteFileTest")); - testClasses.put(new ITestAbfsNetworkStatistics(), - ITestAbfsNetworkStatistics.class.getMethod( - "testAbfsHttpResponseStatistics")); //fs.open - testClasses.put(new ITestAbfsStatistics(), - ITestAbfsStatistics.class.getMethod("testOpenAppendRenameExists")); - //setacl, setpermission, rename, getaclstatus + "testDefaultCreateOverwriteFileTest")); //create testClasses.put(new ITestAzureBlobFilesystemAcl(), ITestAzureBlobFilesystemAcl.class.getMethod( - "testDefaultAclRenamedFile")); + "testDefaultAclRenamedFile")); //rename testClasses.put(new ITestAzureBlobFileSystemDelete(), ITestAzureBlobFileSystemDelete.class.getMethod( "testDeleteFirstLevelDirectory")); //delete testClasses.put(new ITestAzureBlobFileSystemCreate(), ITestAzureBlobFileSystemCreate.class.getMethod( - "testCreateNonRecursive")); //mkdir - - - //add other ops' testClasses and testMethods that have listener registered + "testCreateNonRecursive")); //mkdirs + testClasses.put(new ITestAzureBlobFileSystemAttributes(), + ITestAzureBlobFileSystemAttributes.class.getMethod( + "testSetGetXAttr")); //setxattr, getxattr + testClasses.put(new ITestAzureBlobFilesystemAcl(), + ITestAzureBlobFilesystemAcl.class.getMethod( + "testEnsureAclOperationWorksForRoot")); // setacl, getaclstatus, + // setowner, setpermission, modifyaclentries, + // removeaclentries, removedefaultacl, removeacl for (AbstractAbfsIntegrationTest testClass : testClasses.keySet()) { + System.out.println(testClass.methodName.getMethodName()); testClass.setup(); testClasses.get(testClass).invoke(testClass); testClass.teardown(); } + testExternalOps(); + } + + @Test + //rename this test + public void testExternalOps() throws Exception { + //validate tracing header for access, hasPathCapability, + // getIsNamespaceEnabled + AzureBlobFileSystem fs = getFileSystem(); + fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() + .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), + AbfsOperationConstants.ACCESS, false, 0)); + fs.access(new Path("/"), FsAction.READ); + + fs.setListenerOperation(AbfsOperationConstants.PATH); + //unset namespaceEnabled config to call getAcl -> test tracing header + fs.getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN); + fs.hasPathCapability(new Path("/"), CommonPathCapabilities.FS_ACLS); + + ITestAzureBlobFileSystemAppend test2 = new ITestAzureBlobFileSystemAppend(); + test2.setup(); + test2.testTracingForAppend(); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java index 721ddc20b292b..a9d4767d9acf7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java @@ -1,6 +1,7 @@ package org.apache.hadoop.fs.azurebfs.utils; import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; import org.assertj.core.api.Assertions; public class TracingHeaderValidator implements Listener { @@ -25,6 +26,7 @@ public void updatePrimaryRequestID(String primaryRequestID) { this.primaryRequestID = primaryRequestID; } + @Override public TracingHeaderValidator getClone() { TracingHeaderValidator tracingHeaderValidator = @@ -62,9 +64,14 @@ private void validateTracingHeader(String tracingContextHeader) { String[] id_list = tracingContextHeader.split(":"); validateBasicFormat(id_list); if (needsPrimaryRequestID) { - Assertions.assertThat(id_list[3]) - .describedAs("PrimaryReqID should be common for these requests") - .isEqualTo(primaryRequestID); + if (!operation.equals(AbfsOperationConstants.READ)) { + Assertions.assertThat(primaryRequestID) + .describedAs("Should have primaryReqId").isNotEmpty(); + } else { + Assertions.assertThat(id_list[3]) + .describedAs("PrimaryReqID should be common for these requests") + .isEqualTo(primaryRequestID); + } Assertions.assertThat(id_list[2]).describedAs( "FilesystemID should be same for requests with same filesystem") .isEqualTo(fileSystemID); @@ -85,7 +92,7 @@ public void setOperation(String operation) { } private void validateBasicFormat(String[] id_list) { - System.out.println("basic test"); + System.out.println("basic test " + operation); Assertions.assertThat(id_list) .describedAs("header should have 7 elements").hasSize(7); @@ -102,6 +109,10 @@ private void validateBasicFormat(String[] id_list) { .matches(GUID_PATTERN); Assertions.assertThat(id_list[2]).describedAs("Filesystem ID incorrect") .isEqualTo(fileSystemID); + if (needsPrimaryRequestID && !operation.equals(AbfsOperationConstants.READ)) { + Assertions.assertThat(id_list[3]).describedAs("should have primaryReqId") + .isNotEmpty(); + } Assertions.assertThat(id_list[5]).describedAs("Operation name incorrect") .isEqualTo(operation); int retryCount = Integer.parseInt(id_list[6]); From 9b4f55809d54a4f7348859cede4c44a68ccd16ec Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 17 Nov 2020 13:20:35 +0530 Subject: [PATCH 42/77] test tc for appendblob=true --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 1 + .../azurebfs/services/AbfsOutputStream.java | 1 + .../ITestAzureBlobFileSystemFlush.java | 26 +++++++++++++++++++ .../fs/azurebfs/TestTracingContext.java | 24 ++++++++++++----- 4 files changed, 46 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index b4df573911f8a..77c20409da511 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -1509,6 +1509,7 @@ private Hashtable parseCommaSeparatedXmsProperties(String xMsPro private boolean isKeyForDirectorySet(String key, Set dirSet) { for (String dir : dirSet) { + System.out.println("dir " + dir + " and key = " + key); if (dir.isEmpty() || key.startsWith(dir + AbfsHttpConstants.FORWARD_SLASH)) { return true; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index efb47a4a5f969..9657523842e3a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -213,6 +213,7 @@ public synchronized void write(final byte[] data, final int off, final int lengt if (writableBytes <= numberOfBytesToWrite) { System.arraycopy(data, currentOffset, buffer, bufferIndex, writableBytes); bufferIndex += writableBytes; + System.out.println("going to write"); writeCurrentBufferToService(); currentOffset += writableBytes; numberOfBytesToWrite = numberOfBytesToWrite - writableBytes; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index 8d15bb21849dc..e0312fa451d23 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -29,8 +29,13 @@ import java.util.concurrent.Future; import java.io.IOException; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations; +import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.hamcrest.core.IsEqual; import org.hamcrest.core.IsNot; import org.junit.Test; @@ -41,6 +46,9 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_APPEND_BLOB_KEY; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI; + /** * Test flush operation. * This class cannot be run in parallel test mode--check comments in @@ -299,6 +307,24 @@ public void testHsyncWithFlushEnabled() throws Exception { } } + @Test + public void testTracingHeaderForAppendBlob() throws Exception { + Configuration config = new Configuration(this.getRawConfiguration()); + config.set(FS_AZURE_APPEND_BLOB_KEY, "abfss:/"); + config.set(TestConfigurationKeys.FS_AZURE_TEST_APPENDBLOB_ENABLED, "true"); + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(config); + + byte[] buf = new byte[10]; + new Random().nextBytes(buf); + FSDataOutputStream out = fs.create(new Path("/testFile")); + ((AbfsOutputStream)out.getWrappedStream()).registerListener(new + TracingHeaderValidator(fs.getAbfsStore().getAbfsConfiguration() + .getClientCorrelationID(), fs.getFileSystemID(), + AbfsOperationConstants.CREATE, false, 0)); + out.write(buf); + out.hsync(); + } + @Test public void testStreamCapabilitiesWithFlushDisabled() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 353e9cbf84ac5..720060cfe6a46 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -4,11 +4,14 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonPathCapabilities; import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; import org.apache.hadoop.fs.azurebfs.enums.Trilean; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; @@ -24,6 +27,7 @@ import java.lang.reflect.Method; import java.util.HashMap; import java.util.Map; +import java.util.Random; public class TestTracingContext extends AbstractAbfsIntegrationTest { private static final String[] CLIENT_CORRELATIONID_LIST = { @@ -118,6 +122,9 @@ public void runCorrelationTestForAllMethods() throws Exception { ITestAbfsReadWriteAndSeek.class.getMethod("testReadAndWriteWithDifferentBufferSizesAndSeek")); testClasses.put(new ITestAzureBlobFileSystemAppend(), //append ITestAzureBlobFileSystemAppend.class.getMethod("testTracingForAppend")); + testClasses.put(new ITestAzureBlobFileSystemFlush(), + ITestAzureBlobFileSystemFlush.class.getMethod( + "testTracingHeaderForAppendBlob")); //outputstream (appendblob) testClasses.put(new ITestAzureBlobFileSystemCreate(), ITestAzureBlobFileSystemCreate.class.getMethod( "testDefaultCreateOverwriteFileTest")); //create @@ -157,15 +164,20 @@ public void testExternalOps() throws Exception { fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), AbfsOperationConstants.ACCESS, false, 0)); - fs.access(new Path("/"), FsAction.READ); +// fs.access(new Path("/"), FsAction.READ); fs.setListenerOperation(AbfsOperationConstants.PATH); //unset namespaceEnabled config to call getAcl -> test tracing header - fs.getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN); - fs.hasPathCapability(new Path("/"), CommonPathCapabilities.FS_ACLS); +// fs.getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN); +// fs.hasPathCapability(new Path("/"), CommonPathCapabilities.FS_ACLS); + +// ITestAzureBlobFileSystemAppend test2 = new ITestAzureBlobFileSystemAppend(); +// test2.setup(); +// test2.testTracingForAppend(); + + Configuration config = new Configuration(this.getRawConfiguration()); + config.set(TestConfigurationKeys.FS_AZURE_TEST_APPENDBLOB_ENABLED, "true"); + AzureBlobFileSystem fs1 = getFileSystem(config); - ITestAzureBlobFileSystemAppend test2 = new ITestAzureBlobFileSystemAppend(); - test2.setup(); - test2.testTracingForAppend(); } } From 858695bd6dba12363a7a57e4fe406697f3ce6f8e Mon Sep 17 00:00:00 2001 From: Sumangala Date: Mon, 23 Nov 2020 11:09:08 +0530 Subject: [PATCH 43/77] clean up code --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 1 - .../fs/azurebfs/services/AbfsInputStream.java | 17 +++++----------- .../azurebfs/services/AbfsOutputStream.java | 8 -------- .../fs/azurebfs/utils/TracingContext.java | 9 --------- .../azurebfs/AbstractAbfsIntegrationTest.java | 4 +++- .../ITestAzureBlobFileSystemAppend.java | 2 -- .../ITestAzureBlobFileSystemFlush.java | 1 - .../fs/azurebfs/TestTracingContext.java | 20 +++++-------------- .../utils/TracingHeaderValidator.java | 14 +++---------- 9 files changed, 16 insertions(+), 60 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 77c20409da511..b4df573911f8a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -1509,7 +1509,6 @@ private Hashtable parseCommaSeparatedXmsProperties(String xMsPro private boolean isKeyForDirectorySet(String key, Set dirSet) { for (String dir : dirSet) { - System.out.println("dir " + dir + " and key = " + key); if (dir.isEmpty() || key.startsWith(dir + AbfsHttpConstants.FORWARD_SLASH)) { return true; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 7bb4e9bac003c..287e07f7d4df1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -120,10 +120,6 @@ public void registerListener(Listener listener1) { tracingContext.setListener(listener); } - public String getStreamID() { - return inputStreamID; - } - @Override public int read() throws IOException { byte[] b = new byte[1]; @@ -206,8 +202,6 @@ private int readOneBlock(final byte[] b, final int off, final int len) throws IO // Enable readAhead when reading sequentially if (-1 == fCursorAfterLastRead || fCursorAfterLastRead == fCursor || b.length >= bufferSize) { - System.out.println(fCursorAfterLastRead + " " + fCursor + " b.l " + b.length + - " " + bufferSize); bytesRead = readInternal(fCursor, buffer, 0, bufferSize, false); } else { bytesRead = readInternal(fCursor, buffer, 0, b.length, true); @@ -262,7 +256,6 @@ private int readInternal(final long position, final byte[] b, final int offset, nextOffset, nextSize); ReadBufferManager.getBufferManager().queueReadAhead(this, nextOffset, (int) nextSize, new TracingContext(readAheadTracingContext)); - System.out.println("----------------1-------------------"); nextOffset = nextOffset + nextSize; numReadAheads--; } @@ -272,7 +265,6 @@ private int readInternal(final long position, final byte[] b, final int offset, bytesFromReadAhead += receivedBytes; if (receivedBytes > 0) { incrementReadOps(); - System.out.println("Received data from read ahead"); LOG.debug("Received data from read ahead, not doing remote read"); if (streamStatistics != null) { streamStatistics.readAheadBytesRead(receivedBytes); @@ -282,12 +274,9 @@ private int readInternal(final long position, final byte[] b, final int offset, // got nothing from read-ahead, do our own read now receivedBytes = readRemote(position, b, offset, length, new TracingContext(tracingContext)); - System.out.println("----------------2-----------------"); return receivedBytes; } else { LOG.debug("read ahead disabled, reading remote"); - System.out.println("----------------3-----------------"); - tracingContext.printTC(); return readRemote(position, b, offset, length, new TracingContext(tracingContext)); } } @@ -318,7 +307,6 @@ int readRemote(long position, byte[] b, int offset, int length, TracingContext t op = client.read(path, position, b, offset, length, tolerateOobAppends ? "*" : eTag, cachedSasToken.get(), tracingContext); - System.out.println("----------------4-----------------"); cachedSasToken.update(op.getSasToken()); if (streamStatistics != null) { streamStatistics.remoteReadOperation(); @@ -536,6 +524,11 @@ protected void setCachedSasToken(final CachedSASToken cachedSasToken) { this.cachedSasToken = cachedSasToken; } + @VisibleForTesting + public String getStreamID() { + return inputStreamID; + } + /** * Getter for AbfsInputStreamStatistics. * diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 9657523842e3a..46d624d0f18cf 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -145,7 +145,6 @@ public AbfsOutputStream( this.outputStreamID = getOutputStreamID(); this.tracingContext = new TracingContext(tracingContext); this.tracingContext.setStreamID(outputStreamID); - System.out.println("----------------5----------------"); } private String getOutputStreamID() { @@ -213,7 +212,6 @@ public synchronized void write(final byte[] data, final int off, final int lengt if (writableBytes <= numberOfBytesToWrite) { System.arraycopy(data, currentOffset, buffer, bufferIndex, writableBytes); bufferIndex += writableBytes; - System.out.println("going to write"); writeCurrentBufferToService(); currentOffset += writableBytes; numberOfBytesToWrite = numberOfBytesToWrite - writableBytes; @@ -346,10 +344,8 @@ private synchronized void flushInternalAsync() throws IOException { private void writeAppendBlobCurrentBufferToService() throws IOException { if (bufferIndex == 0) { - System.out.println("bufferIndex is 0"); return; } - System.out.println("bufferIndex not 0"); outputStreamStatistics.writeCurrentBuffer(); final byte[] bytes = buffer; @@ -362,7 +358,6 @@ private void writeAppendBlobCurrentBufferToService() throws IOException { AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "writeCurrentBufferToService", "append")) { - System.out.println("----------------6----------------"); AbfsRestOperation op = client.append(path, offset, bytes, 0, bytesLength, cachedSasToken.get(), this.isAppendBlob, new TracingContext(tracingContext)); @@ -388,7 +383,6 @@ private void writeAppendBlobCurrentBufferToService() throws IOException { private synchronized void writeCurrentBufferToService() throws IOException { if (this.isAppendBlob) { - System.out.println("append blob true"); writeAppendBlobCurrentBufferToService(); return; } @@ -418,7 +412,6 @@ public Void call() throws Exception { AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "writeCurrentBufferToService", "append")) { - System.out.println("----------------7----------------"); AbfsRestOperation op = client.append(path, offset, bytes, 0, bytesLength, cachedSasToken.get(), false, new TracingContext(tracingContext)); @@ -483,7 +476,6 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset, try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "flushWrittenBytesToServiceInternal", "flush")) { - System.out.println("----------------8----------------"); AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, isClose, cachedSasToken.get(), new TracingContext(tracingContext)); cachedSasToken.update(op.getSasToken()); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index a2ff11614c7d6..6a0da57cc3f13 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -76,11 +76,6 @@ public TracingContext(TracingContext originalTracingContext) { } } - public void printTC() { - System.out.println(clientCorrelationID + ":" + clientRequestID + - ":" + primaryRequestID); - } - public String validateClientCorrelationID(String clientCorrelationID) { if ((clientCorrelationID.length() > MAX_CLIENT_CORRELATION_ID_LENGTH) || (!clientCorrelationID.matches(CLIENT_CORRELATION_ID_PATTERN))) { @@ -132,8 +127,4 @@ public String toString() { return header; } -// public void validateTracingHeader() { -// if (listener != null) -// listener.afterOp(toString()); -// } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index 1dc0f60fbe3a2..4c5d7eed51d51 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -25,6 +25,7 @@ import java.util.UUID; import java.util.concurrent.Callable; +import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import org.junit.After; @@ -159,7 +160,8 @@ public TracingContext getTestTracingContext(AzureBlobFileSystem fs, TracingContextFormat format = abfsConf == null? TracingContextFormat.ALL_ID_FORMAT : abfsConf.getTracingContextFormat(); - return new TracingContext(corrID, fsID, "TS", isCont, format, null); + return new TracingContext(corrID, fsID, AbfsOperationConstants.TESTOP, isCont, + format, null); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java index 78a76f1017d26..63102b219cec5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java @@ -58,9 +58,7 @@ public void testAppendWithLength0() throws Exception { try(FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { final byte[] b = new byte[1024]; new Random().nextBytes(b); - System.out.println("going to write"); stream.write(b, 1000, 0); - System.out.println("done write"); assertEquals(0, stream.getPos()); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index e0312fa451d23..138217b6ebcdf 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -243,7 +243,6 @@ private void testFlush(boolean disableOutputStreamFlush) throws Exception { boolean isAppendBlob = true; if (!fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(testFilePath).toString())) { isAppendBlob = false; - System.out.println("append false"); } try (FSDataOutputStream stream = fs.create(testFilePath)) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 720060cfe6a46..0f01bcacbb0c1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -114,8 +114,8 @@ public void runCorrelationTestForAllMethods() throws Exception { //map to avoid creating new instance and calling setup() for each test Map testClasses = new HashMap<>(); -// testClasses.put(new ITestAzureBlobFileSystemListStatus(), //liststatus -// ITestAzureBlobFileSystemListStatus.class.getMethod("testListPath")); + testClasses.put(new ITestAzureBlobFileSystemListStatus(), //liststatus + ITestAzureBlobFileSystemListStatus.class.getMethod("testListPath")); testClasses.put(new ITestAbfsReadWriteAndSeek(32), //open, read, write ITestAbfsReadWriteAndSeek.class.getMethod("testReadAheadRequestID")); testClasses.put(new ITestAbfsReadWriteAndSeek(32), //read @@ -147,7 +147,6 @@ public void runCorrelationTestForAllMethods() throws Exception { // removeaclentries, removedefaultacl, removeacl for (AbstractAbfsIntegrationTest testClass : testClasses.keySet()) { - System.out.println(testClass.methodName.getMethodName()); testClass.setup(); testClasses.get(testClass).invoke(testClass); testClass.teardown(); @@ -164,20 +163,11 @@ public void testExternalOps() throws Exception { fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), AbfsOperationConstants.ACCESS, false, 0)); -// fs.access(new Path("/"), FsAction.READ); + fs.access(new Path("/"), FsAction.READ); fs.setListenerOperation(AbfsOperationConstants.PATH); //unset namespaceEnabled config to call getAcl -> test tracing header -// fs.getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN); -// fs.hasPathCapability(new Path("/"), CommonPathCapabilities.FS_ACLS); - -// ITestAzureBlobFileSystemAppend test2 = new ITestAzureBlobFileSystemAppend(); -// test2.setup(); -// test2.testTracingForAppend(); - - Configuration config = new Configuration(this.getRawConfiguration()); - config.set(TestConfigurationKeys.FS_AZURE_TEST_APPENDBLOB_ENABLED, "true"); - AzureBlobFileSystem fs1 = getFileSystem(config); - + fs.getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN); + fs.hasPathCapability(new Path("/"), CommonPathCapabilities.FS_ACLS); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java index a9d4767d9acf7..8ad2e2a40f37c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java @@ -36,12 +36,6 @@ public TracingHeaderValidator getClone() { return tracingHeaderValidator; } - public TracingHeaderValidator getClone(String operation) { - TracingHeaderValidator tracingHeaderValidator = getClone(); - tracingHeaderValidator.operation = operation; - return tracingHeaderValidator; - } - public TracingHeaderValidator(String clientCorrelationID, String fileSystemID, String operation, boolean needsPrimaryRequestID, int retryNum) { @@ -67,7 +61,9 @@ private void validateTracingHeader(String tracingContextHeader) { if (!operation.equals(AbfsOperationConstants.READ)) { Assertions.assertThat(primaryRequestID) .describedAs("Should have primaryReqId").isNotEmpty(); - } else { + // } else { + } + if (!primaryRequestID.isEmpty() && !id_list[3].isEmpty()){ Assertions.assertThat(id_list[3]) .describedAs("PrimaryReqID should be common for these requests") .isEqualTo(primaryRequestID); @@ -75,14 +71,11 @@ private void validateTracingHeader(String tracingContextHeader) { Assertions.assertThat(id_list[2]).describedAs( "FilesystemID should be same for requests with same filesystem") .isEqualTo(fileSystemID); - System.out.println("primaryreq " + primaryRequestID); } if (!streamID.isEmpty()) { - System.out.println("check stream" + streamID); Assertions.assertThat(id_list[4]) .describedAs("Stream id should be common for these requests") .isEqualTo(streamID); - System.out.println("streamid " + streamID); } } @@ -92,7 +85,6 @@ public void setOperation(String operation) { } private void validateBasicFormat(String[] id_list) { - System.out.println("basic test " + operation); Assertions.assertThat(id_list) .describedAs("header should have 7 elements").hasSize(7); From a9a4f425c7b3f97f0ad44426374e3dcfaa34b019 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Mon, 23 Nov 2020 14:49:05 +0530 Subject: [PATCH 44/77] simplify preq test --- .../hadoop/fs/azurebfs/utils/Listener.java | 2 +- .../fs/azurebfs/utils/TracingContext.java | 2 +- .../azurebfs/AbstractAbfsIntegrationTest.java | 3 +- .../azurebfs/ITestAbfsReadWriteAndSeek.java | 2 - .../fs/azurebfs/TestTracingContext.java | 3 +- .../utils/TracingHeaderValidator.java | 51 ++++++++----------- 6 files changed, 24 insertions(+), 39 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java index 5e44adf31a64e..d9c418205a221 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java @@ -1,7 +1,7 @@ package org.apache.hadoop.fs.azurebfs.utils; public interface Listener { - void afterOp(String header); + void callTracingHeaderValidator(String header); void updatePrimaryRequestID(String primaryRequestID); Listener getClone(); void setOperation(String operation); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index 6a0da57cc3f13..89744fdc15dee 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -122,7 +122,7 @@ public String toString() { case TWO_ID_FORMAT: header = clientCorrelationID + ":" + header; } if (listener != null) { - listener.afterOp(header); + listener.callTracingHeaderValidator(header); } return header; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index 4c5d7eed51d51..665263baf18fa 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -143,8 +143,7 @@ protected AbstractAbfsIntegrationTest() throws Exception { } protected boolean getIsNamespaceEnabled(AzureBlobFileSystem fs) throws IOException { - return fs.getAbfsStore().getIsNamespaceEnabled(getTestTracingContext(fs, - false)); + return fs.getIsNamespaceEnabled(getTestTracingContext(fs, false)); } public TracingContext getTestTracingContext(AzureBlobFileSystem fs, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index d14ae05ee5a86..6a7c3ac11ce42 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -131,8 +131,6 @@ private void testReadWriteAndSeek(int bufferSize) throws Exception { inputStream.seek(0); result = inputStream.read(readBuffer, 0, bufferSize); -// inputStream.seek(bufferSize - 1); -// result += inputStream.read(readBuffer, bufferSize-1, 1); } assertNotEquals("data read in final read()", -1, result); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 0f01bcacbb0c1..f793979bd1b18 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -118,7 +118,7 @@ public void runCorrelationTestForAllMethods() throws Exception { ITestAzureBlobFileSystemListStatus.class.getMethod("testListPath")); testClasses.put(new ITestAbfsReadWriteAndSeek(32), //open, read, write ITestAbfsReadWriteAndSeek.class.getMethod("testReadAheadRequestID")); - testClasses.put(new ITestAbfsReadWriteAndSeek(32), //read + testClasses.put(new ITestAbfsReadWriteAndSeek(32), //read (bypassreadahead) ITestAbfsReadWriteAndSeek.class.getMethod("testReadAndWriteWithDifferentBufferSizesAndSeek")); testClasses.put(new ITestAzureBlobFileSystemAppend(), //append ITestAzureBlobFileSystemAppend.class.getMethod("testTracingForAppend")); @@ -151,7 +151,6 @@ public void runCorrelationTestForAllMethods() throws Exception { testClasses.get(testClass).invoke(testClass); testClass.teardown(); } - testExternalOps(); } @Test diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java index 8ad2e2a40f37c..df8397493652c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java @@ -17,16 +17,10 @@ public class TracingHeaderValidator implements Listener { // {3}[0-9a-f]{12}[)}]?$ @Override - public void afterOp(String tracingContextHeader) { + public void callTracingHeaderValidator(String tracingContextHeader) { validateTracingHeader(tracingContextHeader); } - @Override - public void updatePrimaryRequestID(String primaryRequestID) { - this.primaryRequestID = primaryRequestID; - } - - @Override public TracingHeaderValidator getClone() { TracingHeaderValidator tracingHeaderValidator = @@ -57,31 +51,16 @@ public TracingHeaderValidator(String clientCorrelationID, private void validateTracingHeader(String tracingContextHeader) { String[] id_list = tracingContextHeader.split(":"); validateBasicFormat(id_list); - if (needsPrimaryRequestID) { - if (!operation.equals(AbfsOperationConstants.READ)) { - Assertions.assertThat(primaryRequestID) - .describedAs("Should have primaryReqId").isNotEmpty(); - // } else { - } - if (!primaryRequestID.isEmpty() && !id_list[3].isEmpty()){ - Assertions.assertThat(id_list[3]) - .describedAs("PrimaryReqID should be common for these requests") - .isEqualTo(primaryRequestID); - } - Assertions.assertThat(id_list[2]).describedAs( - "FilesystemID should be same for requests with same filesystem") - .isEqualTo(fileSystemID); + if (!primaryRequestID.isEmpty() && !id_list[3].isEmpty()){ + Assertions.assertThat(id_list[3]) + .describedAs("PrimaryReqID should be common for these requests") + .isEqualTo(primaryRequestID); + } + if (!streamID.isEmpty()) { + Assertions.assertThat(id_list[4]) + .describedAs("Stream id should be common for these requests") + .isEqualTo(streamID); } - if (!streamID.isEmpty()) { - Assertions.assertThat(id_list[4]) - .describedAs("Stream id should be common for these requests") - .isEqualTo(streamID); - } - } - - @VisibleForTesting - public void setOperation(String operation) { - this.operation = operation; } private void validateBasicFormat(String[] id_list) { @@ -112,4 +91,14 @@ private void validateBasicFormat(String[] id_list) { .describedAs("Retry count incorrect") .isEqualTo(retryNum); } + + @Override + public void setOperation(String operation) { + this.operation = operation; + } + + @Override + public void updatePrimaryRequestID(String primaryRequestID) { + this.primaryRequestID = primaryRequestID; + } } From e18bab62749da71b56ecb6a4a3ec86983cdc7139 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Wed, 25 Nov 2020 11:15:26 +0530 Subject: [PATCH 45/77] fix matchers error --- .../hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java | 4 ++-- .../hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index 50788ec5f4c67..9a96f477607bd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -242,13 +242,13 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { doReturn(idempotencyRetOp).when(mockClient).deleteIdempotencyCheckOp(any()); when(mockClient.deletePath("/NonExistingPath", false, - null, any(TracingContext.class))).thenCallRealMethod(); + null, getTestTracingContext(fs, false))).thenCallRealMethod(); Assertions.assertThat(mockClient.deletePath( "/NonExistingPath", false, null, - getTestTracingContext(fs, true)) + getTestTracingContext(fs, false)) .getResult() .getStatusCode()) .describedAs("Idempotency check reports successful " diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java index b738b7aebb998..f8e8cd453b5f4 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java @@ -1302,7 +1302,7 @@ public void testSetOwnerForNonNamespaceEnabledAccount() throws Exception { TracingHeaderValidator tracingHeaderValidator = new TracingHeaderValidator( conf.getClientCorrelationID(), fs.getFileSystemID(), - AbfsOperationConstants.GETFILESTATUS, false, 1); + AbfsOperationConstants.GETFILESTATUS, false, 0); fs.registerListener(tracingHeaderValidator); FileStatus oldFileStatus = fs.getFileStatus(filePath); tracingHeaderValidator.setOperation(AbfsOperationConstants.SETOWNER); From 22c3a8411e623bac3e2909620a13e666184e85cf Mon Sep 17 00:00:00 2001 From: Sumangala Date: Wed, 25 Nov 2020 17:18:45 +0530 Subject: [PATCH 46/77] pr revw changes --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 27 ++++------- .../fs/azurebfs/AzureBlobFileSystem.java | 46 +++++++++---------- .../fs/azurebfs/AzureBlobFileSystemStore.java | 3 +- .../constants/AbfsOperationConstants.java | 28 ----------- .../constants/FileSystemConfigurations.java | 4 +- .../constants/HdfsOperationConstants.java | 28 +++++++++++ .../fs/azurebfs/services/AbfsInputStream.java | 4 +- .../azurebfs/utils/TracingContextFormat.java | 12 +---- .../azurebfs/AbstractAbfsIntegrationTest.java | 4 +- .../azurebfs/ITestAbfsNetworkStatistics.java | 2 - .../azurebfs/ITestAbfsReadWriteAndSeek.java | 10 ++-- .../fs/azurebfs/ITestAbfsStatistics.java | 2 - .../ITestAzureBlobFileSystemAppend.java | 8 ++-- .../ITestAzureBlobFileSystemAttributes.java | 6 +-- .../ITestAzureBlobFileSystemCheckAccess.java | 7 +-- .../ITestAzureBlobFileSystemCreate.java | 8 ++-- .../ITestAzureBlobFileSystemDelete.java | 5 +- .../azurebfs/ITestAzureBlobFileSystemE2E.java | 6 +-- .../ITestAzureBlobFileSystemFlush.java | 6 +-- .../ITestAzureBlobFileSystemListStatus.java | 4 +- .../azurebfs/ITestAzureBlobFilesystemAcl.java | 24 +++++----- .../fs/azurebfs/TestTracingContext.java | 15 ++---- .../services/TestAbfsInputStream.java | 5 +- .../services/TestAbfsOutputStream.java | 17 ++++--- .../utils/TracingHeaderValidator.java | 5 +- 25 files changed, 124 insertions(+), 162 deletions(-) delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperationConstants.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HdfsOperationConstants.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index a101e1186649b..08912a2b0e794 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -231,12 +231,6 @@ public class AbfsConfiguration{ DefaultValue = EMPTY_STRING) private String clientCorrelationID; - @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_TRACINGCONTEXT_FORMAT, - DefaultValue = 1)// DEFAULT_TRACINGCONTEXT_FORMATSIZE) - private int inputTracingContextFormat; - - private final TracingContextFormat tracingContextFormat; - @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ENABLE_DELEGATION_TOKEN, DefaultValue = DEFAULT_ENABLE_DELEGATION_TOKEN) private boolean enableDelegationToken; @@ -284,8 +278,6 @@ public AbfsConfiguration(final Configuration rawConfig, String accountName) field.set(this, validateBoolean(field)); } } - tracingContextFormat = - TracingContextFormat.valueOf(inputTracingContextFormat); } public Trilean getIsNamespaceEnabledAccount() { @@ -308,17 +300,6 @@ public String getClientCorrelationID() { return clientCorrelationID; } - /** - * Config to allow user to pick format of x-ms-client-request-id header - * 0 -> client-req-id - * 1 -> all IDs (default) - * 2 -> client-corr-id : client-req-id - * @return tracingContextFormat config - */ - public TracingContextFormat getTracingContextFormat() { - return tracingContextFormat; - } - /** * Appends an account name to a configuration key yielding the * account-specific form. @@ -659,6 +640,14 @@ public DelegatingSSLSocketFactory.SSLChannelMode getPreferredSSLFactoryOption() return getEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, DEFAULT_FS_AZURE_SSL_CHANNEL_MODE); } + /** + * Enum config to allow user to pick format of x-ms-client-request-id header + * @return tracingContextFormat config if valid, else default ALL_ID_FORMAT + */ + public TracingContextFormat getTracingContextFormat() { + return getEnum(FS_AZURE_TRACINGCONTEXT_FORMAT, TracingContextFormat.ALL_ID_FORMAT); + } + public AuthType getAuthType(String accountName) { return getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 9c421c495c9fc..5bc49de514108 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -62,7 +62,7 @@ import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; @@ -192,7 +192,7 @@ public FSDataInputStream open(final Path path, final int bufferSize) throws IOEx try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.OPEN, tracingContextFormat, + fileSystemID, HdfsOperationConstants.OPEN, tracingContextFormat, listener); InputStream inputStream = abfsStore.openFileForRead(qualifiedPath, statistics, tracingContext); return new FSDataInputStream(inputStream); @@ -218,7 +218,7 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.CREATE, overwrite, + fileSystemID, HdfsOperationConstants.CREATE, overwrite, tracingContextFormat, listener); OutputStream outputStream = abfsStore.createFile(qualifiedPath, statistics, overwrite, permission == null ? FsPermission.getFileDefault() : permission, FsPermission.getUMask(getConf()), @@ -286,7 +286,7 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.APPEND, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.APPEND, tracingContextFormat, listener); OutputStream outputStream = abfsStore.openFileForWrite(qualifiedPath, statistics, false, tracingContext); return new FSDataOutputStream(outputStream, statistics); @@ -329,7 +329,7 @@ public boolean rename(final Path src, final Path dst) throws IOException { // Non-HNS account need to check dst status on driver side. TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.RENAME, true, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.RENAME, true, tracingContextFormat, listener); if (!abfsStore.getIsNamespaceEnabled(tracingContext) && dstFileStatus == null) { dstFileStatus = tryGetFileStatus(qualifiedDstPath); } @@ -382,7 +382,7 @@ public boolean delete(final Path f, final boolean recursive) throws IOException try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.DELETE, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.DELETE, tracingContextFormat, listener); abfsStore.delete(qualifiedPath, recursive, tracingContext); return true; } catch (AzureBlobFileSystemException ex) { @@ -401,7 +401,7 @@ public FileStatus[] listStatus(final Path f) throws IOException { try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.LISTSTATUS, true, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.LISTSTATUS, true, tracingContextFormat, listener); FileStatus[] result = abfsStore.listStatus(qualifiedPath, tracingContext); return result; } catch (AzureBlobFileSystemException ex) { @@ -470,7 +470,7 @@ public boolean mkdirs(final Path f, final FsPermission permission) throws IOExce try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.MKDIR, false, + fileSystemID, HdfsOperationConstants.MKDIR, false, tracingContextFormat, listener); abfsStore.createDirectory(qualifiedPath, permission == null ? FsPermission.getDirDefault() : permission, FsPermission.getUMask(getConf()), tracingContext); @@ -503,7 +503,7 @@ public FileStatus getFileStatus(final Path f) throws IOException { try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.GETFILESTATUS, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.GET_FILESTATUS, tracingContextFormat, listener); return abfsStore.getFileStatus(qualifiedPath, tracingContext); } catch(AzureBlobFileSystemException ex) { checkException(f, ex); @@ -679,7 +679,7 @@ public void setOwner(final Path path, final String owner, final String group) LOG.debug( "AzureBlobFileSystem.setOwner path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.SETOWNER, true, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.SET_OWNER, true, tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { super.setOwner(path, owner, group); @@ -725,7 +725,7 @@ public void setXAttr(final Path path, final String name, final byte[] value, fin try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.SETATTR, true, + fileSystemID, HdfsOperationConstants.SET_ATTR, true, tracingContextFormat, listener); Hashtable properties = abfsStore.getPathStatus(qualifiedPath, tracingContext); String xAttrName = ensureValidAttributeName(name); @@ -764,7 +764,7 @@ public byte[] getXAttr(final Path path, final String name) byte[] value = null; try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.GETATTR, true, + fileSystemID, HdfsOperationConstants.GET_ATTR, true, tracingContextFormat, listener); Hashtable properties = abfsStore.getPathStatus(qualifiedPath, tracingContext); String xAttrName = ensureValidAttributeName(name); @@ -794,7 +794,7 @@ public void setPermission(final Path path, final FsPermission permission) throws IOException { LOG.debug("AzureBlobFileSystem.setPermission path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.PERMISSION, true, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.SET_PERMISSION, true, tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { super.setPermission(path, permission); @@ -830,7 +830,7 @@ public void modifyAclEntries(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.modifyAclEntries path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.MODIFYACL,true, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.MODIFY_ACL,true, tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -865,7 +865,7 @@ public void removeAclEntries(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.removeAclEntries path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.REMOVEACLENTRIES, true, + fileSystemID, HdfsOperationConstants.REMOVE_ACL_ENTRIES, true, tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { @@ -897,7 +897,7 @@ public void removeAclEntries(final Path path, final List aclSpec) public void removeDefaultAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeDefaultAcl path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.REMOVEDEFAULTACL, true, + fileSystemID, HdfsOperationConstants.REMOVE_DEFAULT_ACL, true, tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { @@ -927,7 +927,7 @@ public void removeDefaultAcl(final Path path) throws IOException { public void removeAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeAcl path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.REMOVEACL, true, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.REMOVE_ACL, true, tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -959,7 +959,7 @@ public void setAcl(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.setAcl path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.SETACL, true, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.SET_ACL, true, tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -991,7 +991,7 @@ public void setAcl(final Path path, final List aclSpec) public AclStatus getAclStatus(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.getAclStatus path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.GETACLSTATUS, true, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.GET_ACL_STATUS, true, tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -1027,7 +1027,7 @@ public void access(final Path path, final FsAction mode) throws IOException { Path qualifiedPath = makeQualified(path); try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.ACCESS, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.ACCESS, tracingContextFormat, listener); this.abfsStore.access(qualifiedPath, mode, tracingContext); } catch (AzureBlobFileSystemException ex) { checkCheckAccessException(path, ex); @@ -1062,7 +1062,7 @@ private boolean fileSystemExists() throws IOException { "AzureBlobFileSystem.fileSystemExists uri: {}", uri); try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.GETFILESTATUS, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.GET_FILESTATUS, tracingContextFormat, listener); abfsStore.getFilesystemProperties(tracingContext); } catch (AzureBlobFileSystemException ex) { try { @@ -1083,7 +1083,7 @@ private void createFileSystem() throws IOException { "AzureBlobFileSystem.createFileSystem uri: {}", uri); try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.CREATEFILESYSTEM, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.CREATE_FILESYSTEM, tracingContextFormat, listener); abfsStore.createFilesystem(tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(null, ex); @@ -1335,7 +1335,7 @@ public boolean hasPathCapability(final Path path, final String capability) return true; case CommonPathCapabilities.FS_ACLS: TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, AbfsOperationConstants.PATH, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.HAS_PATH_CAPABILITY, tracingContextFormat, listener); return getIsNamespaceEnabled(tracingContext); default: return super.hasPathCapability(p, capability); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 5cc724e32a368..ae25418792b0c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -1029,7 +1029,8 @@ public void setOwner(final Path path, final String owner, final String group, final AbfsRestOperation op = client.setOwner(getRelativePath(path), transformedOwner, - transformedGroup, tracingContext); + transformedGroup, + tracingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperationConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperationConstants.java deleted file mode 100644 index 4587f287d1e84..0000000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsOperationConstants.java +++ /dev/null @@ -1,28 +0,0 @@ -package org.apache.hadoop.fs.azurebfs.constants; - -public final class AbfsOperationConstants { - public static final String ACCESS = "AS"; - public static final String APPEND = "AP"; - public static final String CREATE = "CR"; - public static final String CREATEFILESYSTEM = "CF"; - public static final String DELETE = "DL"; - public static final String GETACLSTATUS = "GS"; - public static final String GETATTR = "GA"; - public static final String GETFILESTATUS = "FS"; - public static final String LISTSTATUS = "LS"; - public static final String MKDIR = "MK"; - public static final String MODIFYACL = "MA"; - public static final String OPEN = "OP"; - public static final String PATH = "PA"; - public static final String PERMISSION = "PR"; - public static final String READ = "RD"; - public static final String REMOVEACL = "RA"; - public static final String REMOVEACLENTRIES = "RE"; - public static final String REMOVEDEFAULTACL = "DA"; - public static final String RENAME = "RN"; - public static final String SETATTR = "ST"; - public static final String SETOWNER = "SO"; - public static final String SETACL = "SA"; - public static final String TESTOP = "TS"; - public static final String WRITE = "WR"; -} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index 6af8a828179d5..01fa6b7d754f2 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -20,6 +20,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; @@ -81,9 +82,6 @@ public final class FileSystemConfigurations { public static final DelegatingSSLSocketFactory.SSLChannelMode DEFAULT_FS_AZURE_SSL_CHANNEL_MODE = DelegatingSSLSocketFactory.SSLChannelMode.Default; - public static final int DEFAULT_TRACINGCONTEXT_FORMAT = - TracingContextFormat.ALL_ID_FORMAT.ordinal(); - //ordinal has associated risks - check public static final boolean DEFAULT_ENABLE_DELEGATION_TOKEN = false; public static final boolean DEFAULT_ENABLE_HTTPS = true; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HdfsOperationConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HdfsOperationConstants.java new file mode 100644 index 0000000000000..9029796190076 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HdfsOperationConstants.java @@ -0,0 +1,28 @@ +package org.apache.hadoop.fs.azurebfs.constants; + +public final class HdfsOperationConstants { + public static final String ACCESS = "AS"; + public static final String APPEND = "AP"; + public static final String CREATE = "CR"; + public static final String CREATE_FILESYSTEM = "CF"; + public static final String DELETE = "DL"; + public static final String GET_ACL_STATUS = "GA"; + public static final String GET_ATTR = "GR"; + public static final String GET_FILESTATUS = "GF"; + public static final String LISTSTATUS = "LS"; + public static final String MKDIR = "MK"; + public static final String MODIFY_ACL = "MA"; + public static final String OPEN = "OP"; + public static final String HAS_PATH_CAPABILITY = "PC"; + public static final String SET_PERMISSION = "SP"; + public static final String READ = "RE"; + public static final String REMOVE_ACL = "RA"; + public static final String REMOVE_ACL_ENTRIES = "RT"; + public static final String REMOVE_DEFAULT_ACL = "RD"; + public static final String RENAME = "RN"; + public static final String SET_ATTR = "SR"; + public static final String SET_OWNER = "SO"; + public static final String SET_ACL = "SA"; + public static final String TEST_OP = "TS"; + public static final String WRITE = "WR"; +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 062da3c975eca..aae24963b4c15 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -28,7 +28,7 @@ import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.Listener; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.slf4j.Logger; @@ -103,7 +103,7 @@ public AbfsInputStream( this.streamStatistics = abfsInputStreamContext.getStreamStatistics(); this.inputStreamID = getInputStreamID(); this.tracingContext = new TracingContext(tracingContext); - this.tracingContext.setOperation(AbfsOperationConstants.READ); + this.tracingContext.setOperation(HdfsOperationConstants.READ); this.tracingContext.setStreamID(inputStreamID); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java index 22c91bdf86aa7..b98302b96ecd8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java @@ -4,14 +4,4 @@ public enum TracingContextFormat { SINGLE_ID_FORMAT, ALL_ID_FORMAT, TWO_ID_FORMAT; - - //save list as calling values() is expensive - private static final TracingContextFormat[] formatValues = - TracingContextFormat.values(); - private static final int FORMAT_COUNT = formatValues.length; - - public static TracingContextFormat valueOf(int number) { - return number < FORMAT_COUNT? formatValues[number] : - TracingContextFormat.ALL_ID_FORMAT; - } -} +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index 665263baf18fa..ddcbb0bc6a712 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -25,7 +25,7 @@ import java.util.UUID; import java.util.concurrent.Callable; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import org.junit.After; @@ -159,7 +159,7 @@ public TracingContext getTestTracingContext(AzureBlobFileSystem fs, TracingContextFormat format = abfsConf == null? TracingContextFormat.ALL_ID_FORMAT : abfsConf.getTracingContextFormat(); - return new TracingContext(corrID, fsID, AbfsOperationConstants.TESTOP, isCont, + return new TracingContext(corrID, fsID, HdfsOperationConstants.TEST_OP, isCont, format, null); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java index 82af8fbd56d70..c2dbe937b812b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java @@ -21,8 +21,6 @@ import java.io.IOException; import java.util.Map; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.junit.Test; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index 6a7c3ac11ce42..53c3aaa8a2306 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -21,7 +21,7 @@ import java.util.Arrays; import java.util.Random; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; @@ -78,7 +78,7 @@ public void testReadAheadRequestID() throws java.io.IOException { try (FSDataOutputStream stream = fs.create(TEST_PATH)) { ((AbfsOutputStream)stream.getWrappedStream()).registerListener(new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), - fs.getFileSystemID(), AbfsOperationConstants.CREATE, + fs.getFileSystemID(), HdfsOperationConstants.CREATE, false, 0)); stream.write(b); } @@ -87,12 +87,12 @@ public void testReadAheadRequestID() throws java.io.IOException { int result; TracingHeaderValidator tracingHeaderValidator = new TracingHeaderValidator( abfsConfiguration.getClientCorrelationID(), fs.getFileSystemID(), - AbfsOperationConstants.OPEN, false, 0); + HdfsOperationConstants.OPEN, false, 0); fs.registerListener(tracingHeaderValidator); try (FSDataInputStream inputStream = fs.open(TEST_PATH)) { ((AbfsInputStream)inputStream.getWrappedStream()).registerListener(new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), - fs.getFileSystemID(), AbfsOperationConstants.READ, + fs.getFileSystemID(), HdfsOperationConstants.READ, false, 0)); result = inputStream.read(readBuffer, 0, bufferSize*4); } @@ -117,7 +117,7 @@ private void testReadWriteAndSeek(int bufferSize) throws Exception { try (FSDataInputStream inputStream = fs.open(TEST_PATH)) { ((AbfsInputStream) inputStream.getWrappedStream()).registerListener(new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), - fs.getFileSystemID(), AbfsOperationConstants.READ, true, + fs.getFileSystemID(), HdfsOperationConstants.READ, true, 0, ((AbfsInputStream) inputStream.getWrappedStream()).getStreamID())); inputStream.seek(bufferSize); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java index 21cb584498a9b..b08bc108abad9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java @@ -21,8 +21,6 @@ import java.io.IOException; import java.util.Map; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; import org.apache.hadoop.fs.Path; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java index 63102b219cec5..a309477943ca8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java @@ -22,9 +22,7 @@ import java.io.IOException; import java.util.Random; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; @@ -73,7 +71,7 @@ public void testAppendFileAfterDelete() throws Exception { fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), - AbfsOperationConstants.APPEND, false, 0)); + HdfsOperationConstants.APPEND, false, 0)); fs.append(filePath); } @@ -91,7 +89,7 @@ public void testTracingForAppend() throws IOException { fs.create(TEST_FILE_PATH); fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() .getAbfsConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), AbfsOperationConstants.APPEND, + fs.getFileSystemID(), HdfsOperationConstants.APPEND, false, 0)); FSDataOutputStream in = fs.append(TEST_FILE_PATH, 10); byte[] buf = new byte[100]; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java index 1763f87c7e6a5..a02a29381fd6f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.XAttrSetFlag; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Assume; import org.junit.Test; @@ -59,10 +59,10 @@ public void testSetGetXAttr() throws Exception { // after setting the xAttr on the file, the value should be retrievable fs.registerListener(new TracingHeaderValidator(conf.getClientCorrelationID(), - fs.getFileSystemID(), AbfsOperationConstants.SETATTR, + fs.getFileSystemID(), HdfsOperationConstants.SET_ATTR, true, 0)); fs.setXAttr(testFile, attributeName1, attributeValue1); - fs.setListenerOperation(AbfsOperationConstants.GETATTR); + fs.setListenerOperation(HdfsOperationConstants.GET_ATTR); assertArrayEquals(attributeValue1, fs.getXAttr(testFile, attributeName1)); fs.registerListener(null); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java index 15c53158362b7..08f9e0d6c7fca 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java @@ -22,7 +22,7 @@ import java.lang.reflect.Field; import java.util.List; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.collect.Lists; @@ -191,7 +191,8 @@ public void testCheckAccessForAccountWithoutNS() throws Exception { // acts as noop AzureBlobFileSystemStore mockAbfsStore = Mockito.mock(AzureBlobFileSystemStore.class); - Mockito.when(mockAbfsStore.getIsNamespaceEnabled()).thenReturn(true); + Mockito.when(mockAbfsStore.getIsNamespaceEnabled(getTestTracingContext(getFileSystem(), + false))).thenReturn(true); Field abfsStoreField = AzureBlobFileSystem.class.getDeclaredField( "abfsStore"); abfsStoreField.setAccessible(true); @@ -223,7 +224,7 @@ public void testFsActionEXECUTE() throws Exception { AzureBlobFileSystem fs = (AzureBlobFileSystem)testUserFs; fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), - AbfsOperationConstants.ACCESS, false, 0)); + HdfsOperationConstants.ACCESS, false, 0)); assertAccessible(testFilePath, FsAction.EXECUTE); fs.registerListener(null); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 504a052749644..b23335a5a41eb 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -25,7 +25,7 @@ import java.util.EnumSet; import java.util.UUID; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; @@ -100,7 +100,7 @@ public void testCreateNonRecursive() throws Exception { } catch (FileNotFoundException expected) { } fs.registerListener(new TracingHeaderValidator(conf.getClientCorrelationID(), - fs.getFileSystemID(), AbfsOperationConstants.MKDIR, + fs.getFileSystemID(), HdfsOperationConstants.MKDIR, false, 0)); fs.mkdirs(TEST_FOLDER_PATH); fs.registerListener(null); @@ -273,7 +273,7 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) // Case 2: Not Overwrite - File pre-exists fs.registerListener(new TracingHeaderValidator(conf.getClientCorrelationID(), - fs.getFileSystemID(), AbfsOperationConstants.CREATE, false, 0)); + fs.getFileSystemID(), HdfsOperationConstants.CREATE, false, 0)); intercept(FileAlreadyExistsException.class, () -> fs.create(nonOverwriteFile, false)); fs.registerListener(null); @@ -303,7 +303,7 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) // Case 4: Overwrite - File pre-exists fs.registerListener(new TracingHeaderValidator(conf.getClientCorrelationID(), - fs.getFileSystemID(), AbfsOperationConstants.CREATE, true, + fs.getFileSystemID(), HdfsOperationConstants.CREATE, true, 0)); fs.create(overwriteFilePath, true); fs.registerListener(null); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index 9a96f477607bd..0ed0708c912cc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -26,8 +26,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.assertj.core.api.Assertions; import org.junit.Assume; @@ -150,7 +149,7 @@ public Void call() throws Exception { Path dir = new Path("/test"); fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), - AbfsOperationConstants.DELETE, false, 0)); + HdfsOperationConstants.DELETE, false, 0)); // first try a non-recursive delete, expect failure intercept(FileAlreadyExistsException.class, () -> fs.delete(dir, false)); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java index 399ea50935453..86d107e568d21 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java @@ -23,7 +23,7 @@ import java.util.Arrays; import java.util.Random; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; @@ -144,11 +144,11 @@ public void testWriteWithBufferOffset() throws Exception { final byte[] r = new byte[TEST_DEFAULT_READ_BUFFER_SIZE]; AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); TracingHeaderValidator tracingHeaderValidator = new TracingHeaderValidator - (conf.getClientCorrelationID(), fs.getFileSystemID(), AbfsOperationConstants.OPEN, + (conf.getClientCorrelationID(), fs.getFileSystemID(), HdfsOperationConstants.OPEN, false, 0); fs.registerListener(tracingHeaderValidator); FSDataInputStream inputStream = fs.open(testFilePath, TEST_DEFAULT_BUFFER_SIZE); - tracingHeaderValidator.setOperation(AbfsOperationConstants.READ); + tracingHeaderValidator.setOperation(HdfsOperationConstants.READ); ((AbfsInputStream)inputStream.getWrappedStream()).registerListener(tracingHeaderValidator); int result = inputStream.read(r); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index 138217b6ebcdf..ebc52b594eb51 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -31,8 +31,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.StreamCapabilities; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; @@ -47,7 +46,6 @@ import org.apache.hadoop.fs.Path; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_APPEND_BLOB_KEY; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI; /** * Test flush operation. @@ -319,7 +317,7 @@ public void testTracingHeaderForAppendBlob() throws Exception { ((AbfsOutputStream)out.getWrappedStream()).registerListener(new TracingHeaderValidator(fs.getAbfsStore().getAbfsConfiguration() .getClientCorrelationID(), fs.getFileSystemID(), - AbfsOperationConstants.CREATE, false, 0)); + HdfsOperationConstants.CREATE, false, 0)); out.write(buf); out.hsync(); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java index cad6e4be58441..2814faa59bb23 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java @@ -29,7 +29,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; @@ -89,7 +89,7 @@ public Void call() throws Exception { AbfsConfiguration conf = fs1.getAbfsStore().getAbfsConfiguration(); fs1.registerListener(new TracingHeaderValidator( conf.getClientCorrelationID(), - fs1.getFileSystemID(), AbfsOperationConstants.LISTSTATUS, true, + fs1.getFileSystemID(), HdfsOperationConstants.LISTSTATUS, true, 0)); FileStatus[] files = fs1.listStatus(new Path("/")); assertEquals(TEST_FILES_NUMBER, files.length /* user directory */); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java index bee1107326d8e..61f5f42f58431 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java @@ -24,7 +24,7 @@ import java.util.List; import java.util.UUID; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Assume; import org.junit.Ignore; @@ -1221,7 +1221,7 @@ public void testDefaultAclRenamedFile() throws Exception { fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), - AbfsOperationConstants.RENAME, true, 0)); + HdfsOperationConstants.RENAME, true, 0)); fs.rename(filePath, renamedFilePath); fs.registerListener(null); AclEntry[] expected = new AclEntry[] { }; @@ -1266,27 +1266,27 @@ public void testEnsureAclOperationWorksForRoot() throws Exception { fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() .getAbfsConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), AbfsOperationConstants.SETACL, true, 0)); + fs.getFileSystemID(), HdfsOperationConstants.SET_ACL, true, 0)); fs.setAcl(rootPath, aclSpec1); - fs.setListenerOperation(AbfsOperationConstants.GETACLSTATUS); + fs.setListenerOperation(HdfsOperationConstants.GET_ACL_STATUS); fs.getAclStatus(rootPath); - fs.setListenerOperation(AbfsOperationConstants.SETOWNER); + fs.setListenerOperation(HdfsOperationConstants.SET_OWNER); fs.setOwner(rootPath, TEST_OWNER, TEST_GROUP); - fs.setListenerOperation(AbfsOperationConstants.PERMISSION); + fs.setListenerOperation(HdfsOperationConstants.SET_PERMISSION); fs.setPermission(rootPath, new FsPermission("777")); List aclSpec2 = Lists.newArrayList( aclEntry(DEFAULT, USER, FOO, ALL), aclEntry(ACCESS, USER, BAR, ALL)); - fs.setListenerOperation(AbfsOperationConstants.MODIFYACL); + fs.setListenerOperation(HdfsOperationConstants.MODIFY_ACL); fs.modifyAclEntries(rootPath, aclSpec2); - fs.setListenerOperation(AbfsOperationConstants.REMOVEACLENTRIES); + fs.setListenerOperation(HdfsOperationConstants.REMOVE_ACL_ENTRIES); fs.removeAclEntries(rootPath, aclSpec2); - fs.setListenerOperation(AbfsOperationConstants.REMOVEDEFAULTACL); + fs.setListenerOperation(HdfsOperationConstants.REMOVE_DEFAULT_ACL); fs.removeDefaultAcl(rootPath); - fs.setListenerOperation(AbfsOperationConstants.REMOVEACL); + fs.setListenerOperation(HdfsOperationConstants.REMOVE_ACL); fs.removeAcl(rootPath); } @@ -1302,10 +1302,10 @@ public void testSetOwnerForNonNamespaceEnabledAccount() throws Exception { TracingHeaderValidator tracingHeaderValidator = new TracingHeaderValidator( conf.getClientCorrelationID(), fs.getFileSystemID(), - AbfsOperationConstants.GETFILESTATUS, false, 0); + HdfsOperationConstants.GET_FILESTATUS, false, 0); fs.registerListener(tracingHeaderValidator); FileStatus oldFileStatus = fs.getFileStatus(filePath); - tracingHeaderValidator.setOperation(AbfsOperationConstants.SETOWNER); + tracingHeaderValidator.setOperation(HdfsOperationConstants.SET_OWNER); fs.setOwner(filePath, TEST_OWNER, TEST_GROUP); fs.registerListener(null); FileStatus newFileStatus = fs.getFileStatus(filePath); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index f793979bd1b18..7f2d71ff4bbb7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -1,17 +1,11 @@ package org.apache.hadoop.fs.azurebfs; import com.google.common.base.Preconditions; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonPathCapabilities; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; -import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; import org.apache.hadoop.fs.azurebfs.enums.Trilean; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; @@ -27,7 +21,6 @@ import java.lang.reflect.Method; import java.util.HashMap; import java.util.Map; -import java.util.Random; public class TestTracingContext extends AbstractAbfsIntegrationTest { private static final String[] CLIENT_CORRELATIONID_LIST = { @@ -57,7 +50,7 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, boolean includeInHeader) throws IOException { AzureBlobFileSystem fs = getFileSystem(); TracingContext tracingContext = new TracingContext(clientCorrelationId, - fs.getFileSystemID(), AbfsOperationConstants.TESTOP, + fs.getFileSystemID(), HdfsOperationConstants.TEST_OP, TracingContextFormat.ALL_ID_FORMAT,null); String correlationID = tracingContext.toString().split(":")[0]; if (includeInHeader) { @@ -161,10 +154,10 @@ public void testExternalOps() throws Exception { AzureBlobFileSystem fs = getFileSystem(); fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), - AbfsOperationConstants.ACCESS, false, 0)); + HdfsOperationConstants.ACCESS, false, 0)); fs.access(new Path("/"), FsAction.READ); - fs.setListenerOperation(AbfsOperationConstants.PATH); + fs.setListenerOperation(HdfsOperationConstants.HAS_PATH_CAPABILITY); //unset namespaceEnabled config to call getAcl -> test tracing header fs.getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN); fs.hasPathCapability(new Path("/"), CommonPathCapabilities.FS_ACLS); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index 7ab5194fe6094..a528d6fd6365f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -203,7 +203,7 @@ public void testFailedReadAheadEviction() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class)); + any(String.class), any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testFailedReadAheadEviction.txt"); @@ -218,7 +218,8 @@ public void testFailedReadAheadEviction() throws Exception { // at java.util.Stack.peek(Stack.java:102) // at java.util.Stack.pop(Stack.java:84) // at org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.queueReadAhead - ReadBufferManager.getBufferManager().queueReadAhead(inputStream, 0, ONE_KB); + ReadBufferManager.getBufferManager().queueReadAhead(inputStream, 0, + ONE_KB, getTestTracingContext(getFileSystem(), true)); } /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index 2f530971ecceb..b255b299a051f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -23,8 +23,7 @@ import java.util.HashSet; import java.util.Random; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Test; @@ -85,7 +84,7 @@ public void verifyShortWriteRequest() throws Exception { conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperationConstants.WRITE, + "test-fs-id", HdfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), null); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); @@ -144,7 +143,7 @@ public void verifyWriteRequest() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperationConstants.WRITE, + "test-fs-id", HdfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), null); when(client.getAbfsPerfTracker()).thenReturn(tracker); @@ -213,7 +212,7 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperationConstants.WRITE, + "test-fs-id", HdfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), null); when(client.getAbfsPerfTracker()).thenReturn(tracker); @@ -296,7 +295,7 @@ public void verifyWriteRequestOfBufferSize() throws Exception { populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), new TracingContext(abfsConf.getClientCorrelationID(), "test-fs-id", - AbfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), + HdfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), null)); final byte[] b = new byte[BUFFER_SIZE]; @@ -351,7 +350,7 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, true), new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperationConstants.OPEN, + "test-fs-id", HdfsOperationConstants.OPEN, abfsConf.getTracingContextFormat(), null)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -396,7 +395,7 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperationConstants.WRITE, + "test-fs-id", HdfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), null); when(client.getAbfsPerfTracker()).thenReturn(tracker); @@ -463,7 +462,7 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", AbfsOperationConstants.WRITE, + "test-fs-id", HdfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), null); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java index df8397493652c..69849bc04717c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java @@ -1,7 +1,6 @@ package org.apache.hadoop.fs.azurebfs.utils; -import com.google.common.annotations.VisibleForTesting; -import org.apache.hadoop.fs.azurebfs.constants.AbfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.assertj.core.api.Assertions; public class TracingHeaderValidator implements Listener { @@ -80,7 +79,7 @@ private void validateBasicFormat(String[] id_list) { .matches(GUID_PATTERN); Assertions.assertThat(id_list[2]).describedAs("Filesystem ID incorrect") .isEqualTo(fileSystemID); - if (needsPrimaryRequestID && !operation.equals(AbfsOperationConstants.READ)) { + if (needsPrimaryRequestID && !operation.equals(HdfsOperationConstants.READ)) { Assertions.assertThat(id_list[3]).describedAs("should have primaryReqId") .isNotEmpty(); } From 3a3a40eab3ff65ef6748c973f0df37e55d2af0ca Mon Sep 17 00:00:00 2001 From: Sumangala Date: Thu, 26 Nov 2020 08:19:03 +0530 Subject: [PATCH 47/77] fix some test failures --- .../azurebfs/services/AbfsHttpOperation.java | 2 +- .../azurebfs/services/AbfsOutputStream.java | 1 + .../hadoop/fs/azurebfs/utils/Listener.java | 2 +- .../fs/azurebfs/utils/TracingContext.java | 3 ++- .../hadoop/fs/azurebfs/ITestAbfsClient.java | 2 +- .../fs/azurebfs/TestTracingContext.java | 2 +- .../services/TestAbfsOutputStream.java | 2 ++ .../utils/TracingHeaderValidator.java | 27 ++++++++++++++++--- 8 files changed, 33 insertions(+), 8 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index 0f69139a1cef0..2a5def3a5ce81 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -151,7 +151,7 @@ public String getResponseHeader(String httpHeader) { return connection.getHeaderField(httpHeader); } - @com.google.common.annotations.VisibleForTesting + @org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting public String getRequestHeader(String httpHeader) { return connection.getRequestProperties().get(httpHeader).toString(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index c99dffbb5a47e..b7ea2ac43aa6e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -478,6 +478,7 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset, AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, isClose, cachedSasToken.get(), new TracingContext(tracingContext)); + System.out.println(op + "--"); cachedSasToken.update(op.getSasToken()); perfInfo.registerResult(op.getResult()).registerSuccess(true); } catch (AzureBlobFileSystemException ex) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java index d9c418205a221..4f74c7f6da5c0 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java @@ -1,7 +1,7 @@ package org.apache.hadoop.fs.azurebfs.utils; public interface Listener { - void callTracingHeaderValidator(String header); + void callTracingHeaderValidator(String header, TracingContextFormat format); void updatePrimaryRequestID(String primaryRequestID); Listener getClone(); void setOperation(String operation); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index 89744fdc15dee..d6269d5546f37 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -114,6 +114,7 @@ public void setListener(Listener listener) { public String toString() { String header = clientRequestID; //case 0, no IDs for correlation + System.out.println("---TC--- " + format); switch (format) { case ALL_ID_FORMAT: header = clientCorrelationID + ":" + header + ":" + fileSystemID + ":" + primaryRequestID @@ -122,7 +123,7 @@ public String toString() { case TWO_ID_FORMAT: header = clientCorrelationID + ":" + header; } if (listener != null) { - listener.callTracingHeaderValidator(header); + listener.callTracingHeaderValidator(header, format); } return header; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java index a83c9bace44a0..9963a7eb39cce 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java @@ -32,7 +32,7 @@ import org.junit.Ignore; import org.junit.Test; -import com.google.common.base.Preconditions; +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 7f2d71ff4bbb7..3d2ae0001dc52 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -1,6 +1,6 @@ package org.apache.hadoop.fs.azurebfs; -import com.google.common.base.Preconditions; +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.fs.CommonPathCapabilities; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index b255b299a051f..29dee96334f09 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -471,6 +471,8 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), any(TracingContext.class))).thenReturn(op); + System.out.println(op + "op++"); + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), tracingContext); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java index 69849bc04717c..498a89d1e417f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java @@ -12,11 +12,14 @@ public class TracingHeaderValidator implements Listener { String operation; int retryNum; String GUID_PATTERN = "[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}"; + private TracingContextFormat format; // client-req-id as per docs: ^[{(]?[0-9a-f]{8}[-]?([0-9a-f]{4}[-]?) // {3}[0-9a-f]{12}[)}]?$ @Override - public void callTracingHeaderValidator(String tracingContextHeader) { + public void callTracingHeaderValidator(String tracingContextHeader, + TracingContextFormat format) { + this.format = format; validateTracingHeader(tracingContextHeader); } @@ -50,6 +53,8 @@ public TracingHeaderValidator(String clientCorrelationID, private void validateTracingHeader(String tracingContextHeader) { String[] id_list = tracingContextHeader.split(":"); validateBasicFormat(id_list); + if (format != TracingContextFormat.ALL_ID_FORMAT) + return; if (!primaryRequestID.isEmpty() && !id_list[3].isEmpty()){ Assertions.assertThat(id_list[3]) .describedAs("PrimaryReqID should be common for these requests") @@ -63,8 +68,20 @@ private void validateTracingHeader(String tracingContextHeader) { } private void validateBasicFormat(String[] id_list) { - Assertions.assertThat(id_list) - .describedAs("header should have 7 elements").hasSize(7); + if (format == TracingContextFormat.ALL_ID_FORMAT) { + Assertions.assertThat(id_list) + .describedAs("header should have 7 elements").hasSize(7); + } else if (format == TracingContextFormat.TWO_ID_FORMAT) { + Assertions.assertThat(id_list) + .describedAs("header should have 2 elements").hasSize(2); + } else { + Assertions.assertThat(id_list) + .describedAs("header should have 1 element").hasSize(1); + Assertions.assertThat(id_list[0]) + .describedAs("Client request ID is a guid") + .matches(GUID_PATTERN); + return; + } if(clientCorrelationID.matches("[a-zA-Z0-9-]*")) { Assertions.assertThat(id_list[0]).describedAs("Correlation ID should match config") @@ -77,6 +94,10 @@ private void validateBasicFormat(String[] id_list) { Assertions.assertThat(id_list[1]) .describedAs("Client request ID is a guid") .matches(GUID_PATTERN); + + if (format != TracingContextFormat.ALL_ID_FORMAT) + return; + Assertions.assertThat(id_list[2]).describedAs("Filesystem ID incorrect") .isEqualTo(fileSystemID); if (needsPrimaryRequestID && !operation.equals(HdfsOperationConstants.READ)) { From 472d0903972128c0634702712ab4d7fb84442b3d Mon Sep 17 00:00:00 2001 From: Sumangala Date: Fri, 27 Nov 2020 09:36:06 +0530 Subject: [PATCH 48/77] code cleanup --- .../azurebfs/services/AbfsOutputStream.java | 1 - .../fs/azurebfs/utils/TracingContext.java | 1 - .../azurebfs/utils/TracingContextFormat.java | 2 +- .../azurebfs/AbstractAbfsIntegrationTest.java | 17 ++++++----- .../azurebfs/ITestAbfsReadWriteAndSeek.java | 1 - .../fs/azurebfs/ITestAbfsStatistics.java | 1 - .../ITestAzureBlobFileSystemAppend.java | 7 +---- .../ITestAzureBlobFileSystemDelete.java | 1 - .../azurebfs/ITestAzureBlobFileSystemE2E.java | 7 ----- .../ITestAzureBlobFileSystemListStatus.java | 13 +++------ .../azurebfs/ITestAzureBlobFilesystemAcl.java | 1 - .../fs/azurebfs/TestTracingContext.java | 29 +++++-------------- .../services/TestAbfsInputStream.java | 21 +++++--------- .../services/TestAbfsOutputStream.java | 24 +++++++-------- 14 files changed, 39 insertions(+), 87 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index b7ea2ac43aa6e..c99dffbb5a47e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -478,7 +478,6 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset, AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, isClose, cachedSasToken.get(), new TracingContext(tracingContext)); - System.out.println(op + "--"); cachedSasToken.update(op.getSasToken()); perfInfo.registerResult(op.getResult()).registerSuccess(true); } catch (AzureBlobFileSystemException ex) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index d6269d5546f37..91fece4718273 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -114,7 +114,6 @@ public void setListener(Listener listener) { public String toString() { String header = clientRequestID; //case 0, no IDs for correlation - System.out.println("---TC--- " + format); switch (format) { case ALL_ID_FORMAT: header = clientCorrelationID + ":" + header + ":" + fileSystemID + ":" + primaryRequestID diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java index b98302b96ecd8..12e6560436aea 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java @@ -4,4 +4,4 @@ public enum TracingContextFormat { SINGLE_ID_FORMAT, ALL_ID_FORMAT, TWO_ID_FORMAT; -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index ddcbb0bc6a712..a72715345b26f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -148,16 +148,17 @@ protected boolean getIsNamespaceEnabled(AzureBlobFileSystem fs) throws IOExcepti public TracingContext getTestTracingContext(AzureBlobFileSystem fs, boolean isCont) { - if (fs == null) { - return new TracingContext("test-corr-id", "test-fs-id", "TS", false, - TracingContextFormat.ALL_ID_FORMAT, null); - } +// if (fs == null) { +// return new TracingContext("test-corr-id", "test-fs-id", "TS", false, +// TracingContextFormat.ALL_ID_FORMAT, null); +// } String fsID = fs.getFileSystemID(); AbfsConfiguration abfsConf = fs.getAbfsStore().getAbfsConfiguration(); - String corrID = abfsConf == null? "test-corr-id" : - abfsConf.getClientCorrelationID(); - TracingContextFormat format = abfsConf == null? TracingContextFormat.ALL_ID_FORMAT : - abfsConf.getTracingContextFormat(); +// String corrID = abfsConf == null? "test-corr-id" : + String corrID = abfsConf.getClientCorrelationID(); + TracingContextFormat format = abfsConf.getTracingContextFormat(); +// TracingContextFormat format = abfsConf == null? TracingContextFormat.ALL_ID_FORMAT : +// abfsConf.getTracingContextFormat(); return new TracingContext(corrID, fsID, HdfsOperationConstants.TEST_OP, isCont, format, null); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index 53c3aaa8a2306..bc3b9bf8d3056 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -131,7 +131,6 @@ private void testReadWriteAndSeek(int bufferSize) throws Exception { inputStream.seek(0); result = inputStream.read(readBuffer, 0, bufferSize); - } assertNotEquals("data read in final read()", -1, result); assertArrayEquals(readBuffer, b); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java index b08bc108abad9..42205807c1b3e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java @@ -176,7 +176,6 @@ public void testOpenAppendRenameExists() throws IOException { + "exists methods on Abfs"); AzureBlobFileSystem fs = getFileSystem(); - AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); Path createFilePath = path(getMethodName()); Path destCreateFilePath = path(getMethodName() + "New"); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java index a309477943ca8..b7576895bf105 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java @@ -69,9 +69,6 @@ public void testAppendFileAfterDelete() throws Exception { ContractTestUtils.touch(fs, filePath); fs.delete(filePath, false); - fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() - .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), - HdfsOperationConstants.APPEND, false, 0)); fs.append(filePath); } @@ -91,8 +88,6 @@ public void testTracingForAppend() throws IOException { .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), HdfsOperationConstants.APPEND, false, 0)); - FSDataOutputStream in = fs.append(TEST_FILE_PATH, 10); - byte[] buf = new byte[100]; - in.write(buf, 0, 5); + fs.append(TEST_FILE_PATH, 10); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index 0ed0708c912cc..80fb5c61ab116 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -72,7 +72,6 @@ public ITestAzureBlobFileSystemDelete() throws Exception { @Test public void testDeleteRoot() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); - AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); fs.mkdirs(new Path("/testFolder0")); fs.mkdirs(new Path("/testFolder1")); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java index 86d107e568d21..905ad919d5c95 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java @@ -142,14 +142,7 @@ public void testWriteWithBufferOffset() throws Exception { } final byte[] r = new byte[TEST_DEFAULT_READ_BUFFER_SIZE]; - AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); - TracingHeaderValidator tracingHeaderValidator = new TracingHeaderValidator - (conf.getClientCorrelationID(), fs.getFileSystemID(), HdfsOperationConstants.OPEN, - false, 0); - fs.registerListener(tracingHeaderValidator); FSDataInputStream inputStream = fs.open(testFilePath, TEST_DEFAULT_BUFFER_SIZE); - tracingHeaderValidator.setOperation(HdfsOperationConstants.READ); - ((AbfsInputStream)inputStream.getWrappedStream()).registerListener(tracingHeaderValidator); int result = inputStream.read(r); assertNotEquals(-1, result); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java index 2814faa59bb23..efd0233d9fe26 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java @@ -82,16 +82,11 @@ public Void call() throws Exception { } es.shutdownNow(); - Configuration config = new Configuration(this.getRawConfiguration()); - config.set(FS_AZURE_CLIENT_CORRELATIONID, "validconfig"); - AzureBlobFileSystem fs1 = - (AzureBlobFileSystem) FileSystem.newInstance(config); - AbfsConfiguration conf = fs1.getAbfsStore().getAbfsConfiguration(); - fs1.registerListener(new TracingHeaderValidator( - conf.getClientCorrelationID(), - fs1.getFileSystemID(), HdfsOperationConstants.LISTSTATUS, true, + fs.registerListener(new TracingHeaderValidator( + getConfiguration().getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.LISTSTATUS, true, 0)); - FileStatus[] files = fs1.listStatus(new Path("/")); + FileStatus[] files = fs.listStatus(new Path("/")); assertEquals(TEST_FILES_NUMBER, files.length /* user directory */); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java index 61f5f42f58431..7b1b0022f04e9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java @@ -1226,7 +1226,6 @@ public void testDefaultAclRenamedFile() throws Exception { fs.registerListener(null); AclEntry[] expected = new AclEntry[] { }; AclStatus s = fs.getAclStatus(renamedFilePath); - AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(expected, returned); assertPermission(fs, renamedFilePath, (short) RW_R); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 3d2ae0001dc52..159e5332b626b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -17,7 +17,9 @@ import org.junit.Ignore; import org.junit.Test; +import java.io.FileNotFoundException; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.HashMap; import java.util.Map; @@ -80,23 +82,6 @@ false, true, getOctalNotation(FsPermission.getDefault()), Assertions.assertThat(requestHeader) .describedAs("Client Request Header should match TracingContext") .isEqualTo(tracingContext.toString()); - - // use fn below or pass listener to run all TracingHeaderValidator checks - checkRequiredIDs(requestHeader); - } - - private void checkRequiredIDs(String requestHeader) { - String[] id_list = requestHeader.split(":"); - - Assertions.assertThat(id_list[1]) - .describedAs("client-req-id should be a guid") - .matches(GUID_PATTERN); - Assertions.assertThat(id_list[2]) - .describedAs("filesystem-id should not be empty") - .isNotEmpty(); - Assertions.assertThat(id_list[1]) - .describedAs("client-request-id should be unique") - .isNotEqualTo(prevClientRequestID); } @Ignore @@ -104,7 +89,7 @@ private void checkRequiredIDs(String requestHeader) { //call test methods from the respective test classes //can be ignored when running all tests as these get covered public void runCorrelationTestForAllMethods() throws Exception { - //map to avoid creating new instance and calling setup() for each test + //map to group together creating new instance and calling setup() for tests Map testClasses = new HashMap<>(); testClasses.put(new ITestAzureBlobFileSystemListStatus(), //liststatus @@ -144,21 +129,21 @@ public void runCorrelationTestForAllMethods() throws Exception { testClasses.get(testClass).invoke(testClass); testClass.teardown(); } + testExternalOps(); } @Test //rename this test public void testExternalOps() throws Exception { - //validate tracing header for access, hasPathCapability, - // getIsNamespaceEnabled + //validate tracing header for access, hasPathCapability AzureBlobFileSystem fs = getFileSystem(); fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), HdfsOperationConstants.ACCESS, false, 0)); - fs.access(new Path("/"), FsAction.READ); + fs.access(new Path("/"), FsAction.ALL); fs.setListenerOperation(HdfsOperationConstants.HAS_PATH_CAPABILITY); - //unset namespaceEnabled config to call getAcl -> test tracing header + //unset namespaceEnabled config to call getAcl fs.getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN); fs.hasPathCapability(new Path("/"), CommonPathCapabilities.FS_ACLS); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index a528d6fd6365f..446c826669834 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -112,8 +112,7 @@ private void verifyReadCallCount(AbfsClient client, int count) throws Thread.sleep(1000); verify(client, times(count)).read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), - any(String.class), any(String.class), - any(TracingContext.class)); + any(String.class), any(String.class), any(TracingContext.class)); } private void checkEvictedStatus(AbfsInputStream inputStream, int position, boolean expectedToThrowException) @@ -168,8 +167,7 @@ public void testFailedReadAhead() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class), - any(TracingContext.class)); + any(String.class), any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testFailedReadAhead.txt"); @@ -248,8 +246,7 @@ public void testOlderReadAheadFailure() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class), - any(TracingContext.class)); + any(String.class), any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testOlderReadAheadFailure.txt"); @@ -303,8 +300,7 @@ public void testSuccessfulReadAhead() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class), - any(TracingContext.class)); + any(String.class), any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testSuccessfulReadAhead.txt"); int beforeReadCompletedListSize = ReadBufferManager.getBufferManager().getCompletedReadListSize(); @@ -362,8 +358,7 @@ public void testReadAheadManagerForFailedReadAhead() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class), - any(TracingContext.class)); + any(String.class), any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testReadAheadManagerForFailedReadAhead.txt"); @@ -416,8 +411,7 @@ public void testReadAheadManagerForOlderReadAheadFailure() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class), - any(TracingContext.class)); + any(String.class), any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testReadAheadManagerForOlderReadAheadFailure.txt"); @@ -471,8 +465,7 @@ public void testReadAheadManagerForSuccessfulReadAhead() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class), - any(TracingContext.class)); + any(String.class), any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testSuccessfulReadAhead.txt"); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index 29dee96334f09..f899dd59b790b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -83,9 +83,6 @@ public void verifyShortWriteRequest() throws Exception { final Configuration conf = new Configuration(); conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); - TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", HdfsOperationConstants.WRITE, - abfsConf.getTracingContextFormat(), null); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), @@ -94,8 +91,10 @@ public void verifyShortWriteRequest() throws Exception { any(TracingContext.class))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, - populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, - false), tracingContext); + populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), + new TracingContext(abfsConf.getClientCorrelationID(), "test-fs-id", + HdfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), + null)); final byte[] b = new byte[WRITE_SIZE]; new Random().nextBytes(b); out.write(b); @@ -461,21 +460,18 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { final Configuration conf = new Configuration(); conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); - TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", HdfsOperationConstants.WRITE, - abfsConf.getTracingContextFormat(), null); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), - any(TracingContext.class))).thenReturn(op); - - System.out.println(op + "op++"); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), + anyString(), any(TracingContext.class))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, - populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, - false), tracingContext); + populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), + new TracingContext(abfsConf.getClientCorrelationID(), "test-fs-id", + HdfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), + null)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); From 21e2a8682ca9c9684f68e4c06185f16c991f85d7 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Fri, 27 Nov 2020 12:10:32 +0530 Subject: [PATCH 49/77] fix sastoken matcher --- .../fs/azurebfs/services/AbfsOutputStream.java | 4 ++-- .../azurebfs/ITestAzureBlobFileSystemDelete.java | 6 ++++-- .../azurebfs/services/TestAbfsOutputStream.java | 15 ++++++++------- 3 files changed, 14 insertions(+), 11 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index c99dffbb5a47e..9011b71802297 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -476,8 +476,8 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset, try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "flushWrittenBytesToServiceInternal", "flush")) { - AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, isClose, cachedSasToken.get(), - new TracingContext(tracingContext)); + AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, + isClose, cachedSasToken.get(), new TracingContext(tracingContext)); cachedSasToken.update(op.getSasToken()); perfInfo.registerResult(op.getResult()).registerSuccess(true); } catch (AzureBlobFileSystemException ex) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index 80fb5c61ab116..ff0368b03dd25 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -27,6 +27,7 @@ import java.util.concurrent.Future; import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.assertj.core.api.Assertions; import org.junit.Assume; @@ -239,14 +240,15 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { when(idempotencyRetOp.getResult()).thenReturn(http200Op); doReturn(idempotencyRetOp).when(mockClient).deleteIdempotencyCheckOp(any()); + TracingContext tracingContext = getTestTracingContext(fs, false); when(mockClient.deletePath("/NonExistingPath", false, - null, getTestTracingContext(fs, false))).thenCallRealMethod(); + null, tracingContext)).thenCallRealMethod(); Assertions.assertThat(mockClient.deletePath( "/NonExistingPath", false, null, - getTestTracingContext(fs, false)) + tracingContext) .getResult() .getStatusCode()) .describedAs("Idempotency check reports successful " diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index f899dd59b790b..86bbb9397baf0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -25,6 +25,7 @@ import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import org.junit.Test; import org.mockito.ArgumentCaptor; @@ -87,7 +88,7 @@ public void verifyShortWriteRequest() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), any(TracingContext.class))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, @@ -141,14 +142,14 @@ public void verifyWriteRequest() throws Exception { conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); - TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), + TracingContext tracingContext = new TracingContext("test-corr-id", "test-fs-id", HdfsOperationConstants.WRITE, - abfsConf.getTracingContextFormat(), null); + TracingContextFormat.ALL_ID_FORMAT, null); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), any(TracingContext.class))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, @@ -217,7 +218,7 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), any(TracingContext.class))).thenReturn(op); when(op.getSasToken()).thenReturn("testToken"); when(op.getResult()).thenReturn(httpOp); @@ -400,7 +401,7 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), any(TracingContext.class))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext( @@ -465,7 +466,7 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), - anyString(), any(TracingContext.class))).thenReturn(op); + any(), any(TracingContext.class))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), From 8eaad735b62f9a5ae18c6f726f3e58b0c1bce9c3 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Mon, 30 Nov 2020 14:37:17 +0530 Subject: [PATCH 50/77] access test, formatting --- .../fs/azurebfs/utils/TracingContext.java | 2 +- .../ITestAzureBlobFileSystemCheckAccess.java | 4 - .../fs/azurebfs/TestTracingContext.java | 81 ++++++++++++------- 3 files changed, 52 insertions(+), 35 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index 91fece4718273..2b20b2402ea26 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -48,7 +48,7 @@ public TracingContext(String clientCorrelationID, String fileSystemID, this.clientCorrelationID = validateClientCorrelationID(clientCorrelationID); streamID = EMPTY_STRING; retryCount = 0; - primaryRequestID = ""; + primaryRequestID = EMPTY_STRING; format = tracingContextFormat; this.listener = listener; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java index 08f9e0d6c7fca..2dfa949506670 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java @@ -222,11 +222,7 @@ public void testFsActionEXECUTE() throws Exception { Path testFilePath = setupTestDirectoryAndUserAccess("/test3.txt", FsAction.EXECUTE); AzureBlobFileSystem fs = (AzureBlobFileSystem)testUserFs; - fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() - .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), - HdfsOperationConstants.ACCESS, false, 0)); assertAccessible(testFilePath, FsAction.EXECUTE); - fs.registerListener(null); assertInaccessible(testFilePath, FsAction.READ); assertInaccessible(testFilePath, FsAction.WRITE); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 159e5332b626b..e08777faf9701 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -1,5 +1,6 @@ package org.apache.hadoop.fs.azurebfs; +import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.fs.CommonPathCapabilities; import org.apache.hadoop.fs.Path; @@ -14,12 +15,11 @@ import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.assertj.core.api.Assertions; +import org.junit.Assume; import org.junit.Ignore; import org.junit.Test; -import java.io.FileNotFoundException; import java.io.IOException; -import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.HashMap; import java.util.Map; @@ -30,7 +30,6 @@ public class TestTracingContext extends AbstractAbfsIntegrationTest { private static final int HTTP_CREATED = 201; private final String EMPTY_STRING = ""; String GUID_PATTERN = "[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}"; - String prevClientRequestID = ""; public TestTracingContext() throws Exception { super(); @@ -45,7 +44,13 @@ public void testClientCorrelationID() throws IOException { private String getOctalNotation(FsPermission fsPermission) { Preconditions.checkNotNull(fsPermission, "fsPermission"); - return String.format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal()); + return String + .format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal()); + } + + private String getRelativePath(final Path path) { + Preconditions.checkNotNull(path, "path"); + return path.toUri().getPath(); } public void checkCorrelationConfigValidation(String clientCorrelationId, @@ -53,7 +58,7 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, AzureBlobFileSystem fs = getFileSystem(); TracingContext tracingContext = new TracingContext(clientCorrelationId, fs.getFileSystemID(), HdfsOperationConstants.TEST_OP, - TracingContextFormat.ALL_ID_FORMAT,null); + TracingContextFormat.ALL_ID_FORMAT, null); String correlationID = tracingContext.toString().split(":")[0]; if (includeInHeader) { Assertions.assertThat(correlationID) @@ -65,19 +70,26 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, .isEqualTo(EMPTY_STRING); } + boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(tracingContext); + String path = getRelativePath(new Path("/testDir")); + String permission = isNamespaceEnabled ? + getOctalNotation(FsPermission.getDirDefault()) : + null; + String umask = isNamespaceEnabled ? + getOctalNotation(FsPermission.getUMask(fs.getConf())) : + null; + //request should not fail for invalid clientCorrelationID - fs.getAbfsStore().setNamespaceEnabled(Trilean.getTrilean(true)); - AbfsRestOperation op = fs.getAbfsStore().getClient().createPath("/testDir", - false, true, getOctalNotation(FsPermission.getDefault()), - getOctalNotation(FsPermission.getUMask(getRawConfiguration())), - false, null, tracingContext); + AbfsRestOperation op = fs.getAbfsClient() + .createPath(path, false, true, permission, umask, false, null, + tracingContext); int statusCode = op.getResult().getStatusCode(); Assertions.assertThat(statusCode).describedAs("Request should not fail") .isEqualTo(HTTP_CREATED); - String requestHeader = op.getResult().getRequestHeader( - HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID) + String requestHeader = op.getResult() + .getRequestHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID) .replace("[", "").replace("]", ""); Assertions.assertThat(requestHeader) .describedAs("Client Request Header should match TracingContext") @@ -95,29 +107,30 @@ public void runCorrelationTestForAllMethods() throws Exception { testClasses.put(new ITestAzureBlobFileSystemListStatus(), //liststatus ITestAzureBlobFileSystemListStatus.class.getMethod("testListPath")); testClasses.put(new ITestAbfsReadWriteAndSeek(32), //open, read, write - ITestAbfsReadWriteAndSeek.class.getMethod("testReadAheadRequestID")); + ITestAbfsReadWriteAndSeek.class.getMethod("testReadAheadRequestID")); testClasses.put(new ITestAbfsReadWriteAndSeek(32), //read (bypassreadahead) - ITestAbfsReadWriteAndSeek.class.getMethod("testReadAndWriteWithDifferentBufferSizesAndSeek")); + ITestAbfsReadWriteAndSeek.class + .getMethod("testReadAndWriteWithDifferentBufferSizesAndSeek")); testClasses.put(new ITestAzureBlobFileSystemAppend(), //append ITestAzureBlobFileSystemAppend.class.getMethod("testTracingForAppend")); testClasses.put(new ITestAzureBlobFileSystemFlush(), ITestAzureBlobFileSystemFlush.class.getMethod( "testTracingHeaderForAppendBlob")); //outputstream (appendblob) testClasses.put(new ITestAzureBlobFileSystemCreate(), - ITestAzureBlobFileSystemCreate.class.getMethod( - "testDefaultCreateOverwriteFileTest")); //create + ITestAzureBlobFileSystemCreate.class + .getMethod("testDefaultCreateOverwriteFileTest")); //create testClasses.put(new ITestAzureBlobFilesystemAcl(), - ITestAzureBlobFilesystemAcl.class.getMethod( - "testDefaultAclRenamedFile")); //rename + ITestAzureBlobFilesystemAcl.class + .getMethod("testDefaultAclRenamedFile")); //rename testClasses.put(new ITestAzureBlobFileSystemDelete(), - ITestAzureBlobFileSystemDelete.class.getMethod( - "testDeleteFirstLevelDirectory")); //delete + ITestAzureBlobFileSystemDelete.class + .getMethod("testDeleteFirstLevelDirectory")); //delete testClasses.put(new ITestAzureBlobFileSystemCreate(), - ITestAzureBlobFileSystemCreate.class.getMethod( - "testCreateNonRecursive")); //mkdirs + ITestAzureBlobFileSystemCreate.class + .getMethod("testCreateNonRecursive")); //mkdirs testClasses.put(new ITestAzureBlobFileSystemAttributes(), - ITestAzureBlobFileSystemAttributes.class.getMethod( - "testSetGetXAttr")); //setxattr, getxattr + ITestAzureBlobFileSystemAttributes.class + .getMethod("testSetGetXAttr")); //setxattr, getxattr testClasses.put(new ITestAzureBlobFilesystemAcl(), ITestAzureBlobFilesystemAcl.class.getMethod( "testEnsureAclOperationWorksForRoot")); // setacl, getaclstatus, @@ -137,14 +150,22 @@ public void runCorrelationTestForAllMethods() throws Exception { public void testExternalOps() throws Exception { //validate tracing header for access, hasPathCapability AzureBlobFileSystem fs = getFileSystem(); - fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() - .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), - HdfsOperationConstants.ACCESS, false, 0)); - fs.access(new Path("/"), FsAction.ALL); - fs.setListenerOperation(HdfsOperationConstants.HAS_PATH_CAPABILITY); - //unset namespaceEnabled config to call getAcl + fs.registerListener(new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.HAS_PATH_CAPABILITY, false, + 0)); + + // unset namespaceEnabled to call getAcl -> trigger tracing header validator fs.getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN); fs.hasPathCapability(new Path("/"), CommonPathCapabilities.FS_ACLS); + + Assume.assumeTrue(getIsNamespaceEnabled(getFileSystem())); + Assume.assumeTrue(getConfiguration().isCheckAccessEnabled()); + Assume.assumeTrue(getAuthType() == AuthType.OAuth); + + fs.setListenerOperation(HdfsOperationConstants.ACCESS); + fs.getAbfsStore().setNamespaceEnabled(Trilean.TRUE); + fs.access(new Path("/"), FsAction.READ); } } From db8d89586959545ff69e1c9ab95260759c8bf1a4 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Mon, 30 Nov 2020 16:11:03 +0530 Subject: [PATCH 51/77] format PR diff --- .../constants/FileSystemConfigurations.java | 2 - .../azurebfs/services/AbfsOutputStream.java | 18 +- .../azurebfs/services/AbfsRestOperation.java | 7 +- .../fs/azurebfs/utils/TracingContext.java | 31 +-- .../azurebfs/utils/TracingContextFormat.java | 9 +- .../azurebfs/AbstractAbfsIntegrationTest.java | 20 +- .../hadoop/fs/azurebfs/ITestAbfsClient.java | 15 +- .../ITestAbfsInputStreamStatistics.java | 16 +- .../azurebfs/ITestAbfsReadWriteAndSeek.java | 31 ++- .../ITestAzureBlobFileSystemAppend.java | 7 +- .../ITestAzureBlobFileSystemAttributes.java | 6 +- .../ITestAzureBlobFileSystemBackCompat.java | 2 +- .../ITestAzureBlobFileSystemCheckAccess.java | 8 +- .../ITestAzureBlobFileSystemCreate.java | 23 +- .../ITestAzureBlobFileSystemDelete.java | 10 +- .../azurebfs/ITestAzureBlobFileSystemE2E.java | 3 - .../ITestAzureBlobFileSystemFlush.java | 11 +- .../ITestAzureBlobFileSystemListStatus.java | 10 +- .../ITestAzureBlobFileSystemRandomRead.java | 4 +- .../azurebfs/ITestAzureBlobFilesystemAcl.java | 10 +- .../azurebfs/ITestFileSystemProperties.java | 20 +- .../fs/azurebfs/ITestGetNameSpaceEnabled.java | 11 +- .../fs/azurebfs/ITestSharedKeyAuth.java | 4 +- .../azurebfs/ITestWasbAbfsCompatibility.java | 8 +- .../services/TestAbfsInputStream.java | 13 +- .../services/TestAbfsOutputStream.java | 204 ++++++++++-------- .../utils/TracingHeaderValidator.java | 40 ++-- 27 files changed, 274 insertions(+), 269 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index 01fa6b7d754f2..fa0ee6a89212d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -20,8 +20,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 9011b71802297..a04c23c20938b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -358,9 +358,9 @@ private void writeAppendBlobCurrentBufferToService() throws IOException { AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "writeCurrentBufferToService", "append")) { - AbfsRestOperation op = client.append(path, offset, bytes, 0, - bytesLength, cachedSasToken.get(), this.isAppendBlob, - new TracingContext(tracingContext)); + AbfsRestOperation op = client + .append(path, offset, bytes, 0, bytesLength, cachedSasToken.get(), + this.isAppendBlob, new TracingContext(tracingContext)); cachedSasToken.update(op.getSasToken()); outputStreamStatistics.uploadSuccessful(bytesLength); perfInfo.registerResult(op.getResult()); @@ -412,9 +412,9 @@ public Void call() throws Exception { AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "writeCurrentBufferToService", "append")) { - AbfsRestOperation op = client.append(path, offset, bytes, 0, - bytesLength, cachedSasToken.get(), false, - new TracingContext(tracingContext)); + AbfsRestOperation op = client + .append(path, offset, bytes, 0, bytesLength, cachedSasToken.get(), + false, new TracingContext(tracingContext)); cachedSasToken.update(op.getSasToken()); perfInfo.registerResult(op.getResult()); byteBufferPool.putBuffer(ByteBuffer.wrap(bytes)); @@ -475,9 +475,9 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset, AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "flushWrittenBytesToServiceInternal", "flush")) { - - AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, - isClose, cachedSasToken.get(), new TracingContext(tracingContext)); + AbfsRestOperation op = client + .flush(path, offset, retainUncommitedData, isClose, + cachedSasToken.get(), new TracingContext(tracingContext)); cachedSasToken.update(op.getSasToken()); perfInfo.registerResult(op.getResult()).registerSuccess(true); } catch (AzureBlobFileSystemException ex) { 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 37fb10a41db87..d450703eeac82 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 @@ -122,8 +122,7 @@ String getSasToken() { * @param requestHeaders The HTTP request headers. * @param sasToken A sasToken for optional re-use by AbfsInputStream/AbfsOutputStream. */ - @VisibleForTesting - protected AbfsRestOperation(final AbfsRestOperationType operationType, + AbfsRestOperation(final AbfsRestOperationType operationType, final AbfsClient client, final String method, final URL url, @@ -206,7 +205,7 @@ public void execute(TracingContext tracingContext) throws AzureBlobFileSystemExc LOG.trace("{} REST operation complete", operationType); } - protected void updateClientRequestHeader(AbfsHttpOperation httpOperation, + private void updateClientRequestHeader(AbfsHttpOperation httpOperation, TracingContext tracingContext) { tracingContext.generateClientRequestID(); httpOperation.getConnection() @@ -219,7 +218,7 @@ protected void updateClientRequestHeader(AbfsHttpOperation httpOperation, * fails, there may be a retry. The retryCount is incremented with each * attempt. */ - protected boolean executeHttpOperation(final int retryCount, + private boolean executeHttpOperation(final int retryCount, TracingContext tracingContext) throws AzureBlobFileSystemException { AbfsHttpOperation httpOperation = null; try { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index 2b20b2402ea26..af20ae8aae3b0 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

* Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -42,7 +42,8 @@ public class TracingContext { public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*"; public TracingContext(String clientCorrelationID, String fileSystemID, - String hadoopOpName, TracingContextFormat tracingContextFormat, Listener listener) { + String hadoopOpName, TracingContextFormat tracingContextFormat, + Listener listener) { this.fileSystemID = fileSystemID; this.hadoopOpName = hadoopOpName; this.clientCorrelationID = validateClientCorrelationID(clientCorrelationID); @@ -56,9 +57,9 @@ public TracingContext(String clientCorrelationID, String fileSystemID, public TracingContext(String clientCorrelationID, String fileSystemID, String hadoopOpName, boolean needsPrimaryReqId, TracingContextFormat tracingContextFormat, Listener listener) { - this(clientCorrelationID, fileSystemID, hadoopOpName, - tracingContextFormat, listener); - primaryRequestID = needsPrimaryReqId? UUID.randomUUID().toString() : ""; + this(clientCorrelationID, fileSystemID, hadoopOpName, tracingContextFormat, + listener); + primaryRequestID = needsPrimaryReqId ? UUID.randomUUID().toString() : ""; if (listener != null) listener.updatePrimaryRequestID(primaryRequestID); } @@ -71,7 +72,7 @@ public TracingContext(TracingContext originalTracingContext) { this.retryCount = 0; this.primaryRequestID = originalTracingContext.primaryRequestID; this.format = originalTracingContext.format; - if(originalTracingContext.listener != null) { + if (originalTracingContext.listener != null) { this.listener = originalTracingContext.listener.getClone(); } } @@ -92,7 +93,7 @@ public void generateClientRequestID() { public void setPrimaryRequestID() { primaryRequestID = UUID.randomUUID().toString(); - if(listener != null) + if (listener != null) listener.updatePrimaryRequestID(primaryRequestID); } @@ -115,11 +116,13 @@ public void setListener(Listener listener) { public String toString() { String header = clientRequestID; //case 0, no IDs for correlation switch (format) { - case ALL_ID_FORMAT: header = - clientCorrelationID + ":" + header + ":" + fileSystemID + ":" + primaryRequestID - + ":" + streamID + ":" + hadoopOpName + ":" + retryCount; - break; - case TWO_ID_FORMAT: header = clientCorrelationID + ":" + header; + case ALL_ID_FORMAT: + header = clientCorrelationID + ":" + header + ":" + fileSystemID + ":" + + primaryRequestID + ":" + streamID + ":" + hadoopOpName + ":" + + retryCount; + break; + case TWO_ID_FORMAT: + header = clientCorrelationID + ":" + header; } if (listener != null) { listener.callTracingHeaderValidator(header, format); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java index 12e6560436aea..0be91c9252415 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java @@ -1,7 +1,10 @@ package org.apache.hadoop.fs.azurebfs.utils; public enum TracingContextFormat { - SINGLE_ID_FORMAT, - ALL_ID_FORMAT, - TWO_ID_FORMAT; + SINGLE_ID_FORMAT, // + + ALL_ID_FORMAT, // :: + // :::: + + TWO_ID_FORMAT; // : } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index a72715345b26f..c06a53afe9100 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -27,7 +27,6 @@ import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -142,26 +141,17 @@ protected AbstractAbfsIntegrationTest() throws Exception { } } - protected boolean getIsNamespaceEnabled(AzureBlobFileSystem fs) throws IOException { + protected boolean getIsNamespaceEnabled(AzureBlobFileSystem fs) + throws IOException { return fs.getIsNamespaceEnabled(getTestTracingContext(fs, false)); } public TracingContext getTestTracingContext(AzureBlobFileSystem fs, boolean isCont) { -// if (fs == null) { -// return new TracingContext("test-corr-id", "test-fs-id", "TS", false, -// TracingContextFormat.ALL_ID_FORMAT, null); -// } - String fsID = fs.getFileSystemID(); AbfsConfiguration abfsConf = fs.getAbfsStore().getAbfsConfiguration(); -// String corrID = abfsConf == null? "test-corr-id" : - String corrID = abfsConf.getClientCorrelationID(); - TracingContextFormat format = abfsConf.getTracingContextFormat(); -// TracingContextFormat format = abfsConf == null? TracingContextFormat.ALL_ID_FORMAT : -// abfsConf.getTracingContextFormat(); - - return new TracingContext(corrID, fsID, HdfsOperationConstants.TEST_OP, isCont, - format, null); + return new TracingContext(abfsConf.getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.TEST_OP, isCont, + abfsConf.getTracingContextFormat(), null); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java index 9963a7eb39cce..62720b30710ce 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java @@ -32,21 +32,14 @@ import org.junit.Ignore; import org.junit.Test; -import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; - -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; -import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema; 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.apache.hadoop.fs.permission.FsPermission; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_CORRELATIONID; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -69,8 +62,9 @@ public void testContinuationTokenHavingEqualSign() throws Exception { AbfsClient abfsClient = fs.getAbfsClient(); try { - AbfsRestOperation op = abfsClient.listPath("/", true, LIST_MAX_RESULTS, - "===========", getTestTracingContext(fs, true)); + AbfsRestOperation op = abfsClient + .listPath("/", true, LIST_MAX_RESULTS, "===========", + getTestTracingContext(fs, true)); Assert.assertTrue(false); } catch (AbfsRestOperationException ex) { Assert.assertEquals("InvalidQueryParameterValue", ex.getErrorCode().getErrorCode()); @@ -144,7 +138,8 @@ private List listPath(String directory) throws IOException { return getFileSystem().getAbfsClient() .listPath(directory, false, getListMaxResults(), null, - getTestTracingContext(getFileSystem(), true)).getResult().getListResultSchema().paths(); + getTestTracingContext(getFileSystem(), true)).getResult() + .getListResultSchema().paths(); } private int getListMaxResults() throws IOException { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java index 338a1e0f9a02f..342d3c770f815 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java @@ -20,7 +20,6 @@ import java.io.IOException; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.assertj.core.api.Assertions; import org.junit.Test; import org.slf4j.Logger; @@ -64,7 +63,7 @@ public void testInitValues() throws IOException { outputStream = createAbfsOutputStreamWithFlushEnabled(fs, initValuesPath); inputStream = abfss.openFileForRead(initValuesPath, fs.getFsStatistics(), - getTestTracingContext(fs, false)); + getTestTracingContext(fs, false)); AbfsInputStreamStatisticsImpl stats = (AbfsInputStreamStatisticsImpl) inputStream.getStreamStatistics(); @@ -201,7 +200,7 @@ public void testReadStatistics() throws IOException { out.write(defBuffer); out.hflush(); in = abfss.openFileForRead(readStatPath, fs.getFsStatistics(), - getTestTracingContext(fs, false)); + getTestTracingContext(fs, false)); /* * Doing file read 10 times. @@ -272,16 +271,15 @@ public void testWithNullStreamStatistics() throws IOException { out.hflush(); // AbfsRestOperation Instance required for eTag. - AbfsRestOperation abfsRestOperation = - fs.getAbfsClient().getPathStatus(nullStatFilePath.toUri().getPath(), false, - getTestTracingContext(fs, false)); + AbfsRestOperation abfsRestOperation = fs.getAbfsClient() + .getPathStatus(nullStatFilePath.toUri().getPath(), false, + getTestTracingContext(fs, false)); // AbfsInputStream with no StreamStatistics. in = new AbfsInputStream(fs.getAbfsClient(), null, - nullStatFilePath.toUri().getPath(), ONE_KB, - abfsInputStreamContext, + nullStatFilePath.toUri().getPath(), ONE_KB, abfsInputStreamContext, abfsRestOperation.getResult().getResponseHeader("ETag"), - getTestTracingContext(fs, false)); + getTestTracingContext(fs, false)); // Verifying that AbfsInputStream Operations works with null statistics. assertNotEquals("AbfsInputStream read() with null statistics should " diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index bc3b9bf8d3056..e100a91059e8a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -76,24 +76,21 @@ public void testReadAheadRequestID() throws java.io.IOException { final byte[] b = new byte[bufferSize * 10]; new Random().nextBytes(b); try (FSDataOutputStream stream = fs.create(TEST_PATH)) { - ((AbfsOutputStream)stream.getWrappedStream()).registerListener(new - TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.CREATE, - false, 0)); + ((AbfsOutputStream) stream.getWrappedStream()).registerListener( + new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.CREATE, false, 0)); stream.write(b); } final byte[] readBuffer = new byte[4 * bufferSize]; int result; - TracingHeaderValidator tracingHeaderValidator = new TracingHeaderValidator( - abfsConfiguration.getClientCorrelationID(), fs.getFileSystemID(), - HdfsOperationConstants.OPEN, false, 0); - fs.registerListener(tracingHeaderValidator); + fs.registerListener( + new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.OPEN, false, 0)); try (FSDataInputStream inputStream = fs.open(TEST_PATH)) { - ((AbfsInputStream)inputStream.getWrappedStream()).registerListener(new - TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.READ, - false, 0)); + ((AbfsInputStream) inputStream.getWrappedStream()).registerListener( + new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.READ, false, 0)); result = inputStream.read(readBuffer, 0, bufferSize*4); } fs.registerListener(null); @@ -115,11 +112,11 @@ private void testReadWriteAndSeek(int bufferSize) throws Exception { final byte[] readBuffer = new byte[2 * bufferSize]; int result; try (FSDataInputStream inputStream = fs.open(TEST_PATH)) { - ((AbfsInputStream) inputStream.getWrappedStream()).registerListener(new - TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.READ, true, - 0, - ((AbfsInputStream) inputStream.getWrappedStream()).getStreamID())); + ((AbfsInputStream) inputStream.getWrappedStream()).registerListener( + new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.READ, true, 0, + ((AbfsInputStream) inputStream.getWrappedStream()) + .getStreamID())); inputStream.seek(bufferSize); result = inputStream.read(readBuffer, bufferSize, bufferSize); assertNotEquals(-1, result); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java index b7576895bf105..d5b82aa1c9fbc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java @@ -84,10 +84,9 @@ public void testAppendDirectory() throws Exception { public void testTracingForAppend() throws IOException { AzureBlobFileSystem fs = getFileSystem(); fs.create(TEST_FILE_PATH); - fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() - .getAbfsConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.APPEND, - false, 0)); + fs.registerListener(new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.APPEND, false, 0)); fs.append(TEST_FILE_PATH, 10); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java index a02a29381fd6f..edb5fc23614b7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java @@ -58,9 +58,9 @@ public void testSetGetXAttr() throws Exception { assertNull(fs.getXAttr(testFile, attributeName1)); // after setting the xAttr on the file, the value should be retrievable - fs.registerListener(new TracingHeaderValidator(conf.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.SET_ATTR, - true, 0)); + fs.registerListener( + new TracingHeaderValidator(conf.getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.SET_ATTR, true, 0)); fs.setXAttr(testFile, attributeName1, attributeValue1); fs.setListenerOperation(HdfsOperationConstants.GET_ATTR); assertArrayEquals(attributeValue1, fs.getXAttr(testFile, attributeName1)); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java index e47a37e22691e..cd50b7fc0becd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java @@ -43,7 +43,7 @@ public ITestAzureBlobFileSystemBackCompat() throws Exception { public void testBlobBackCompat() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); Assume.assumeFalse("This test does not support namespace enabled account", - getIsNamespaceEnabled(getFileSystem())); + getIsNamespaceEnabled(getFileSystem())); String storageConnectionString = getBlobConnectionString(); CloudStorageAccount storageAccount = CloudStorageAccount.parse(storageConnectionString); CloudBlobClient blobClient = storageAccount.createCloudBlobClient(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java index 2dfa949506670..8443e525483a3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCheckAccess.java @@ -22,8 +22,6 @@ import java.lang.reflect.Field; import java.util.List; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.collect.Lists; import org.junit.Assume; @@ -191,8 +189,9 @@ public void testCheckAccessForAccountWithoutNS() throws Exception { // acts as noop AzureBlobFileSystemStore mockAbfsStore = Mockito.mock(AzureBlobFileSystemStore.class); - Mockito.when(mockAbfsStore.getIsNamespaceEnabled(getTestTracingContext(getFileSystem(), - false))).thenReturn(true); + Mockito.when(mockAbfsStore + .getIsNamespaceEnabled(getTestTracingContext(getFileSystem(), false))) + .thenReturn(true); Field abfsStoreField = AzureBlobFileSystem.class.getDeclaredField( "abfsStore"); abfsStoreField.setAccessible(true); @@ -221,7 +220,6 @@ public void testFsActionEXECUTE() throws Exception { checkPrerequisites(); Path testFilePath = setupTestDirectoryAndUserAccess("/test3.txt", FsAction.EXECUTE); - AzureBlobFileSystem fs = (AzureBlobFileSystem)testUserFs; assertAccessible(testFilePath, FsAction.EXECUTE); assertInaccessible(testFilePath, FsAction.READ); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index b23335a5a41eb..ecad317871f01 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -92,16 +92,15 @@ public void testEnsureFileCreatedImmediately() throws Exception { @SuppressWarnings("deprecation") public void testCreateNonRecursive() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); - AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE); try { fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null); fail("Should've thrown"); } catch (FileNotFoundException expected) { } - fs.registerListener(new TracingHeaderValidator(conf.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.MKDIR, - false, 0)); + fs.registerListener(new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.MKDIR, false, 0)); fs.mkdirs(TEST_FOLDER_PATH); fs.registerListener(null); @@ -250,7 +249,6 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(currentFs.getUri(), config); - AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); long totalConnectionMadeBeforeTest = fs.getInstrumentationMap() .get(CONNECTIONS_MADE.getStatName()); @@ -272,7 +270,8 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) fs.getInstrumentationMap()); // Case 2: Not Overwrite - File pre-exists - fs.registerListener(new TracingHeaderValidator(conf.getClientCorrelationID(), + fs.registerListener(new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), HdfsOperationConstants.CREATE, false, 0)); intercept(FileAlreadyExistsException.class, () -> fs.create(nonOverwriteFile, false)); @@ -302,9 +301,9 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) fs.getInstrumentationMap()); // Case 4: Overwrite - File pre-exists - fs.registerListener(new TracingHeaderValidator(conf.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.CREATE, true, - 0)); + fs.registerListener(new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.CREATE, true, 0)); fs.create(overwriteFilePath, true); fs.registerListener(null); @@ -363,8 +362,8 @@ public void testNegativeScenariosForCreateOverwriteDisabled() AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); abfsStore = setAzureBlobSystemStoreField(abfsStore, "client", mockClient); - boolean isNamespaceEnabled = - abfsStore.getIsNamespaceEnabled(getTestTracingContext(fs, false)); + boolean isNamespaceEnabled = abfsStore + .getIsNamespaceEnabled(getTestTracingContext(fs, false)); AbfsRestOperation successOp = mock( AbfsRestOperation.class); @@ -480,7 +479,7 @@ private void validateCreateFileException(final Class ex intercept( exceptionClass, () -> abfsStore.createFile(testPath, null, true, permission, umask, - getTestTracingContext(getFileSystem(), true))); + getTestTracingContext(getFileSystem(), true))); } private AbfsRestOperationException getMockAbfsRestOperationException(int status) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index ff0368b03dd25..b7ca2ebbb9fc2 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -147,9 +147,9 @@ public Void call() throws Exception { es.shutdownNow(); Path dir = new Path("/test"); - fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() - .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), - HdfsOperationConstants.DELETE, false, 0)); + fs.registerListener(new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.DELETE, false, 0)); // first try a non-recursive delete, expect failure intercept(FileAlreadyExistsException.class, () -> fs.delete(dir, false)); @@ -241,8 +241,8 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { doReturn(idempotencyRetOp).when(mockClient).deleteIdempotencyCheckOp(any()); TracingContext tracingContext = getTestTracingContext(fs, false); - when(mockClient.deletePath("/NonExistingPath", false, - null, tracingContext)).thenCallRealMethod(); + when(mockClient.deletePath("/NonExistingPath", false, null, tracingContext)) + .thenCallRealMethod(); Assertions.assertThat(mockClient.deletePath( "/NonExistingPath", diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java index 905ad919d5c95..05c3855f5c89d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java @@ -23,9 +23,6 @@ import java.util.Arrays; import java.util.Random; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; -import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; import org.apache.hadoop.conf.Configuration; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index ebc52b594eb51..dc5789c4ca0e2 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -309,15 +309,16 @@ public void testTracingHeaderForAppendBlob() throws Exception { Configuration config = new Configuration(this.getRawConfiguration()); config.set(FS_AZURE_APPEND_BLOB_KEY, "abfss:/"); config.set(TestConfigurationKeys.FS_AZURE_TEST_APPENDBLOB_ENABLED, "true"); - AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(config); + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem + .newInstance(config); byte[] buf = new byte[10]; new Random().nextBytes(buf); FSDataOutputStream out = fs.create(new Path("/testFile")); - ((AbfsOutputStream)out.getWrappedStream()).registerListener(new - TracingHeaderValidator(fs.getAbfsStore().getAbfsConfiguration() - .getClientCorrelationID(), fs.getFileSystemID(), - HdfsOperationConstants.CREATE, false, 0)); + ((AbfsOutputStream) out.getWrappedStream()).registerListener( + new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.CREATE, false, 0)); out.write(buf); out.hsync(); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java index efd0233d9fe26..e6cc3bf07790c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java @@ -27,8 +27,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; @@ -39,7 +37,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_CORRELATIONID; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists; @@ -82,10 +79,9 @@ public Void call() throws Exception { } es.shutdownNow(); - fs.registerListener(new TracingHeaderValidator( - getConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.LISTSTATUS, true, - 0)); + fs.registerListener( + new TracingHeaderValidator(getConfiguration().getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.LISTSTATUS, true, 0)); FileStatus[] files = fs.listStatus(new Path("/")); assertEquals(TEST_FILES_NUMBER, files.length /* user directory */); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java index f5938c00fc798..abe18c0f461b2 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java @@ -98,7 +98,7 @@ public void testBasicRead() throws Exception { @Test public void testRandomRead() throws Exception { Assume.assumeFalse("This test does not support namespace enabled account", - getIsNamespaceEnabled(getFileSystem())); + getIsNamespaceEnabled(getFileSystem())); assumeHugeFileExists(); try ( FSDataInputStream inputStreamV1 @@ -416,7 +416,7 @@ public void testSequentialReadAfterReverseSeekPerformance() @Ignore("HADOOP-16915") public void testRandomReadPerformance() throws Exception { Assume.assumeFalse("This test does not support namespace enabled account", - getIsNamespaceEnabled(getFileSystem())); + getIsNamespaceEnabled(getFileSystem())); createTestFile(); assumeHugeFileExists(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java index 7b1b0022f04e9..b274ed210cacf 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java @@ -1219,9 +1219,9 @@ public void testDefaultAclRenamedFile() throws Exception { fs.setPermission(filePath, FsPermission.createImmutable((short) RW_R)); Path renamedFilePath = new Path(dirPath, "file1"); - fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() - .getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), - HdfsOperationConstants.RENAME, true, 0)); + fs.registerListener(new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.RENAME, true, 0)); fs.rename(filePath, renamedFilePath); fs.registerListener(null); AclEntry[] expected = new AclEntry[] { }; @@ -1263,8 +1263,8 @@ public void testEnsureAclOperationWorksForRoot() throws Exception { aclEntry(DEFAULT, GROUP, FOO, ALL), aclEntry(ACCESS, GROUP, BAR, ALL)); - fs.registerListener(new TracingHeaderValidator(fs.getAbfsStore() - .getAbfsConfiguration().getClientCorrelationID(), + fs.registerListener(new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), fs.getFileSystemID(), HdfsOperationConstants.SET_ACL, true, 0)); fs.setAcl(rootPath, aclSpec1); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java index 22cbd38f3165b..9b1c85cce9e48 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java @@ -67,8 +67,8 @@ public void testBase64FileSystemProperties() throws Exception { properties.put("key", "{ value: value }"); TracingContext tracingContext = getTestTracingContext(fs, true); fs.getAbfsStore().setFilesystemProperties(properties, tracingContext); - Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties( - tracingContext); + Hashtable fetchedProperties = fs.getAbfsStore() + .getFilesystemProperties(tracingContext); assertEquals(properties, fetchedProperties); } @@ -81,8 +81,8 @@ public void testBase64PathProperties() throws Exception { touch(TEST_PATH); TracingContext tracingContext = getTestTracingContext(fs, true); fs.getAbfsStore().setPathProperties(TEST_PATH, properties, tracingContext); - Hashtable fetchedProperties = - fs.getAbfsStore().getPathStatus(TEST_PATH, tracingContext); + Hashtable fetchedProperties = fs.getAbfsStore() + .getPathStatus(TEST_PATH, tracingContext); assertEquals(properties, fetchedProperties); } @@ -94,8 +94,8 @@ public void testBase64InvalidFileSystemProperties() throws Exception { properties.put("key", "{ value: valueæ­² }"); TracingContext tracingContext = getTestTracingContext(fs, true); fs.getAbfsStore().setFilesystemProperties(properties, tracingContext); - Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties( - tracingContext); + Hashtable fetchedProperties = fs.getAbfsStore() + .getFilesystemProperties(tracingContext); assertEquals(properties, fetchedProperties); } @@ -108,8 +108,8 @@ public void testBase64InvalidPathProperties() throws Exception { touch(TEST_PATH); TracingContext tracingContext = getTestTracingContext(fs, true); fs.getAbfsStore().setPathProperties(TEST_PATH, properties, tracingContext); - Hashtable fetchedProperties = fs.getAbfsStore().getPathStatus(TEST_PATH, - tracingContext); + Hashtable fetchedProperties = fs.getAbfsStore() + .getPathStatus(TEST_PATH, tracingContext); assertEquals(properties, fetchedProperties); } @@ -121,8 +121,8 @@ public void testSetFileSystemProperties() throws Exception { properties.put("containerForDevTest", "true"); TracingContext tracingContext = getTestTracingContext(fs, true); fs.getAbfsStore().setFilesystemProperties(properties, tracingContext); - Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties( - tracingContext); + Hashtable fetchedProperties = fs.getAbfsStore() + .getFilesystemProperties(tracingContext); assertEquals(properties, fetchedProperties); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java index bb2e330e7b00e..bf88ae4abbc5a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java @@ -67,7 +67,7 @@ public void testXNSAccount() throws IOException { Assume.assumeTrue("Skip this test because the account being used for test is a non XNS account", isUsingXNSAccount); assertTrue("Expecting getIsNamespaceEnabled() return true", - getIsNamespaceEnabled(getFileSystem())); + getIsNamespaceEnabled(getFileSystem())); } @Test @@ -180,7 +180,8 @@ private void ensureGetAclCallIsMadeOnceForInvalidConf(String invalidConf) .setNamespaceEnabled(Trilean.getTrilean(invalidConf)); AbfsClient mockClient = callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient(); - verify(mockClient, times(1)).getAclStatus(anyString(), any(TracingContext.class)); + verify(mockClient, times(1)) + .getAclStatus(anyString(), any(TracingContext.class)); } private void ensureGetAclCallIsNeverMadeForValidConf(String validConf) @@ -189,14 +190,16 @@ private void ensureGetAclCallIsNeverMadeForValidConf(String validConf) .setNamespaceEnabled(Trilean.getTrilean(validConf)); AbfsClient mockClient = callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient(); - verify(mockClient, never()).getAclStatus(anyString(), any(TracingContext.class)); + verify(mockClient, never()) + .getAclStatus(anyString(), any(TracingContext.class)); } private void unsetConfAndEnsureGetAclCallIsMadeOnce() throws IOException { this.getFileSystem().getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN); AbfsClient mockClient = callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient(); - verify(mockClient, times(1)).getAclStatus(anyString(), any(TracingContext.class)); + verify(mockClient, times(1)) + .getAclStatus(anyString(), any(TracingContext.class)); } private AbfsClient callAbfsGetIsNamespaceEnabledAndReturnMockAbfsClient() diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java index 3798b52c760c5..fedddcc4b16fb 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java @@ -54,8 +54,8 @@ public void testWithWrongSharedKey() throws Exception { + "Authorization header is formed correctly including the " + "signature.\", 403", () -> { - abfsClient.getAclStatus("/", getTestTracingContext(getFileSystem(), - false)); + abfsClient + .getAclStatus("/", getTestTracingContext(getFileSystem(), false)); }); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java index 94d64b050105c..35f64fb3f12fe 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java @@ -58,7 +58,7 @@ public void testListFileStatus() throws Exception { AzureBlobFileSystem fs = getFileSystem(); // test only valid for non-namespace enabled account Assume.assumeFalse("Namespace enabled account does not support this test,", - getIsNamespaceEnabled(fs)); + getIsNamespaceEnabled(fs)); NativeAzureFileSystem wasb = getWasbFileSystem(); @@ -93,7 +93,7 @@ public void testReadFile() throws Exception { AzureBlobFileSystem abfs = getFileSystem(); // test only valid for non-namespace enabled account Assume.assumeFalse("Namespace enabled account does not support this test", - getIsNamespaceEnabled(abfs)); + getIsNamespaceEnabled(abfs)); NativeAzureFileSystem wasb = getWasbFileSystem(); @@ -133,7 +133,7 @@ public void testDir() throws Exception { AzureBlobFileSystem abfs = getFileSystem(); // test only valid for non-namespace enabled account Assume.assumeFalse("Namespace enabled account does not support this test", - getIsNamespaceEnabled(abfs)); + getIsNamespaceEnabled(abfs)); NativeAzureFileSystem wasb = getWasbFileSystem(); @@ -168,7 +168,7 @@ public void testSetWorkingDirectory() throws Exception { AzureBlobFileSystem abfs = getFileSystem(); // test only valid for non-namespace enabled account Assume.assumeFalse("Namespace enabled account does not support this test", - getIsNamespaceEnabled(abfs)); + getIsNamespaceEnabled(abfs)); NativeAzureFileSystem wasb = getWasbFileSystem(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index 446c826669834..0917446f8f3e8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -77,7 +77,8 @@ private AbfsClient getMockAbfsClient() { return client; } - private AbfsInputStream getAbfsInputStream(AbfsClient mockAbfsClient, String fileName) throws IOException { + private AbfsInputStream getAbfsInputStream(AbfsClient mockAbfsClient, + String fileName) throws IOException { AbfsInputStreamContext inputStreamContext = new AbfsInputStreamContext(-1); // Create AbfsInputStream with the client instance AbfsInputStream inputStream = new AbfsInputStream( @@ -100,9 +101,11 @@ private void queueReadAheads(AbfsInputStream inputStream) { ReadBufferManager.getBufferManager() .queueReadAhead(inputStream, 0, ONE_KB, inputStream.tracingContext); ReadBufferManager.getBufferManager() - .queueReadAhead(inputStream, ONE_KB, ONE_KB, inputStream.tracingContext); + .queueReadAhead(inputStream, ONE_KB, ONE_KB, + inputStream.tracingContext); ReadBufferManager.getBufferManager() - .queueReadAhead(inputStream, TWO_KB, TWO_KB, inputStream.tracingContext); + .queueReadAhead(inputStream, TWO_KB, TWO_KB, + inputStream.tracingContext); } private void verifyReadCallCount(AbfsClient client, int count) throws @@ -216,8 +219,8 @@ public void testFailedReadAheadEviction() throws Exception { // at java.util.Stack.peek(Stack.java:102) // at java.util.Stack.pop(Stack.java:84) // at org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.queueReadAhead - ReadBufferManager.getBufferManager().queueReadAhead(inputStream, 0, - ONE_KB, getTestTracingContext(getFileSystem(), true)); + ReadBufferManager.getBufferManager().queueReadAhead(inputStream, 0, ONE_KB, + getTestTracingContext(getFileSystem(), true)); } /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index 86bbb9397baf0..5192492f5e54a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -87,9 +87,10 @@ public void verifyShortWriteRequest() throws Exception { AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), - anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); + anyInt(), any(), anyBoolean(), any(TracingContext.class))) + .thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), - any(TracingContext.class))).thenReturn(op); + any(TracingContext.class))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), @@ -107,8 +108,8 @@ public void verifyShortWriteRequest() throws Exception { ArgumentCaptor acByteArray = ArgumentCaptor.forClass(byte[].class); ArgumentCaptor acAppendBlobAppend = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acSASToken = ArgumentCaptor.forClass(String.class); - ArgumentCaptor acTracingContext = - ArgumentCaptor.forClass(TracingContext.class); + ArgumentCaptor acTracingContext = ArgumentCaptor + .forClass(TracingContext.class); final byte[] b1 = new byte[2*WRITE_SIZE]; @@ -119,9 +120,11 @@ public void verifyShortWriteRequest() throws Exception { out.hsync(); - verify(client, times(2)).append(acString.capture(), acLong.capture(), acByteArray.capture(), acBufferOffset.capture(), acBufferLength.capture(), - acSASToken.capture(), - acAppendBlobAppend.capture(), acTracingContext.capture()); + verify(client, times(2)) + .append(acString.capture(), acLong.capture(), acByteArray.capture(), + acBufferOffset.capture(), acBufferLength.capture(), + acSASToken.capture(), acAppendBlobAppend.capture(), + acTracingContext.capture()); assertThat(Arrays.asList(PATH, PATH)).describedAs("Path of the requests").isEqualTo(acString.getAllValues()); assertThat(Arrays.asList(Long.valueOf(0), Long.valueOf(WRITE_SIZE))).describedAs("Write Position").isEqualTo(acLong.getAllValues()); assertThat(Arrays.asList(0, 0)).describedAs("Buffer Offset").isEqualTo(acBufferOffset.getAllValues()); @@ -143,18 +146,19 @@ public void verifyWriteRequest() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext("test-corr-id", - "test-fs-id", HdfsOperationConstants.WRITE, - TracingContextFormat.ALL_ID_FORMAT, null); + "test-fs-id", HdfsOperationConstants.WRITE, + TracingContextFormat.ALL_ID_FORMAT, null); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), - anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); + anyInt(), any(), anyBoolean(), any(TracingContext.class))) + .thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), - any(TracingContext.class))).thenReturn(op); + any(TracingContext.class))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, - populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, - false), tracingContext); + populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), + tracingContext); final byte[] b = new byte[WRITE_SIZE]; new Random().nextBytes(b); @@ -170,12 +174,14 @@ public void verifyWriteRequest() throws Exception { ArgumentCaptor acByteArray = ArgumentCaptor.forClass(byte[].class); ArgumentCaptor acAppendBlobAppend = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acSASToken = ArgumentCaptor.forClass(String.class); - ArgumentCaptor acTracingContext = - ArgumentCaptor.forClass(TracingContext.class); - - verify(client, times(2)).append(acString.capture(), acLong.capture(), acByteArray.capture(), acBufferOffset.capture(), acBufferLength.capture(), - acSASToken.capture(), - acAppendBlobAppend.capture(), acTracingContext.capture()); + ArgumentCaptor acTracingContext = ArgumentCaptor + .forClass(TracingContext.class); + + verify(client, times(2)) + .append(acString.capture(), acLong.capture(), acByteArray.capture(), + acBufferOffset.capture(), acBufferLength.capture(), + acSASToken.capture(), acAppendBlobAppend.capture(), + acTracingContext.capture()); assertThat(Arrays.asList(PATH, PATH)).describedAs("Path").isEqualTo(acString.getAllValues()); assertThat(new HashSet(Arrays.asList(Long.valueOf(0), Long.valueOf(BUFFER_SIZE)))).describedAs("Position").isEqualTo(new HashSet( acLong.getAllValues())); @@ -189,8 +195,10 @@ public void verifyWriteRequest() throws Exception { ArgumentCaptor acFlushClose = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); - verify(client, times(1)).flush(acFlushString.capture(), acFlushLong.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture(), acTracingContext.capture()); + verify(client, times(1)) + .flush(acFlushString.capture(), acFlushLong.capture(), + acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), + acFlushSASToken.capture(), acTracingContext.capture()); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushString.getAllValues()); assertThat(Arrays.asList(Long.valueOf(5*WRITE_SIZE))).describedAs("position").isEqualTo(acFlushLong.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); @@ -211,21 +219,22 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); - TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", HdfsOperationConstants.WRITE, - abfsConf.getTracingContextFormat(), null); + TracingContext tracingContext = new TracingContext( + abfsConf.getClientCorrelationID(), "test-fs-id", + HdfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), null); when(client.getAbfsPerfTracker()).thenReturn(tracker); - when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), - anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); + when(client + .append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), + any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), - any(TracingContext.class))).thenReturn(op); + any(TracingContext.class))).thenReturn(op); when(op.getSasToken()).thenReturn("testToken"); when(op.getResult()).thenReturn(httpOp); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, - populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, - false), tracingContext); + populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), + tracingContext); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -241,12 +250,14 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { ArgumentCaptor acByteArray = ArgumentCaptor.forClass(byte[].class); ArgumentCaptor acAppendBlobAppend = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acSASToken = ArgumentCaptor.forClass(String.class); - ArgumentCaptor acTracingContext = - ArgumentCaptor.forClass(TracingContext.class); - - verify(client, times(2)).append(acString.capture(), acLong.capture(), acByteArray.capture(), acBufferOffset.capture(), acBufferLength.capture(), - acSASToken.capture(), - acAppendBlobAppend.capture(), acTracingContext.capture()); + ArgumentCaptor acTracingContext = ArgumentCaptor + .forClass(TracingContext.class); + + verify(client, times(2)) + .append(acString.capture(), acLong.capture(), acByteArray.capture(), + acBufferOffset.capture(), acBufferLength.capture(), + acSASToken.capture(), acAppendBlobAppend.capture(), + acTracingContext.capture()); assertThat(Arrays.asList(PATH, PATH)).describedAs("path").isEqualTo(acString.getAllValues()); assertThat(new HashSet(Arrays.asList(Long.valueOf(0), Long.valueOf(BUFFER_SIZE)))).describedAs("Position").isEqualTo(new HashSet( acLong.getAllValues())); @@ -259,8 +270,10 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { ArgumentCaptor acFlushClose = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); - verify(client, times(1)).flush(acFlushString.capture(), acFlushLong.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture(), acTracingContext.capture()); + verify(client, times(1)) + .flush(acFlushString.capture(), acFlushLong.capture(), + acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), + acFlushSASToken.capture(), acTracingContext.capture()); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushString.getAllValues()); assertThat(Arrays.asList(Long.valueOf(2*BUFFER_SIZE))).describedAs("position").isEqualTo(acFlushLong.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); @@ -284,16 +297,17 @@ public void verifyWriteRequestOfBufferSize() throws Exception { AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); - when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), - anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), + when(client + .append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), + any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); + when(client + .flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), any(TracingContext.class))).thenReturn(op); when(op.getSasToken()).thenReturn("testToken"); when(op.getResult()).thenReturn(httpOp); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, - populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, - false), + populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), new TracingContext(abfsConf.getClientCorrelationID(), "test-fs-id", HdfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), null)); @@ -313,12 +327,14 @@ public void verifyWriteRequestOfBufferSize() throws Exception { ArgumentCaptor acByteArray = ArgumentCaptor.forClass(byte[].class); ArgumentCaptor acAppendBlobAppend = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acSASToken = ArgumentCaptor.forClass(String.class); - ArgumentCaptor acTracingContext = - ArgumentCaptor.forClass(TracingContext.class); - - verify(client, times(2)).append(acString.capture(), acLong.capture(), acByteArray.capture(), acBufferOffset.capture(), acBufferLength.capture(), - acSASToken.capture(), - acAppendBlobAppend.capture(), acTracingContext.capture()); + ArgumentCaptor acTracingContext = ArgumentCaptor + .forClass(TracingContext.class); + + verify(client, times(2)) + .append(acString.capture(), acLong.capture(), acByteArray.capture(), + acBufferOffset.capture(), acBufferLength.capture(), + acSASToken.capture(), acAppendBlobAppend.capture(), + acTracingContext.capture()); assertThat(Arrays.asList(PATH, PATH)).describedAs("File Path").isEqualTo(acString.getAllValues()); assertThat(new HashSet(Arrays.asList(Long.valueOf(0), Long.valueOf(BUFFER_SIZE)))).describedAs("Position in file").isEqualTo( new HashSet(acLong.getAllValues())); @@ -342,16 +358,18 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); - when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), - anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), + when(client + .append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), + any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); + when(client + .flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), anyString(), any(TracingContext.class))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, - populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, - true), new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", HdfsOperationConstants.OPEN, - abfsConf.getTracingContextFormat(), null)); + populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, true), + new TracingContext(abfsConf.getClientCorrelationID(), "test-fs-id", + HdfsOperationConstants.OPEN, abfsConf.getTracingContextFormat(), + null)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -367,12 +385,14 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { ArgumentCaptor acByteArray = ArgumentCaptor.forClass(byte[].class); ArgumentCaptor acAppendBlobAppend = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acSASToken = ArgumentCaptor.forClass(String.class); - ArgumentCaptor acTracingContext = - ArgumentCaptor.forClass(TracingContext.class); - - verify(client, times(2)).append(acString.capture(), acLong.capture(), acByteArray.capture(), acBufferOffset.capture(), acBufferLength.capture(), - acSASToken.capture(), - acAppendBlobAppend.capture(), acTracingContext.capture()); + ArgumentCaptor acTracingContext = ArgumentCaptor + .forClass(TracingContext.class); + + verify(client, times(2)) + .append(acString.capture(), acLong.capture(), acByteArray.capture(), + acBufferOffset.capture(), acBufferLength.capture(), + acSASToken.capture(), acAppendBlobAppend.capture(), + acTracingContext.capture()); assertThat(Arrays.asList(PATH, PATH)).describedAs("File Path").isEqualTo(acString.getAllValues()); assertThat(Arrays.asList(Long.valueOf(0), Long.valueOf(BUFFER_SIZE))).describedAs("File Position").isEqualTo(acLong.getAllValues()); assertThat(Arrays.asList(0, 0)).describedAs("Buffer Offset").isEqualTo(acBufferOffset.getAllValues()); @@ -394,19 +414,20 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { conf.set(accountKey1, accountValue1); abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); - TracingContext tracingContext = new TracingContext(abfsConf.getClientCorrelationID(), - "test-fs-id", HdfsOperationConstants.WRITE, - abfsConf.getTracingContextFormat(), null); + TracingContext tracingContext = new TracingContext( + abfsConf.getClientCorrelationID(), "test-fs-id", + HdfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), null); when(client.getAbfsPerfTracker()).thenReturn(tracker); - when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), - anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); + when(client + .append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), + any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), - any(TracingContext.class))).thenReturn(op); + any(TracingContext.class))).thenReturn(op); - AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext( - BUFFER_SIZE, true, false, false), - tracingContext); + AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, + populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), + tracingContext); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -422,12 +443,14 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { ArgumentCaptor acByteArray = ArgumentCaptor.forClass(byte[].class); ArgumentCaptor acAppendBlobAppend = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acSASToken = ArgumentCaptor.forClass(String.class); - ArgumentCaptor acTracingContext = - ArgumentCaptor.forClass(TracingContext.class); - - verify(client, times(2)).append(acString.capture(), acLong.capture(), acByteArray.capture(), acBufferOffset.capture(), acBufferLength.capture(), - acSASToken.capture(), - acAppendBlobAppend.capture(), acTracingContext.capture()); + ArgumentCaptor acTracingContext = ArgumentCaptor + .forClass(TracingContext.class); + + verify(client, times(2)) + .append(acString.capture(), acLong.capture(), acByteArray.capture(), + acBufferOffset.capture(), acBufferLength.capture(), + acSASToken.capture(), acAppendBlobAppend.capture(), + acTracingContext.capture()); assertThat(Arrays.asList(PATH, PATH)).describedAs("File Path").isEqualTo(acString.getAllValues()); assertThat(new HashSet(Arrays.asList(Long.valueOf(0), Long.valueOf(BUFFER_SIZE)))).describedAs("File Position").isEqualTo( new HashSet(acLong.getAllValues())); @@ -440,8 +463,10 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { ArgumentCaptor acFlushClose = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); - verify(client, times(1)).flush(acFlushString.capture(), acFlushLong.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture(), acTracingContext.capture()); + verify(client, times(1)) + .flush(acFlushString.capture(), acFlushLong.capture(), + acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), + acFlushSASToken.capture(), acTracingContext.capture()); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushString.getAllValues()); assertThat(Arrays.asList(Long.valueOf(2*BUFFER_SIZE))).describedAs("position").isEqualTo(acFlushLong.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); @@ -463,10 +488,11 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); - when(client.append(anyString(), anyLong(), any(byte[].class), anyInt(), - anyInt(), any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), - any(), any(TracingContext.class))).thenReturn(op); + when(client + .append(anyString(), anyLong(), any(byte[].class), anyInt(), anyInt(), + any(), anyBoolean(), any(TracingContext.class))).thenReturn(op); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), + any(TracingContext.class))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), @@ -490,12 +516,14 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { ArgumentCaptor acByteArray = ArgumentCaptor.forClass(byte[].class); ArgumentCaptor acAppendBlobAppend = ArgumentCaptor.forClass(Boolean.class); ArgumentCaptor acSASToken = ArgumentCaptor.forClass(String.class); - ArgumentCaptor acTracingContext = - ArgumentCaptor.forClass(TracingContext.class); - - verify(client, times(2)).append(acString.capture(), acLong.capture(), acByteArray.capture(), acBufferOffset.capture(), acBufferLength.capture(), - acSASToken.capture(), - acAppendBlobAppend.capture(), acTracingContext.capture()); + ArgumentCaptor acTracingContext = ArgumentCaptor + .forClass(TracingContext.class); + + verify(client, times(2)) + .append(acString.capture(), acLong.capture(), acByteArray.capture(), + acBufferOffset.capture(), acBufferLength.capture(), + acSASToken.capture(), acAppendBlobAppend.capture(), + acTracingContext.capture()); assertThat(Arrays.asList(PATH, PATH)).describedAs("path").isEqualTo(acString.getAllValues()); assertThat(new HashSet(Arrays.asList(Long.valueOf(0), Long.valueOf(BUFFER_SIZE)))).describedAs("Position").isEqualTo( new HashSet(acLong.getAllValues())); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java index 498a89d1e417f..3e6e8b1d95396 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java @@ -25,16 +25,15 @@ public void callTracingHeaderValidator(String tracingContextHeader, @Override public TracingHeaderValidator getClone() { - TracingHeaderValidator tracingHeaderValidator = - new TracingHeaderValidator(clientCorrelationID, fileSystemID, - operation, needsPrimaryRequestID, retryNum, streamID); + TracingHeaderValidator tracingHeaderValidator = new TracingHeaderValidator( + clientCorrelationID, fileSystemID, operation, needsPrimaryRequestID, + retryNum, streamID); tracingHeaderValidator.primaryRequestID = primaryRequestID; return tracingHeaderValidator; } - public TracingHeaderValidator(String clientCorrelationID, - String fileSystemID, String operation, - boolean needsPrimaryRequestID, int retryNum) { + public TracingHeaderValidator(String clientCorrelationID, String fileSystemID, + String operation, boolean needsPrimaryRequestID, int retryNum) { this.clientCorrelationID = clientCorrelationID; this.fileSystemID = fileSystemID; this.operation = operation; @@ -42,9 +41,9 @@ public TracingHeaderValidator(String clientCorrelationID, this.needsPrimaryRequestID = needsPrimaryRequestID; } - public TracingHeaderValidator(String clientCorrelationID, - String fileSystemID, String operation, boolean needsPrimaryRequestID, - int retryNum, String streamID) { + public TracingHeaderValidator(String clientCorrelationID, String fileSystemID, + String operation, boolean needsPrimaryRequestID, int retryNum, + String streamID) { this(clientCorrelationID, fileSystemID, operation, needsPrimaryRequestID, retryNum); this.streamID = streamID; @@ -55,7 +54,7 @@ private void validateTracingHeader(String tracingContextHeader) { validateBasicFormat(id_list); if (format != TracingContextFormat.ALL_ID_FORMAT) return; - if (!primaryRequestID.isEmpty() && !id_list[3].isEmpty()){ + if (!primaryRequestID.isEmpty() && !id_list[3].isEmpty()) { Assertions.assertThat(id_list[3]) .describedAs("PrimaryReqID should be common for these requests") .isEqualTo(primaryRequestID); @@ -75,24 +74,23 @@ private void validateBasicFormat(String[] id_list) { Assertions.assertThat(id_list) .describedAs("header should have 2 elements").hasSize(2); } else { - Assertions.assertThat(id_list) - .describedAs("header should have 1 element").hasSize(1); + Assertions.assertThat(id_list).describedAs("header should have 1 element") + .hasSize(1); Assertions.assertThat(id_list[0]) - .describedAs("Client request ID is a guid") - .matches(GUID_PATTERN); + .describedAs("Client request ID is a guid").matches(GUID_PATTERN); return; } - if(clientCorrelationID.matches("[a-zA-Z0-9-]*")) { - Assertions.assertThat(id_list[0]).describedAs("Correlation ID should match config") + if (clientCorrelationID.matches("[a-zA-Z0-9-]*")) { + Assertions.assertThat(id_list[0]) + .describedAs("Correlation ID should match config") .isEqualTo(clientCorrelationID); } else { Assertions.assertThat(id_list[0]) .describedAs("Invalid config should be replaced with empty string") .isEmpty(); } - Assertions.assertThat(id_list[1]) - .describedAs("Client request ID is a guid") + Assertions.assertThat(id_list[1]).describedAs("Client request ID is a guid") .matches(GUID_PATTERN); if (format != TracingContextFormat.ALL_ID_FORMAT) @@ -100,15 +98,15 @@ private void validateBasicFormat(String[] id_list) { Assertions.assertThat(id_list[2]).describedAs("Filesystem ID incorrect") .isEqualTo(fileSystemID); - if (needsPrimaryRequestID && !operation.equals(HdfsOperationConstants.READ)) { + if (needsPrimaryRequestID && !operation + .equals(HdfsOperationConstants.READ)) { Assertions.assertThat(id_list[3]).describedAs("should have primaryReqId") .isNotEmpty(); } Assertions.assertThat(id_list[5]).describedAs("Operation name incorrect") .isEqualTo(operation); int retryCount = Integer.parseInt(id_list[6]); - Assertions.assertThat(retryCount) - .describedAs("Retry count incorrect") + Assertions.assertThat(retryCount).describedAs("Retry count incorrect") .isEqualTo(retryNum); } From 0521969b26ead9397ff6856bba4b7f1feaca5394 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Mon, 30 Nov 2020 17:50:53 +0530 Subject: [PATCH 52/77] more formatting --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 1 + .../fs/azurebfs/AzureBlobFileSystem.java | 82 +++++---- .../fs/azurebfs/AzureBlobFileSystemStore.java | 173 ++++++++++-------- 3 files changed, 149 insertions(+), 107 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 08912a2b0e794..2039ceac92150 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -235,6 +235,7 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_ENABLE_DELEGATION_TOKEN) private boolean enableDelegationToken; + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ALWAYS_USE_HTTPS, DefaultValue = DEFAULT_ENABLE_HTTPS) private boolean alwaysUseHttps; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 5bc49de514108..e87261a248d14 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -193,8 +193,9 @@ public FSDataInputStream open(final Path path, final int bufferSize) throws IOEx try { TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, HdfsOperationConstants.OPEN, tracingContextFormat, - listener); - InputStream inputStream = abfsStore.openFileForRead(qualifiedPath, statistics, tracingContext); + listener); + InputStream inputStream = abfsStore + .openFileForRead(qualifiedPath, statistics, tracingContext); return new FSDataInputStream(inputStream); } catch(AzureBlobFileSystemException ex) { checkException(path, ex); @@ -219,10 +220,10 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi try { TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, HdfsOperationConstants.CREATE, overwrite, - tracingContextFormat, listener); + tracingContextFormat, listener); OutputStream outputStream = abfsStore.createFile(qualifiedPath, statistics, overwrite, - permission == null ? FsPermission.getFileDefault() : permission, FsPermission.getUMask(getConf()), - tracingContext); + permission == null ? FsPermission.getFileDefault() : permission, + FsPermission.getUMask(getConf()), tracingContext); statIncrement(FILES_CREATED); return new FSDataOutputStream(outputStream, statistics); } catch(AzureBlobFileSystemException ex) { @@ -286,9 +287,10 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.APPEND, tracingContextFormat, listener); - OutputStream outputStream = abfsStore.openFileForWrite(qualifiedPath, statistics, false, - tracingContext); + fileSystemID, HdfsOperationConstants.APPEND, tracingContextFormat, + listener); + OutputStream outputStream = abfsStore + .openFileForWrite(qualifiedPath, statistics, false, tracingContext); return new FSDataOutputStream(outputStream, statistics); } catch(AzureBlobFileSystemException ex) { checkException(f, ex); @@ -329,7 +331,8 @@ public boolean rename(final Path src, final Path dst) throws IOException { // Non-HNS account need to check dst status on driver side. TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.RENAME, true, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.RENAME, true, tracingContextFormat, + listener); if (!abfsStore.getIsNamespaceEnabled(tracingContext) && dstFileStatus == null) { dstFileStatus = tryGetFileStatus(qualifiedDstPath); } @@ -382,7 +385,8 @@ public boolean delete(final Path f, final boolean recursive) throws IOException try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.DELETE, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.DELETE, tracingContextFormat, + listener); abfsStore.delete(qualifiedPath, recursive, tracingContext); return true; } catch (AzureBlobFileSystemException ex) { @@ -401,7 +405,8 @@ public FileStatus[] listStatus(final Path f) throws IOException { try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.LISTSTATUS, true, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.LISTSTATUS, true, + tracingContextFormat, listener); FileStatus[] result = abfsStore.listStatus(qualifiedPath, tracingContext); return result; } catch (AzureBlobFileSystemException ex) { @@ -472,7 +477,8 @@ public boolean mkdirs(final Path f, final FsPermission permission) throws IOExce TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, HdfsOperationConstants.MKDIR, false, tracingContextFormat, listener); - abfsStore.createDirectory(qualifiedPath, permission == null ? FsPermission.getDirDefault() : permission, + abfsStore.createDirectory(qualifiedPath, + permission == null ? FsPermission.getDirDefault() : permission, FsPermission.getUMask(getConf()), tracingContext); statIncrement(DIRECTORIES_CREATED); return true; @@ -503,7 +509,8 @@ public FileStatus getFileStatus(final Path f) throws IOException { try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.GET_FILESTATUS, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.GET_FILESTATUS, + tracingContextFormat, listener); return abfsStore.getFileStatus(qualifiedPath, tracingContext); } catch(AzureBlobFileSystemException ex) { checkException(f, ex); @@ -679,7 +686,8 @@ public void setOwner(final Path path, final String owner, final String group) LOG.debug( "AzureBlobFileSystem.setOwner path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.SET_OWNER, true, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.SET_OWNER, true, + tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { super.setOwner(path, owner, group); @@ -727,7 +735,8 @@ public void setXAttr(final Path path, final String name, final byte[] value, fin TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, HdfsOperationConstants.SET_ATTR, true, tracingContextFormat, listener); - Hashtable properties = abfsStore.getPathStatus(qualifiedPath, tracingContext); + Hashtable properties = abfsStore + .getPathStatus(qualifiedPath, tracingContext); String xAttrName = ensureValidAttributeName(name); boolean xAttrExists = properties.containsKey(xAttrName); XAttrSetFlag.validate(name, xAttrExists, flag); @@ -766,7 +775,8 @@ public byte[] getXAttr(final Path path, final String name) TracingContext tracingContext = new TracingContext(clientCorrelationID, fileSystemID, HdfsOperationConstants.GET_ATTR, true, tracingContextFormat, listener); - Hashtable properties = abfsStore.getPathStatus(qualifiedPath, tracingContext); + Hashtable properties = abfsStore + .getPathStatus(qualifiedPath, tracingContext); String xAttrName = ensureValidAttributeName(name); if (properties.containsKey(xAttrName)) { String xAttrValue = properties.get(xAttrName); @@ -794,7 +804,8 @@ public void setPermission(final Path path, final FsPermission permission) throws IOException { LOG.debug("AzureBlobFileSystem.setPermission path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.SET_PERMISSION, true, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.SET_PERMISSION, true, + tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { super.setPermission(path, permission); @@ -808,8 +819,7 @@ public void setPermission(final Path path, final FsPermission permission) Path qualifiedPath = makeQualified(path); try { - abfsStore.setPermission(qualifiedPath, - permission, tracingContext); + abfsStore.setPermission(qualifiedPath, permission, tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -830,7 +840,8 @@ public void modifyAclEntries(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.modifyAclEntries path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.MODIFY_ACL,true, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.MODIFY_ACL, true, + tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -845,8 +856,7 @@ public void modifyAclEntries(final Path path, final List aclSpec) Path qualifiedPath = makeQualified(path); try { - abfsStore.modifyAclEntries(qualifiedPath, - aclSpec, tracingContext); + abfsStore.modifyAclEntries(qualifiedPath, aclSpec, tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -927,7 +937,8 @@ public void removeDefaultAcl(final Path path) throws IOException { public void removeAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeAcl path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.REMOVE_ACL, true, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.REMOVE_ACL, true, + tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -959,7 +970,8 @@ public void setAcl(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.setAcl path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.SET_ACL, true, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.SET_ACL, true, + tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -991,7 +1003,8 @@ public void setAcl(final Path path, final List aclSpec) public AclStatus getAclStatus(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.getAclStatus path: {}", path); TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.GET_ACL_STATUS, true, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.GET_ACL_STATUS, true, + tracingContextFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -1027,7 +1040,8 @@ public void access(final Path path, final FsAction mode) throws IOException { Path qualifiedPath = makeQualified(path); try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.ACCESS, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.ACCESS, tracingContextFormat, + listener); this.abfsStore.access(qualifiedPath, mode, tracingContext); } catch (AzureBlobFileSystemException ex) { checkCheckAccessException(path, ex); @@ -1062,7 +1076,8 @@ private boolean fileSystemExists() throws IOException { "AzureBlobFileSystem.fileSystemExists uri: {}", uri); try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.GET_FILESTATUS, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.GET_FILESTATUS, + tracingContextFormat, listener); abfsStore.getFilesystemProperties(tracingContext); } catch (AzureBlobFileSystemException ex) { try { @@ -1083,7 +1098,8 @@ private void createFileSystem() throws IOException { "AzureBlobFileSystem.createFileSystem uri: {}", uri); try { TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.CREATE_FILESYSTEM, tracingContextFormat, listener); + fileSystemID, HdfsOperationConstants.CREATE_FILESYSTEM, + tracingContextFormat, listener); abfsStore.createFilesystem(tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(null, ex); @@ -1310,7 +1326,8 @@ AbfsDelegationTokenManager getDelegationTokenManager() { } @VisibleForTesting - boolean getIsNamespaceEnabled(TracingContext tracingContext) throws AzureBlobFileSystemException { + boolean getIsNamespaceEnabled(TracingContext tracingContext) + throws AzureBlobFileSystemException { return abfsStore.getIsNamespaceEnabled(tracingContext); } @@ -1334,9 +1351,10 @@ public boolean hasPathCapability(final Path path, final String capability) case CommonPathCapabilities.FS_APPEND: return true; case CommonPathCapabilities.FS_ACLS: - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.HAS_PATH_CAPABILITY, tracingContextFormat, listener); - return getIsNamespaceEnabled(tracingContext); + return getIsNamespaceEnabled( + new TracingContext(clientCorrelationID, fileSystemID, + HdfsOperationConstants.HAS_PATH_CAPABILITY, tracingContextFormat, + listener)); default: return super.hasPathCapability(p, capability); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index ae25418792b0c..8ce735877c109 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -280,7 +280,8 @@ private String[] authorityParts(URI uri) throws InvalidUriAuthorityException, In return authorityParts; } - public boolean getIsNamespaceEnabled(TracingContext tracingContext) throws AzureBlobFileSystemException { + public boolean getIsNamespaceEnabled(TracingContext tracingContext) + throws AzureBlobFileSystemException { try { return this.isNamespaceEnabled.toBoolean(); } catch (TrileanConversionException e) { @@ -291,7 +292,8 @@ public boolean getIsNamespaceEnabled(TracingContext tracingContext) throws Azure LOG.debug("Get root ACL status"); try (AbfsPerfInfo perfInfo = startTracking("getIsNamespaceEnabled", "getAclStatus")) { - AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.ROOT_PATH, tracingContext); + AbfsRestOperation op = client + .getAclStatus(AbfsHttpConstants.ROOT_PATH, tracingContext); perfInfo.registerResult(op.getResult()); isNamespaceEnabled = Trilean.getTrilean(true); perfInfo.registerSuccess(true); @@ -344,7 +346,8 @@ public AbfsConfiguration getAbfsConfiguration() { return this.abfsConfiguration; } - public Hashtable getFilesystemProperties(TracingContext tracingContext) throws AzureBlobFileSystemException { + public Hashtable getFilesystemProperties( + TracingContext tracingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("getFilesystemProperties", "getFilesystemProperties")) { LOG.debug("getFilesystemProperties for filesystem: {}", @@ -352,7 +355,8 @@ public Hashtable getFilesystemProperties(TracingContext tracingC final Hashtable parsedXmsProperties; - final AbfsRestOperation op = client.getFilesystemProperties(tracingContext); + final AbfsRestOperation op = client + .getFilesystemProperties(tracingContext); perfInfo.registerResult(op.getResult()); final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); @@ -364,7 +368,8 @@ public Hashtable getFilesystemProperties(TracingContext tracingC } } - public void setFilesystemProperties(final Hashtable properties, TracingContext tracingContext) + public void setFilesystemProperties( + final Hashtable properties, TracingContext tracingContext) throws AzureBlobFileSystemException { if (properties == null || properties.isEmpty()) { LOG.trace("setFilesystemProperties no properties present"); @@ -384,21 +389,22 @@ public void setFilesystemProperties(final Hashtable properties, throw new InvalidAbfsRestOperationException(ex); } - final AbfsRestOperation op = client.setFilesystemProperties(commaSeparatedProperties, tracingContext); + final AbfsRestOperation op = client + .setFilesystemProperties(commaSeparatedProperties, tracingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } - public Hashtable getPathStatus(final Path path, TracingContext tracingContext) - throws AzureBlobFileSystemException { + public Hashtable getPathStatus(final Path path, + TracingContext tracingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("getPathStatus", "getPathStatus")){ LOG.debug("getPathStatus for filesystem: {} path: {}", client.getFileSystem(), path); final Hashtable parsedXmsProperties; - final AbfsRestOperation op = client.getPathStatus(getRelativePath(path), true, - tracingContext); + final AbfsRestOperation op = client + .getPathStatus(getRelativePath(path), true, tracingContext); perfInfo.registerResult(op.getResult()); final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); @@ -411,8 +417,9 @@ public Hashtable getPathStatus(final Path path, TracingContext t } } - public void setPathProperties(final Path path, final Hashtable properties, - TracingContext tracingContext) throws AzureBlobFileSystemException { + public void setPathProperties(final Path path, + final Hashtable properties, TracingContext tracingContext) + throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("setPathProperties", "setPathProperties")){ LOG.debug("setFilesystemProperties for filesystem: {} path: {} with properties: {}", client.getFileSystem(), @@ -425,13 +432,15 @@ public void setPathProperties(final Path path, final Hashtable p } catch (CharacterCodingException ex) { throw new InvalidAbfsRestOperationException(ex); } - final AbfsRestOperation op = client.setPathProperties(getRelativePath(path), commaSeparatedProperties, + final AbfsRestOperation op = client + .setPathProperties(getRelativePath(path), commaSeparatedProperties, tracingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } - public void createFilesystem(TracingContext tracingContext) throws AzureBlobFileSystemException { + public void createFilesystem(TracingContext tracingContext) + throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("createFilesystem", "createFilesystem")){ LOG.debug("createFilesystem for filesystem: {}", client.getFileSystem()); @@ -441,7 +450,8 @@ public void createFilesystem(TracingContext tracingContext) throws AzureBlobFile } } - public void deleteFilesystem(TracingContext tracingContext) throws AzureBlobFileSystemException { + public void deleteFilesystem(TracingContext tracingContext) + throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("deleteFilesystem", "deleteFilesystem")) { LOG.debug("deleteFilesystem for filesystem: {}", client.getFileSystem()); @@ -452,9 +462,9 @@ public void deleteFilesystem(TracingContext tracingContext) throws AzureBlobFile } public OutputStream createFile(final Path path, - final FileSystem.Statistics statistics, - final boolean overwrite, final FsPermission permission, - final FsPermission umask, TracingContext tracingContext) throws AzureBlobFileSystemException { + final FileSystem.Statistics statistics, final boolean overwrite, + final FsPermission permission, final FsPermission umask, + TracingContext tracingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}", @@ -527,16 +537,16 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa final FileSystem.Statistics statistics, final String permission, final String umask, - final boolean isAppendBlob, TracingContext tracingContext) throws AzureBlobFileSystemException { + final boolean isAppendBlob, + TracingContext tracingContext) throws AzureBlobFileSystemException { AbfsRestOperation op; try { // Trigger a create with overwrite=false first so that eTag fetch can be // avoided for cases when no pre-existing file is present (major portion // of create file traffic falls into the case of no pre-existing file). - - op = client.createPath(relativePath, true, false, - permission, umask, isAppendBlob, null, tracingContext); + op = client.createPath(relativePath, true, false, permission, umask, + isAppendBlob, null, tracingContext); } catch (AbfsRestOperationException e) { if (e.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { @@ -598,8 +608,8 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext(boolean isAppend .build(); } - public void createDirectory(final Path path, final FsPermission permission, final FsPermission umask, - TracingContext tracingContext) + public void createDirectory(final Path path, final FsPermission permission, + final FsPermission umask, TracingContext tracingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("createDirectory", "createPath")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); @@ -618,8 +628,8 @@ public void createDirectory(final Path path, final FsPermission permission, fina } } - public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statistics statistics, - TracingContext tracingContext) + public AbfsInputStream openFileForRead(final Path path, + final FileSystem.Statistics statistics, TracingContext tracingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("openFileForRead", "getPathStatus")) { LOG.debug("openFileForRead filesystem: {} path: {}", @@ -628,7 +638,8 @@ public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statist String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getPathStatus(relativePath, false, tracingContext); + final AbfsRestOperation op = client + .getPathStatus(relativePath, false, tracingContext); perfInfo.registerResult(op.getResult()); final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); @@ -662,9 +673,9 @@ private AbfsInputStreamContext populateAbfsInputStreamContext() { .build(); } - public OutputStream openFileForWrite(final Path path, final FileSystem.Statistics statistics, - final boolean overwrite, TracingContext tracingContext) throws - AzureBlobFileSystemException { + public OutputStream openFileForWrite(final Path path, + final FileSystem.Statistics statistics, final boolean overwrite, + TracingContext tracingContext) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("openFileForWrite", "getPathStatus")) { LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {}", client.getFileSystem(), @@ -673,7 +684,8 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getPathStatus(relativePath, false, tracingContext); + final AbfsRestOperation op = client + .getPathStatus(relativePath, false, tracingContext); perfInfo.registerResult(op.getResult()); final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); @@ -706,8 +718,8 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic } } - public void rename(final Path source, final Path destination, TracingContext tracingContext) throws - AzureBlobFileSystemException { + public void rename(final Path source, final Path destination, + TracingContext tracingContext) throws AzureBlobFileSystemException { final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue; @@ -729,8 +741,9 @@ public void rename(final Path source, final Path destination, TracingContext tra do { try (AbfsPerfInfo perfInfo = startTracking("rename", "renamePath")) { - AbfsRestOperation op = client.renamePath(sourceRelativePath, - destinationRelativePath, continuation, tracingContext); + AbfsRestOperation op = client + .renamePath(sourceRelativePath, destinationRelativePath, + continuation, tracingContext); perfInfo.registerResult(op.getResult()); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); perfInfo.registerSuccess(true); @@ -744,8 +757,8 @@ public void rename(final Path source, final Path destination, TracingContext tra } while (shouldContinue); } - public void delete(final Path path, final boolean recursive, TracingContext tracingContext) - throws AzureBlobFileSystemException { + public void delete(final Path path, final boolean recursive, + TracingContext tracingContext) throws AzureBlobFileSystemException { final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue = true; @@ -761,8 +774,8 @@ public void delete(final Path path, final boolean recursive, TracingContext trac do { try (AbfsPerfInfo perfInfo = startTracking("delete", "deletePath")) { - AbfsRestOperation op = client.deletePath( - relativePath, recursive, continuation, tracingContext); + AbfsRestOperation op = client + .deletePath(relativePath, recursive, continuation, tracingContext); perfInfo.registerResult(op.getResult()); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); perfInfo.registerSuccess(true); @@ -776,7 +789,8 @@ public void delete(final Path path, final boolean recursive, TracingContext trac } while (shouldContinue); } - public FileStatus getFileStatus(final Path path, TracingContext tracingContext) throws IOException { + public FileStatus getFileStatus(final Path path, + TracingContext tracingContext) throws IOException { try (AbfsPerfInfo perfInfo = startTracking("getFileStatus", "undetermined")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); LOG.debug("getFileStatus filesystem: {} path: {} isNamespaceEnabled: {}", @@ -849,7 +863,8 @@ public FileStatus getFileStatus(final Path path, TracingContext tracingContext) * @param path The list path. * @return the entries in the path. * */ - public FileStatus[] listStatus(final Path path, TracingContext tracingContext) throws IOException { + public FileStatus[] listStatus(final Path path, TracingContext tracingContext) + throws IOException { return listStatus(path, null, tracingContext); } @@ -866,7 +881,7 @@ public FileStatus[] listStatus(final Path path, TracingContext tracingContext) t * */ @InterfaceStability.Unstable public FileStatus[] listStatus(final Path path, final String startFrom, - TracingContext tracingContext) throws IOException { + TracingContext tracingContext) throws IOException { final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue = true; @@ -890,7 +905,8 @@ public FileStatus[] listStatus(final Path path, final String startFrom, do { try (AbfsPerfInfo perfInfo = startTracking("listStatus", "listPath")) { AbfsRestOperation op = client.listPath(relativePath, false, - abfsConfiguration.getListMaxResults(), continuation, tracingContext); + abfsConfiguration.getListMaxResults(), continuation, + tracingContext); perfInfo.registerResult(op.getResult()); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); ListResultSchema retrievedSchema = op.getResult().getListResultSchema(); @@ -1008,8 +1024,7 @@ private String generateContinuationTokenForNonXns(String path, final String firs } public void setOwner(final Path path, final String owner, final String group, - TracingContext tracingContext) throws - AzureBlobFileSystemException { + TracingContext tracingContext) throws AzureBlobFileSystemException { if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); @@ -1037,8 +1052,7 @@ public void setOwner(final Path path, final String owner, final String group, } public void setPermission(final Path path, final FsPermission permission, - TracingContext tracingContext) throws - AzureBlobFileSystemException { + TracingContext tracingContext) throws AzureBlobFileSystemException { if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); @@ -1053,15 +1067,15 @@ public void setPermission(final Path path, final FsPermission permission, permission); final AbfsRestOperation op = client.setPermission(getRelativePath(path), - String.format(AbfsHttpConstants.PERMISSION_FORMAT, permission.toOctal()), tracingContext); + String.format(AbfsHttpConstants.PERMISSION_FORMAT, + permission.toOctal()), tracingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } public void modifyAclEntries(final Path path, final List aclSpec, - TracingContext tracingContext) throws - AzureBlobFileSystemException { + TracingContext tracingContext) throws AzureBlobFileSystemException { if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); @@ -1081,7 +1095,8 @@ public void modifyAclEntries(final Path path, final List aclSpec, String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getAclStatus(relativePath, useUpn, tracingContext); + final AbfsRestOperation op = client + .getAclStatus(relativePath, useUpn, tracingContext); perfInfoGet.registerResult(op.getResult()); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); @@ -1092,9 +1107,9 @@ public void modifyAclEntries(final Path path, final List aclSpec, perfInfoGet.registerSuccess(true).finishTracking(); try (AbfsPerfInfo perfInfoSet = startTracking("modifyAclEntries", "setAcl")) { - final AbfsRestOperation setAclOp - = client.setAcl(relativePath, - AbfsAclHelper.serializeAclSpec(aclEntries), eTag, tracingContext); + final AbfsRestOperation setAclOp = client + .setAcl(relativePath, AbfsAclHelper.serializeAclSpec(aclEntries), + eTag, tracingContext); perfInfoSet.registerResult(setAclOp.getResult()) .registerSuccess(true) .registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT); @@ -1103,7 +1118,7 @@ public void modifyAclEntries(final Path path, final List aclSpec, } public void removeAclEntries(final Path path, final List aclSpec, - TracingContext tracingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) throws AzureBlobFileSystemException { if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); @@ -1123,7 +1138,8 @@ public void removeAclEntries(final Path path, final List aclSpec, String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getAclStatus(relativePath, isUpnFormat, tracingContext); + final AbfsRestOperation op = client + .getAclStatus(relativePath, isUpnFormat, tracingContext); perfInfoGet.registerResult(op.getResult()); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); @@ -1134,9 +1150,9 @@ public void removeAclEntries(final Path path, final List aclSpec, perfInfoGet.registerSuccess(true).finishTracking(); try (AbfsPerfInfo perfInfoSet = startTracking("removeAclEntries", "setAcl")) { - final AbfsRestOperation setAclOp = - client.setAcl(relativePath, - AbfsAclHelper.serializeAclSpec(aclEntries), eTag, tracingContext); + final AbfsRestOperation setAclOp = client + .setAcl(relativePath, AbfsAclHelper.serializeAclSpec(aclEntries), + eTag, tracingContext); perfInfoSet.registerResult(setAclOp.getResult()) .registerSuccess(true) .registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT); @@ -1144,7 +1160,8 @@ public void removeAclEntries(final Path path, final List aclSpec, } } - public void removeDefaultAcl(final Path path, TracingContext tracingContext) throws AzureBlobFileSystemException { + public void removeDefaultAcl(final Path path, TracingContext tracingContext) + throws AzureBlobFileSystemException { if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); @@ -1159,7 +1176,8 @@ public void removeDefaultAcl(final Path path, TracingContext tracingContext) thr String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getAclStatus(relativePath, tracingContext); + final AbfsRestOperation op = client + .getAclStatus(relativePath, tracingContext); perfInfoGet.registerResult(op.getResult()); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); final Map aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); @@ -1176,9 +1194,9 @@ public void removeDefaultAcl(final Path path, TracingContext tracingContext) thr perfInfoGet.registerSuccess(true).finishTracking(); try (AbfsPerfInfo perfInfoSet = startTracking("removeDefaultAcl", "setAcl")) { - final AbfsRestOperation setAclOp = - client.setAcl(relativePath, - AbfsAclHelper.serializeAclSpec(aclEntries), eTag, tracingContext); + final AbfsRestOperation setAclOp = client + .setAcl(relativePath, AbfsAclHelper.serializeAclSpec(aclEntries), + eTag, tracingContext); perfInfoSet.registerResult(setAclOp.getResult()) .registerSuccess(true) .registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT); @@ -1186,7 +1204,8 @@ public void removeDefaultAcl(final Path path, TracingContext tracingContext) thr } } - public void removeAcl(final Path path, TracingContext tracingContext) throws AzureBlobFileSystemException { + public void removeAcl(final Path path, TracingContext tracingContext) + throws AzureBlobFileSystemException { if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); @@ -1201,7 +1220,8 @@ public void removeAcl(final Path path, TracingContext tracingContext) throws Azu String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getAclStatus(relativePath, tracingContext); + final AbfsRestOperation op = client + .getAclStatus(relativePath, tracingContext); perfInfoGet.registerResult(op.getResult()); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); @@ -1215,9 +1235,9 @@ public void removeAcl(final Path path, TracingContext tracingContext) throws Azu perfInfoGet.registerSuccess(true).finishTracking(); try (AbfsPerfInfo perfInfoSet = startTracking("removeAcl", "setAcl")) { - final AbfsRestOperation setAclOp = - client.setAcl(relativePath, - AbfsAclHelper.serializeAclSpec(newAclEntries), eTag, tracingContext); + final AbfsRestOperation setAclOp = client + .setAcl(relativePath, AbfsAclHelper.serializeAclSpec(newAclEntries), + eTag, tracingContext); perfInfoSet.registerResult(setAclOp.getResult()) .registerSuccess(true) .registerAggregates(perfInfoGet.getTrackingStart(), GET_SET_AGGREGATE_COUNT); @@ -1226,7 +1246,7 @@ public void removeAcl(final Path path, TracingContext tracingContext) throws Azu } public void setAcl(final Path path, final List aclSpec, - TracingContext tracingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) throws AzureBlobFileSystemException { if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); @@ -1246,7 +1266,8 @@ public void setAcl(final Path path, final List aclSpec, String relativePath = getRelativePath(path); - final AbfsRestOperation op = client.getAclStatus(relativePath, isUpnFormat, tracingContext); + final AbfsRestOperation op = client + .getAclStatus(relativePath, isUpnFormat, tracingContext); perfInfoGet.registerResult(op.getResult()); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); @@ -1267,7 +1288,8 @@ public void setAcl(final Path path, final List aclSpec, } } - public AclStatus getAclStatus(final Path path, TracingContext tracingContext) throws IOException { + public AclStatus getAclStatus(final Path path, TracingContext tracingContext) + throws IOException { if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "This operation is only valid for storage accounts with the hierarchical namespace enabled."); @@ -1280,7 +1302,8 @@ public AclStatus getAclStatus(final Path path, TracingContext tracingContext) th client.getFileSystem(), path); - AbfsRestOperation op = client.getAclStatus(getRelativePath(path), tracingContext); + AbfsRestOperation op = client + .getAclStatus(getRelativePath(path), tracingContext); AbfsHttpOperation result = op.getResult(); perfInfo.registerResult(result); @@ -1313,8 +1336,8 @@ public AclStatus getAclStatus(final Path path, TracingContext tracingContext) th } } - public void access(final Path path, final FsAction mode, TracingContext tracingContext) - throws AzureBlobFileSystemException { + public void access(final Path path, final FsAction mode, + TracingContext tracingContext) throws AzureBlobFileSystemException { LOG.debug("access for filesystem: {}, path: {}, mode: {}", this.client.getFileSystem(), path, mode); if (!this.abfsConfiguration.isCheckAccessEnabled() From f3f91f4390a365f85ed64b34cd667163fa583c1b Mon Sep 17 00:00:00 2001 From: Sumangala Date: Thu, 3 Dec 2020 12:06:37 +0530 Subject: [PATCH 53/77] merge conflict --- .../hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java | 4 ++-- .../fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java | 6 ++++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index 8d21274ed8dfd..2b2860fb1d2c0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -147,10 +147,10 @@ protected boolean getIsNamespaceEnabled(AzureBlobFileSystem fs) } public TracingContext getTestTracingContext(AzureBlobFileSystem fs, - boolean isCont) { + boolean needsPrimaryReqId) { AbfsConfiguration abfsConf = fs.getAbfsStore().getAbfsConfiguration(); return new TracingContext(abfsConf.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.TEST_OP, isCont, + fs.getFileSystemID(), HdfsOperationConstants.TEST_OP, needsPrimaryReqId, abfsConf.getTracingContextFormat(), null); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java index 75adaf344c939..a65676d128df9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java @@ -391,7 +391,8 @@ public void testSignatureMask() throws Exception { String src = "/testABC/test.xt"; fs.create(new Path(src)); AbfsRestOperation abfsHttpRestOperation = fs.getAbfsClient() - .renamePath(src, "/testABC" + "/abc.txt", null); + .renamePath(src, "/testABC" + "/abc.txt", null, + getTestTracingContext(fs, false)); AbfsHttpOperation result = abfsHttpRestOperation.getResult(); String url = result.getSignatureMaskedUrl(); String encodedUrl = result.getSignatureMaskedEncodedUrl(); @@ -407,7 +408,8 @@ public void testSignatureMask() throws Exception { public void testSignatureMaskOnExceptionMessage() throws Exception { intercept(IOException.class, "sig=XXXX", () -> getFileSystem().getAbfsClient() - .renamePath("testABC/test.xt", "testABC/abc.txt", null)); + .renamePath("testABC/test.xt", "testABC/abc.txt", null, + getTestTracingContext(getFileSystem(), false))); } } From 18ea7f0445555dfbf43aa175a0aa642521ff39f9 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Thu, 3 Dec 2020 13:14:19 +0530 Subject: [PATCH 54/77] pr changes --- .../hadoop/fs/azurebfs/services/AbfsHttpOperation.java | 4 ++-- .../hadoop/fs/azurebfs/services/AbfsInputStream.java | 10 +++++----- .../hadoop/fs/azurebfs/services/AbfsOutputStream.java | 2 +- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index b1a2a7807f914..de950b531be2b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -26,11 +26,11 @@ import java.net.URL; import java.net.URLEncoder; import java.util.List; -import java.util.UUID; import javax.net.ssl.HttpsURLConnection; import javax.net.ssl.SSLSocketFactory; +import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.codehaus.jackson.JsonFactory; import org.codehaus.jackson.JsonParser; @@ -155,7 +155,7 @@ public String getResponseHeader(String httpHeader) { return connection.getHeaderField(httpHeader); } - @org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting + @VisibleForTesting public String getRequestHeader(String httpHeader) { return connection.getRequestProperties().get(httpHeader).toString(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 9f8a2568de80a..6891388d46d42 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -24,13 +24,10 @@ import java.net.HttpURLConnection; import java.util.UUID; -import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.utils.Listener; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,9 +36,12 @@ import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.FileSystem.Statistics; import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.utils.CachedSASToken; +import org.apache.hadoop.fs.azurebfs.utils.Listener; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import static org.apache.hadoop.util.StringUtils.toLowerCase; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index a04c23c20938b..37f78ce53e83b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -36,7 +36,6 @@ import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; -import org.apache.hadoop.fs.azurebfs.utils.Listener; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,6 +43,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.utils.CachedSASToken; +import org.apache.hadoop.fs.azurebfs.utils.Listener; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.io.ElasticByteBufferPool; import org.apache.hadoop.fs.FileSystem.Statistics; From af74d8e464bfd736614dabbdd54cd37113d40126 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Thu, 3 Dec 2020 14:58:17 +0530 Subject: [PATCH 55/77] stream id test --- .../hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java | 8 ++++++-- .../hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java | 4 +++- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index e100a91059e8a..f8e35d88e93c1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -78,7 +78,9 @@ public void testReadAheadRequestID() throws java.io.IOException { try (FSDataOutputStream stream = fs.create(TEST_PATH)) { ((AbfsOutputStream) stream.getWrappedStream()).registerListener( new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.CREATE, false, 0)); + fs.getFileSystemID(), HdfsOperationConstants.CREATE, false, 0, + ((AbfsOutputStream) stream.getWrappedStream()) + .getStreamID())); stream.write(b); } @@ -90,7 +92,9 @@ public void testReadAheadRequestID() throws java.io.IOException { try (FSDataInputStream inputStream = fs.open(TEST_PATH)) { ((AbfsInputStream) inputStream.getWrappedStream()).registerListener( new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.READ, false, 0)); + fs.getFileSystemID(), HdfsOperationConstants.READ, false, 0, + ((AbfsInputStream) inputStream.getWrappedStream()) + .getStreamID())); result = inputStream.read(readBuffer, 0, bufferSize*4); } fs.registerListener(null); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index dc5789c4ca0e2..1b49404b326d8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -33,6 +33,7 @@ import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.hamcrest.core.IsEqual; @@ -318,7 +319,8 @@ public void testTracingHeaderForAppendBlob() throws Exception { ((AbfsOutputStream) out.getWrappedStream()).registerListener( new TracingHeaderValidator( fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.CREATE, false, 0)); + fs.getFileSystemID(), HdfsOperationConstants.CREATE, false, 0, + ((AbfsOutputStream) out.getWrappedStream()).getStreamID())); out.write(buf); out.hsync(); } From 3a4eb41661d9ffc9819a5de66de59ca56c0658d7 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Fri, 4 Dec 2020 14:20:24 +0530 Subject: [PATCH 56/77] documentation md --- .../azurebfs/constants/ConfigurationKeys.java | 3 +-- .../hadoop-azure/src/site/markdown/abfs.md | 22 +++++++++++++++++++ .../utils/TracingHeaderValidator.java | 3 ++- 3 files changed, 25 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index 05e3ac9ddfb17..838226c7493a0 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -92,8 +92,7 @@ public final class ConfigurationKeys { * x-ms-client-request-Id header. Defaults to empty string if the length and * character constraints are not satisfied. **/ public static final String FS_AZURE_CLIENT_CORRELATIONID = "fs.azure.client.correlationid"; - public static final String FS_AZURE_TRACINGCONTEXT_FORMAT = "fs.azure" - + ".tracingcontext.format"; + public static final String FS_AZURE_TRACINGCONTEXT_FORMAT = "fs.azure.tracingcontext.format"; public static final String FS_AZURE_CLUSTER_NAME = "fs.azure.cluster.name"; public static final String FS_AZURE_CLUSTER_TYPE = "fs.azure.cluster.type"; public static final String FS_AZURE_SSL_CHANNEL_MODE_KEY = "fs.azure.ssl.channel.mode"; diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md index a4188111e0ebb..e6e13adc7e078 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md @@ -729,6 +729,28 @@ Consult the javadocs for `org.apache.hadoop.fs.azurebfs.constants.ConfigurationK `org.apache.hadoop.fs.azurebfs.AbfsConfiguration` for the full list of configuration options and their default values. +### Client Correlation Options + +#### 1. Client CorrelationId Option + +Config `fs.azure.client.correlationid` provides an option to correlate client +requests using this client-provided identifier. This Id will be visible in Azure +Storage Analytics logs in the `request-id-header` field. +Reference: [Storage Analytics log format](https://docs.microsoft.com/en-us/rest/api/storageservices/storage-analytics-log-format) + +This config accepts a string which can be maximum of 72 characters and should +contain alphanumeric characters and/or hyphens only. Defaults to empty string if +input is invalid. + +#### 1. Correlation IDs Display Options + +Config `fs.azure.tracingcontext.format` provides an option to select the + format of IDs included in the `request-id-header`. This config accepts a + String value corresponding to the following enum options. + `SINGLE_ID_FORMAT`: clientRequestId + `ALL_ID_FORMAT` : all IDs (default) + `TWO_ID_FORMAT` : clientCorrelationId:clientRequestId + ### Flush Options #### 1. Azure Blob File System Flush Options diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java index 3e6e8b1d95396..990e50b007a59 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java @@ -106,7 +106,8 @@ private void validateBasicFormat(String[] id_list) { Assertions.assertThat(id_list[5]).describedAs("Operation name incorrect") .isEqualTo(operation); int retryCount = Integer.parseInt(id_list[6]); - Assertions.assertThat(retryCount).describedAs("Retry count incorrect") + Assertions.assertThat(retryCount) + .describedAs("Retry was required due to issue on server side") .isEqualTo(retryNum); } From f90bbb399437d461a9bc4ee614c27a3d03bae227 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Sat, 5 Dec 2020 12:27:23 +0530 Subject: [PATCH 57/77] fix yetus bugs --- .../fs/azurebfs/AzureBlobFileSystem.java | 2 +- .../fs/azurebfs/AzureBlobFileSystemStore.java | 4 +- .../constants/FileSystemConfigurations.java | 2 + .../constants/HdfsOperationConstants.java | 20 +++++ .../fs/azurebfs/services/AbfsClient.java | 4 +- .../fs/azurebfs/services/AbfsInputStream.java | 9 ++- .../azurebfs/services/AbfsOutputStream.java | 5 +- .../azurebfs/services/AbfsRestOperation.java | 2 +- .../hadoop/fs/azurebfs/utils/Listener.java | 18 +++++ .../fs/azurebfs/utils/TracingContext.java | 20 +++-- .../azurebfs/utils/TracingContextFormat.java | 18 +++++ .../azurebfs/ITestAbfsReadWriteAndSeek.java | 68 ++++++++--------- .../ITestAzureBlobFileSystemFlush.java | 1 - .../fs/azurebfs/TestTracingContext.java | 40 +++++++--- .../services/TestAbfsInputStream.java | 6 +- .../utils/TracingHeaderValidator.java | 74 ++++++++++++------- 16 files changed, 202 insertions(+), 91 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index e87261a248d14..1e78cd1ffad41 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -179,7 +179,7 @@ public boolean isSecureScheme() { public URI getUri() { return this.uri; } - + public void registerListener(Listener listener1) { listener = listener1; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 78c9db16a3fee..c338406a55185 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -47,7 +47,6 @@ import java.util.Locale; import java.util.Map; import java.util.Set; -import java.util.UUID; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -864,6 +863,7 @@ public FileStatus getFileStatus(final Path path, /** * @param path The list path. + * @param tracingContext Tracks identifiers for request header * @return the entries in the path. * */ public FileStatus[] listStatus(final Path path, TracingContext tracingContext) @@ -879,7 +879,7 @@ public FileStatus[] listStatus(final Path path, TracingContext tracingContext) * Notice that if startFrom is a non-existent entry name, then the list response contains * all entries after this non-existent entry in lexical order: * listStatus(Path("/folder"), "cfile") will return "/folder/hfile" and "/folder/ifile". - * + * @param tracingContext Tracks identifiers for request header * @return the entries in the path start from "startFrom" in lexical order. * */ @InterfaceStability.Unstable diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index 49fc58ba56b15..2fa8ba77df0b4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -98,5 +98,7 @@ public final class FileSystemConfigurations { public static final boolean DEFAULT_DELETE_CONSIDERED_IDEMPOTENT = true; public static final int DEFAULT_CLOCK_SKEW_WITH_SERVER_IN_MS = 5 * 60 * 1000; // 5 mins + public static final int STREAM_ID_LEN = 12; + private FileSystemConfigurations() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HdfsOperationConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HdfsOperationConstants.java index 9029796190076..297fe3a35e8dd 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HdfsOperationConstants.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HdfsOperationConstants.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.fs.azurebfs.constants; public final class HdfsOperationConstants { @@ -25,4 +43,6 @@ public final class HdfsOperationConstants { public static final String SET_ACL = "SA"; public static final String TEST_OP = "TS"; public static final String WRITE = "WR"; + + private HdfsOperationConstants() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index e28b3aeca79dc..4c57d246d6a11 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -365,6 +365,7 @@ public AbfsRestOperation renamePath(String source, final String destination, fin * @param renameRequestStartTime startTime for the rename request * @param op Rename request REST operation response * @param destination rename destination path + * @param tracingContext Tracks identifiers for request header * @return REST operation response post idempotency check * @throws AzureBlobFileSystemException if GetFileStatus hits any exception */ @@ -381,7 +382,7 @@ public AbfsRestOperation renameIdempotencyCheckOp( try { final AbfsRestOperation destStatusOp = getPathStatus(destination, - false , tracingContext); + false, tracingContext); if (destStatusOp.getResult().getStatusCode() == HttpURLConnection.HTTP_OK) { String lmt = destStatusOp.getResult().getResponseHeader( @@ -766,6 +767,7 @@ public AbfsRestOperation getAclStatus(final String path, final boolean useUPN, * * @param path Path for which access check needs to be performed * @param rwx The permission to be checked on the path + * @param tracingContext Tracks identifiers for request header * @return The {@link AbfsRestOperation} object for the operation * @throws AzureBlobFileSystemException in case of bad requests */ diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 6891388d46d42..bda5cd55bfffb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -43,6 +43,7 @@ import org.apache.hadoop.fs.azurebfs.utils.Listener; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.STREAM_ID_LEN; import static org.apache.hadoop.util.StringUtils.toLowerCase; /** @@ -75,7 +76,7 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer, private int limit = 0; // offset of next byte to be read into buffer from service (i.e., upper marker+1 // of valid bytes in buffer) private boolean closed = false; - public TracingContext tracingContext; + private TracingContext tracingContext; /** Stream statistics. */ private final AbfsInputStreamStatistics streamStatistics; @@ -122,7 +123,7 @@ public String getPath() { } private String getInputStreamID() { - return StringUtils.right(UUID.randomUUID().toString(), 12); + return StringUtils.right(UUID.randomUUID().toString(), STREAM_ID_LEN); } public void registerListener(Listener listener1) { @@ -473,6 +474,10 @@ public synchronized long getPos() throws IOException { return fCursor - limit + bCursor; } + public TracingContext getTracingContext() { + return tracingContext; + } + /** * Seeks a different copy of the data. Returns true if * found a new source, false otherwise. diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 37f78ce53e83b..370d70b2854bb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -51,6 +51,7 @@ import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.Syncable; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.STREAM_ID_LEN; import static org.apache.hadoop.io.IOUtils.wrapException; /** @@ -148,7 +149,7 @@ public AbfsOutputStream( } private String getOutputStreamID() { - return StringUtils.right(UUID.randomUUID().toString(), 12); + return StringUtils.right(UUID.randomUUID().toString(), STREAM_ID_LEN); } /** @@ -282,7 +283,7 @@ public void hflush() throws IOException { flushInternal(false); } } - + public String getStreamID() { return outputStreamID; } 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 89b04eb3f1200..11c2b02245ebc 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 @@ -23,7 +23,6 @@ import java.net.URL; import java.net.UnknownHostException; import java.util.List; -import java.util.UUID; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; @@ -170,6 +169,7 @@ String getSasToken() { } /** + * @param tracingContext Tracks identifiers for request header * Executes the REST operation with retry, by issuing one or more * HTTP operations. */ diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java index 4f74c7f6da5c0..333a410d6c130 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.fs.azurebfs.utils; public interface Listener { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index af20ae8aae3b0..78daa181d54be 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -60,8 +60,9 @@ public TracingContext(String clientCorrelationID, String fileSystemID, this(clientCorrelationID, fileSystemID, hadoopOpName, tracingContextFormat, listener); primaryRequestID = needsPrimaryReqId ? UUID.randomUUID().toString() : ""; - if (listener != null) + if (listener != null) { listener.updatePrimaryRequestID(primaryRequestID); + } } public TracingContext(TracingContext originalTracingContext) { @@ -93,8 +94,9 @@ public void generateClientRequestID() { public void setPrimaryRequestID() { primaryRequestID = UUID.randomUUID().toString(); - if (listener != null) + if (listener != null) { listener.updatePrimaryRequestID(primaryRequestID); + } } public void setStreamID(String stream) { @@ -114,15 +116,19 @@ public void setListener(Listener listener) { } public String toString() { - String header = clientRequestID; //case 0, no IDs for correlation + String header; switch (format) { case ALL_ID_FORMAT: - header = clientCorrelationID + ":" + header + ":" + fileSystemID + ":" - + primaryRequestID + ":" + streamID + ":" + hadoopOpName + ":" - + retryCount; + header = + clientCorrelationID + ":" + clientRequestID + ":" + fileSystemID + ":" + + primaryRequestID + ":" + streamID + ":" + hadoopOpName + ":" + + retryCount; break; case TWO_ID_FORMAT: - header = clientCorrelationID + ":" + header; + header = clientCorrelationID + ":" + clientRequestID; + break; + default: + header = clientRequestID; //case SINGLE_ID_FORMAT } if (listener != null) { listener.callTracingHeaderValidator(header, format); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java index 0be91c9252415..99f3b9ab23d32 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.fs.azurebfs.utils; public enum TracingContextFormat { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index f8e35d88e93c1..1226a3e355b7d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -66,40 +66,6 @@ public void testReadAndWriteWithDifferentBufferSizesAndSeek() throws Exception { testReadWriteAndSeek(size); } - @Test - public void testReadAheadRequestID() throws java.io.IOException { - final AzureBlobFileSystem fs = getFileSystem(); - final AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration(); - int bufferSize = 32; - abfsConfiguration.setReadBufferSize(bufferSize); - - final byte[] b = new byte[bufferSize * 10]; - new Random().nextBytes(b); - try (FSDataOutputStream stream = fs.create(TEST_PATH)) { - ((AbfsOutputStream) stream.getWrappedStream()).registerListener( - new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.CREATE, false, 0, - ((AbfsOutputStream) stream.getWrappedStream()) - .getStreamID())); - stream.write(b); - } - - final byte[] readBuffer = new byte[4 * bufferSize]; - int result; - fs.registerListener( - new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.OPEN, false, 0)); - try (FSDataInputStream inputStream = fs.open(TEST_PATH)) { - ((AbfsInputStream) inputStream.getWrappedStream()).registerListener( - new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.READ, false, 0, - ((AbfsInputStream) inputStream.getWrappedStream()) - .getStreamID())); - result = inputStream.read(readBuffer, 0, bufferSize*4); - } - fs.registerListener(null); - } - private void testReadWriteAndSeek(int bufferSize) throws Exception { final AzureBlobFileSystem fs = getFileSystem(); final AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration(); @@ -136,4 +102,38 @@ private void testReadWriteAndSeek(int bufferSize) throws Exception { assertNotEquals("data read in final read()", -1, result); assertArrayEquals(readBuffer, b); } + + @Test + public void testReadAheadRequestID() throws java.io.IOException { + final AzureBlobFileSystem fs = getFileSystem(); + final AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration(); + int bufferSize = MIN_BUFFER_SIZE; + abfsConfiguration.setReadBufferSize(bufferSize); + + final byte[] b = new byte[bufferSize * 10]; + new Random().nextBytes(b); + try (FSDataOutputStream stream = fs.create(TEST_PATH)) { + ((AbfsOutputStream) stream.getWrappedStream()).registerListener( + new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.CREATE, false, 0, + ((AbfsOutputStream) stream.getWrappedStream()) + .getStreamID())); + stream.write(b); + } + + final byte[] readBuffer = new byte[4 * bufferSize]; + int result; + fs.registerListener( + new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.OPEN, false, 0)); + try (FSDataInputStream inputStream = fs.open(TEST_PATH)) { + ((AbfsInputStream) inputStream.getWrappedStream()).registerListener( + new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.READ, false, 0, + ((AbfsInputStream) inputStream.getWrappedStream()) + .getStreamID())); + result = inputStream.read(readBuffer, 0, bufferSize*4); + } + fs.registerListener(null); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index 1b49404b326d8..b29d769932c16 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -33,7 +33,6 @@ import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; -import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.hamcrest.core.IsEqual; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index e08777faf9701..909446099bc0a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.fs.azurebfs; import org.apache.hadoop.fs.azurebfs.services.AuthType; @@ -24,12 +42,13 @@ import java.util.HashMap; import java.util.Map; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE; + public class TestTracingContext extends AbstractAbfsIntegrationTest { private static final String[] CLIENT_CORRELATIONID_LIST = { "valid-corr-id-123", "inval!d", ""}; private static final int HTTP_CREATED = 201; - private final String EMPTY_STRING = ""; - String GUID_PATTERN = "[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}"; public TestTracingContext() throws Exception { super(); @@ -72,12 +91,12 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(tracingContext); String path = getRelativePath(new Path("/testDir")); - String permission = isNamespaceEnabled ? - getOctalNotation(FsPermission.getDirDefault()) : - null; - String umask = isNamespaceEnabled ? - getOctalNotation(FsPermission.getUMask(fs.getConf())) : - null; + String permission = isNamespaceEnabled + ? getOctalNotation(FsPermission.getDirDefault()) + : null; + String umask = isNamespaceEnabled + ? getOctalNotation(FsPermission.getUMask(fs.getConf())) + : null; //request should not fail for invalid clientCorrelationID AbfsRestOperation op = fs.getAbfsClient() @@ -106,9 +125,10 @@ public void runCorrelationTestForAllMethods() throws Exception { testClasses.put(new ITestAzureBlobFileSystemListStatus(), //liststatus ITestAzureBlobFileSystemListStatus.class.getMethod("testListPath")); - testClasses.put(new ITestAbfsReadWriteAndSeek(32), //open, read, write + testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE), //open, + // read, write ITestAbfsReadWriteAndSeek.class.getMethod("testReadAheadRequestID")); - testClasses.put(new ITestAbfsReadWriteAndSeek(32), //read (bypassreadahead) + testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE), //read (bypassreadahead) ITestAbfsReadWriteAndSeek.class .getMethod("testReadAndWriteWithDifferentBufferSizesAndSeek")); testClasses.put(new ITestAzureBlobFileSystemAppend(), //append diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index e82320456b701..16297786788f7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -142,13 +142,13 @@ public AbfsInputStream getAbfsInputStream(AbfsClient abfsClient, private void queueReadAheads(AbfsInputStream inputStream) { // Mimic AbfsInputStream readAhead queue requests ReadBufferManager.getBufferManager() - .queueReadAhead(inputStream, 0, ONE_KB, inputStream.tracingContext); + .queueReadAhead(inputStream, 0, ONE_KB, inputStream.getTracingContext()); ReadBufferManager.getBufferManager() .queueReadAhead(inputStream, ONE_KB, ONE_KB, - inputStream.tracingContext); + inputStream.getTracingContext()); ReadBufferManager.getBufferManager() .queueReadAhead(inputStream, TWO_KB, TWO_KB, - inputStream.tracingContext); + inputStream.getTracingContext()); } private void verifyReadCallCount(AbfsClient client, int count) throws diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java index 990e50b007a59..daee2029552a7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java @@ -1,17 +1,35 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.fs.azurebfs.utils; import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.assertj.core.api.Assertions; public class TracingHeaderValidator implements Listener { - String clientCorrelationID; - String fileSystemID; - String primaryRequestID = ""; - boolean needsPrimaryRequestID; - String streamID = ""; - String operation; - int retryNum; - String GUID_PATTERN = "[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}"; + private String clientCorrelationID; + private String fileSystemID; + private String primaryRequestID = ""; + private boolean needsPrimaryRequestID; + private String streamID = ""; + private String operation; + private int retryNum; + private String GUID_PATTERN = "^[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}$"; private TracingContextFormat format; // client-req-id as per docs: ^[{(]?[0-9a-f]{8}[-]?([0-9a-f]{4}[-]?) // {3}[0-9a-f]{12}[)}]?$ @@ -50,62 +68,64 @@ public TracingHeaderValidator(String clientCorrelationID, String fileSystemID, } private void validateTracingHeader(String tracingContextHeader) { - String[] id_list = tracingContextHeader.split(":"); - validateBasicFormat(id_list); - if (format != TracingContextFormat.ALL_ID_FORMAT) + String[] idList = tracingContextHeader.split(":"); + validateBasicFormat(idList); + if (format != TracingContextFormat.ALL_ID_FORMAT) { return; - if (!primaryRequestID.isEmpty() && !id_list[3].isEmpty()) { - Assertions.assertThat(id_list[3]) + } + if (!primaryRequestID.isEmpty() && !idList[3].isEmpty()) { + Assertions.assertThat(idList[3]) .describedAs("PrimaryReqID should be common for these requests") .isEqualTo(primaryRequestID); } if (!streamID.isEmpty()) { - Assertions.assertThat(id_list[4]) + Assertions.assertThat(idList[4]) .describedAs("Stream id should be common for these requests") .isEqualTo(streamID); } } - private void validateBasicFormat(String[] id_list) { + private void validateBasicFormat(String[] idList) { if (format == TracingContextFormat.ALL_ID_FORMAT) { - Assertions.assertThat(id_list) + Assertions.assertThat(idList) .describedAs("header should have 7 elements").hasSize(7); } else if (format == TracingContextFormat.TWO_ID_FORMAT) { - Assertions.assertThat(id_list) + Assertions.assertThat(idList) .describedAs("header should have 2 elements").hasSize(2); } else { - Assertions.assertThat(id_list).describedAs("header should have 1 element") + Assertions.assertThat(idList).describedAs("header should have 1 element") .hasSize(1); - Assertions.assertThat(id_list[0]) + Assertions.assertThat(idList[0]) .describedAs("Client request ID is a guid").matches(GUID_PATTERN); return; } if (clientCorrelationID.matches("[a-zA-Z0-9-]*")) { - Assertions.assertThat(id_list[0]) + Assertions.assertThat(idList[0]) .describedAs("Correlation ID should match config") .isEqualTo(clientCorrelationID); } else { - Assertions.assertThat(id_list[0]) + Assertions.assertThat(idList[0]) .describedAs("Invalid config should be replaced with empty string") .isEmpty(); } - Assertions.assertThat(id_list[1]).describedAs("Client request ID is a guid") + Assertions.assertThat(idList[1]).describedAs("Client request ID is a guid") .matches(GUID_PATTERN); - if (format != TracingContextFormat.ALL_ID_FORMAT) + if (format != TracingContextFormat.ALL_ID_FORMAT) { return; + } - Assertions.assertThat(id_list[2]).describedAs("Filesystem ID incorrect") + Assertions.assertThat(idList[2]).describedAs("Filesystem ID incorrect") .isEqualTo(fileSystemID); if (needsPrimaryRequestID && !operation .equals(HdfsOperationConstants.READ)) { - Assertions.assertThat(id_list[3]).describedAs("should have primaryReqId") + Assertions.assertThat(idList[3]).describedAs("should have primaryReqId") .isNotEmpty(); } - Assertions.assertThat(id_list[5]).describedAs("Operation name incorrect") + Assertions.assertThat(idList[5]).describedAs("Operation name incorrect") .isEqualTo(operation); - int retryCount = Integer.parseInt(id_list[6]); + int retryCount = Integer.parseInt(idList[6]); Assertions.assertThat(retryCount) .describedAs("Retry was required due to issue on server side") .isEqualTo(retryNum); From a5b64ed6af46fca88dbe7eca9c60b318cf8728ba Mon Sep 17 00:00:00 2001 From: Sumangala Date: Mon, 7 Dec 2020 15:41:36 +0530 Subject: [PATCH 58/77] fix randomread getTC failure --- .../azurebfs/services/AbfsHttpOperation.java | 2 +- .../azurebfs/AbstractAbfsIntegrationTest.java | 19 +++++++++++++++---- .../services/TestAbfsInputStream.java | 2 +- 3 files changed, 17 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index de950b531be2b..d56a57c2feacf 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -30,7 +30,7 @@ import javax.net.ssl.HttpsURLConnection; import javax.net.ssl.SSLSocketFactory; -import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.codehaus.jackson.JsonFactory; import org.codehaus.jackson.JsonParser; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index 2b2860fb1d2c0..8d042c7fa3012 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -148,10 +149,20 @@ protected boolean getIsNamespaceEnabled(AzureBlobFileSystem fs) public TracingContext getTestTracingContext(AzureBlobFileSystem fs, boolean needsPrimaryReqId) { - AbfsConfiguration abfsConf = fs.getAbfsStore().getAbfsConfiguration(); - return new TracingContext(abfsConf.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.TEST_OP, needsPrimaryReqId, - abfsConf.getTracingContextFormat(), null); + String correlationId, fsId; + TracingContextFormat format; + if (fs == null) { + correlationId = "test-corr-id"; + fsId = "test-filesystem-id"; + format = TracingContextFormat.ALL_ID_FORMAT; + } else { + AbfsConfiguration abfsConf = fs.getAbfsStore().getAbfsConfiguration(); + correlationId = abfsConf.getClientCorrelationID(); + fsId = fs.getFileSystemID(); + format = abfsConf.getTracingContextFormat(); + } + return new TracingContext(correlationId, fsId, + HdfsOperationConstants.TEST_OP, needsPrimaryReqId, format, null); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index 16297786788f7..2ca9932abdd09 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -103,7 +103,7 @@ private AbfsInputStream getAbfsInputStream(AbfsClient mockAbfsClient, THREE_KB, inputStreamContext.withReadBufferSize(ONE_KB).withReadAheadQueueDepth(10).withReadAheadBlockSize(ONE_KB), "eTag", - getTestTracingContext(getFileSystem(), false)); + getTestTracingContext(null, false)); inputStream.setCachedSasToken( TestCachedSASToken.getTestCachedSASTokenInstance()); From d2bf54c1423aea99a21fa8194debea985f6ced01 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Thu, 24 Dec 2020 10:29:00 +0530 Subject: [PATCH 59/77] addressing pr comments --- .../hadoop/fs/azurebfs/AzureBlobFileSystem.java | 3 ++- .../hadoop/fs/azurebfs/services/AbfsInputStream.java | 11 ++++++----- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 1e78cd1ffad41..accaed4ad5ef1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -1287,7 +1287,8 @@ FileSystem.Statistics getFsStatistics() { return this.statistics; } - public void setListenerOperation(String operation) { + @VisibleForTesting + void setListenerOperation(String operation) { listener.setOperation(operation); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index bda5cd55bfffb..5a9ecde665a33 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -126,11 +126,6 @@ private String getInputStreamID() { return StringUtils.right(UUID.randomUUID().toString(), STREAM_ID_LEN); } - public void registerListener(Listener listener1) { - listener = listener1; - tracingContext.setListener(listener); - } - @Override public int read() throws IOException { byte[] b = new byte[1]; @@ -561,6 +556,12 @@ public AbfsInputStreamStatistics getStreamStatistics() { return streamStatistics; } + @VisibleForTesting + public void registerListener(Listener listener1) { + listener = listener1; + tracingContext.setListener(listener); + } + /** * Getter for bytes read from readAhead buffer that fills asynchronously. * From 98e0fb0b0324cd7240cd17e956cdf1919495b8c2 Mon Sep 17 00:00:00 2001 From: sumangala Date: Tue, 23 Feb 2021 12:47:35 +0530 Subject: [PATCH 60/77] new file conflicts --- .../ITestAbfsInputStreamStatistics.java | 3 ++- .../ITestAbfsListStatusRemoteIterator.java | 25 ++++++++++++------- .../ITestAzureBlobFileSystemDelete.java | 7 +++--- .../services/ITestAbfsInputStream.java | 4 ++- .../ITestAbfsInputStreamReadFooter.java | 7 ++++-- .../ITestAbfsInputStreamSmallFileReads.java | 7 ++++-- 6 files changed, 35 insertions(+), 18 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java index 51c5c2f6d9d12..3817250561cb1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java @@ -389,7 +389,8 @@ public void testActionHttpGetRequest() throws IOException { abfsOutputStream.hflush(); abfsInputStream = - abfss.openFileForRead(actionHttpGetRequestPath, fs.getFsStatistics()); + abfss.openFileForRead(actionHttpGetRequestPath, + fs.getFsStatistics(), getTestTracingContext(fs, false)); abfsInputStream.read(); AbfsInputStreamStatisticsImpl abfsInputStreamStatistics = (AbfsInputStreamStatisticsImpl) abfsInputStream.getStreamStatistics(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java index 6d5e4cf3bce2d..1a647b5929b88 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java @@ -29,6 +29,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.assertj.core.api.Assertions; import org.junit.Test; import org.mockito.Mockito; @@ -64,7 +65,8 @@ public void testAbfsIteratorWithHasNext() throws Exception { ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore()); RemoteIterator fsItr = new AbfsListStatusRemoteIterator( - getFileSystem().getFileStatus(testDir), listngSupport); + getFileSystem().getFileStatus(testDir), listngSupport, + getTestTracingContext(getFileSystem(), true)); Assertions.assertThat(fsItr) .describedAs("RemoteIterator should be instance of " + "AbfsListStatusRemoteIterator by default") @@ -88,7 +90,8 @@ public void testAbfsIteratorWithHasNext() throws Exception { verify(listngSupport, Mockito.atLeast(minNumberOfInvokations)) .listStatus(any(Path.class), nullable(String.class), anyList(), anyBoolean(), - nullable(String.class)); + nullable(String.class), + any(TracingContext.class)); } @Test @@ -100,7 +103,8 @@ public void testAbfsIteratorWithoutHasNext() throws Exception { ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore()); RemoteIterator fsItr = new AbfsListStatusRemoteIterator( - getFileSystem().getFileStatus(testDir), listngSupport); + getFileSystem().getFileStatus(testDir), listngSupport, + getTestTracingContext(getFileSystem(), true)); Assertions.assertThat(fsItr) .describedAs("RemoteIterator should be instance of " + "AbfsListStatusRemoteIterator by default") @@ -129,7 +133,8 @@ public void testAbfsIteratorWithoutHasNext() throws Exception { verify(listngSupport, Mockito.atLeast(minNumberOfInvokations)) .listStatus(any(Path.class), nullable(String.class), anyList(), anyBoolean(), - nullable(String.class)); + nullable(String.class), + any(TracingContext.class)); } @Test @@ -205,7 +210,8 @@ public void testNextWhenNoMoreElementsPresent() throws Exception { setPageSize(10); RemoteIterator fsItr = new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir), - getFileSystem().getAbfsStore()); + getFileSystem().getAbfsStore(), + getTestTracingContext(getFileSystem(), true)); fsItr = Mockito.spy(fsItr); Mockito.doReturn(false).when(fsItr).hasNext(); @@ -253,7 +259,7 @@ public void testIOException() throws Exception { ListingSupport lsSupport =getMockListingSupport(exceptionMessage); RemoteIterator fsItr = new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir), - lsSupport); + lsSupport, getTestTracingContext(getFileSystem(), true)); Assertions.assertThatThrownBy(() -> fsItr.next()) .describedAs( @@ -276,19 +282,20 @@ public void testNonExistingPath() throws Throwable { private ListingSupport getMockListingSupport(String exceptionMessage) { return new ListingSupport() { @Override - public FileStatus[] listStatus(Path path) throws IOException { + public FileStatus[] listStatus(Path path, TracingContext tracingContext) throws IOException { return null; } @Override - public FileStatus[] listStatus(Path path, String startFrom) + public FileStatus[] listStatus(Path path, String startFrom, TracingContext tracingContext) throws IOException { return null; } @Override public String listStatus(Path path, String startFrom, - List fileStatuses, boolean fetchAll, String continuation) + List fileStatuses, boolean fetchAll, + String continuation, TracingContext tracingContext) throws IOException { throw new IOException(exceptionMessage); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index 5131c93e3a805..4c2ccba424580 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -229,7 +229,7 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { intercept(AbfsRestOperationException.class, () -> fs.getAbfsStore().delete( new Path("/NonExistingPath"), - false)); + false, getTestTracingContext(fs, false))); intercept(AbfsRestOperationException.class, () -> client.deletePath( @@ -249,7 +249,8 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { mockStore, "abfsPerfTracker", TestAbfsPerfTracker.getAPerfTrackerInstance(this.getConfiguration())); - doCallRealMethod().when(mockStore).delete(new Path("/NonExistingPath"), false); + doCallRealMethod().when(mockStore).delete(new Path("/NonExistingPath"), + false, getTestTracingContext(fs, false)); // Case 2: Mimic retried case // Idempotency check on Delete always returns success @@ -276,7 +277,7 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { .isEqualTo(idempotencyRetOp.getResult().getStatusCode()); // Call from AzureBlobFileSystemStore should not fail either - mockStore.delete(new Path("/NonExistingPath"), false); + mockStore.delete(new Path("/NonExistingPath"), false, getTestTracingContext(fs, false)); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStream.java index 44b0a362dc6c9..1a835439bd75b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStream.java @@ -31,6 +31,7 @@ import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Test; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; @@ -118,7 +119,8 @@ private void testExceptionInOptimization(final FileSystem fs, doThrow(new IOException()) .doCallRealMethod() .when(abfsInputStream) - .readRemote(anyLong(), any(), anyInt(), anyInt()); + .readRemote(anyLong(), any(), anyInt(), anyInt(), + any(TracingContext.class)); iStream = new FSDataInputStream(abfsInputStream); verifyBeforeSeek(abfsInputStream); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java index 09a810c5fc9e8..81799ddb2d3dc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Map; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Test; import org.apache.hadoop.fs.FSDataInputStream; @@ -270,7 +271,8 @@ private void testPartialReadWithNoData(final FileSystem fs, .getWrappedStream(); abfsInputStream = spy(abfsInputStream); doReturn(10).doReturn(10).doCallRealMethod().when(abfsInputStream) - .readRemote(anyLong(), any(), anyInt(), anyInt()); + .readRemote(anyLong(), any(), anyInt(), anyInt(), + any(TracingContext.class)); iStream = new FSDataInputStream(abfsInputStream); seek(iStream, seekPos); @@ -319,7 +321,8 @@ private void testPartialReadWithSomeDat(final FileSystem fs, - someDataLength; doReturn(10).doReturn(secondReturnSize).doCallRealMethod() .when(abfsInputStream) - .readRemote(anyLong(), any(), anyInt(), anyInt()); + .readRemote(anyLong(), any(), anyInt(), anyInt(), + any(TracingContext.class)); iStream = new FSDataInputStream(abfsInputStream); seek(iStream, seekPos); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamSmallFileReads.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamSmallFileReads.java index ff03c0e78f4a3..3b20731ea5ad0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamSmallFileReads.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamSmallFileReads.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Map; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Test; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; @@ -251,7 +252,8 @@ private void partialReadWithNoData(final FileSystem fs, .doReturn(10) .doCallRealMethod() .when(abfsInputStream) - .readRemote(anyLong(), any(), anyInt(), anyInt()); + .readRemote(anyLong(), any(), anyInt(), anyInt(), + any(TracingContext.class)); iStream = new FSDataInputStream(abfsInputStream); seek(iStream, seekPos); @@ -301,7 +303,8 @@ private void partialReadWithSomeData(final FileSystem fs, .doReturn(secondReturnSize) .doCallRealMethod() .when(abfsInputStream) - .readRemote(anyLong(), any(), anyInt(), anyInt()); + .readRemote(anyLong(), any(), anyInt(), anyInt(), + any(TracingContext.class)); iStream = new FSDataInputStream(abfsInputStream); seek(iStream, seekPos); From ec869fe0358b07d4548d4e6318c9d0f7d7f1cbd0 Mon Sep 17 00:00:00 2001 From: sumangala Date: Wed, 3 Mar 2021 11:16:03 +0530 Subject: [PATCH 61/77] imports --- .../apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java | 6 +++--- .../hadoop/fs/azurebfs/services/ListingSupport.java | 3 +++ .../apache/hadoop/fs/azurebfs/utils/TracingContext.java | 7 ++++--- hadoop-tools/hadoop-azure/src/site/markdown/abfs.md | 8 ++++---- .../hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java | 6 +++--- .../hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java | 8 ++++---- .../fs/azurebfs/ITestAzureBlobFileSystemAppend.java | 4 ++-- .../fs/azurebfs/ITestAzureBlobFileSystemAttributes.java | 5 +++-- .../fs/azurebfs/ITestAzureBlobFileSystemCreate.java | 8 ++++---- .../fs/azurebfs/ITestAzureBlobFileSystemDelete.java | 6 +++--- .../fs/azurebfs/ITestAzureBlobFileSystemListStatus.java | 4 ++-- .../hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java | 2 +- .../hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java | 4 ++-- .../hadoop/fs/azurebfs/ITestFileSystemProperties.java | 2 +- .../hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java | 4 ++-- .../azurebfs/services/ITestAbfsInputStreamReadFooter.java | 2 +- .../services/ITestAbfsInputStreamSmallFileReads.java | 2 +- .../hadoop/fs/azurebfs/services/TestAbfsInputStream.java | 2 +- .../hadoop/fs/azurebfs/services/TestAbfsOutputStream.java | 8 ++++---- .../hadoop/fs/azurebfs/utils/TracingHeaderValidator.java | 3 ++- 20 files changed, 50 insertions(+), 44 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index c8701c15adeed..bd6dcdaf60f91 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -33,8 +33,8 @@ import java.util.Collections; import java.util.EnumSet; import java.util.Map; -import java.util.UUID; import java.util.Optional; +import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -44,8 +44,6 @@ import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; -import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; -import org.apache.hadoop.fs.azurebfs.utils.Listener; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,7 +78,9 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager; import org.apache.hadoop.fs.azurebfs.services.AbfsCounters; +import org.apache.hadoop.fs.azurebfs.utils.Listener; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl; import org.apache.hadoop.fs.impl.OpenFileParameters; import org.apache.hadoop.fs.permission.AclEntry; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListingSupport.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListingSupport.java index 9355d5cca2ddd..dc070a1d405d8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListingSupport.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListingSupport.java @@ -33,6 +33,7 @@ public interface ListingSupport { /** * @param path The list path. + * @param tracingContext TracingContext instance to track identifiers * @return the entries in the path. * @throws IOException in case of error */ @@ -50,6 +51,7 @@ public interface ListingSupport { * non-existent entry in lexical order: listStatus * (Path("/folder"), "cfile") will return * "/folder/hfile" and "/folder/ifile". + * @param tracingContext TracingContext instance to track identifiers * @return the entries in the path start from "startFrom" in lexical order. * @throws IOException in case of error */ @@ -72,6 +74,7 @@ public interface ListingSupport { * filled with just one page os results or the entire * result. * @param continuation Contiuation token. null means start rom the begining. + * @param tracingContext TracingContext instance to track identifiers * @return Continuation tokem * @throws IOException in case of error */ diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index 78daa181d54be..c9bfa9894ba0f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -20,12 +20,13 @@ import java.util.UUID; -import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; - -import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; + +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; + public class TracingContext { private final String clientCorrelationID; private final String fileSystemID; diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md index 5a44e8e905f62..cdc82b96c8791 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md @@ -744,10 +744,10 @@ input is invalid. #### 1. Correlation IDs Display Options -Config `fs.azure.tracingcontext.format` provides an option to select the - format of IDs included in the `request-id-header`. This config accepts a - String value corresponding to the following enum options. - `SINGLE_ID_FORMAT`: clientRequestId +Config `fs.azure.tracingcontext.format` provides an option to select the format +of IDs included in the `request-id-header`. This config accepts a String value +corresponding to the following enum options. + `SINGLE_ID_FORMAT` : clientRequestId `ALL_ID_FORMAT` : all IDs (default) `TWO_ID_FORMAT` : clientCorrelationId:clientRequestId diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index 8d042c7fa3012..ae36ec7da30f3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -25,9 +25,6 @@ import java.util.UUID; import java.util.concurrent.Callable; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -47,7 +44,10 @@ import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import org.apache.hadoop.fs.azurebfs.utils.UriUtils; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.permission.FsPermission; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index 1226a3e355b7d..d82438d6a601f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -21,10 +21,6 @@ import java.util.Arrays; import java.util.Random; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; -import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; -import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -32,6 +28,10 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; +import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.APPENDBLOB_MAX_WRITE_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java index d5b82aa1c9fbc..8297eb80bb26a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java @@ -22,12 +22,12 @@ import java.io.IOException; import java.util.Random; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.fs.contract.ContractTestUtils; /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java index edb5fc23614b7..fe0a962f2b4e7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java @@ -21,12 +21,13 @@ import java.io.IOException; import java.util.EnumSet; +import org.junit.Assume; +import org.junit.Test; + import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; -import org.junit.Assume; -import org.junit.Test; import static org.apache.hadoop.test.LambdaTestUtils.intercept; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index ecad317871f01..a771f989022ac 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -25,9 +25,6 @@ import java.util.EnumSet; import java.util.UUID; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; import org.apache.hadoop.conf.Configuration; @@ -40,12 +37,15 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConcurrentWriteOperationDetectedException; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; -import org.apache.hadoop.fs.azurebfs.services.TestAbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; +import org.apache.hadoop.fs.azurebfs.services.TestAbfsClient; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import static java.net.HttpURLConnection.HTTP_CONFLICT; import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index 4c2ccba424580..90e13c7e122cc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -26,13 +26,11 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.Test; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; @@ -40,6 +38,8 @@ import org.apache.hadoop.fs.azurebfs.services.TestAbfsClient; import org.apache.hadoop.fs.azurebfs.services.TestAbfsPerfTracker; import org.apache.hadoop.fs.azurebfs.utils.TestMockHelpers; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java index 2e0aa4d7d24a0..899c0b9f49624 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java @@ -27,8 +27,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Test; import org.apache.hadoop.conf.Configuration; @@ -37,6 +35,8 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.fs.contract.ContractTestUtils; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_LIST_MAX_RESULTS; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java index 707633a992695..62bac31904546 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java @@ -22,7 +22,6 @@ import java.io.InputStream; import java.util.Map; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Assume; import org.junit.Test; import org.slf4j.Logger; @@ -37,6 +36,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; import org.apache.hadoop.fs.azurebfs.services.AuthType; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.io.IOUtils; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java index b274ed210cacf..a6eef220b3fd9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java @@ -24,8 +24,6 @@ import java.util.List; import java.util.UUID; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Assume; import org.junit.Ignore; import org.junit.Test; @@ -34,7 +32,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.FsAction; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java index 9b1c85cce9e48..c478118757107 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java @@ -20,13 +20,13 @@ import java.util.Hashtable; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Test; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; /** * Test FileSystemProperties. diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java index bf88ae4abbc5a..01227691c3139 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java @@ -21,8 +21,6 @@ import java.io.IOException; import java.util.UUID; -import org.apache.hadoop.fs.azurebfs.enums.Trilean; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Assume; import org.junit.Test; import org.assertj.core.api.Assertions; @@ -33,6 +31,8 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.enums.Trilean; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java index 81799ddb2d3dc..1cb8ed3ea8cee 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.util.Map; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Test; import org.apache.hadoop.fs.FSDataInputStream; @@ -29,6 +28,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import static java.lang.Math.max; import static java.lang.Math.min; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamSmallFileReads.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamSmallFileReads.java index 3b20731ea5ad0..d85da5aaeabd7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamSmallFileReads.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamSmallFileReads.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.util.Map; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Test; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; @@ -29,6 +28,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index 2ca9932abdd09..5fd3532624730 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -20,7 +20,6 @@ import java.io.IOException; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Assert; import org.junit.Test; import java.util.Arrays; @@ -38,6 +37,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TimeoutException; import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; import org.apache.hadoop.fs.azurebfs.utils.TestCachedSASToken; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index 621b6e36f075d..066e901adfa18 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -22,16 +22,16 @@ import java.util.Arrays; import java.util.Random; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import org.junit.Test; import org.mockito.ArgumentCaptor; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; -import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.refEq; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java index daee2029552a7..db99ffc75c5d1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java @@ -18,9 +18,10 @@ package org.apache.hadoop.fs.azurebfs.utils; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.assertj.core.api.Assertions; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; + public class TracingHeaderValidator implements Listener { private String clientCorrelationID; private String fileSystemID; From 22e9a40330fae359dc839b8062efe8f0e841a548 Mon Sep 17 00:00:00 2001 From: sumangala Date: Wed, 10 Mar 2021 15:54:01 +0530 Subject: [PATCH 62/77] import --- .../fs/azurebfs/AzureBlobFileSystem.java | 2 +- .../azurebfs/services/AbfsHttpOperation.java | 2 +- .../azurebfs/services/ReadBufferManager.java | 2 +- .../ITestAbfsListStatusRemoteIterator.java | 2 +- .../fs/azurebfs/TestTracingContext.java | 23 ++++++++++--------- 5 files changed, 16 insertions(+), 15 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index bd6dcdaf60f91..aae2dd474ff1c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -66,9 +66,9 @@ import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.FileSystemOperationUnhandledException; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index cb58e6a16d0fc..92cc53f958e99 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -30,8 +30,8 @@ import javax.net.ssl.HttpsURLConnection; import javax.net.ssl.SSLSocketFactory; -import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.codehaus.jackson.JsonFactory; import org.codehaus.jackson.JsonParser; import org.codehaus.jackson.JsonToken; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java index 08f0dd65aecbb..e7656c14493cb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java @@ -18,7 +18,6 @@ package org.apache.hadoop.fs.azurebfs.services; import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,6 +30,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.locks.ReentrantLock; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java index 1a647b5929b88..e2973912e2323 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java @@ -29,7 +29,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.assertj.core.api.Assertions; import org.junit.Test; import org.mockito.Mockito; @@ -39,6 +38,7 @@ import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator; import org.apache.hadoop.fs.azurebfs.services.ListingSupport; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 909446099bc0a..50b5841b01b7e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -18,8 +18,16 @@ package org.apache.hadoop.fs.azurebfs; -import org.apache.hadoop.fs.azurebfs.services.AuthType; -import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import java.io.IOException; +import java.lang.reflect.Method; +import java.util.HashMap; +import java.util.Map; + +import org.assertj.core.api.Assertions; +import org.junit.Assume; +import org.junit.Ignore; +import org.junit.Test; + import org.apache.hadoop.fs.CommonPathCapabilities; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; @@ -27,20 +35,13 @@ import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.enums.Trilean; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; +import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; -import org.assertj.core.api.Assertions; -import org.junit.Assume; -import org.junit.Ignore; -import org.junit.Test; - -import java.io.IOException; -import java.lang.reflect.Method; -import java.util.HashMap; -import java.util.Map; +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE; From a42e5b19a39e6858c790ced0a39edf43a2a262da Mon Sep 17 00:00:00 2001 From: sumangala Date: Wed, 10 Mar 2021 16:27:40 +0530 Subject: [PATCH 63/77] use write code for out --- .../apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index a27e6cd399856..62000f4a9de27 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -39,6 +39,7 @@ import org.slf4j.LoggerFactory; import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; @@ -166,6 +167,7 @@ public AbfsOutputStream( this.outputStreamID = getOutputStreamID(); this.tracingContext = new TracingContext(tracingContext); this.tracingContext.setStreamID(outputStreamID); + this.tracingContext.setOperation(HdfsOperationConstants.WRITE); } private String getOutputStreamID() { From 51e1a8ff7bda74d05ba2ccd8e6c497acbfac4ad8 Mon Sep 17 00:00:00 2001 From: sumangala Date: Thu, 11 Mar 2021 10:04:25 +0530 Subject: [PATCH 64/77] handle invocation ex + write tests --- .../fs/azurebfs/ITestAbfsReadWriteAndSeek.java | 2 +- .../azurebfs/ITestAzureBlobFileSystemFlush.java | 2 +- .../hadoop/fs/azurebfs/TestTracingContext.java | 16 ++++++++++++---- 3 files changed, 14 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index d82438d6a601f..d04acbf13b688 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -115,7 +115,7 @@ public void testReadAheadRequestID() throws java.io.IOException { try (FSDataOutputStream stream = fs.create(TEST_PATH)) { ((AbfsOutputStream) stream.getWrappedStream()).registerListener( new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.CREATE, false, 0, + fs.getFileSystemID(), HdfsOperationConstants.WRITE, false, 0, ((AbfsOutputStream) stream.getWrappedStream()) .getStreamID())); stream.write(b); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index e33d5fb1db8da..5c66531b17523 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -320,7 +320,7 @@ public void testTracingHeaderForAppendBlob() throws Exception { ((AbfsOutputStream) out.getWrappedStream()).registerListener( new TracingHeaderValidator( fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.CREATE, false, 0, + fs.getFileSystemID(), HdfsOperationConstants.WRITE, false, 0, ((AbfsOutputStream) out.getWrappedStream()).getStreamID())); out.write(buf); out.hsync(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 50b5841b01b7e..13403d1ccf992 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -19,12 +19,14 @@ package org.apache.hadoop.fs.azurebfs; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.HashMap; import java.util.Map; import org.assertj.core.api.Assertions; import org.junit.Assume; +import org.junit.AssumptionViolatedException; import org.junit.Ignore; import org.junit.Test; @@ -159,15 +161,21 @@ public void runCorrelationTestForAllMethods() throws Exception { // removeaclentries, removedefaultacl, removeacl for (AbstractAbfsIntegrationTest testClass : testClasses.keySet()) { - testClass.setup(); - testClasses.get(testClass).invoke(testClass); - testClass.teardown(); + try { + testClass.setup(); + testClasses.get(testClass).invoke(testClass); + testClass.teardown(); + } catch (InvocationTargetException e) { + if (!(e.getCause() instanceof AssumptionViolatedException)) { + throw new IOException(testClasses.get(testClass).getName() + + " failed tracing context validation test"); + } + } } testExternalOps(); } @Test - //rename this test public void testExternalOps() throws Exception { //validate tracing header for access, hasPathCapability AzureBlobFileSystem fs = getFileSystem(); From 4ed465f8ce965343be63194a5034070ef932dfd2 Mon Sep 17 00:00:00 2001 From: sumangala Date: Thu, 11 Mar 2021 10:15:47 +0530 Subject: [PATCH 65/77] checkstyle --- .../fs/azurebfs/services/AbfsListStatusRemoteIterator.java | 6 ++---- .../hadoop/fs/azurebfs/utils/TracingHeaderValidator.java | 5 ++--- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java index 0ed8dd3939e53..835217f945af5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java @@ -29,14 +29,12 @@ import java.util.concurrent.TimeUnit; import javax.activation.UnsupportedDataTypeException; -import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; public class AbfsListStatusRemoteIterator implements RemoteIterator { @@ -51,12 +49,12 @@ public class AbfsListStatusRemoteIterator private final FileStatus fileStatus; private final ListingSupport listingSupport; private final ArrayBlockingQueue iteratorsQueue; + private final TracingContext tracingContext; private volatile boolean isAsyncInProgress = false; private boolean isIterationComplete = false; private String continuation; private Iterator currIterator; - private TracingContext tracingContext; public AbfsListStatusRemoteIterator(final FileStatus fileStatus, final ListingSupport listingSupport, TracingContext tracingContext) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java index db99ffc75c5d1..53a29d63531c7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java @@ -30,10 +30,9 @@ public class TracingHeaderValidator implements Listener { private String streamID = ""; private String operation; private int retryNum; - private String GUID_PATTERN = "^[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}$"; private TracingContextFormat format; - // client-req-id as per docs: ^[{(]?[0-9a-f]{8}[-]?([0-9a-f]{4}[-]?) - // {3}[0-9a-f]{12}[)}]?$ + + private static final String GUID_PATTERN = "^[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}$"; @Override public void callTracingHeaderValidator(String tracingContextHeader, From 0e9770052dd12cd551c581e1b382b28ec1712147 Mon Sep 17 00:00:00 2001 From: sumangala Date: Thu, 11 Mar 2021 12:03:05 +0530 Subject: [PATCH 66/77] minor chkstyle --- .../org/apache/hadoop/fs/azurebfs/TestTracingContext.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 13403d1ccf992..2488732d14a5a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -167,8 +167,8 @@ public void runCorrelationTestForAllMethods() throws Exception { testClass.teardown(); } catch (InvocationTargetException e) { if (!(e.getCause() instanceof AssumptionViolatedException)) { - throw new IOException(testClasses.get(testClass).getName() + - " failed tracing context validation test"); + throw new IOException(testClasses.get(testClass).getName() + + " failed tracing context validation test"); } } } From 3447977af7444bdd033ba091124bc70b130954e3 Mon Sep 17 00:00:00 2001 From: sumangala Date: Thu, 15 Apr 2021 23:36:00 +0530 Subject: [PATCH 67/77] lease rm acquire op --- .../hadoop/fs/azurebfs/AzureBlobFileSystemStore.java | 7 ++----- .../azurebfs/constants/HdfsOperationConstants.java | 1 - .../hadoop/fs/azurebfs/services/AbfsLease.java | 9 +++++++++ .../hadoop/fs/azurebfs/utils/TracingContext.java | 12 ++++++------ .../fs/azurebfs/ITestAzureBlobFileSystemLease.java | 8 +++++--- 5 files changed, 22 insertions(+), 15 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index b3e5cb9fc93a4..301861ccf96cd 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -781,6 +781,7 @@ public OutputStream openFileForWrite(final Path path, * Break any current lease on an ABFS file. * * @param path file name + * @param tracingContext TracingContext instance to track correlation IDs * @throws AzureBlobFileSystemException on any exception while breaking the lease */ public void breakLease(final Path path, final TracingContext tracingContext) throws AzureBlobFileSystemException { @@ -1441,7 +1442,6 @@ public boolean isAtomicRenameKey(String key) { public boolean isInfiniteLeaseKey(String key) { if (azureInfiniteLeaseDirSet.isEmpty()) { - System.out.println("=========="); return false; } return isKeyForDirectorySet(key, azureInfiniteLeaseDirSet); @@ -1623,7 +1623,6 @@ private Hashtable parseCommaSeparatedXmsProperties(String xMsPro private boolean isKeyForDirectorySet(String key, Set dirSet) { for (String dir : dirSet) { - System.out.println(dir + "> " + key); if (dir.isEmpty() || key.startsWith(dir + AbfsHttpConstants.FORWARD_SLASH)) { return true; } @@ -1742,17 +1741,15 @@ private void updateInfiniteLeaseDirs() { abfsConfiguration.getAzureInfiniteLeaseDirs().split(AbfsHttpConstants.COMMA))); // remove the empty string, since isKeyForDirectory returns true for empty strings // and we don't want to default to enabling infinite lease dirs -// this.azureInfiniteLeaseDirSet.remove(""); + this.azureInfiniteLeaseDirSet.remove(""); } private AbfsLease maybeCreateLease(String relativePath, TracingContext tracingContext) throws AzureBlobFileSystemException { boolean enableInfiniteLease = isInfiniteLeaseKey(relativePath); if (!enableInfiniteLease) { - System.out.println("-------------------"); return null; } - tracingContext.setOperation(HdfsOperationConstants.ACQUIRE_LEASE); AbfsLease lease = new AbfsLease(client, relativePath, tracingContext); leaseRefs.put(lease, null); return lease; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HdfsOperationConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HdfsOperationConstants.java index b1112eb2526c3..cb2fc81ab0d83 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HdfsOperationConstants.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HdfsOperationConstants.java @@ -20,7 +20,6 @@ public final class HdfsOperationConstants { public static final String ACCESS = "AS"; - public static final String ACQUIRE_LEASE = "AL"; public static final String APPEND = "AP"; public static final String BREAK_LEASE = "BL"; public static final String CREATE = "CR"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java index 42c708e90c85d..8e0c82686eff9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java @@ -194,4 +194,13 @@ public String getLeaseID() { public int getAcquireRetryCount() { return acquireRetryCount; } + + public void setListenerOperation(String op) { + tracingContext.setOperation(op); + } + + @VisibleForTesting + public TracingContext getTracingContext() { + return tracingContext; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index 49c937d3a51c2..d43c2e3c91508 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -29,12 +29,12 @@ /** * The TracingContext class to correlate Store requests using unique - * identifiers and common resources such as filesystem and stream + * identifiers and resources common to requests (e.g. filesystem, stream) * * Implementing new HDFS method: - * Create and pass an instance of TracingContext to method in outer layer of - * ABFS driver (AzureBlobFileSystem/AbfsInputStream/AbfsOutputStream). - * Identifiers appear in X_MS_CLIENT_REQUEST_ID header and debug logs. + * Create TracingContext instance in method of outer layer of + * ABFS driver (AzureBlobFileSystem/AbfsInputStream/AbfsOutputStream), to be + * passed through ABFS layers up to AbfsRestOperation. * * Add new operations to HdfsOperationConstants file. * @@ -49,7 +49,7 @@ public class TracingContext { private final String clientCorrelationID; private final String fileSystemID; - private String clientRequestID = ""; + private String clientRequestID = EMPTY_STRING; private String primaryRequestID; private String streamID; private int retryCount; @@ -67,7 +67,7 @@ public class TracingContext { * @param fileSystemID Unique guid for AzureBlobFileSystem instance * @param hadoopOpName Code indicating the high-level Hadoop operation that * triggered the current Store request - * @param tracingContextFormat Format of IDs to be + * @param tracingContextFormat Format of IDs to be printed in header and logs * @param listener Holds instance of TracingHeaderValidator during testing, * null otherwise */ diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index f8e677009a2eb..95373fca26030 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -21,6 +21,7 @@ import java.util.concurrent.RejectedExecutionException; import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Assert; @@ -313,14 +314,15 @@ public void testAcquireRetry() throws Exception { fs.mkdirs(testFilePath.getParent()); fs.createNewFile(testFilePath); TracingContext tracingContext = getTestTracingContext(fs, false); - fs.registerListener(new TracingHeaderValidator( - fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.ACQUIRE_LEASE, false, 0)); + tracingContext.setListener(new TracingHeaderValidator(getConfiguration().getClientCorrelationID(), + fs.getFileSystemID(), HdfsOperationConstants.TEST_OP, true, 0)); AbfsLease lease = new AbfsLease(fs.getAbfsClient(), testFilePath.toUri().getPath(), tracingContext); Assert.assertNotNull("Did not successfully lease file", lease.getLeaseID()); + lease.setListenerOperation(HdfsOperationConstants.RELEASE_LEASE); lease.free(); + lease.getTracingContext().setListener(null); Assert.assertEquals("Unexpected acquire retry count", 0, lease.getAcquireRetryCount()); AbfsClient mockClient = spy(fs.getAbfsClient()); From c9217d9492486f401cbcb9320e8d473f98595c1b Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 26 Apr 2021 12:31:52 +0530 Subject: [PATCH 68/77] part of merge fix --- .../apache/hadoop/fs/azurebfs/ITestAbfsDurationTrackers.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsDurationTrackers.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsDurationTrackers.java index c8b687d233cb5..0997b3dbd44d4 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsDurationTrackers.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsDurationTrackers.java @@ -76,7 +76,8 @@ public void testAbfsHttpCallsDurations() throws IOException { out.hflush(); // GET the file. - in = fs.getAbfsStore().openFileForRead(testFilePath, fs.getFsStatistics()); + in = fs.getAbfsStore().openFileForRead(testFilePath, + fs.getFsStatistics(), getTestTracingContext(fs, false)); int res = in.read(); LOG.info("Result of Read: {}", res); From fd631219582b5bc6db47b69a4aa496682ac335a4 Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 26 Apr 2021 15:35:03 +0530 Subject: [PATCH 69/77] add active lease fn tests --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 1 - .../hadoop/fs/azurebfs/services/AbfsLease.java | 8 ++------ .../azurebfs/ITestAzureBlobFileSystemLease.java | 16 +++++++++++----- 3 files changed, 13 insertions(+), 12 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 301861ccf96cd..553fdd713f4be 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -52,7 +52,6 @@ import java.util.WeakHashMap; import java.util.concurrent.ExecutionException; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java index 8e0c82686eff9..84f581a7fe202 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java @@ -21,8 +21,6 @@ import java.io.IOException; import java.util.concurrent.TimeUnit; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableScheduledFuture; @@ -31,8 +29,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.io.retry.RetryPolicy; @@ -195,10 +195,6 @@ public int getAcquireRetryCount() { return acquireRetryCount; } - public void setListenerOperation(String op) { - tracingContext.setOperation(op); - } - @VisibleForTesting public TracingContext getTracingContext() { return tracingContext; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index 95373fca26030..53005259956d7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -21,7 +21,7 @@ import java.util.concurrent.RejectedExecutionException; import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; +import org.apache.hadoop.fs.azurebfs.utils.Listener; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Assert; @@ -215,7 +215,11 @@ public void testWriteAfterBreakLease() throws Exception { out.write(0); out.hsync(); + fs.registerListener(new TracingHeaderValidator( + getConfiguration().getClientCorrelationID(), fs.getFileSystemID(), + HdfsOperationConstants.BREAK_LEASE, false, 0)); fs.breakLease(testFilePath); + fs.registerListener(null); LambdaTestUtils.intercept(IOException.class, ERR_LEASE_EXPIRED, () -> { out.write(1); @@ -313,14 +317,16 @@ public void testAcquireRetry() throws Exception { final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); fs.mkdirs(testFilePath.getParent()); fs.createNewFile(testFilePath); - TracingContext tracingContext = getTestTracingContext(fs, false); - tracingContext.setListener(new TracingHeaderValidator(getConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.TEST_OP, true, 0)); + TracingContext tracingContext = getTestTracingContext(fs, true); + Listener listener = new TracingHeaderValidator( + getConfiguration().getClientCorrelationID(), fs.getFileSystemID(), + HdfsOperationConstants.TEST_OP, true, 0); + tracingContext.setListener(listener); AbfsLease lease = new AbfsLease(fs.getAbfsClient(), testFilePath.toUri().getPath(), tracingContext); Assert.assertNotNull("Did not successfully lease file", lease.getLeaseID()); - lease.setListenerOperation(HdfsOperationConstants.RELEASE_LEASE); + listener.setOperation(HdfsOperationConstants.RELEASE_LEASE); lease.free(); lease.getTracingContext().setListener(null); Assert.assertEquals("Unexpected acquire retry count", 0, lease.getAcquireRetryCount()); From d52ee795da47e98178b2498357fb9f55a4e78ef1 Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 26 Apr 2021 17:02:07 +0530 Subject: [PATCH 70/77] javadoc --- .../apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 f8a73f8af38b4..f89c8bf370565 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 @@ -172,7 +172,7 @@ String getSasToken() { /** * Execute a AbfsRestOperation. Track the Duration of a request if * abfsCounters isn't null. - * + * @param tracingContext TracingContext instance to track correlation IDs */ public void execute(TracingContext tracingContext) throws AzureBlobFileSystemException { @@ -192,6 +192,7 @@ public void execute(TracingContext tracingContext) /** * Executes the REST operation with retry, by issuing one or more * HTTP operations. + * @param tracingContext TracingContext instance to track correlation IDs */ private void completeExecute(TracingContext tracingContext) throws AzureBlobFileSystemException { From d15d2cf2c07691ebbdbd4450ebe65b2e146fb8ef Mon Sep 17 00:00:00 2001 From: sumangala Date: Sun, 30 May 2021 15:06:46 +0530 Subject: [PATCH 71/77] merge --- .../fs/azurebfs/ITestCustomerProvidedKey.java | 86 ++++++++++++------- 1 file changed, 56 insertions(+), 30 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java index 9229905b4623c..7e610c107a687 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java @@ -34,6 +34,7 @@ import java.util.Optional; import java.util.Random; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.collect.Lists; @@ -112,11 +113,13 @@ public void testReadWithCPK() throws Exception { AbfsClient abfsClient = fs.getAbfsClient(); int length = FILE_SIZE; byte[] buffer = new byte[length]; - final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false); + TracingContext tracingContext = getTestTracingContext(fs, false); + final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false, + tracingContext); final String eTag = op.getResult() .getResponseHeader(HttpHeaderConfigurations.ETAG); AbfsRestOperation abfsRestOperation = abfsClient - .read(fileName, 0, buffer, 0, length, eTag, null); + .read(fileName, 0, buffer, 0, length, eTag, null, tracingContext); assertCPKHeaders(abfsRestOperation, true); assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); @@ -160,11 +163,13 @@ public void testReadWithoutCPK() throws Exception { AbfsClient abfsClient = fs.getAbfsClient(); int length = INT_512; byte[] buffer = new byte[length * 4]; - final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false); + TracingContext tracingContext = getTestTracingContext(fs, false); + final AbfsRestOperation op = abfsClient + .getPathStatus(fileName, false, tracingContext); final String eTag = op.getResult() .getResponseHeader(HttpHeaderConfigurations.ETAG); AbfsRestOperation abfsRestOperation = abfsClient - .read(fileName, 0, buffer, 0, length, eTag, null); + .read(fileName, 0, buffer, 0, length, eTag, null, tracingContext); assertCPKHeaders(abfsRestOperation, false); assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); @@ -182,7 +187,8 @@ public void testReadWithoutCPK() throws Exception { try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) { LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null); + abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null, + getTestTracingContext(fs, false)); }); } } @@ -200,7 +206,7 @@ public void testAppendWithCPK() throws Exception { byte[] buffer = getRandomBytesArray(5); AbfsClient abfsClient = fs.getAbfsClient(); AbfsRestOperation abfsRestOperation = abfsClient - .append(fileName, buffer, appendRequestParameters, null); + .append(fileName, buffer, appendRequestParameters, null, getTestTracingContext(fs, false)); assertCPKHeaders(abfsRestOperation, true); assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); @@ -216,7 +222,8 @@ public void testAppendWithCPK() throws Exception { try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) { LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient2.append(fileName, buffer, appendRequestParameters, null); + abfsClient2.append(fileName, buffer, appendRequestParameters, null, + getTestTracingContext(fs, false)); }); } @@ -225,7 +232,8 @@ public void testAppendWithCPK() throws Exception { try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) { LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient3.append(fileName, buffer, appendRequestParameters, null); + abfsClient3.append(fileName, buffer, appendRequestParameters, null, + getTestTracingContext(fs, false)); }); } } @@ -243,7 +251,8 @@ public void testAppendWithoutCPK() throws Exception { byte[] buffer = getRandomBytesArray(5); AbfsClient abfsClient = fs.getAbfsClient(); AbfsRestOperation abfsRestOperation = abfsClient - .append(fileName, buffer, appendRequestParameters, null); + .append(fileName, buffer, appendRequestParameters, null, + getTestTracingContext(fs, false)); assertCPKHeaders(abfsRestOperation, false); assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256, ""); @@ -259,7 +268,8 @@ public void testAppendWithoutCPK() throws Exception { try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) { LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient2.append(fileName, buffer, appendRequestParameters, null); + abfsClient2.append(fileName, buffer, appendRequestParameters, null, + getTestTracingContext(fs, false)); }); } } @@ -405,7 +415,8 @@ private void testListPath(final boolean isWithCPK) throws Exception { createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE); AbfsClient abfsClient = fs.getAbfsClient(); AbfsRestOperation abfsRestOperation = abfsClient - .listPath(testDirName, false, INT_50, null); + .listPath(testDirName, false, INT_50, null, + getTestTracingContext(fs, false)); assertListstatus(fs, abfsRestOperation, testPath); // Trying with different CPK headers @@ -415,7 +426,8 @@ private void testListPath(final boolean isWithCPK) throws Exception { "different-1234567890123456789012"); AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); AbfsClient abfsClient2 = fs2.getAbfsClient(); - abfsRestOperation = abfsClient2.listPath(testDirName, false, INT_50, null); + abfsRestOperation = abfsClient2.listPath(testDirName, false, INT_50, + null, tracingContext); assertListstatus(fs, abfsRestOperation, testPath); if (isWithCPK) { @@ -424,7 +436,7 @@ private void testListPath(final boolean isWithCPK) throws Exception { AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient(); abfsRestOperation = abfsClient3 - .listPath(testDirName, false, INT_50, null); + .listPath(testDirName, false, INT_50, null, tracingContext); assertListstatus(fs, abfsRestOperation, testPath); } } @@ -463,11 +475,13 @@ private void testCreatePath(final boolean isWithCPK) throws Exception { FsAction.EXECUTE, FsAction.EXECUTE); FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE, FsAction.NONE); - boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(); + TracingContext tracingContext = getTestTracingContext(fs, false); + boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(tracingContext); AbfsRestOperation abfsRestOperation = abfsClient .createPath(testFileName, true, true, isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null, false, null); + isNamespaceEnabled ? getOctalNotation(umask) : null, false, null, + tracingContext); assertCPKHeaders(abfsRestOperation, isWithCPK); assertResponseHeader(abfsRestOperation, isWithCPK, X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); @@ -505,7 +519,8 @@ private void testRenamePath(final boolean isWithCPK) throws Exception { String newName = "/newName"; AbfsClient abfsClient = fs.getAbfsClient(); AbfsRestOperation abfsRestOperation = abfsClient - .renamePath(testFileName, newName, null); + .renamePath(testFileName, newName, null, + getTestTracingContext(fs, false)); assertCPKHeaders(abfsRestOperation, false); assertNoCPKResponseHeadersPresent(abfsRestOperation); @@ -548,7 +563,8 @@ private void testFlush(final boolean isWithCPK) throws Exception { try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) { LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient2.flush(testFileName, 0, false, false, null, null); + abfsClient2.flush(testFileName, 0, false, false, null, null, + getTestTracingContext(fs, false)); }); } @@ -558,14 +574,16 @@ private void testFlush(final boolean isWithCPK) throws Exception { try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) { LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient3.flush(testFileName, 0, false, false, null, null); + abfsClient3.flush(testFileName, 0, false, false, null, null, + getTestTracingContext(fs, false)); }); } } // With correct CPK AbfsRestOperation abfsRestOperation = abfsClient - .flush(testFileName, 0, false, false, null, null); + .flush(testFileName, 0, false, false, null, null, + getTestTracingContext(fs, false)); assertCPKHeaders(abfsRestOperation, isWithCPK); assertResponseHeader(abfsRestOperation, isWithCPK, X_MS_ENCRYPTION_KEY_SHA256, expectedCPKSha); @@ -594,7 +612,8 @@ private void testSetPathProperties(final boolean isWithCPK) throws Exception { properties.put("key", "val"); AbfsRestOperation abfsRestOperation = abfsClient .setPathProperties(testFileName, - convertXmsPropertiesToCommaSeparatedString(properties)); + convertXmsPropertiesToCommaSeparatedString(properties), + getTestTracingContext(fs, false)); assertCPKHeaders(abfsRestOperation, isWithCPK); assertResponseHeader(abfsRestOperation, isWithCPK, X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); @@ -619,8 +638,9 @@ private void testGetPathStatusFile(final boolean isWithCPK) throws Exception { createFileAndGetContent(fs, testFileName, FILE_SIZE); AbfsClient abfsClient = fs.getAbfsClient(); + TracingContext tracingContext = getTestTracingContext(fs, false); AbfsRestOperation abfsRestOperation = abfsClient - .getPathStatus(testFileName, false); + .getPathStatus(testFileName, false, tracingContext); assertCPKHeaders(abfsRestOperation, false); assertResponseHeader(abfsRestOperation, isWithCPK, X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); @@ -629,7 +649,7 @@ private void testGetPathStatusFile(final boolean isWithCPK) throws Exception { assertResponseHeader(abfsRestOperation, false, X_MS_REQUEST_SERVER_ENCRYPTED, ""); - abfsRestOperation = abfsClient.getPathStatus(testFileName, true); + abfsRestOperation = abfsClient.getPathStatus(testFileName, true, tracingContext); assertCPKHeaders(abfsRestOperation, isWithCPK); assertResponseHeader(abfsRestOperation, isWithCPK, X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); @@ -660,7 +680,8 @@ private void testDeletePath(final boolean isWithCPK) throws Exception { AbfsClient abfsClient = fs.getAbfsClient(); AbfsRestOperation abfsRestOperation = abfsClient - .deletePath(testFileName, false, null); + .deletePath(testFileName, false, null, + getTestTracingContext(fs, false)); assertCPKHeaders(abfsRestOperation, false); assertNoCPKResponseHeadersPresent(abfsRestOperation); @@ -681,13 +702,14 @@ public void testSetPermissionWithoutCPK() throws Exception { private void testSetPermission(final boolean isWithCPK) throws Exception { final AzureBlobFileSystem fs = getAbfs(isWithCPK); final String testFileName = "/" + methodName.getMethodName(); - Assume.assumeTrue(fs.getIsNamespaceEnabled()); + Assume.assumeTrue(fs.getIsNamespaceEnabled(getTestTracingContext(fs, false))); createFileAndGetContent(fs, testFileName, FILE_SIZE); AbfsClient abfsClient = fs.getAbfsClient(); FsPermission permission = new FsPermission(FsAction.EXECUTE, FsAction.EXECUTE, FsAction.EXECUTE); AbfsRestOperation abfsRestOperation = abfsClient - .setPermission(testFileName, permission.toString()); + .setPermission(testFileName, permission.toString(), + getTestTracingContext(fs, false)); assertCPKHeaders(abfsRestOperation, false); assertNoCPKResponseHeadersPresent(abfsRestOperation); } @@ -705,7 +727,8 @@ public void testSetAclWithoutCPK() throws Exception { private void testSetAcl(final boolean isWithCPK) throws Exception { final AzureBlobFileSystem fs = getAbfs(isWithCPK); final String testFileName = "/" + methodName.getMethodName(); - Assume.assumeTrue(fs.getIsNamespaceEnabled()); + TracingContext tracingContext = getTestTracingContext(fs, false); + Assume.assumeTrue(fs.getIsNamespaceEnabled(tracingContext)); createFileAndGetContent(fs, testFileName, FILE_SIZE); AbfsClient abfsClient = fs.getAbfsClient(); @@ -714,7 +737,8 @@ private void testSetAcl(final boolean isWithCPK) throws Exception { .deserializeAclSpec(AclEntry.aclSpecToString(aclSpec)); AbfsRestOperation abfsRestOperation = abfsClient - .setAcl(testFileName, AbfsAclHelper.serializeAclSpec(aclEntries)); + .setAcl(testFileName, AbfsAclHelper.serializeAclSpec(aclEntries), + tracingContext); assertCPKHeaders(abfsRestOperation, false); assertNoCPKResponseHeadersPresent(abfsRestOperation); } @@ -732,10 +756,12 @@ public void testGetAclWithoutCPK() throws Exception { private void testGetAcl(final boolean isWithCPK) throws Exception { final AzureBlobFileSystem fs = getAbfs(isWithCPK); final String testFileName = "/" + methodName.getMethodName(); - Assume.assumeTrue(fs.getIsNamespaceEnabled()); + TracingContext tracingContext = getTestTracingContext(fs, false); + Assume.assumeTrue(fs.getIsNamespaceEnabled(tracingContext)); createFileAndGetContent(fs, testFileName, FILE_SIZE); AbfsClient abfsClient = fs.getAbfsClient(); - AbfsRestOperation abfsRestOperation = abfsClient.getAclStatus(testFileName); + AbfsRestOperation abfsRestOperation = + abfsClient.getAclStatus(testFileName, tracingContext); assertCPKHeaders(abfsRestOperation, false); assertNoCPKResponseHeadersPresent(abfsRestOperation); } @@ -763,7 +789,7 @@ private void testCheckAccess(final boolean isWithCPK) throws Exception { fs.create(new Path(testFileName)); AbfsClient abfsClient = fs.getAbfsClient(); AbfsRestOperation abfsRestOperation = abfsClient - .checkAccess(testFileName, "rwx"); + .checkAccess(testFileName, "rwx", getTestTracingContext(fs, false)); assertCPKHeaders(abfsRestOperation, false); assertNoCPKResponseHeadersPresent(abfsRestOperation); } From d0b33c151b9331b9f542aa2a94d48b1a25e14d8c Mon Sep 17 00:00:00 2001 From: sumangala Date: Sun, 30 May 2021 15:10:12 +0530 Subject: [PATCH 72/77] typo --- .../org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java | 1 + 1 file changed, 1 insertion(+) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java index 7e610c107a687..2c0bd31bf8eeb 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java @@ -426,6 +426,7 @@ private void testListPath(final boolean isWithCPK) throws Exception { "different-1234567890123456789012"); AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); AbfsClient abfsClient2 = fs2.getAbfsClient(); + TracingContext tracingContext = getTestTracingContext(fs, false); abfsRestOperation = abfsClient2.listPath(testDirName, false, INT_50, null, tracingContext); assertListstatus(fs, abfsRestOperation, testPath); From 1567e3e82240a242a6f11970d6d313db575a3787 Mon Sep 17 00:00:00 2001 From: sumangala Date: Fri, 25 Jun 2021 00:31:16 +0530 Subject: [PATCH 73/77] address review comments --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 12 +- .../fs/azurebfs/AzureBlobFileSystem.java | 183 +++++++++--------- .../azurebfs/constants/ConfigurationKeys.java | 2 +- .../azurebfs/constants/FSOperationType.java | 60 ++++++ .../constants/HdfsOperationConstants.java | 50 ----- .../azurebfs/services/AbfsHttpOperation.java | 5 - .../fs/azurebfs/services/AbfsInputStream.java | 14 +- .../fs/azurebfs/services/AbfsLease.java | 4 +- .../azurebfs/services/AbfsOutputStream.java | 14 +- .../azurebfs/services/AbfsRestOperation.java | 4 +- .../hadoop/fs/azurebfs/utils/Listener.java | 6 +- .../fs/azurebfs/utils/TracingContext.java | 35 ++-- ...xtFormat.java => TracingHeaderFormat.java} | 2 +- .../azurebfs/AbstractAbfsIntegrationTest.java | 16 +- .../azurebfs/ITestAbfsReadWriteAndSeek.java | 18 +- .../ITestAzureBlobFileSystemAppend.java | 6 +- .../ITestAzureBlobFileSystemAttributes.java | 8 +- .../ITestAzureBlobFileSystemCreate.java | 14 +- .../ITestAzureBlobFileSystemDelete.java | 6 +- .../ITestAzureBlobFileSystemFlush.java | 6 +- .../ITestAzureBlobFileSystemLease.java | 18 +- .../ITestAzureBlobFileSystemListStatus.java | 8 +- .../azurebfs/ITestAzureBlobFilesystemAcl.java | 30 +-- .../fs/azurebfs/TestTracingContext.java | 32 ++- .../services/TestAbfsOutputStream.java | 36 ++-- .../utils/TracingHeaderValidator.java | 65 +++---- 26 files changed, 330 insertions(+), 324 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FSOperationType.java delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HdfsOperationConstants.java rename hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/{TracingContextFormat.java => TracingHeaderFormat.java} (96%) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index e3c9388bc9c08..1261cc2c8c9be 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -61,7 +61,7 @@ import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy; import org.apache.hadoop.fs.azurebfs.services.KeyProvider; import org.apache.hadoop.fs.azurebfs.services.SimpleKeyProvider; -import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.apache.hadoop.security.ProviderUtils; import org.apache.hadoop.util.ReflectionUtils; @@ -268,7 +268,7 @@ public class AbfsConfiguration{ @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_CLIENT_CORRELATIONID, DefaultValue = EMPTY_STRING) - private String clientCorrelationID; + private String clientCorrelationId; @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ENABLE_DELEGATION_TOKEN, DefaultValue = DEFAULT_ENABLE_DELEGATION_TOKEN) @@ -342,8 +342,8 @@ public String getAccountName() { * Gets client correlation ID provided in config. * @return Client Correlation ID config */ - public String getClientCorrelationID() { - return clientCorrelationID; + public String getClientCorrelationId() { + return clientCorrelationId; } /** @@ -740,8 +740,8 @@ public DelegatingSSLSocketFactory.SSLChannelMode getPreferredSSLFactoryOption() * Enum config to allow user to pick format of x-ms-client-request-id header * @return tracingContextFormat config if valid, else default ALL_ID_FORMAT */ - public TracingContextFormat getTracingContextFormat() { - return getEnum(FS_AZURE_TRACINGCONTEXT_FORMAT, TracingContextFormat.ALL_ID_FORMAT); + public TracingHeaderFormat getTracingHeaderFormat() { + return getEnum(FS_AZURE_TRACINGHEADER_FORMAT, TracingHeaderFormat.ALL_ID_FORMAT); } public AuthType getAuthType(String accountName) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index d3c3ef8ec4741..9206efeae1ef9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -69,7 +69,7 @@ import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.FileSystemOperationUnhandledException; @@ -81,7 +81,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsCounters; import org.apache.hadoop.fs.azurebfs.utils.Listener; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl; import org.apache.hadoop.fs.impl.OpenFileParameters; import org.apache.hadoop.fs.permission.AclEntry; @@ -116,13 +116,13 @@ public class AzureBlobFileSystem extends FileSystem private Path workingDir; private AzureBlobFileSystemStore abfsStore; private boolean isClosed; - private final String fileSystemID = UUID.randomUUID().toString(); + private final String fileSystemId = UUID.randomUUID().toString(); private boolean delegationTokenEnabled = false; private AbfsDelegationTokenManager delegationTokenManager; private AbfsCounters abfsCounters; - private String clientCorrelationID; - private TracingContextFormat tracingContextFormat; + private String clientCorrelationId; + private TracingHeaderFormat tracingHeaderFormat; private Listener listener; @Override @@ -141,14 +141,16 @@ public void initialize(URI uri, Configuration configuration) LOG.trace("AzureBlobFileSystemStore init complete"); final AbfsConfiguration abfsConfiguration = abfsStore.getAbfsConfiguration(); - clientCorrelationID = abfsConfiguration.getClientCorrelationID(); - tracingContextFormat = abfsConfiguration.getTracingContextFormat(); + clientCorrelationId = abfsConfiguration.getClientCorrelationId(); + tracingHeaderFormat = abfsConfiguration.getTracingHeaderFormat(); this.setWorkingDirectory(this.getHomeDirectory()); if (abfsConfiguration.getCreateRemoteFileSystemDuringInitialization()) { - if (this.tryGetFileStatus(new Path(AbfsHttpConstants.ROOT_PATH)) == null) { + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, listener); + if (this.tryGetFileStatus(new Path(AbfsHttpConstants.ROOT_PATH), tracingContext) == null) { try { - this.createFileSystem(); + this.createFileSystem(tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(null, ex, AzureServiceErrorCode.FILE_SYSTEM_ALREADY_EXISTS); } @@ -209,8 +211,8 @@ private FSDataInputStream open(final Path path, Path qualifiedPath = makeQualified(path); try { - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.OPEN, tracingContextFormat, + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.OPEN, tracingHeaderFormat, listener); InputStream inputStream = abfsStore.openFileForRead(qualifiedPath, options, statistics, tracingContext); @@ -249,9 +251,8 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi Path qualifiedPath = makeQualified(f); try { - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.CREATE, overwrite, - tracingContextFormat, listener); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.CREATE, overwrite, tracingHeaderFormat, listener); OutputStream outputStream = abfsStore.createFile(qualifiedPath, statistics, overwrite, permission == null ? FsPermission.getFileDefault() : permission, FsPermission.getUMask(getConf()), tracingContext); @@ -271,7 +272,10 @@ public FSDataOutputStream createNonRecursive(final Path f, final FsPermission pe statIncrement(CALL_CREATE_NON_RECURSIVE); final Path parent = f.getParent(); - final FileStatus parentFileStatus = tryGetFileStatus(parent); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.CREATE_NON_RECURSIVE, tracingHeaderFormat, + listener); + final FileStatus parentFileStatus = tryGetFileStatus(parent, tracingContext); if (parentFileStatus == null) { throw new FileNotFoundException("Cannot create file " @@ -317,8 +321,8 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr Path qualifiedPath = makeQualified(f); try { - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.APPEND, tracingContextFormat, + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.APPEND, tracingHeaderFormat, listener); OutputStream outputStream = abfsStore .openFileForWrite(qualifiedPath, statistics, false, tracingContext); @@ -342,9 +346,12 @@ public boolean rename(final Path src, final Path dst) throws IOException { Path qualifiedSrcPath = makeQualified(src); Path qualifiedDstPath = makeQualified(dst); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.RENAME, true, tracingHeaderFormat, + listener); // rename under same folder; if(makeQualified(parentFolder).equals(qualifiedDstPath)) { - return tryGetFileStatus(qualifiedSrcPath) != null; + return tryGetFileStatus(qualifiedSrcPath, tracingContext) != null; } FileStatus dstFileStatus = null; @@ -353,7 +360,7 @@ public boolean rename(final Path src, final Path dst) throws IOException { // - if it doesn't exist, return false // - if it is file, return true // - if it is dir, return false. - dstFileStatus = tryGetFileStatus(qualifiedDstPath); + dstFileStatus = tryGetFileStatus(qualifiedDstPath, tracingContext); if (dstFileStatus == null) { return false; } @@ -361,11 +368,8 @@ public boolean rename(final Path src, final Path dst) throws IOException { } // Non-HNS account need to check dst status on driver side. - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.RENAME, true, tracingContextFormat, - listener); if (!abfsStore.getIsNamespaceEnabled(tracingContext) && dstFileStatus == null) { - dstFileStatus = tryGetFileStatus(qualifiedDstPath); + dstFileStatus = tryGetFileStatus(qualifiedDstPath, tracingContext); } try { @@ -415,8 +419,8 @@ public boolean delete(final Path f, final boolean recursive) throws IOException } try { - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.DELETE, tracingContextFormat, + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.DELETE, tracingHeaderFormat, listener); abfsStore.delete(qualifiedPath, recursive, tracingContext); return true; @@ -435,9 +439,9 @@ public FileStatus[] listStatus(final Path f) throws IOException { Path qualifiedPath = makeQualified(f); try { - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.LISTSTATUS, true, - tracingContextFormat, listener); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.LISTSTATUS, true, tracingHeaderFormat + , listener); FileStatus[] result = abfsStore.listStatus(qualifiedPath, tracingContext); return result; } catch (AzureBlobFileSystemException ex) { @@ -507,9 +511,9 @@ public boolean mkdirs(final Path f, final FsPermission permission) throws IOExce Path qualifiedPath = makeQualified(f); try { - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.MKDIR, false, - tracingContextFormat, listener); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.MKDIR, false, tracingHeaderFormat, + listener); abfsStore.createDirectory(qualifiedPath, permission == null ? FsPermission.getDirDefault() : permission, FsPermission.getUMask(getConf()), tracingContext); @@ -544,17 +548,22 @@ public synchronized void close() throws IOException { @Override public FileStatus getFileStatus(final Path f) throws IOException { - LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.GET_FILESTATUS, tracingHeaderFormat, + listener); + return getFileStatus(f, tracingContext); + } + + private FileStatus getFileStatus(final Path path, + TracingContext tracingContext) throws IOException { + LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", path); statIncrement(CALL_GET_FILE_STATUS); - Path qualifiedPath = makeQualified(f); + Path qualifiedPath = makeQualified(path); try { - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.GET_FILESTATUS, - tracingContextFormat, listener); return abfsStore.getFileStatus(qualifiedPath, tracingContext); } catch(AzureBlobFileSystemException ex) { - checkException(f, ex); + checkException(path, ex); return null; } } @@ -573,9 +582,9 @@ public void breakLease(final Path f) throws IOException { try (DurationInfo ignored = new DurationInfo(LOG, false, "Break lease for %s", qualifiedPath)) { - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.BREAK_LEASE, - tracingContextFormat, listener); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.BREAK_LEASE, tracingHeaderFormat, + listener); abfsStore.breakLease(qualifiedPath, tracingContext); } catch(AzureBlobFileSystemException ex) { checkException(f, ex); @@ -749,9 +758,9 @@ public void setOwner(final Path path, final String owner, final String group) throws IOException { LOG.debug( "AzureBlobFileSystem.setOwner path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.SET_OWNER, true, - tracingContextFormat, listener); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.SET_OWNER, true, tracingHeaderFormat, + listener); if (!getIsNamespaceEnabled(tracingContext)) { super.setOwner(path, owner, group); @@ -796,9 +805,9 @@ public void setXAttr(final Path path, final String name, final byte[] value, fin Path qualifiedPath = makeQualified(path); try { - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.SET_ATTR, true, - tracingContextFormat, listener); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.SET_ATTR, true, tracingHeaderFormat, + listener); Hashtable properties = abfsStore .getPathStatus(qualifiedPath, tracingContext); String xAttrName = ensureValidAttributeName(name); @@ -836,9 +845,9 @@ public byte[] getXAttr(final Path path, final String name) byte[] value = null; try { - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.GET_ATTR, true, - tracingContextFormat, listener); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.GET_ATTR, true, tracingHeaderFormat, + listener); Hashtable properties = abfsStore .getPathStatus(qualifiedPath, tracingContext); String xAttrName = ensureValidAttributeName(name); @@ -867,9 +876,8 @@ private static String ensureValidAttributeName(String attribute) { public void setPermission(final Path path, final FsPermission permission) throws IOException { LOG.debug("AzureBlobFileSystem.setPermission path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.SET_PERMISSION, true, - tracingContextFormat, listener); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.SET_PERMISSION, true, tracingHeaderFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { super.setPermission(path, permission); @@ -903,9 +911,9 @@ public void setPermission(final Path path, final FsPermission permission) public void modifyAclEntries(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.modifyAclEntries path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.MODIFY_ACL, true, - tracingContextFormat, listener); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.MODIFY_ACL, true, tracingHeaderFormat, + listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -938,9 +946,9 @@ public void modifyAclEntries(final Path path, final List aclSpec) public void removeAclEntries(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.removeAclEntries path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.REMOVE_ACL_ENTRIES, true, - tracingContextFormat, listener); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.REMOVE_ACL_ENTRIES, true, + tracingHeaderFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -970,9 +978,9 @@ public void removeAclEntries(final Path path, final List aclSpec) @Override public void removeDefaultAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeDefaultAcl path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.REMOVE_DEFAULT_ACL, true, - tracingContextFormat, listener); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.REMOVE_DEFAULT_ACL, true, + tracingHeaderFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -1000,9 +1008,9 @@ public void removeDefaultAcl(final Path path) throws IOException { @Override public void removeAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeAcl path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.REMOVE_ACL, true, - tracingContextFormat, listener); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.REMOVE_ACL, true, tracingHeaderFormat, + listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -1033,9 +1041,9 @@ public void removeAcl(final Path path) throws IOException { public void setAcl(final Path path, final List aclSpec) throws IOException { LOG.debug("AzureBlobFileSystem.setAcl path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.SET_ACL, true, - tracingContextFormat, listener); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.SET_ACL, true, tracingHeaderFormat, + listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -1066,9 +1074,8 @@ public void setAcl(final Path path, final List aclSpec) @Override public AclStatus getAclStatus(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.getAclStatus path: {}", path); - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.GET_ACL_STATUS, true, - tracingContextFormat, listener); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.GET_ACL_STATUS, true, tracingHeaderFormat, listener); if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( @@ -1103,8 +1110,8 @@ public void access(final Path path, final FsAction mode) throws IOException { LOG.debug("AzureBlobFileSystem.access path : {}, mode : {}", path, mode); Path qualifiedPath = makeQualified(path); try { - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.ACCESS, tracingContextFormat, + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.ACCESS, tracingHeaderFormat, listener); this.abfsStore.access(qualifiedPath, mode, tracingContext); } catch (AzureBlobFileSystemException ex) { @@ -1130,20 +1137,20 @@ public RemoteIterator listStatusIterator(Path path) throws IOException { LOG.debug("AzureBlobFileSystem.listStatusIterator path : {}", path); if (abfsStore.getAbfsConfiguration().enableAbfsListIterator()) { - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.LISTSTATUS, true, - tracingContextFormat, listener); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.LISTSTATUS, true, tracingHeaderFormat, listener); AbfsListStatusRemoteIterator abfsLsItr = - new AbfsListStatusRemoteIterator(getFileStatus(path), abfsStore, tracingContext); + new AbfsListStatusRemoteIterator(getFileStatus(path, tracingContext), abfsStore, + tracingContext); return RemoteIterators.typeCastingRemoteIterator(abfsLsItr); } else { return super.listStatusIterator(path); } } - private FileStatus tryGetFileStatus(final Path f) { + private FileStatus tryGetFileStatus(final Path f, TracingContext tracingContext) { try { - return getFileStatus(f); + return getFileStatus(f, tracingContext); } catch (IOException ex) { LOG.debug("File not found {}", f); statIncrement(ERROR_IGNORED); @@ -1155,9 +1162,8 @@ private boolean fileSystemExists() throws IOException { LOG.debug( "AzureBlobFileSystem.fileSystemExists uri: {}", uri); try { - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.GET_FILESTATUS, - tracingContextFormat, listener); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.TEST_OP, tracingHeaderFormat, listener); abfsStore.getFilesystemProperties(tracingContext); } catch (AzureBlobFileSystemException ex) { try { @@ -1173,13 +1179,10 @@ private boolean fileSystemExists() throws IOException { return true; } - private void createFileSystem() throws IOException { + private void createFileSystem(TracingContext tracingContext) throws IOException { LOG.debug( "AzureBlobFileSystem.createFileSystem uri: {}", uri); try { - TracingContext tracingContext = new TracingContext(clientCorrelationID, - fileSystemID, HdfsOperationConstants.CREATE_FILESYSTEM, - tracingContextFormat, listener); abfsStore.createFilesystem(tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(null, ex); @@ -1374,7 +1377,7 @@ FileSystem.Statistics getFsStatistics() { } @VisibleForTesting - void setListenerOperation(String operation) { + void setListenerOperation(FSOperationType operation) { listener.setOperation(operation); } @@ -1430,8 +1433,8 @@ Map getInstrumentationMap() { } @VisibleForTesting - String getFileSystemID() { - return fileSystemID; + String getFileSystemId() { + return fileSystemId; } @Override @@ -1445,8 +1448,8 @@ public boolean hasPathCapability(final Path path, final String capability) return true; case CommonPathCapabilities.FS_ACLS: return getIsNamespaceEnabled( - new TracingContext(clientCorrelationID, fileSystemID, - HdfsOperationConstants.HAS_PATH_CAPABILITY, tracingContextFormat, + new TracingContext(clientCorrelationId, fileSystemId, + FSOperationType.HAS_PATH_CAPABILITY, tracingHeaderFormat, listener)); default: return super.hasPathCapability(p, capability); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index 4e3afed511218..df9845130bce5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -114,7 +114,7 @@ public final class ConfigurationKeys { * x-ms-client-request-Id header. Defaults to empty string if the length and * character constraints are not satisfied. **/ public static final String FS_AZURE_CLIENT_CORRELATIONID = "fs.azure.client.correlationid"; - public static final String FS_AZURE_TRACINGCONTEXT_FORMAT = "fs.azure.tracingcontext.format"; + public static final String FS_AZURE_TRACINGHEADER_FORMAT = "fs.azure.tracingheader.format"; public static final String FS_AZURE_CLUSTER_NAME = "fs.azure.cluster.name"; public static final String FS_AZURE_CLUSTER_TYPE = "fs.azure.cluster.type"; public static final String FS_AZURE_SSL_CHANNEL_MODE_KEY = "fs.azure.ssl.channel.mode"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FSOperationType.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FSOperationType.java new file mode 100644 index 0000000000000..6b6e98c9c7082 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FSOperationType.java @@ -0,0 +1,60 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"), you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs.constants; + +public enum FSOperationType { + ACCESS("AS"), + APPEND("AP"), + BREAK_LEASE("BL"), + CREATE("CR"), + CREATE_FILESYSTEM("CF"), + CREATE_NON_RECURSIVE("CN"), + DELETE("DL"), + GET_ACL_STATUS("GA"), + GET_ATTR("GR"), + GET_FILESTATUS("GF"), + LISTSTATUS("LS"), + MKDIR("MK"), + MODIFY_ACL("MA"), + OPEN("OP"), + HAS_PATH_CAPABILITY("PC"), + SET_PERMISSION("SP"), + READ("RE"), + RELEASE_LEASE("RL"), + REMOVE_ACL("RA"), + REMOVE_ACL_ENTRIES("RT"), + REMOVE_DEFAULT_ACL("RD"), + RENAME("RN"), + SET_ATTR("SR"), + SET_OWNER("SO"), + SET_ACL("SA"), + TEST_OP("TS"), + WRITE("WR"); + + private final String opCode; + + FSOperationType(String opCode) { + this.opCode = opCode; + } + + @Override + public String toString() { + return opCode; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HdfsOperationConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HdfsOperationConstants.java deleted file mode 100644 index cb2fc81ab0d83..0000000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HdfsOperationConstants.java +++ /dev/null @@ -1,50 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.constants; - -public final class HdfsOperationConstants { - public static final String ACCESS = "AS"; - public static final String APPEND = "AP"; - public static final String BREAK_LEASE = "BL"; - public static final String CREATE = "CR"; - public static final String CREATE_FILESYSTEM = "CF"; - public static final String DELETE = "DL"; - public static final String GET_ACL_STATUS = "GA"; - public static final String GET_ATTR = "GR"; - public static final String GET_FILESTATUS = "GF"; - public static final String LISTSTATUS = "LS"; - public static final String MKDIR = "MK"; - public static final String MODIFY_ACL = "MA"; - public static final String OPEN = "OP"; - public static final String HAS_PATH_CAPABILITY = "PC"; - public static final String SET_PERMISSION = "SP"; - public static final String READ = "RE"; - public static final String RELEASE_LEASE = "RL"; - public static final String REMOVE_ACL = "RA"; - public static final String REMOVE_ACL_ENTRIES = "RT"; - public static final String REMOVE_DEFAULT_ACL = "RD"; - public static final String RENAME = "RN"; - public static final String SET_ATTR = "SR"; - public static final String SET_OWNER = "SO"; - public static final String SET_ACL = "SA"; - public static final String TEST_OP = "TS"; - public static final String WRITE = "WR"; - - private HdfsOperationConstants() {} -} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index 92cc53f958e99..09c5fe549d1d2 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -166,11 +166,6 @@ public String getResponseHeader(String httpHeader) { return connection.getHeaderField(httpHeader); } - @VisibleForTesting - public String getRequestHeader(String httpHeader) { - return connection.getRequestProperties().get(httpHeader).toString(); - } - // Returns a trace message for the request @Override public String toString() { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index a5b47c55d90e5..f7715eb0c0d08 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -36,7 +36,7 @@ import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.FileSystem.Statistics; import org.apache.hadoop.fs.StreamCapabilities; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.utils.CachedSASToken; @@ -72,7 +72,7 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer, private final String eTag; // eTag of the path when InputStream are created private final boolean tolerateOobAppends; // whether tolerate Oob Appends private final boolean readAheadEnabled; // whether enable readAhead; - private final String inputStreamID; + private final String inputStreamId; private final boolean alwaysReadBufferSize; /* * By default the pread API will do a seek + read as in FSInputStream. @@ -137,10 +137,10 @@ public AbfsInputStream( this.cachedSasToken = new CachedSASToken( abfsInputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds()); this.streamStatistics = abfsInputStreamContext.getStreamStatistics(); - this.inputStreamID = getInputStreamID(); + this.inputStreamId = getInputStreamId(); this.tracingContext = new TracingContext(tracingContext); - this.tracingContext.setOperation(HdfsOperationConstants.READ); - this.tracingContext.setStreamID(inputStreamID); + this.tracingContext.setOperation(FSOperationType.READ); + this.tracingContext.setStreamID(inputStreamId); this.context = abfsInputStreamContext; readAheadBlockSize = abfsInputStreamContext.getReadAheadBlockSize(); @@ -156,7 +156,7 @@ public String getPath() { return path; } - private String getInputStreamID() { + private String getInputStreamId() { return StringUtils.right(UUID.randomUUID().toString(), STREAM_ID_LEN); } @@ -728,7 +728,7 @@ protected void setCachedSasToken(final CachedSASToken cachedSasToken) { @VisibleForTesting public String getStreamID() { - return inputStreamID; + return inputStreamId; } /** diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java index 84f581a7fe202..7421bc2e282ea 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java @@ -29,8 +29,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.io.retry.RetryPolicies; @@ -169,7 +169,7 @@ public void free() { future.cancel(true); } TracingContext tracingContext = new TracingContext(this.tracingContext); - tracingContext.setOperation(HdfsOperationConstants.RELEASE_LEASE); + tracingContext.setOperation(FSOperationType.RELEASE_LEASE); client.releaseLease(path, leaseID, tracingContext); } catch (IOException e) { LOG.warn("Exception when trying to release lease {} on {}. Lease will need to be broken: {}", diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 80cdee05e5597..b9d57a0aab1a8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -40,7 +40,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.PathIOException; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; @@ -96,7 +96,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable, // SAS tokens can be re-used until they expire private CachedSASToken cachedSasToken; - private final String outputStreamID; + private final String outputStreamId; private final TracingContext tracingContext; private Listener listener; @@ -170,13 +170,13 @@ public AbfsOutputStream( if (outputStreamStatistics != null) { this.ioStatistics = outputStreamStatistics.getIOStatistics(); } - this.outputStreamID = getOutputStreamID(); + this.outputStreamId = getOutputStreamId(); this.tracingContext = new TracingContext(tracingContext); - this.tracingContext.setStreamID(outputStreamID); - this.tracingContext.setOperation(HdfsOperationConstants.WRITE); + this.tracingContext.setStreamID(outputStreamId); + this.tracingContext.setOperation(FSOperationType.WRITE); } - private String getOutputStreamID() { + private String getOutputStreamId() { return StringUtils.right(UUID.randomUUID().toString(), STREAM_ID_LEN); } @@ -311,7 +311,7 @@ public void hflush() throws IOException { } public String getStreamID() { - return outputStreamID; + return outputStreamId; } public void registerListener(Listener listener1) { 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 f89c8bf370565..3547ba8acb367 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 @@ -228,10 +228,10 @@ private void completeExecute(TracingContext tracingContext) private void updateClientRequestHeader(AbfsHttpOperation httpOperation, TracingContext tracingContext) { - tracingContext.generateClientRequestID(); + tracingContext.generateClientRequestId(); httpOperation.getConnection() .setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, - tracingContext.toString()); + tracingContext.constructHeader()); } /** diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java index 32ddc5819a584..4c2270a87f100 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java @@ -18,14 +18,16 @@ package org.apache.hadoop.fs.azurebfs.utils; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; + /** * Interface for testing identifiers tracked via TracingContext * Implemented in TracingHeaderValidator */ public interface Listener { - void callTracingHeaderValidator(String header, TracingContextFormat format); + void callTracingHeaderValidator(String header, TracingHeaderFormat format); void updatePrimaryRequestID(String primaryRequestID); Listener getClone(); - void setOperation(String operation); + void setOperation(FSOperationType operation); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index d43c2e3c91508..0df792c2852aa 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -23,6 +23,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; @@ -49,12 +50,12 @@ public class TracingContext { private final String clientCorrelationID; private final String fileSystemID; - private String clientRequestID = EMPTY_STRING; + private String clientRequestId = EMPTY_STRING; private String primaryRequestID; private String streamID; private int retryCount; - private String hadoopOpName; - private final TracingContextFormat format; + private FSOperationType hadoopOpName; + private final TracingHeaderFormat format; private Listener listener = null; private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); @@ -67,12 +68,12 @@ public class TracingContext { * @param fileSystemID Unique guid for AzureBlobFileSystem instance * @param hadoopOpName Code indicating the high-level Hadoop operation that * triggered the current Store request - * @param tracingContextFormat Format of IDs to be printed in header and logs + * @param tracingHeaderFormat Format of IDs to be printed in header and logs * @param listener Holds instance of TracingHeaderValidator during testing, * null otherwise */ public TracingContext(String clientCorrelationID, String fileSystemID, - String hadoopOpName, TracingContextFormat tracingContextFormat, + FSOperationType hadoopOpName, TracingHeaderFormat tracingHeaderFormat, Listener listener) { this.fileSystemID = fileSystemID; this.hadoopOpName = hadoopOpName; @@ -80,14 +81,14 @@ public TracingContext(String clientCorrelationID, String fileSystemID, streamID = EMPTY_STRING; retryCount = 0; primaryRequestID = EMPTY_STRING; - format = tracingContextFormat; + format = tracingHeaderFormat; this.listener = listener; } public TracingContext(String clientCorrelationID, String fileSystemID, - String hadoopOpName, boolean needsPrimaryReqId, - TracingContextFormat tracingContextFormat, Listener listener) { - this(clientCorrelationID, fileSystemID, hadoopOpName, tracingContextFormat, + FSOperationType hadoopOpName, boolean needsPrimaryReqId, + TracingHeaderFormat tracingHeaderFormat, Listener listener) { + this(clientCorrelationID, fileSystemID, hadoopOpName, tracingHeaderFormat, listener); primaryRequestID = needsPrimaryReqId ? UUID.randomUUID().toString() : ""; if (listener != null) { @@ -118,8 +119,8 @@ public String validateClientCorrelationID(String clientCorrelationID) { return clientCorrelationID; } - public void generateClientRequestID() { - clientRequestID = UUID.randomUUID().toString(); + public void generateClientRequestId() { + clientRequestId = UUID.randomUUID().toString(); } public void setPrimaryRequestID() { @@ -133,7 +134,7 @@ public void setStreamID(String stream) { streamID = stream; } - public void setOperation(String operation) { + public void setOperation(FSOperationType operation) { this.hadoopOpName = operation; } @@ -145,22 +146,22 @@ public void setListener(Listener listener) { this.listener = listener; } - public String toString() { + public String constructHeader() { String header; switch (format) { case ALL_ID_FORMAT: header = - clientCorrelationID + ":" + clientRequestID + ":" + fileSystemID + ":" + clientCorrelationID + ":" + clientRequestId + ":" + fileSystemID + ":" + primaryRequestID + ":" + streamID + ":" + hadoopOpName + ":" + retryCount; break; case TWO_ID_FORMAT: - header = clientCorrelationID + ":" + clientRequestID; + header = clientCorrelationID + ":" + clientRequestId; break; default: - header = clientRequestID; //case SINGLE_ID_FORMAT + header = clientRequestId; //case SINGLE_ID_FORMAT } - if (listener != null) { //testing + if (listener != null) { //for testing listener.callTracingHeaderValidator(header, format); } return header; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderFormat.java similarity index 96% rename from hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java rename to hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderFormat.java index 99f3b9ab23d32..93af796f8803d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContextFormat.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderFormat.java @@ -18,7 +18,7 @@ package org.apache.hadoop.fs.azurebfs.utils; -public enum TracingContextFormat { +public enum TracingHeaderFormat { SINGLE_ID_FORMAT, // ALL_ID_FORMAT, // :: diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index f715408835893..2497f8f1b63c5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -35,6 +35,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic; 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.security.AbfsDelegationTokenManager; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; @@ -44,10 +45,9 @@ import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; import org.apache.hadoop.fs.azurebfs.utils.UriUtils; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.permission.FsPermission; @@ -150,19 +150,19 @@ protected boolean getIsNamespaceEnabled(AzureBlobFileSystem fs) public TracingContext getTestTracingContext(AzureBlobFileSystem fs, boolean needsPrimaryReqId) { String correlationId, fsId; - TracingContextFormat format; + TracingHeaderFormat format; if (fs == null) { correlationId = "test-corr-id"; fsId = "test-filesystem-id"; - format = TracingContextFormat.ALL_ID_FORMAT; + format = TracingHeaderFormat.ALL_ID_FORMAT; } else { AbfsConfiguration abfsConf = fs.getAbfsStore().getAbfsConfiguration(); - correlationId = abfsConf.getClientCorrelationID(); - fsId = fs.getFileSystemID(); - format = abfsConf.getTracingContextFormat(); + correlationId = abfsConf.getClientCorrelationId(); + fsId = fs.getFileSystemId(); + format = abfsConf.getTracingHeaderFormat(); } return new TracingContext(correlationId, fsId, - HdfsOperationConstants.TEST_OP, needsPrimaryReqId, format, null); + FSOperationType.TEST_OP, needsPrimaryReqId, format, null); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index d04acbf13b688..f4f0f231037e1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; @@ -83,8 +83,8 @@ private void testReadWriteAndSeek(int bufferSize) throws Exception { int result; try (FSDataInputStream inputStream = fs.open(TEST_PATH)) { ((AbfsInputStream) inputStream.getWrappedStream()).registerListener( - new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.READ, true, 0, + new TracingHeaderValidator(abfsConfiguration.getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.READ, true, 0, ((AbfsInputStream) inputStream.getWrappedStream()) .getStreamID())); inputStream.seek(bufferSize); @@ -114,8 +114,8 @@ public void testReadAheadRequestID() throws java.io.IOException { new Random().nextBytes(b); try (FSDataOutputStream stream = fs.create(TEST_PATH)) { ((AbfsOutputStream) stream.getWrappedStream()).registerListener( - new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.WRITE, false, 0, + new TracingHeaderValidator(abfsConfiguration.getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.WRITE, false, 0, ((AbfsOutputStream) stream.getWrappedStream()) .getStreamID())); stream.write(b); @@ -124,12 +124,12 @@ public void testReadAheadRequestID() throws java.io.IOException { final byte[] readBuffer = new byte[4 * bufferSize]; int result; fs.registerListener( - new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.OPEN, false, 0)); + new TracingHeaderValidator(abfsConfiguration.getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.OPEN, false, 0)); try (FSDataInputStream inputStream = fs.open(TEST_PATH)) { ((AbfsInputStream) inputStream.getWrappedStream()).registerListener( - new TracingHeaderValidator(abfsConfiguration.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.READ, false, 0, + new TracingHeaderValidator(abfsConfiguration.getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.READ, false, 0, ((AbfsInputStream) inputStream.getWrappedStream()) .getStreamID())); result = inputStream.read(readBuffer, 0, bufferSize*4); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java index 8297eb80bb26a..4b1e497f64f84 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java @@ -26,7 +26,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.fs.contract.ContractTestUtils; @@ -85,8 +85,8 @@ public void testTracingForAppend() throws IOException { AzureBlobFileSystem fs = getFileSystem(); fs.create(TEST_FILE_PATH); fs.registerListener(new TracingHeaderValidator( - fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.APPEND, false, 0)); + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.APPEND, false, 0)); fs.append(TEST_FILE_PATH, 10); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java index fe0a962f2b4e7..beb7d0ebaaa8e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java @@ -26,7 +26,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.XAttrSetFlag; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -60,10 +60,10 @@ public void testSetGetXAttr() throws Exception { // after setting the xAttr on the file, the value should be retrievable fs.registerListener( - new TracingHeaderValidator(conf.getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.SET_ATTR, true, 0)); + new TracingHeaderValidator(conf.getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.SET_ATTR, true, 0)); fs.setXAttr(testFile, attributeName1, attributeValue1); - fs.setListenerOperation(HdfsOperationConstants.GET_ATTR); + fs.setListenerOperation(FSOperationType.GET_ATTR); assertArrayEquals(attributeValue1, fs.getXAttr(testFile, attributeName1)); fs.registerListener(null); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index a771f989022ac..86689a1e61940 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -37,7 +37,7 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.test.GenericTestUtils; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConcurrentWriteOperationDetectedException; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; @@ -99,8 +99,8 @@ public void testCreateNonRecursive() throws Exception { } catch (FileNotFoundException expected) { } fs.registerListener(new TracingHeaderValidator( - fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.MKDIR, false, 0)); + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.MKDIR, false, 0)); fs.mkdirs(TEST_FOLDER_PATH); fs.registerListener(null); @@ -271,8 +271,8 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) // Case 2: Not Overwrite - File pre-exists fs.registerListener(new TracingHeaderValidator( - fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.CREATE, false, 0)); + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.CREATE, false, 0)); intercept(FileAlreadyExistsException.class, () -> fs.create(nonOverwriteFile, false)); fs.registerListener(null); @@ -302,8 +302,8 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) // Case 4: Overwrite - File pre-exists fs.registerListener(new TracingHeaderValidator( - fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.CREATE, true, 0)); + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.CREATE, true, 0)); fs.create(overwriteFilePath, true); fs.registerListener(null); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index 90e13c7e122cc..ad144dffc4e82 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -30,7 +30,7 @@ import org.junit.Assume; import org.junit.Test; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; @@ -153,8 +153,8 @@ public Void call() throws Exception { es.shutdownNow(); Path dir = new Path("/test"); fs.registerListener(new TracingHeaderValidator( - fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.DELETE, false, 0)); + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.DELETE, false, 0)); // first try a non-recursive delete, expect failure intercept(FileAlreadyExistsException.class, () -> fs.delete(dir, false)); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index 5c66531b17523..40a551cd60a30 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -31,7 +31,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.StreamCapabilities; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; @@ -319,8 +319,8 @@ public void testTracingHeaderForAppendBlob() throws Exception { FSDataOutputStream out = fs.create(new Path("/testFile")); ((AbfsOutputStream) out.getWrappedStream()).registerListener( new TracingHeaderValidator( - fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.WRITE, false, 0, + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.WRITE, false, 0, ((AbfsOutputStream) out.getWrappedStream()).getStreamID())); out.write(buf); out.hsync(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index 53005259956d7..2894abe4d0e2b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -20,20 +20,20 @@ import java.io.IOException; import java.util.concurrent.RejectedExecutionException; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.utils.Listener; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.junit.Assert; import org.junit.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; 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.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsLease; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; +import org.apache.hadoop.fs.azurebfs.utils.Listener; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; @@ -216,8 +216,8 @@ public void testWriteAfterBreakLease() throws Exception { out.hsync(); fs.registerListener(new TracingHeaderValidator( - getConfiguration().getClientCorrelationID(), fs.getFileSystemID(), - HdfsOperationConstants.BREAK_LEASE, false, 0)); + getConfiguration().getClientCorrelationId(), fs.getFileSystemId(), + FSOperationType.BREAK_LEASE, false, 0)); fs.breakLease(testFilePath); fs.registerListener(null); @@ -319,14 +319,14 @@ public void testAcquireRetry() throws Exception { fs.createNewFile(testFilePath); TracingContext tracingContext = getTestTracingContext(fs, true); Listener listener = new TracingHeaderValidator( - getConfiguration().getClientCorrelationID(), fs.getFileSystemID(), - HdfsOperationConstants.TEST_OP, true, 0); + getConfiguration().getClientCorrelationId(), fs.getFileSystemId(), + FSOperationType.TEST_OP, true, 0); tracingContext.setListener(listener); AbfsLease lease = new AbfsLease(fs.getAbfsClient(), testFilePath.toUri().getPath(), tracingContext); Assert.assertNotNull("Did not successfully lease file", lease.getLeaseID()); - listener.setOperation(HdfsOperationConstants.RELEASE_LEASE); + listener.setOperation(FSOperationType.RELEASE_LEASE); lease.free(); lease.getTracingContext().setListener(null); Assert.assertEquals("Unexpected acquire retry count", 0, lease.getAcquireRetryCount()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java index 899c0b9f49624..0b3215f5e9fe9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java @@ -35,7 +35,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.fs.contract.ContractTestUtils; @@ -67,7 +67,7 @@ public void testListPath() throws Exception { final List> tasks = new ArrayList<>(); ExecutorService es = Executors.newFixedThreadPool(10); - for (int i = 0; i < TEST_FILES_NUMBER; i++) { + for (int i = 0; i < TEST_FILES_NUMBER/100; i++) { final Path fileName = new Path("/test" + i); Callable callable = new Callable() { @Override @@ -86,8 +86,8 @@ public Void call() throws Exception { es.shutdownNow(); fs.registerListener( - new TracingHeaderValidator(getConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.LISTSTATUS, true, 0)); + new TracingHeaderValidator(getConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.LISTSTATUS, true, 0)); FileStatus[] files = fs.listStatus(new Path("/")); assertEquals(TEST_FILES_NUMBER, files.length /* user directory */); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java index d140f3b035f4f..03c3b49d6b253 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java @@ -32,7 +32,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.fs.permission.AclEntry; @@ -1220,8 +1220,8 @@ public void testDefaultAclRenamedFile() throws Exception { Path renamedFilePath = new Path(dirPath, "file1"); fs.registerListener(new TracingHeaderValidator( - fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.RENAME, true, 0)); + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.RENAME, true, 0)); fs.rename(filePath, renamedFilePath); fs.registerListener(null); AclEntry[] expected = new AclEntry[] { }; @@ -1264,28 +1264,28 @@ public void testEnsureAclOperationWorksForRoot() throws Exception { aclEntry(ACCESS, GROUP, BAR, ALL)); fs.registerListener(new TracingHeaderValidator( - fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.SET_ACL, true, 0)); + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.SET_ACL, true, 0)); fs.setAcl(rootPath, aclSpec1); - fs.setListenerOperation(HdfsOperationConstants.GET_ACL_STATUS); + fs.setListenerOperation(FSOperationType.GET_ACL_STATUS); fs.getAclStatus(rootPath); - fs.setListenerOperation(HdfsOperationConstants.SET_OWNER); + fs.setListenerOperation(FSOperationType.SET_OWNER); fs.setOwner(rootPath, TEST_OWNER, TEST_GROUP); - fs.setListenerOperation(HdfsOperationConstants.SET_PERMISSION); + fs.setListenerOperation(FSOperationType.SET_PERMISSION); fs.setPermission(rootPath, new FsPermission("777")); List aclSpec2 = Lists.newArrayList( aclEntry(DEFAULT, USER, FOO, ALL), aclEntry(ACCESS, USER, BAR, ALL)); - fs.setListenerOperation(HdfsOperationConstants.MODIFY_ACL); + fs.setListenerOperation(FSOperationType.MODIFY_ACL); fs.modifyAclEntries(rootPath, aclSpec2); - fs.setListenerOperation(HdfsOperationConstants.REMOVE_ACL_ENTRIES); + fs.setListenerOperation(FSOperationType.REMOVE_ACL_ENTRIES); fs.removeAclEntries(rootPath, aclSpec2); - fs.setListenerOperation(HdfsOperationConstants.REMOVE_DEFAULT_ACL); + fs.setListenerOperation(FSOperationType.REMOVE_DEFAULT_ACL); fs.removeDefaultAcl(rootPath); - fs.setListenerOperation(HdfsOperationConstants.REMOVE_ACL); + fs.setListenerOperation(FSOperationType.REMOVE_ACL); fs.removeAcl(rootPath); } @@ -1300,11 +1300,11 @@ public void testSetOwnerForNonNamespaceEnabledAccount() throws Exception { assertTrue(fs.exists(filePath)); TracingHeaderValidator tracingHeaderValidator = new TracingHeaderValidator( - conf.getClientCorrelationID(), fs.getFileSystemID(), - HdfsOperationConstants.GET_FILESTATUS, false, 0); + conf.getClientCorrelationId(), fs.getFileSystemId(), + FSOperationType.GET_FILESTATUS, false, 0); fs.registerListener(tracingHeaderValidator); FileStatus oldFileStatus = fs.getFileStatus(filePath); - tracingHeaderValidator.setOperation(HdfsOperationConstants.SET_OWNER); + tracingHeaderValidator.setOperation(FSOperationType.SET_OWNER); fs.setOwner(filePath, TEST_OWNER, TEST_GROUP); fs.registerListener(null); FileStatus newFileStatus = fs.getFileStatus(filePath); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 2488732d14a5a..f4c217d81a1b0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -33,13 +33,12 @@ import org.apache.hadoop.fs.CommonPathCapabilities; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; -import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.enums.Trilean; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; @@ -58,7 +57,7 @@ public TestTracingContext() throws Exception { } @Test - public void testClientCorrelationID() throws IOException { + public void testClientCorrelationId() throws IOException { checkCorrelationConfigValidation(CLIENT_CORRELATIONID_LIST[0], true); checkCorrelationConfigValidation(CLIENT_CORRELATIONID_LIST[1], false); checkCorrelationConfigValidation(CLIENT_CORRELATIONID_LIST[2], false); @@ -79,9 +78,9 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, boolean includeInHeader) throws IOException { AzureBlobFileSystem fs = getFileSystem(); TracingContext tracingContext = new TracingContext(clientCorrelationId, - fs.getFileSystemID(), HdfsOperationConstants.TEST_OP, - TracingContextFormat.ALL_ID_FORMAT, null); - String correlationID = tracingContext.toString().split(":")[0]; + fs.getFileSystemId(), FSOperationType.TEST_OP, + TracingHeaderFormat.ALL_ID_FORMAT, null); + String correlationID = tracingContext.constructHeader().split(":")[0]; if (includeInHeader) { Assertions.assertThat(correlationID) .describedAs("Correlation ID should match config when valid") @@ -110,12 +109,11 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, Assertions.assertThat(statusCode).describedAs("Request should not fail") .isEqualTo(HTTP_CREATED); - String requestHeader = op.getResult() - .getRequestHeader(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID) - .replace("[", "").replace("]", ""); + String requestHeader = op.getResult().getClientRequestId().replace("[", "") + .replace("]", ""); Assertions.assertThat(requestHeader) .describedAs("Client Request Header should match TracingContext") - .isEqualTo(tracingContext.toString()); + .isEqualTo(tracingContext.constructHeader()); } @Ignore @@ -123,11 +121,10 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, //call test methods from the respective test classes //can be ignored when running all tests as these get covered public void runCorrelationTestForAllMethods() throws Exception { - //map to group together creating new instance and calling setup() for tests Map testClasses = new HashMap<>(); - testClasses.put(new ITestAzureBlobFileSystemListStatus(), //liststatus - ITestAzureBlobFileSystemListStatus.class.getMethod("testListPath")); +// testClasses.put(new ITestAzureBlobFileSystemListStatus(), //liststatus +// ITestAzureBlobFileSystemListStatus.class.getMethod("testListPath")); testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE), //open, // read, write ITestAbfsReadWriteAndSeek.class.getMethod("testReadAheadRequestID")); @@ -172,7 +169,6 @@ public void runCorrelationTestForAllMethods() throws Exception { } } } - testExternalOps(); } @Test @@ -181,8 +177,8 @@ public void testExternalOps() throws Exception { AzureBlobFileSystem fs = getFileSystem(); fs.registerListener(new TracingHeaderValidator( - fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationID(), - fs.getFileSystemID(), HdfsOperationConstants.HAS_PATH_CAPABILITY, false, + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.HAS_PATH_CAPABILITY, false, 0)); // unset namespaceEnabled to call getAcl -> trigger tracing header validator @@ -193,7 +189,7 @@ public void testExternalOps() throws Exception { Assume.assumeTrue(getConfiguration().isCheckAccessEnabled()); Assume.assumeTrue(getAuthType() == AuthType.OAuth); - fs.setListenerOperation(HdfsOperationConstants.ACCESS); + fs.setListenerOperation(FSOperationType.ACCESS); fs.getAbfsStore().setNamespaceEnabled(Trilean.TRUE); fs.access(new Path("/"), FsAction.READ); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index 24c18bc9f4efb..072831982790f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -28,10 +28,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.azurebfs.utils.TracingContextFormat; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.isNull; @@ -97,8 +97,8 @@ public void verifyShortWriteRequest() throws Exception { AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), - new TracingContext(abfsConf.getClientCorrelationID(), "test-fs-id", - HdfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), + new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", + FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(), null)); final byte[] b = new byte[WRITE_SIZE]; new Random().nextBytes(b); @@ -151,8 +151,8 @@ public void verifyWriteRequest() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext("test-corr-id", - "test-fs-id", HdfsOperationConstants.WRITE, - TracingContextFormat.ALL_ID_FORMAT, null); + "test-fs-id", FSOperationType.WRITE, + TracingHeaderFormat.ALL_ID_FORMAT, null); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(TracingContext.class))).thenReturn(op); @@ -216,8 +216,8 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext( - abfsConf.getClientCorrelationID(), "test-fs-id", - HdfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), null); + abfsConf.getClientCorrelationId(), "test-fs-id", + FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(), null); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(TracingContext.class))).thenReturn(op); @@ -291,8 +291,8 @@ public void verifyWriteRequestOfBufferSize() throws Exception { AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), - new TracingContext(abfsConf.getClientCorrelationID(), "test-fs-id", - HdfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), + new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", + FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(), null)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -339,8 +339,8 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, true), - new TracingContext(abfsConf.getClientCorrelationID(), "test-fs-id", - HdfsOperationConstants.OPEN, abfsConf.getTracingContextFormat(), + new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", + FSOperationType.OPEN, abfsConf.getTracingHeaderFormat(), null)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -379,8 +379,8 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { abfsConf = new AbfsConfiguration(conf, accountName1); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext( - abfsConf.getClientCorrelationID(), "test-fs-id", - HdfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), null); + abfsConf.getClientCorrelationId(), "test-fs-id", + FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(), null); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), @@ -390,8 +390,8 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { isNull(), any(TracingContext.class))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, - populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), new TracingContext(abfsConf.getClientCorrelationID(), "test-fs-id", - HdfsOperationConstants.OPEN, abfsConf.getTracingContextFormat(), + populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", + FSOperationType.OPEN, abfsConf.getTracingHeaderFormat(), null)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); @@ -452,8 +452,8 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), - new TracingContext(abfsConf.getClientCorrelationID(), "test-fs-id", - HdfsOperationConstants.WRITE, abfsConf.getTracingContextFormat(), + new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", + FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(), null)); final byte[] b = new byte[BUFFER_SIZE]; new Random().nextBytes(b); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java index 025da9143610c..ed5cb36454fe0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java @@ -18,29 +18,28 @@ package org.apache.hadoop.fs.azurebfs.utils; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.assertj.core.api.Assertions; -import org.apache.hadoop.fs.azurebfs.constants.HdfsOperationConstants; - /** * Used to validate correlation identifiers provided during testing against * values that get associated with a request through its TracingContext instance */ public class TracingHeaderValidator implements Listener { - private String clientCorrelationID; - private String fileSystemID; - private String primaryRequestID = ""; - private boolean needsPrimaryRequestID; + private String clientCorrelationId; + private String fileSystemId; + private String primaryRequestId = ""; + private boolean needsPrimaryRequestId; private String streamID = ""; - private String operation; + private FSOperationType operation; private int retryNum; - private TracingContextFormat format; + private TracingHeaderFormat format; private static final String GUID_PATTERN = "^[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}$"; @Override public void callTracingHeaderValidator(String tracingContextHeader, - TracingContextFormat format) { + TracingHeaderFormat format) { this.format = format; validateTracingHeader(tracingContextHeader); } @@ -48,25 +47,25 @@ public void callTracingHeaderValidator(String tracingContextHeader, @Override public TracingHeaderValidator getClone() { TracingHeaderValidator tracingHeaderValidator = new TracingHeaderValidator( - clientCorrelationID, fileSystemID, operation, needsPrimaryRequestID, + clientCorrelationId, fileSystemId, operation, needsPrimaryRequestId, retryNum, streamID); - tracingHeaderValidator.primaryRequestID = primaryRequestID; + tracingHeaderValidator.primaryRequestId = primaryRequestId; return tracingHeaderValidator; } - public TracingHeaderValidator(String clientCorrelationID, String fileSystemID, - String operation, boolean needsPrimaryRequestID, int retryNum) { - this.clientCorrelationID = clientCorrelationID; - this.fileSystemID = fileSystemID; + public TracingHeaderValidator(String clientCorrelationId, String fileSystemId, + FSOperationType operation, boolean needsPrimaryRequestId, int retryNum) { + this.clientCorrelationId = clientCorrelationId; + this.fileSystemId = fileSystemId; this.operation = operation; this.retryNum = retryNum; - this.needsPrimaryRequestID = needsPrimaryRequestID; + this.needsPrimaryRequestId = needsPrimaryRequestId; } - public TracingHeaderValidator(String clientCorrelationID, String fileSystemID, - String operation, boolean needsPrimaryRequestID, int retryNum, + public TracingHeaderValidator(String clientCorrelationId, String fileSystemId, + FSOperationType operation, boolean needsPrimaryRequestId, int retryNum, String streamID) { - this(clientCorrelationID, fileSystemID, operation, needsPrimaryRequestID, + this(clientCorrelationId, fileSystemId, operation, needsPrimaryRequestId, retryNum); this.streamID = streamID; } @@ -74,13 +73,13 @@ public TracingHeaderValidator(String clientCorrelationID, String fileSystemID, private void validateTracingHeader(String tracingContextHeader) { String[] idList = tracingContextHeader.split(":"); validateBasicFormat(idList); - if (format != TracingContextFormat.ALL_ID_FORMAT) { + if (format != TracingHeaderFormat.ALL_ID_FORMAT) { return; } - if (!primaryRequestID.isEmpty() && !idList[3].isEmpty()) { + if (!primaryRequestId.isEmpty() && !idList[3].isEmpty()) { Assertions.assertThat(idList[3]) .describedAs("PrimaryReqID should be common for these requests") - .isEqualTo(primaryRequestID); + .isEqualTo(primaryRequestId); } if (!streamID.isEmpty()) { Assertions.assertThat(idList[4]) @@ -90,10 +89,10 @@ private void validateTracingHeader(String tracingContextHeader) { } private void validateBasicFormat(String[] idList) { - if (format == TracingContextFormat.ALL_ID_FORMAT) { + if (format == TracingHeaderFormat.ALL_ID_FORMAT) { Assertions.assertThat(idList) .describedAs("header should have 7 elements").hasSize(7); - } else if (format == TracingContextFormat.TWO_ID_FORMAT) { + } else if (format == TracingHeaderFormat.TWO_ID_FORMAT) { Assertions.assertThat(idList) .describedAs("header should have 2 elements").hasSize(2); } else { @@ -104,10 +103,10 @@ private void validateBasicFormat(String[] idList) { return; } - if (clientCorrelationID.matches("[a-zA-Z0-9-]*")) { + if (clientCorrelationId.matches("[a-zA-Z0-9-]*")) { Assertions.assertThat(idList[0]) .describedAs("Correlation ID should match config") - .isEqualTo(clientCorrelationID); + .isEqualTo(clientCorrelationId); } else { Assertions.assertThat(idList[0]) .describedAs("Invalid config should be replaced with empty string") @@ -116,19 +115,19 @@ private void validateBasicFormat(String[] idList) { Assertions.assertThat(idList[1]).describedAs("Client request ID is a guid") .matches(GUID_PATTERN); - if (format != TracingContextFormat.ALL_ID_FORMAT) { + if (format != TracingHeaderFormat.ALL_ID_FORMAT) { return; } Assertions.assertThat(idList[2]).describedAs("Filesystem ID incorrect") - .isEqualTo(fileSystemID); - if (needsPrimaryRequestID && !operation - .equals(HdfsOperationConstants.READ)) { + .isEqualTo(fileSystemId); + if (needsPrimaryRequestId && !operation + .equals(FSOperationType.READ)) { Assertions.assertThat(idList[3]).describedAs("should have primaryReqId") .isNotEmpty(); } Assertions.assertThat(idList[5]).describedAs("Operation name incorrect") - .isEqualTo(operation); + .isEqualTo(operation.toString()); int retryCount = Integer.parseInt(idList[6]); Assertions.assertThat(retryCount) .describedAs("Retry was required due to issue on server side") @@ -140,12 +139,12 @@ private void validateBasicFormat(String[] idList) { * @param operation Hadoop operation code (String of two characters) */ @Override - public void setOperation(String operation) { + public void setOperation(FSOperationType operation) { this.operation = operation; } @Override public void updatePrimaryRequestID(String primaryRequestID) { - this.primaryRequestID = primaryRequestID; + this.primaryRequestId = primaryRequestID; } } From ecee1809de89ed5b4fc76df79dd64895537aab99 Mon Sep 17 00:00:00 2001 From: sumangala Date: Tue, 29 Jun 2021 01:32:10 +0530 Subject: [PATCH 74/77] revw comments --- .../fs/azurebfs/AzureBlobFileSystem.java | 11 +++- .../fs/azurebfs/services/AbfsInputStream.java | 4 +- .../fs/azurebfs/services/AbfsLease.java | 10 ++-- .../azurebfs/services/AbfsOutputStream.java | 4 +- .../azurebfs/services/AbfsRestOperation.java | 4 +- .../fs/azurebfs/utils/TracingContext.java | 55 ++++++++++--------- .../azurebfs/utils/TracingHeaderFormat.java | 6 +- .../ITestAzureBlobFileSystemListStatus.java | 2 +- .../fs/azurebfs/TestTracingContext.java | 25 +++++---- .../utils/TracingHeaderValidator.java | 4 +- 10 files changed, 70 insertions(+), 55 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 9206efeae1ef9..816ca13fa2d48 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -140,8 +140,10 @@ public void initialize(URI uri, Configuration configuration) configuration, abfsCounters); LOG.trace("AzureBlobFileSystemStore init complete"); - final AbfsConfiguration abfsConfiguration = abfsStore.getAbfsConfiguration(); - clientCorrelationId = abfsConfiguration.getClientCorrelationId(); + final AbfsConfiguration abfsConfiguration = abfsStore + .getAbfsConfiguration(); + clientCorrelationId = TracingContext.validateClientCorrelationID( + abfsConfiguration.getClientCorrelationId()); tracingHeaderFormat = abfsConfiguration.getTracingHeaderFormat(); this.setWorkingDirectory(this.getHomeDirectory()); @@ -1437,6 +1439,11 @@ String getFileSystemId() { return fileSystemId; } + @VisibleForTesting + String getClientCorrelationId() { + return clientCorrelationId; + } + @Override public boolean hasPathCapability(final Path path, final String capability) throws IOException { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index f7715eb0c0d08..5437da36096e6 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -137,7 +137,7 @@ public AbfsInputStream( this.cachedSasToken = new CachedSASToken( abfsInputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds()); this.streamStatistics = abfsInputStreamContext.getStreamStatistics(); - this.inputStreamId = getInputStreamId(); + this.inputStreamId = createInputStreamId(); this.tracingContext = new TracingContext(tracingContext); this.tracingContext.setOperation(FSOperationType.READ); this.tracingContext.setStreamID(inputStreamId); @@ -156,7 +156,7 @@ public String getPath() { return path; } - private String getInputStreamId() { + private String createInputStreamId() { return StringUtils.right(UUID.randomUUID().toString(), STREAM_ID_LEN); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java index 7421bc2e282ea..2e97598ef04f3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java @@ -101,7 +101,8 @@ public AbfsLease(AbfsClient client, String path, int acquireMaxRetries, // Try to get the lease a specified number of times, else throw an error RetryPolicy retryPolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep( acquireMaxRetries, acquireRetryInterval, TimeUnit.SECONDS); - acquireLease(retryPolicy, 0, acquireRetryInterval, 0); + acquireLease(retryPolicy, 0, acquireRetryInterval, 0, + new TracingContext(tracingContext)); while (leaseID == null && exception == null) { try { @@ -120,14 +121,14 @@ public AbfsLease(AbfsClient client, String path, int acquireMaxRetries, } private void acquireLease(RetryPolicy retryPolicy, int numRetries, - int retryInterval, long delay) + int retryInterval, long delay, TracingContext tracingContext) throws LeaseException { LOG.debug("Attempting to acquire lease on {}, retry {}", path, numRetries); if (future != null && !future.isDone()) { throw new LeaseException(ERR_LEASE_FUTURE_EXISTS); } future = client.schedule(() -> client.acquireLease(path, - INFINITE_LEASE_DURATION, new TracingContext(tracingContext)), + INFINITE_LEASE_DURATION, tracingContext), delay, TimeUnit.SECONDS); client.addCallback(future, new FutureCallback() { @Override @@ -143,7 +144,8 @@ public void onFailure(Throwable throwable) { == retryPolicy.shouldRetry(null, numRetries, 0, true).action) { LOG.debug("Failed to acquire lease on {}, retrying: {}", path, throwable); acquireRetryCount++; - acquireLease(retryPolicy, numRetries + 1, retryInterval, retryInterval); + acquireLease(retryPolicy, numRetries + 1, retryInterval, + retryInterval, tracingContext); } else { exception = throwable; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index b9d57a0aab1a8..91b068a78c93f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -170,13 +170,13 @@ public AbfsOutputStream( if (outputStreamStatistics != null) { this.ioStatistics = outputStreamStatistics.getIOStatistics(); } - this.outputStreamId = getOutputStreamId(); + this.outputStreamId = createOutputStreamId(); this.tracingContext = new TracingContext(tracingContext); this.tracingContext.setStreamID(outputStreamId); this.tracingContext.setOperation(FSOperationType.WRITE); } - private String getOutputStreamId() { + private String createOutputStreamId() { return StringUtils.right(UUID.randomUUID().toString(), STREAM_ID_LEN); } 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 3547ba8acb367..c6582b0653f32 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 @@ -226,7 +226,7 @@ private void completeExecute(TracingContext tracingContext) LOG.trace("{} REST operation complete", operationType); } - private void updateClientRequestHeader(AbfsHttpOperation httpOperation, + private void setClientRequestHeader(AbfsHttpOperation httpOperation, TracingContext tracingContext) { tracingContext.generateClientRequestId(); httpOperation.getConnection() @@ -246,7 +246,7 @@ private boolean executeHttpOperation(final int retryCount, // initialize the HTTP request and open the connection httpOperation = new AbfsHttpOperation(url, method, requestHeaders); incrementCounter(AbfsStatistic.CONNECTIONS_MADE, 1); - updateClientRequestHeader(httpOperation, tracingContext); + setClientRequestHeader(httpOperation, tracingContext); switch(client.getAuthType()) { case Custom: diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index 0df792c2852aa..2b0e381e8edf3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -39,7 +39,7 @@ * * Add new operations to HdfsOperationConstants file. * - * PrimaryRequestId can be enabled for individual HDFS API that invoke + * PrimaryRequestId can be enabled for individual Hadoop API that invoke * multiple Store calls. * * Testing: @@ -48,15 +48,16 @@ */ public class TracingContext { - private final String clientCorrelationID; - private final String fileSystemID; - private String clientRequestId = EMPTY_STRING; - private String primaryRequestID; - private String streamID; - private int retryCount; - private FSOperationType hadoopOpName; - private final TracingHeaderFormat format; - private Listener listener = null; + private final String clientCorrelationID; // passed over config by client + private final String fileSystemID; // GUID for fileSystem instance + private String clientRequestId = EMPTY_STRING; // GUID per http request + //Optional, non-empty for methods that trigger two or more Store calls + private String primaryRequestId; + private String streamID; // appears per stream instance (read/write ops) + private int retryCount; // retry number as recorded by AbfsRestOperation + private FSOperationType opType; // two-lettered code representing Hadoop op + private final TracingHeaderFormat format; // header ID display options + private Listener listener = null; // null except when testing private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; @@ -66,33 +67,33 @@ public class TracingContext { * Initialize TracingContext * @param clientCorrelationID Provided over config by client * @param fileSystemID Unique guid for AzureBlobFileSystem instance - * @param hadoopOpName Code indicating the high-level Hadoop operation that + * @param opType Code indicating the high-level Hadoop operation that * triggered the current Store request * @param tracingHeaderFormat Format of IDs to be printed in header and logs * @param listener Holds instance of TracingHeaderValidator during testing, * null otherwise */ public TracingContext(String clientCorrelationID, String fileSystemID, - FSOperationType hadoopOpName, TracingHeaderFormat tracingHeaderFormat, + FSOperationType opType, TracingHeaderFormat tracingHeaderFormat, Listener listener) { this.fileSystemID = fileSystemID; - this.hadoopOpName = hadoopOpName; - this.clientCorrelationID = validateClientCorrelationID(clientCorrelationID); + this.opType = opType; + this.clientCorrelationID = clientCorrelationID; streamID = EMPTY_STRING; retryCount = 0; - primaryRequestID = EMPTY_STRING; + primaryRequestId = EMPTY_STRING; format = tracingHeaderFormat; this.listener = listener; } public TracingContext(String clientCorrelationID, String fileSystemID, - FSOperationType hadoopOpName, boolean needsPrimaryReqId, + FSOperationType opType, boolean needsPrimaryReqId, TracingHeaderFormat tracingHeaderFormat, Listener listener) { - this(clientCorrelationID, fileSystemID, hadoopOpName, tracingHeaderFormat, + this(clientCorrelationID, fileSystemID, opType, tracingHeaderFormat, listener); - primaryRequestID = needsPrimaryReqId ? UUID.randomUUID().toString() : ""; + primaryRequestId = needsPrimaryReqId ? UUID.randomUUID().toString() : ""; if (listener != null) { - listener.updatePrimaryRequestID(primaryRequestID); + listener.updatePrimaryRequestID(primaryRequestId); } } @@ -100,16 +101,16 @@ public TracingContext(TracingContext originalTracingContext) { this.fileSystemID = originalTracingContext.fileSystemID; this.streamID = originalTracingContext.streamID; this.clientCorrelationID = originalTracingContext.clientCorrelationID; - this.hadoopOpName = originalTracingContext.hadoopOpName; + this.opType = originalTracingContext.opType; this.retryCount = 0; - this.primaryRequestID = originalTracingContext.primaryRequestID; + this.primaryRequestId = originalTracingContext.primaryRequestId; this.format = originalTracingContext.format; if (originalTracingContext.listener != null) { this.listener = originalTracingContext.listener.getClone(); } } - public String validateClientCorrelationID(String clientCorrelationID) { + public static String validateClientCorrelationID(String clientCorrelationID) { if ((clientCorrelationID.length() > MAX_CLIENT_CORRELATION_ID_LENGTH) || (!clientCorrelationID.matches(CLIENT_CORRELATION_ID_PATTERN))) { LOG.debug( @@ -124,9 +125,9 @@ public void generateClientRequestId() { } public void setPrimaryRequestID() { - primaryRequestID = UUID.randomUUID().toString(); + primaryRequestId = UUID.randomUUID().toString(); if (listener != null) { - listener.updatePrimaryRequestID(primaryRequestID); + listener.updatePrimaryRequestID(primaryRequestId); } } @@ -135,7 +136,7 @@ public void setStreamID(String stream) { } public void setOperation(FSOperationType operation) { - this.hadoopOpName = operation; + this.opType = operation; } public void setRetryCount(int retryCount) { @@ -149,10 +150,10 @@ public void setListener(Listener listener) { public String constructHeader() { String header; switch (format) { - case ALL_ID_FORMAT: + case ALL_ID_FORMAT: // Optional IDs (e.g. streamId) may be empty header = clientCorrelationID + ":" + clientRequestId + ":" + fileSystemID + ":" - + primaryRequestID + ":" + streamID + ":" + hadoopOpName + ":" + + primaryRequestId + ":" + streamID + ":" + opType + ":" + retryCount; break; case TWO_ID_FORMAT: diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderFormat.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderFormat.java index 93af796f8803d..3f23ae3ed7c14 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderFormat.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderFormat.java @@ -21,8 +21,8 @@ public enum TracingHeaderFormat { SINGLE_ID_FORMAT, // - ALL_ID_FORMAT, // :: - // :::: + TWO_ID_FORMAT, // : - TWO_ID_FORMAT; // : + ALL_ID_FORMAT; // :: + // :::: } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java index 0b3215f5e9fe9..dc9ef9bffb945 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java @@ -67,7 +67,7 @@ public void testListPath() throws Exception { final List> tasks = new ArrayList<>(); ExecutorService es = Executors.newFixedThreadPool(10); - for (int i = 0; i < TEST_FILES_NUMBER/100; i++) { + for (int i = 0; i < TEST_FILES_NUMBER; i++) { final Path fileName = new Path("/test" + i); Callable callable = new Callable() { @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index f4c217d81a1b0..a8704296e113b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -24,6 +24,8 @@ import java.util.HashMap; import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.AssumptionViolatedException; @@ -45,6 +47,7 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_CORRELATIONID; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE; public class TestTracingContext extends AbstractAbfsIntegrationTest { @@ -57,7 +60,7 @@ public TestTracingContext() throws Exception { } @Test - public void testClientCorrelationId() throws IOException { + public void testClientCorrelationId() throws Exception { checkCorrelationConfigValidation(CLIENT_CORRELATIONID_LIST[0], true); checkCorrelationConfigValidation(CLIENT_CORRELATIONID_LIST[1], false); checkCorrelationConfigValidation(CLIENT_CORRELATIONID_LIST[2], false); @@ -75,12 +78,12 @@ private String getRelativePath(final Path path) { } public void checkCorrelationConfigValidation(String clientCorrelationId, - boolean includeInHeader) throws IOException { - AzureBlobFileSystem fs = getFileSystem(); - TracingContext tracingContext = new TracingContext(clientCorrelationId, - fs.getFileSystemId(), FSOperationType.TEST_OP, - TracingHeaderFormat.ALL_ID_FORMAT, null); - String correlationID = tracingContext.constructHeader().split(":")[0]; + boolean includeInHeader) throws Exception { + Configuration conf = getRawConfiguration(); + conf.set(FS_AZURE_CLIENT_CORRELATIONID, clientCorrelationId); + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(conf); + + String correlationID = fs.getClientCorrelationId(); if (includeInHeader) { Assertions.assertThat(correlationID) .describedAs("Correlation ID should match config when valid") @@ -90,7 +93,9 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, .describedAs("Invalid ID should be replaced with empty string") .isEqualTo(EMPTY_STRING); } - + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fs.getFileSystemId(), FSOperationType.TEST_OP, + TracingHeaderFormat.ALL_ID_FORMAT, null); boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(tracingContext); String path = getRelativePath(new Path("/testDir")); String permission = isNamespaceEnabled @@ -123,8 +128,8 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, public void runCorrelationTestForAllMethods() throws Exception { Map testClasses = new HashMap<>(); -// testClasses.put(new ITestAzureBlobFileSystemListStatus(), //liststatus -// ITestAzureBlobFileSystemListStatus.class.getMethod("testListPath")); + testClasses.put(new ITestAzureBlobFileSystemListStatus(), //liststatus + ITestAzureBlobFileSystemListStatus.class.getMethod("testListPath")); testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE), //open, // read, write ITestAbfsReadWriteAndSeek.class.getMethod("testReadAheadRequestID")); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java index ed5cb36454fe0..ca3bdfd900f8a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java @@ -144,7 +144,7 @@ public void setOperation(FSOperationType operation) { } @Override - public void updatePrimaryRequestID(String primaryRequestID) { - this.primaryRequestId = primaryRequestID; + public void updatePrimaryRequestID(String primaryRequestId) { + this.primaryRequestId = primaryRequestId; } } From b69d9e209e0f6646cd1f1ff6a1a84ac4e7f3bfc8 Mon Sep 17 00:00:00 2001 From: sumangala Date: Wed, 30 Jun 2021 14:48:17 +0530 Subject: [PATCH 75/77] correction --- .../azurebfs/services/TestAbfsOutputStream.java | 17 ++++------------- 1 file changed, 4 insertions(+), 13 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index 072831982790f..f01c81b74eeed 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -104,16 +104,6 @@ public void verifyShortWriteRequest() throws Exception { new Random().nextBytes(b); out.write(b); out.hsync(); - ArgumentCaptor acString = ArgumentCaptor.forClass(String.class); - ArgumentCaptor acLong = ArgumentCaptor.forClass(Long.class); - ArgumentCaptor acBufferOffset = ArgumentCaptor.forClass(Integer.class); - ArgumentCaptor acBufferLength = ArgumentCaptor.forClass(Integer.class); - ArgumentCaptor acByteArray = ArgumentCaptor.forClass(byte[].class); - ArgumentCaptor acAppendBlobAppend = ArgumentCaptor.forClass(Boolean.class); - ArgumentCaptor acSASToken = ArgumentCaptor.forClass(String.class); - ArgumentCaptor acTracingContext = ArgumentCaptor - .forClass(TracingContext.class); - final byte[] b1 = new byte[2*WRITE_SIZE]; new Random().nextBytes(b1); @@ -129,12 +119,13 @@ public void verifyShortWriteRequest() throws Exception { WRITE_SIZE, 0, 2 * WRITE_SIZE, APPEND_MODE, false, null); verify(client, times(1)).append( - eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), acTracingContext.capture()); + eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), + any(TracingContext.class)); verify(client, times(1)).append( - eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), acTracingContext.capture()); + eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class)); // confirm there were only 2 invocations in all verify(client, times(2)).append( - eq(PATH), any(byte[].class), any(), any(), acTracingContext.capture()); + eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class)); } /** From 8cf0fe32c89351a55c475cdd91261335591129f8 Mon Sep 17 00:00:00 2001 From: sumangala Date: Thu, 1 Jul 2021 14:30:24 +0530 Subject: [PATCH 76/77] checkstyle --- .../org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java | 4 ++-- .../apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 816ca13fa2d48..a8bf7c16eecfd 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -442,8 +442,8 @@ public FileStatus[] listStatus(final Path f) throws IOException { try { TracingContext tracingContext = new TracingContext(clientCorrelationId, - fileSystemId, FSOperationType.LISTSTATUS, true, tracingHeaderFormat - , listener); + fileSystemId, FSOperationType.LISTSTATUS, true, tracingHeaderFormat, + listener); FileStatus[] result = abfsStore.listStatus(qualifiedPath, tracingContext); return result; } catch (AzureBlobFileSystemException ex) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index 09c5fe549d1d2..7721bde1ad475 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -31,7 +31,6 @@ import javax.net.ssl.SSLSocketFactory; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; -import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.codehaus.jackson.JsonFactory; import org.codehaus.jackson.JsonParser; import org.codehaus.jackson.JsonToken; From 0dc83f4367c6d674a7d8f0bfba0498c26c4551e1 Mon Sep 17 00:00:00 2001 From: sumangala Date: Fri, 2 Jul 2021 17:33:53 +0530 Subject: [PATCH 77/77] set header in tc --- .../azurebfs/services/AbfsHttpOperation.java | 3 +++ .../azurebfs/services/AbfsRestOperation.java | 10 +------ .../fs/azurebfs/utils/TracingContext.java | 26 ++++++++++++++----- .../fs/azurebfs/TestTracingContext.java | 2 +- .../utils/TracingHeaderValidator.java | 4 ++- 5 files changed, 28 insertions(+), 17 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index 7721bde1ad475..5d71c9eee7941 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -400,6 +400,9 @@ public void processResponse(final byte[] buffer, final int offset, final int len } } + public void setRequestProperty(String key, String value) { + this.connection.setRequestProperty(key, value); + } /** * Open the HTTP connection. 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 c6582b0653f32..62576d8b371b0 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 @@ -226,14 +226,6 @@ private void completeExecute(TracingContext tracingContext) LOG.trace("{} REST operation complete", operationType); } - private void setClientRequestHeader(AbfsHttpOperation httpOperation, - TracingContext tracingContext) { - tracingContext.generateClientRequestId(); - httpOperation.getConnection() - .setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, - tracingContext.constructHeader()); - } - /** * Executes a single HTTP operation to complete the REST operation. If it * fails, there may be a retry. The retryCount is incremented with each @@ -246,7 +238,7 @@ private boolean executeHttpOperation(final int retryCount, // initialize the HTTP request and open the connection httpOperation = new AbfsHttpOperation(url, method, requestHeaders); incrementCounter(AbfsStatistic.CONNECTIONS_MADE, 1); - setClientRequestHeader(httpOperation, tracingContext); + tracingContext.constructHeader(httpOperation); switch(client.getAuthType()) { case Custom: diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index 2b0e381e8edf3..5a115451df159 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -24,7 +24,9 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; @@ -58,6 +60,8 @@ public class TracingContext { private FSOperationType opType; // two-lettered code representing Hadoop op private final TracingHeaderFormat format; // header ID display options private Listener listener = null; // null except when testing + //final concatenated ID list set into x-ms-client-request-id header + private String header = EMPTY_STRING; private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; @@ -120,10 +124,6 @@ public static String validateClientCorrelationID(String clientCorrelationID) { return clientCorrelationID; } - public void generateClientRequestId() { - clientRequestId = UUID.randomUUID().toString(); - } - public void setPrimaryRequestID() { primaryRequestId = UUID.randomUUID().toString(); if (listener != null) { @@ -147,8 +147,14 @@ public void setListener(Listener listener) { this.listener = listener; } - public String constructHeader() { - String header; + /** + * Concatenate all identifiers separated by (:) into a string and set into + * X_MS_CLIENT_REQUEST_ID header of the http operation + * @param httpOperation AbfsHttpOperation instance to set header into + * connection + */ + public void constructHeader(AbfsHttpOperation httpOperation) { + clientRequestId = UUID.randomUUID().toString(); switch (format) { case ALL_ID_FORMAT: // Optional IDs (e.g. streamId) may be empty header = @@ -165,6 +171,14 @@ public String constructHeader() { if (listener != null) { //for testing listener.callTracingHeaderValidator(header, format); } + httpOperation.setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, header); + } + + /** + * Return header representing the request associated with the tracingContext + * @return Header string set into X_MS_CLIENT_REQUEST_ID + */ + public String getHeader() { return header; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index a8704296e113b..006004850d0df 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -118,7 +118,7 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, .replace("]", ""); Assertions.assertThat(requestHeader) .describedAs("Client Request Header should match TracingContext") - .isEqualTo(tracingContext.constructHeader()); + .isEqualTo(tracingContext.getHeader()); } @Ignore diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java index ca3bdfd900f8a..e195f1c381a94 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java @@ -21,6 +21,8 @@ import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.assertj.core.api.Assertions; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; + /** * Used to validate correlation identifiers provided during testing against * values that get associated with a request through its TracingContext instance @@ -28,7 +30,7 @@ public class TracingHeaderValidator implements Listener { private String clientCorrelationId; private String fileSystemId; - private String primaryRequestId = ""; + private String primaryRequestId = EMPTY_STRING; private boolean needsPrimaryRequestId; private String streamID = ""; private FSOperationType operation;