diff --git a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml index 2065746b76611..cc72394acc500 100644 --- a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml +++ b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml @@ -44,6 +44,12 @@ + + + FileSystemOperation execute( final String scopeDescription, 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 85d9d96ac2ddb..6cd551d1d7740 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 @@ -55,10 +55,15 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; import org.apache.hadoop.fs.azurebfs.security.ContextProviderEncryptionAdapter; import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter; import org.apache.hadoop.fs.azurebfs.security.NoContextEncryptionAdapter; +import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientHandler; +import org.apache.hadoop.fs.azurebfs.services.AbfsDfsClient; import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; import org.apache.hadoop.fs.azurebfs.utils.NamespaceUtil; import org.apache.hadoop.fs.impl.BackReference; @@ -142,6 +147,7 @@ import org.apache.hadoop.util.concurrent.HadoopExecutors; import org.apache.http.client.utils.URIBuilder; +import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.METADATA_INCOMPLETE_RENAME_FAILURES; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.RENAME_RECOVERY; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_EQUALS; @@ -158,6 +164,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_ABFS_ENDPOINT; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_FOOTER_READ_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BUFFERED_PREAD_DISABLE; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_FNS_ACCOUNT_SERVICE_TYPE; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_IDENTITY_TRANSFORM_CLASS; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT; @@ -170,6 +177,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport { private static final Logger LOG = LoggerFactory.getLogger(AzureBlobFileSystemStore.class); private AbfsClient client; + private AbfsClientHandler clientHandler; private URI uri; private String userName; private String primaryUserGroup; @@ -222,7 +230,8 @@ public AzureBlobFileSystemStore( leaseRefs = Collections.synchronizedMap(new WeakHashMap<>()); try { - this.abfsConfiguration = new AbfsConfiguration(abfsStoreBuilder.configuration, accountName); + this.abfsConfiguration = new AbfsConfiguration( + abfsStoreBuilder.configuration, accountName, identifyAbfsServiceTypeFromUrl()); } catch (IllegalAccessException exception) { throw new FileSystemOperationUnhandledException(exception); } @@ -284,9 +293,26 @@ public AzureBlobFileSystemStore( "abfs-bounded"); } + public void validateConfiguredServiceType(TracingContext tracingContext) + throws AzureBlobFileSystemException { + // Todo: [FnsOverBlob] - Remove this check, Failing FS Init with Blob Endpoint Until FNS over Blob is ready. + if (getConfiguredServiceType() == AbfsServiceType.BLOB) { + throw new InvalidConfigurationValueException(FS_DEFAULT_NAME_KEY); + } + if (getIsNamespaceEnabled(tracingContext) && getConfiguredServiceType() == AbfsServiceType.BLOB) { + // This could be because of either wrongly configured url or wrongly configured fns service type. + if (identifyAbfsServiceTypeFromUrl() == AbfsServiceType.BLOB) { + throw new InvalidConfigurationValueException(FS_DEFAULT_NAME_KEY, "Wrong Domain Suffix for HNS Account"); + } + throw new InvalidConfigurationValueException(FS_AZURE_FNS_ACCOUNT_SERVICE_TYPE, "Wrong Service Type for HNS Accounts"); + } + } + /** * Checks if the given key in Azure Storage should be stored as a page * blob instead of block blob. + * @param key The key to check. + * @return True if the key should be stored as a page blob, false otherwise. */ public boolean isAppendBlobKey(String key) { return isKeyForDirectorySet(key, appendBlobDirSet); @@ -1760,19 +1786,46 @@ private void initializeClient(URI uri, String fileSystemName, } LOG.trace("Initializing AbfsClient for {}", baseUrl); + AbfsDfsClient dfsClient = null; + AbfsBlobClient blobClient = null; if (tokenProvider != null) { - this.client = new AbfsClient(baseUrl, creds, abfsConfiguration, + dfsClient = new AbfsDfsClient(baseUrl, creds, abfsConfiguration, tokenProvider, encryptionContextProvider, populateAbfsClientContext()); + blobClient = abfsConfiguration.isBlobClientInitRequired() + ? new AbfsBlobClient(baseUrl, creds, abfsConfiguration, tokenProvider, + encryptionContextProvider, populateAbfsClientContext()) + : null; } else { - this.client = new AbfsClient(baseUrl, creds, abfsConfiguration, + dfsClient = new AbfsDfsClient(baseUrl, creds, abfsConfiguration, sasTokenProvider, encryptionContextProvider, populateAbfsClientContext()); + blobClient = abfsConfiguration.isBlobClientInitRequired() + ? new AbfsBlobClient(baseUrl, creds, abfsConfiguration, sasTokenProvider, + encryptionContextProvider, populateAbfsClientContext()) + : null; } + this.clientHandler = new AbfsClientHandler(getConfiguredServiceType(), + dfsClient, blobClient); + this.client = clientHandler.getClient(); + LOG.trace("AbfsClient init complete"); } + private AbfsServiceType identifyAbfsServiceTypeFromUrl() { + if (uri.toString().contains(FileSystemUriSchemes.ABFS_BLOB_DOMAIN_NAME)) { + return AbfsServiceType.BLOB; + } + // In case of DFS Domain name or any other custom endpoint, the service + // type is to be identified as default DFS. + return AbfsServiceType.DFS; + } + + private AbfsServiceType getConfiguredServiceType() { + return abfsConfiguration.getFsConfiguredServiceType(); + } + /** * Populate a new AbfsClientContext instance with the desired properties. * diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java index 4f5ee5f9683fc..006e9dcc35522 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java @@ -48,6 +48,15 @@ public final class AbfsHttpConstants { public static final String APPEND_BLOB_TYPE = "appendblob"; public static final String TOKEN_VERSION = "2"; + public static final String CONTAINER = "container"; + public static final String METADATA = "metadata"; + public static final String LIST = "list"; + public static final String BLOCK = "block"; + public static final String BLOCKLIST = "blocklist"; + public static final String LEASE = "lease"; + public static final String BLOCK_BLOB_TYPE = "BlockBlob"; + public static final String BLOCK_TYPE_COMMITTED = "committed"; + public static final String JAVA_VENDOR = "java.vendor"; public static final String JAVA_VERSION = "java.version"; public static final String OS_NAME = "os.name"; @@ -88,6 +97,7 @@ public final class AbfsHttpConstants { public static final String HTTP_HEADER_PREFIX = "x-ms-"; public static final String HASH = "#"; public static final String TRUE = "true"; + public static final String ZERO = "0"; public static final String PLUS_ENCODE = "%20"; public static final String FORWARD_SLASH_ENCODE = "%2F"; @@ -97,6 +107,7 @@ public final class AbfsHttpConstants { public static final String GMT_TIMEZONE = "GMT"; public static final String APPLICATION_JSON = "application/json"; public static final String APPLICATION_OCTET_STREAM = "application/octet-stream"; + public static final String APPLICATION_XML = "application/xml"; public static final String ROOT_PATH = "/"; public static final String ACCESS_MASK = "mask:"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsServiceType.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsServiceType.java new file mode 100644 index 0000000000000..08fea89c62ab1 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsServiceType.java @@ -0,0 +1,31 @@ +/** + * 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; + +/** + * Azure Storage Offers two sets of Rest APIs for interacting with the storage account. + *
    + *
  1. Blob Rest API:
  2. + *
  3. Data Lake Rest API:
  4. + *
+ */ +public enum AbfsServiceType { + DFS, + BLOB; +} 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 2ccc6ade876f1..ea556c24ca6f1 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 @@ -35,6 +35,28 @@ public final class ConfigurationKeys { * path to determine HNS status. */ public static final String FS_AZURE_ACCOUNT_IS_HNS_ENABLED = "fs.azure.account.hns.enabled"; + + /** + * Config to specify which {@link AbfsServiceType} to use with HNS-Disabled Account type. + * Recommendation is to always use Blob Endpoint with HNS-Disabled Account type. + * This will override service endpoint configured in "fs.defaultFS". + * Value {@value} case-insensitive "DFS" or "BLOB". + */ + public static final String FS_AZURE_FNS_ACCOUNT_SERVICE_TYPE = "fs.azure.fns.account.service.type"; + + /** + * Config to specify which {@link AbfsServiceType} to use only for Ingress Operations. + * Other operations will continue to move to the configured service endpoint. + * Value {@value} case-insensitive "DFS" or "BLOB". + */ + public static final String FS_AZURE_INGRESS_SERVICE_TYPE = "fs.azure.ingress.service.type"; + + /** + * Config to be set only for cases where traffic over dfs endpoint is experiencing compatibility issues. + * Value {@value} case-insensitive "True" or "False". + */ + public static final String FS_AZURE_ENABLE_DFSTOBLOB_FALLBACK = "fs.azure.enable.dfstoblob.fallback"; + /** * Enable or disable expect hundred continue header. * Value: {@value}. 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 index 6b6e98c9c7082..8c9c8af75b53d 100644 --- 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 @@ -45,7 +45,8 @@ public enum FSOperationType { SET_OWNER("SO"), SET_ACL("SA"), TEST_OP("TS"), - WRITE("WR"); + WRITE("WR"), + INIT("IN"); private final String opCode; 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 ade0dc39cfe18..0b071351ef5aa 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 @@ -32,6 +32,7 @@ public final class FileSystemConfigurations { public static final String DEFAULT_FS_AZURE_ACCOUNT_IS_HNS_ENABLED = ""; + public static final boolean DEFAULT_FS_AZURE_ENABLE_DFSTOBLOB_FALLBACK = false; public static final boolean DEFAULT_FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED = true; public static final String USER_HOME_DIRECTORY_PREFIX = "/user"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemUriSchemes.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemUriSchemes.java index c7a0cdad605ab..0b5cba72f126d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemUriSchemes.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemUriSchemes.java @@ -38,5 +38,8 @@ public final class FileSystemUriSchemes { public static final String WASB_SECURE_SCHEME = "wasbs"; public static final String WASB_DNS_PREFIX = "blob"; + public static final String ABFS_DFS_DOMAIN_NAME = "dfs.core.windows.net"; + public static final String ABFS_BLOB_DOMAIN_NAME = "blob.core.windows.net"; + private FileSystemUriSchemes() {} -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java index b3c2b21d3c277..0b5406d90ae55 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java @@ -75,5 +75,11 @@ public final class HttpHeaderConfigurations { public static final String EXPECT = "Expect"; public static final String X_MS_RANGE_GET_CONTENT_MD5 = "x-ms-range-get-content-md5"; + public static final String X_MS_SOURCE_LEASE_ID = "x-ms-source-lease-id"; + public static final String X_MS_BLOB_TYPE = "x-ms-blob-type"; + public static final String X_MS_META_HDI_ISFOLDER = "x-ms-meta-hdi_isfolder"; + public static final String X_MS_METADATA_PREFIX = "x-ms-meta-"; + public static final String X_MS_COPY_SOURCE = "x-ms-copy-source"; + private HttpHeaderConfigurations() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java index f7e37dcb6d50d..94b7525bde58c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java @@ -48,5 +48,15 @@ public final class HttpQueryParams { public static final String QUERY_PARAM_SUOID = "suoid"; public static final String QUERY_PARAM_SIGNATURE = "sig"; + public static final String QUERY_PARAM_RESTYPE = "restype"; + public static final String QUERY_PARAM_COMP = "comp"; + public static final String QUERY_PARAM_INCLUDE = "include"; + public static final String QUERY_PARAM_PREFIX = "prefix"; + public static final String QUERY_PARAM_MARKER = "marker"; + public static final String QUERY_PARAM_DELIMITER = "delimiter"; + public static final String QUERY_PARAM_MAX_RESULTS = "maxresults"; + public static final String QUERY_PARAM_BLOCKID = "blockid"; + public static final String QUERY_PARAM_BLOCKLISTTYPE = "blocklisttype"; + private HttpQueryParams() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidConfigurationValueException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidConfigurationValueException.java index 7591bac59e292..d52a66a5ea998 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidConfigurationValueException.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidConfigurationValueException.java @@ -34,4 +34,8 @@ public InvalidConfigurationValueException(String configKey, Exception innerExcep public InvalidConfigurationValueException(String configKey) { super("Invalid configuration value detected for " + configKey); } + + public InvalidConfigurationValueException(String configKey, String messaage) { + super(String.format("Invalid configuration value detected for \"%s\". %s ", configKey, messaage)); + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java index 9da6427d65c2c..2155fb10b39cd 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java @@ -37,6 +37,11 @@ public enum Mode { private boolean isExpectHeaderEnabled; private boolean isRetryDueToExpect; + // Following parameters are used by AbfsBlobClient only. + private String blockId; + private String eTag; + + // Constructor to be used for interacting with AbfsDfsClient public AppendRequestParameters(final long position, final int offset, final int length, @@ -52,6 +57,31 @@ public AppendRequestParameters(final long position, this.leaseId = leaseId; this.isExpectHeaderEnabled = isExpectHeaderEnabled; this.isRetryDueToExpect = false; + this.blockId = null; + this.eTag = null; + } + + // Constructor to be used for interacting with AbfsBlobClient + @SuppressWarnings("checkstyle:ParameterNumber") + public AppendRequestParameters(final long position, + final int offset, + final int length, + final Mode mode, + final boolean isAppendBlob, + final String leaseId, + final boolean isExpectHeaderEnabled, + final String blockId, + final String eTag) { + this.position = position; + this.offset = offset; + this.length = length; + this.mode = mode; + this.isAppendBlob = isAppendBlob; + this.leaseId = leaseId; + this.isExpectHeaderEnabled = isExpectHeaderEnabled; + this.isRetryDueToExpect = false; + this.blockId = blockId; + this.eTag = eTag; } public long getPosition() { @@ -86,6 +116,14 @@ public boolean isRetryDueToExpect() { return isRetryDueToExpect; } + public String getBlockId() { + return blockId; + } + + public String getETag() { + return eTag; + } + public void setRetryDueToExpect(boolean retryDueToExpect) { isRetryDueToExpect = retryDueToExpect; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java index 1a1a27c53b641..8b392b5d3020b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java @@ -122,6 +122,7 @@ public static AzureADToken getTokenUsingClientCreds(String authEndpoint, * @param tenantGuid (optional) The guid of the AAD tenant. Can be {@code null}. * @param clientId (optional) The clientId guid of the MSI service * principal to use. Can be {@code null}. + * @param authority MSI authority to be used. Can be {@code null}. * @param bypassCache {@code boolean} specifying whether a cached token is acceptable or a fresh token * request should me made to AAD * @return {@link AzureADToken} obtained using the creds diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/IdentityTransformerInterface.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/IdentityTransformerInterface.java index 00f93eae30bd4..e1b8aa80f606e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/IdentityTransformerInterface.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/IdentityTransformerInterface.java @@ -34,6 +34,7 @@ public interface IdentityTransformerInterface { * @param isUserName indicate whether the input originalIdentity is an owner name or owning group name. * @param localIdentity the local user or group, should be parsed from UserGroupInformation. * @return owner or group after transformation. + * @throws IOException if transformation fails. */ String transformIdentityForGetRequest(String originalIdentity, boolean isUserName, String localIdentity) throws IOException; @@ -56,6 +57,7 @@ String transformIdentityForGetRequest(String originalIdentity, boolean isUserNam * @param aclEntries list of AclEntry. * @param localUser local user name. * @param localGroup local primary group. + * @throws IOException if transformation fails. */ void transformAclEntriesForGetRequest(final List aclEntries, String localUser, String localGroup) throws IOException; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/RefreshTokenBasedTokenProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/RefreshTokenBasedTokenProvider.java index a24a4447d3098..6062a5cea23a8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/RefreshTokenBasedTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/RefreshTokenBasedTokenProvider.java @@ -42,6 +42,7 @@ public class RefreshTokenBasedTokenProvider extends AccessTokenProvider { * * @param clientId the client ID (GUID) of the client web app obtained from Azure Active Directory configuration * @param refreshToken the refresh token + * @param authEndpoint the authentication endpoint */ public RefreshTokenBasedTokenProvider(final String authEndpoint, String clientId, String refreshToken) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java new file mode 100644 index 0000000000000..c2ed21204af69 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java @@ -0,0 +1,873 @@ +/** + * 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.services; + +import java.io.Closeable; +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; + +import org.apache.commons.lang3.NotImplementedException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams; +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; +import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; +import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider; +import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; +import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +import static java.net.HttpURLConnection.HTTP_CONFLICT; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ACQUIRE_LEASE_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPLICATION_JSON; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPLICATION_OCTET_STREAM; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPLICATION_XML; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCKLIST; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK_BLOB_TYPE; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK_TYPE_COMMITTED; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BREAK_LEASE_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COMMA; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CONTAINER; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DEFAULT_LEASE_BREAK_PERIOD; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_DELETE; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_GET; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_HEAD; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.LEASE; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.METADATA; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.RELEASE_LEASE_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.RENEW_LEASE_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.SINGLE_WHITE_SPACE; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.STAR; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.TRUE; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ZERO; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.ACCEPT; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.CONTENT_LENGTH; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.CONTENT_TYPE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.IF_MATCH; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.IF_NONE_MATCH; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.RANGE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.USER_AGENT; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_BLOB_TYPE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_COPY_SOURCE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_BREAK_PERIOD; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_DURATION; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_ID; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_METADATA_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_META_HDI_ISFOLDER; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_PROPOSED_LEASE_ID; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SOURCE_LEASE_ID; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_BLOCKID; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_BLOCKLISTTYPE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_CLOSE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_COMP; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_RESTYPE; + +/** + * AbfsClient interacting with Blob endpoint. + */ +public class AbfsBlobClient extends AbfsClient implements Closeable { + + public AbfsBlobClient(final URL baseUrl, + final SharedKeyCredentials sharedKeyCredentials, + final AbfsConfiguration abfsConfiguration, + final AccessTokenProvider tokenProvider, + final EncryptionContextProvider encryptionContextProvider, + final AbfsClientContext abfsClientContext) throws IOException { + super(baseUrl, sharedKeyCredentials, abfsConfiguration, tokenProvider, + encryptionContextProvider, abfsClientContext); + } + + public AbfsBlobClient(final URL baseUrl, + final SharedKeyCredentials sharedKeyCredentials, + final AbfsConfiguration abfsConfiguration, + final SASTokenProvider sasTokenProvider, + final EncryptionContextProvider encryptionContextProvider, + final AbfsClientContext abfsClientContext) throws IOException { + super(baseUrl, sharedKeyCredentials, abfsConfiguration, sasTokenProvider, + encryptionContextProvider, abfsClientContext); + } + + @Override + public void close() throws IOException { + super.close(); + } + + public List createDefaultHeaders() { + return this.createDefaultHeaders(getxMsVersion()); + } + + /** + * Create request headers for Rest Operation using the specified API version. + * Blob Endpoint API responses are in JSON/XML format. + * @param xMsVersion API version to be used. + * @return default request headers + */ + @Override + public List createDefaultHeaders(AbfsHttpConstants.ApiVersion xMsVersion) { + List requestHeaders = super.createCommonHeaders(xMsVersion); + requestHeaders.add(new AbfsHttpHeader(ACCEPT, APPLICATION_JSON + + COMMA + SINGLE_WHITE_SPACE + APPLICATION_OCTET_STREAM + + COMMA + SINGLE_WHITE_SPACE + APPLICATION_XML)); + return requestHeaders; + } + + /** + * Get Rest Operation for API . + * Creates a storage container as filesystem root. + * @param tracingContext for tracing the service call. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation createFilesystem(TracingContext tracingContext) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESTYPE, CONTAINER); + + final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.CreateContainer, + HTTP_METHOD_PUT, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * Sets user-defined properties of the filesystem. + * @param properties comma separated list of metadata key-value pairs. + * @param tracingContext for tracing the service call. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation setFilesystemProperties(final String properties, + TracingContext tracingContext) throws AzureBlobFileSystemException { + List requestHeaders = createDefaultHeaders(); + List metadataRequestHeaders = getMetadataHeadersList(properties); + requestHeaders.addAll(metadataRequestHeaders); + + AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESTYPE, CONTAINER); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, METADATA); + + final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.SetContainerMetadata, + HTTP_METHOD_PUT, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * Gets all the properties of the filesystem. + * @param tracingContext for tracing the service call. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + * */ + @Override + public AbfsRestOperation getFilesystemProperties(TracingContext tracingContext) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESTYPE, CONTAINER); + + final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.GetContainerProperties, + HTTP_METHOD_HEAD, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * Deletes the Container acting as current filesystem. + * @param tracingContext for tracing the service call. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESTYPE, CONTAINER); + + final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.DeleteContainer, + HTTP_METHOD_DELETE, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * Creates a file or directory(marker file) at specified path. + * @param path of the directory to be created. + * @param tracingContext for tracing the service call. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation createPath(final String path, + final boolean isFile, + final boolean overwrite, + final AzureBlobFileSystemStore.Permissions permissions, + final boolean isAppendBlob, + final String eTag, + final ContextEncryptionAdapter contextEncryptionAdapter, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, ZERO)); + requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_TYPE, BLOCK_BLOB_TYPE)); + if (!overwrite) { + requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, AbfsHttpConstants.STAR)); + } + if (eTag != null && !eTag.isEmpty()) { + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_MATCH, eTag)); + } + if (!isFile) { + requestHeaders.add(new AbfsHttpHeader(X_MS_META_HDI_ISFOLDER, TRUE)); + } + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + appendSASTokenToQuery(path, SASTokenProvider.CREATE_FILE_OPERATION, abfsUriQueryBuilder); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.PutBlob, + HTTP_METHOD_PUT, url, requestHeaders); + try { + op.execute(tracingContext); + } catch (AzureBlobFileSystemException ex) { + // If we have no HTTP response, throw the original exception. + if (!op.hasResult()) { + throw ex; + } + if (!isFile && op.getResult().getStatusCode() == HTTP_CONFLICT) { + // This ensures that we don't throw ex only for existing directory but if a blob exists we throw exception. + final AbfsHttpOperation opResult = this.getPathStatus( + path, true, tracingContext, null).getResult(); + if (checkIsDir(opResult)) { + return op; + } + } + throw ex; + } + return op; + } + + /** + * Get Rest Operation for API . + * @param relativePath to return only blobs with names that begin with the specified prefix. + * @param recursive to return all blobs in the path, including those in subdirectories. + * @param listMaxResults maximum number of blobs to return. + * @param continuation marker to specify the continuation token. + * @param tracingContext for tracing the service call. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation or response parsing fails. + */ + @Override + public AbfsRestOperation listPath(final String relativePath, final boolean recursive, + final int listMaxResults, final String continuation, TracingContext tracingContext) + throws AzureBlobFileSystemException { + // Todo: [FnsOverBlob] To be implemented as part of response handling of blob endpoint APIs. + throw new NotImplementedException("Blob Endpoint Support is not yet implemented"); + } + + /** + * Get Rest Operation for API . + * @param path on which lease has to be acquired. + * @param duration for which lease has to be acquired. + * @param tracingContext for tracing the service call. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation acquireLease(final String path, final int duration, + TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, ACQUIRE_LEASE_ACTION)); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_DURATION, Integer.toString(duration))); + requestHeaders.add(new AbfsHttpHeader(X_MS_PROPOSED_LEASE_ID, UUID.randomUUID().toString())); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.LeaseBlob, + HTTP_METHOD_PUT, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * @param path on which lease has to be renewed. + * @param leaseId of the lease to be renewed. + * @param tracingContext for tracing the service call. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation renewLease(final String path, final String leaseId, + TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, RENEW_LEASE_ACTION)); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.LeaseBlob, + HTTP_METHOD_PUT, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * @param path on which lease has to be released. + * @param leaseId of the lease to be released. + * @param tracingContext for tracing the service call. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation releaseLease(final String path, final String leaseId, + TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, RELEASE_LEASE_ACTION)); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.LeaseBlob, + HTTP_METHOD_PUT, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * @param path on which lease has to be broken. + * @param tracingContext for tracing the service call. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation breakLease(final String path, + TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, BREAK_LEASE_ACTION)); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_BREAK_PERIOD, DEFAULT_LEASE_BREAK_PERIOD)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.LeaseBlob, + HTTP_METHOD_PUT, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + @Override + public AbfsClientRenameResult renamePath(final String source, + final String destination, + final String continuation, + final TracingContext tracingContext, + final String sourceEtag, + final boolean isMetadataIncompleteState, + final boolean isNamespaceEnabled) throws IOException { + // Todo: [FnsOverBlob] To be implemented as part of rename-delete over blob endpoint work. + throw new NotImplementedException("Rename operation on Blob endpoint will be implemented in future."); + } + + /** + * Get Rest Operation for API . + * Read the contents of the file at specified path + * @param path of the file to be read. + * @param position in the file from where data has to be read. + * @param buffer to store the data read. + * @param bufferOffset offset in the buffer to start storing the data. + * @param bufferLength length of data to be read. + * @param eTag to specify conditional headers. + * @param cachedSasToken to be used for the authenticating operation. + * @param contextEncryptionAdapter to provide encryption context. + * @param tracingContext for tracing the service call. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @SuppressWarnings("checkstyle:ParameterNumber") + @Override + public AbfsRestOperation read(final String path, + final long position, + final byte[] buffer, + final int bufferOffset, + final int bufferLength, + final String eTag, + final String cachedSasToken, + final ContextEncryptionAdapter contextEncryptionAdapter, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + AbfsHttpHeader rangeHeader = new AbfsHttpHeader(RANGE, String.format( + "bytes=%d-%d", position, position + bufferLength - 1)); + requestHeaders.add(rangeHeader); + requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.READ_OPERATION, + abfsUriQueryBuilder, cachedSasToken); + + URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.GetBlob, + HTTP_METHOD_GET, url, requestHeaders, + buffer, bufferOffset, bufferLength, + sasTokenForReuse); + op.execute(tracingContext); + + return op; + } + + /** + * Get Rest Operation for API . + * Uploads data to be appended to a file. + * @param path to which data has to be appended. + * @param buffer containing data to be appended. + * @param reqParams containing parameters for append operation like offset, length etc. + * @param cachedSasToken to be used for the authenticating operation. + * @param contextEncryptionAdapter to provide encryption context. + * @param tracingContext for tracing the service call. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation append(final String path, + final byte[] buffer, + final AppendRequestParameters reqParams, + final String cachedSasToken, + final ContextEncryptionAdapter contextEncryptionAdapter, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, String.valueOf(buffer.length))); + requestHeaders.add(new AbfsHttpHeader(IF_MATCH, reqParams.getETag())); + if (reqParams.getLeaseId() != null) { + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, reqParams.getLeaseId())); + } + if (reqParams.isExpectHeaderEnabled()) { + requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); + } + + if (reqParams.isRetryDueToExpect()) { + String userAgentRetry = getUserAgent(); + userAgentRetry = userAgentRetry.replace(HUNDRED_CONTINUE_USER_AGENT, EMPTY_STRING); + requestHeaders.removeIf(header -> header.getName().equalsIgnoreCase(USER_AGENT)); + requestHeaders.add(new AbfsHttpHeader(USER_AGENT, userAgentRetry)); + } + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCK); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_BLOCKID, reqParams.getBlockId()); + + String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.WRITE_OPERATION, + abfsUriQueryBuilder, cachedSasToken); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.PutBlock, + HTTP_METHOD_PUT, url, requestHeaders, + buffer, reqParams.getoffset(), reqParams.getLength(), + sasTokenForReuse); + + try { + op.execute(tracingContext); + } catch (AzureBlobFileSystemException e) { + /* + If the http response code indicates a user error we retry + the same append request with expect header being disabled. + When "100-continue" header is enabled but a non Http 100 response comes, + the response message might not get set correctly by the server. + So, this handling is to avoid breaking of backward compatibility + if someone has taken dependency on the exception message, + which is created using the error string present in the response header. + */ + int responseStatusCode = ((AbfsRestOperationException) e).getStatusCode(); + if (checkUserError(responseStatusCode) && reqParams.isExpectHeaderEnabled()) { + LOG.debug("User error, retrying without 100 continue enabled for the given path {}", path); + reqParams.setExpectHeaderEnabled(false); + reqParams.setRetryDueToExpect(true); + return this.append(path, buffer, reqParams, cachedSasToken, + contextEncryptionAdapter, tracingContext); + } + else { + throw e; + } + } + return op; + } + + /** + * Redirect to flush specific to blob endpoint + */ + @Override + public AbfsRestOperation flush(final String path, + final long position, + final boolean retainUncommittedData, + final boolean isClose, + final String cachedSasToken, + final String leaseId, + final ContextEncryptionAdapter contextEncryptionAdapter, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + return this.flush(new byte[0], path, isClose, cachedSasToken, leaseId, "", + tracingContext); + } + + /** + * Get Rest Operation for API . + * The flush operation to commit the blocks. + * @param buffer This has the xml in byte format with the blockIds to be flushed. + * @param path The path to flush the data to. + * @param isClose True when the stream is closed. + * @param cachedSasToken The cachedSasToken if available. + * @param leaseId The leaseId of the blob if available. + * @param eTag The etag of the blob. + * @param tracingContext for tracing the service call. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation flush(byte[] buffer, + final String path, + boolean isClose, + final String cachedSasToken, + final String leaseId, + final String eTag, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, String.valueOf(buffer.length))); + requestHeaders.add(new AbfsHttpHeader(CONTENT_TYPE, APPLICATION_XML)); + requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag)); + if (leaseId != null) { + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId)); + } + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, String.valueOf(isClose)); + String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.WRITE_OPERATION, + abfsUriQueryBuilder, cachedSasToken); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.PutBlockList, + HTTP_METHOD_PUT, url, requestHeaders, + buffer, 0, buffer.length, + sasTokenForReuse); + + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * Set the properties of a file or directory. + * @param path on which properties have to be set. + * @param properties comma separated list of metadata key-value pairs. + * @param tracingContext for tracing the service call. + * @param contextEncryptionAdapter to provide encryption context. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation setPathProperties(final String path, + final String properties, + final TracingContext tracingContext, + final ContextEncryptionAdapter contextEncryptionAdapter) + throws AzureBlobFileSystemException { + List requestHeaders = createDefaultHeaders(); + List metadataRequestHeaders = getMetadataHeadersList(properties); + requestHeaders.addAll(metadataRequestHeaders); + + AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, METADATA); + appendSASTokenToQuery(path, SASTokenProvider.SET_PROPERTIES_OPERATION, abfsUriQueryBuilder); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.SetPathProperties, + HTTP_METHOD_PUT, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * Get the properties of a file or directory. + * @param path of which properties have to be fetched. + * @param includeProperties to include user defined properties. + * @param tracingContext for tracing the service call. + * @param contextEncryptionAdapter to provide encryption context. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation getPathStatus(final String path, + final boolean includeProperties, + final TracingContext tracingContext, + final ContextEncryptionAdapter contextEncryptionAdapter) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN, + String.valueOf(getAbfsConfiguration().isUpnUsed())); + appendSASTokenToQuery(path, SASTokenProvider.GET_PROPERTIES_OPERATION, + abfsUriQueryBuilder); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.GetPathStatus, + HTTP_METHOD_HEAD, url, requestHeaders); + try { + op.execute(tracingContext); + } catch (AzureBlobFileSystemException ex) { + // If we have no HTTP response, throw the original exception. + if (!op.hasResult()) { + throw ex; + } + if (op.getResult().getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { + // This path could be present as an implicit directory in FNS. + AbfsRestOperation listOp = listPath(path, false, 1, null, tracingContext); + // Todo: [FnsOverBlob] To be implemented as part of response handling of blob endpoint APIs. + return listOp; + } + throw ex; + } + return op; + } + + @Override + public AbfsRestOperation deletePath(final String path, + final boolean recursive, + final String continuation, + TracingContext tracingContext, + final boolean isNamespaceEnabled) throws AzureBlobFileSystemException { + // Todo: [FnsOverBlob] To be implemented as part of rename-delete over blob endpoint work. + throw new NotImplementedException("Delete operation on Blob endpoint will be implemented in future."); + } + + @Override + public AbfsRestOperation setOwner(final String path, + final String owner, + final String group, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + throw new UnsupportedOperationException( + "SetOwner operation is only supported on HNS enabled Accounts."); + } + + @Override + public AbfsRestOperation setPermission(final String path, + final String permission, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + throw new UnsupportedOperationException( + "SetPermission operation is only supported on HNS enabled Accounts."); + } + + @Override + public AbfsRestOperation setAcl(final String path, + final String aclSpecString, + final String eTag, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + throw new UnsupportedOperationException( + "SetAcl operation is only supported on HNS enabled Accounts."); + } + + @Override + public AbfsRestOperation getAclStatus(final String path, final boolean useUPN, + TracingContext tracingContext) throws AzureBlobFileSystemException { + throw new UnsupportedOperationException( + "GetAclStatus operation is only supported on HNS enabled Accounts."); + } + + @Override + public AbfsRestOperation checkAccess(String path, String rwx, TracingContext tracingContext) + throws AzureBlobFileSystemException { + throw new UnsupportedOperationException( + "CheckAccess operation is only supported on HNS enabled Accounts."); + } + + @Override + public boolean checkIsDir(AbfsHttpOperation result) { + boolean isDirectory = (result.getResponseHeader(X_MS_META_HDI_ISFOLDER) != null); + if (isDirectory) { + return true; + } + return false; + } + + /** + * Returns true if the status code lies in the range of user error. + * In the case of HTTP_CONFLICT for PutBlockList we fall back to DFS and hence + * this retry handling is not needed. + * @param responseStatusCode http response status code. + * @return True or False. + */ + @Override + public boolean checkUserError(int responseStatusCode) { + return (responseStatusCode >= HttpURLConnection.HTTP_BAD_REQUEST + && responseStatusCode < HttpURLConnection.HTTP_INTERNAL_ERROR + && responseStatusCode != HttpURLConnection.HTTP_CONFLICT); + } + + /** + * Get Rest Operation for API . + * Get the list of committed block ids of the blob. + * @param path The path to get the list of blockId's. + * @param tracingContext for tracing the service call. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + public AbfsRestOperation getBlockList(final String path, TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + String operation = SASTokenProvider.READ_OPERATION; + appendSASTokenToQuery(path, operation, abfsUriQueryBuilder); + + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_BLOCKLISTTYPE, BLOCK_TYPE_COMMITTED); + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.GetBlockList, HTTP_METHOD_GET, url, + requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * This is an asynchronous API, it returns copyId and expects client + * to poll the server on the destination and check the copy-progress. + * @param sourceBlobPath path of source to be copied. + * @param destinationBlobPath path of the destination. + * @param srcLeaseId if source path has an active lease. + * @param tracingContext for tracing the service call. + * @return executed rest operation containing response from server. + * This method owns the logic of triggering copyBlob API. The caller of this + * method have to own the logic of polling the destination with the copyId + * returned in the response from this method. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + public AbfsRestOperation copyBlob(Path sourceBlobPath, + Path destinationBlobPath, + final String srcLeaseId, TracingContext tracingContext) throws AzureBlobFileSystemException { + AbfsUriQueryBuilder abfsUriQueryBuilderDst = createDefaultUriQueryBuilder(); + AbfsUriQueryBuilder abfsUriQueryBuilderSrc = new AbfsUriQueryBuilder(); + String dstBlobRelativePath = destinationBlobPath.toUri().getPath(); + String srcBlobRelativePath = sourceBlobPath.toUri().getPath(); + appendSASTokenToQuery(dstBlobRelativePath, + SASTokenProvider.WRITE_OPERATION, abfsUriQueryBuilderDst); + appendSASTokenToQuery(srcBlobRelativePath, + SASTokenProvider.READ_OPERATION, abfsUriQueryBuilderSrc); + final URL url = createRequestUrl(dstBlobRelativePath, + abfsUriQueryBuilderDst.toString()); + final String sourcePathUrl = createRequestUrl(srcBlobRelativePath, + abfsUriQueryBuilderSrc.toString()).toString(); + List requestHeaders = createDefaultHeaders(); + if (srcLeaseId != null) { + requestHeaders.add(new AbfsHttpHeader(X_MS_SOURCE_LEASE_ID, srcLeaseId)); + } + requestHeaders.add(new AbfsHttpHeader(X_MS_COPY_SOURCE, sourcePathUrl)); + requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, STAR)); + + final AbfsRestOperation op = getAbfsRestOperation(AbfsRestOperationType.CopyBlob, HTTP_METHOD_PUT, + url, requestHeaders); + + return op; + } + + /** + * Get Rest Operation for API . + * Deletes the blob at the given path. + * @param blobPath path of the blob to be deleted. + * @param leaseId if path has an active lease. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + public AbfsRestOperation deleteBlobPath(final Path blobPath, + final String leaseId, final TracingContext tracingContext) throws AzureBlobFileSystemException { + AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + String blobRelativePath = blobPath.toUri().getPath(); + appendSASTokenToQuery(blobRelativePath, + SASTokenProvider.DELETE_OPERATION, abfsUriQueryBuilder); + final URL url = createRequestUrl(blobRelativePath, abfsUriQueryBuilder.toString()); + final List requestHeaders = createDefaultHeaders(); + if (leaseId != null) { + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId)); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.DeleteBlob, HTTP_METHOD_DELETE, url, + requestHeaders); + op.execute(tracingContext); + return op; + } + + private List getMetadataHeadersList(final String properties) { + List metadataRequestHeaders = new ArrayList(); + String[] propertiesArray = properties.split(","); + for (String property : propertiesArray) { + String[] keyValue = property.split("="); + metadataRequestHeaders.add(new AbfsHttpHeader(X_MS_METADATA_PREFIX + keyValue[0], keyValue[1])); + } + return metadataRequestHeaders; + } +} 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 f76f0ca6e87f7..0feaef349be29 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 @@ -35,7 +35,6 @@ import java.util.Locale; import java.util.Timer; import java.util.TimerTask; -import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; @@ -69,7 +68,6 @@ import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; -import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams; 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.InvalidUriException; @@ -83,7 +81,6 @@ import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.apache.hadoop.util.concurrent.HadoopExecutors; -import static org.apache.commons.lang3.StringUtils.isEmpty; import static org.apache.commons.lang3.StringUtils.isNotEmpty; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.RENAME_PATH_ATTEMPTS; import static org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.extractEtagHeader; @@ -94,14 +91,12 @@ import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*; import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.*; -import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND; -import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND; import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_ABBREVIATION; /** * AbfsClient. */ -public class AbfsClient implements Closeable { +public abstract class AbfsClient implements Closeable { public static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); public static final String HUNDRED_CONTINUE_USER_AGENT = SINGLE_WHITE_SPACE + HUNDRED_CONTINUE + SEMICOLON; @@ -144,7 +139,7 @@ public class AbfsClient implements Closeable { /** * logging the rename failure if metadata is in an incomplete state. */ - private static final LogExactlyOnce ABFS_METADATA_INCOMPLETE_RENAME_FAILURE = new LogExactlyOnce(LOG); + protected static final LogExactlyOnce ABFS_METADATA_INCOMPLETE_RENAME_FAILURE = new LogExactlyOnce(LOG); private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, @@ -317,22 +312,25 @@ AbfsThrottlingIntercept getIntercept() { * @return default request headers */ @VisibleForTesting - protected List createDefaultHeaders() { - return createDefaultHeaders(this.xMsVersion); - } + protected abstract List createDefaultHeaders(); /** * Create request headers for Rest Operation using the specified API version. - * @param xMsVersion + * @param xMsVersion Azure services API version to be used. * @return default request headers */ - private List createDefaultHeaders(ApiVersion xMsVersion) { + @VisibleForTesting + public abstract List createDefaultHeaders(ApiVersion xMsVersion); + + /** + * Create request headers common to both service endpoints. + * @param xMsVersion azure services API version to be used. + * @return common request headers + */ + protected List createCommonHeaders(ApiVersion xMsVersion) { final List requestHeaders = new ArrayList(); requestHeaders.add(new AbfsHttpHeader(X_MS_VERSION, xMsVersion.toString())); - requestHeaders.add(new AbfsHttpHeader(ACCEPT, APPLICATION_JSON - + COMMA + SINGLE_WHITE_SPACE + APPLICATION_OCTET_STREAM)); - requestHeaders.add(new AbfsHttpHeader(ACCEPT_CHARSET, - UTF_8)); + requestHeaders.add(new AbfsHttpHeader(ACCEPT_CHARSET, UTF_8)); requestHeaders.add(new AbfsHttpHeader(CONTENT_TYPE, EMPTY_STRING)); requestHeaders.add(new AbfsHttpHeader(USER_AGENT, userAgent)); return requestHeaders; @@ -354,8 +352,15 @@ private List createDefaultHeaders(ApiVersion xMsVersion) { *

  • getPathStatus for fs.setXAttr and fs.getXAttr
  • *
  • read
  • * + * @param path path of the file / directory to be created / overwritten. + * @param requestHeaders list of headers to be added to the request. + * @param isCreateFileRequest defines if file or directory has to be created / overwritten. + * @param contextEncryptionAdapter object that contains the encryptionContext and + * encryptionKey created from the developer provided implementation of {@link EncryptionContextProvider} + * @param tracingContext to trace service calls. + * @throws AzureBlobFileSystemException if namespace is not enabled. */ - private void addEncryptionKeyRequestHeaders(String path, + protected void addEncryptionKeyRequestHeaders(String path, List requestHeaders, boolean isCreateFileRequest, ContextEncryptionAdapter contextEncryptionAdapter, TracingContext tracingContext) throws AzureBlobFileSystemException { @@ -390,106 +395,25 @@ private void addEncryptionKeyRequestHeaders(String path, SERVER_SIDE_ENCRYPTION_ALGORITHM)); } - AbfsUriQueryBuilder createDefaultUriQueryBuilder() { + protected AbfsUriQueryBuilder createDefaultUriQueryBuilder() { final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_TIMEOUT, DEFAULT_TIMEOUT); return abfsUriQueryBuilder; } - public AbfsRestOperation createFilesystem(TracingContext tracingContext) - throws AzureBlobFileSystemException { - final List requestHeaders = createDefaultHeaders(); - - final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); - - final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.CreateFileSystem, - HTTP_METHOD_PUT, url, requestHeaders); - op.execute(tracingContext); - return op; - } - - public AbfsRestOperation setFilesystemProperties(final String properties, - TracingContext tracingContext) throws AzureBlobFileSystemException { - final List requestHeaders = createDefaultHeaders(); - // JDK7 does not support PATCH, so to work around the issue we will use - // PUT and specify the real method in the X-Http-Method-Override header. - requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, - HTTP_METHOD_PATCH)); - - requestHeaders.add(new AbfsHttpHeader(X_MS_PROPERTIES, - properties)); - - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); + public abstract AbfsRestOperation createFilesystem(TracingContext tracingContext) + throws AzureBlobFileSystemException; - final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.SetFileSystemProperties, - HTTP_METHOD_PUT, - url, - requestHeaders); - op.execute(tracingContext); - return op; - } + public abstract AbfsRestOperation setFilesystemProperties(String properties, + TracingContext tracingContext) throws AzureBlobFileSystemException; - public AbfsRestOperation listPath(final String relativePath, final boolean recursive, final int listMaxResults, - final String continuation, TracingContext tracingContext) - throws IOException { - final List requestHeaders = createDefaultHeaders(); + public abstract AbfsRestOperation listPath(String relativePath, boolean recursive, int listMaxResults, + String continuation, TracingContext tracingContext) + throws IOException; - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_DIRECTORY, getDirectoryQueryParameter(relativePath)); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_RECURSIVE, String.valueOf(recursive)); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_MAXRESULTS, String.valueOf(listMaxResults)); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN, String.valueOf(abfsConfiguration.isUpnUsed())); - appendSASTokenToQuery(relativePath, SASTokenProvider.LIST_OPERATION, abfsUriQueryBuilder); - - final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.ListPaths, - HTTP_METHOD_GET, - url, - requestHeaders); - op.execute(tracingContext); - return op; - } + public abstract AbfsRestOperation getFilesystemProperties(TracingContext tracingContext) throws AzureBlobFileSystemException; - public AbfsRestOperation getFilesystemProperties(TracingContext tracingContext) throws AzureBlobFileSystemException { - final List requestHeaders = createDefaultHeaders(); - - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); - - final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.GetFileSystemProperties, - HTTP_METHOD_HEAD, - url, - requestHeaders); - op.execute(tracingContext); - return op; - } - - public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws AzureBlobFileSystemException { - final List requestHeaders = createDefaultHeaders(); - - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); - - final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.DeleteFileSystem, - HTTP_METHOD_DELETE, - url, - requestHeaders); - op.execute(tracingContext); - return op; - } + public abstract AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws AzureBlobFileSystemException; /** * Method for calling createPath API to the backend. Method can be called from: @@ -518,150 +442,25 @@ public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws * @throws AzureBlobFileSystemException throws back the exception it receives from the * {@link AbfsRestOperation#execute(TracingContext)} method call. */ - public AbfsRestOperation createPath(final String path, - final boolean isFile, - final boolean overwrite, - final Permissions permissions, - final boolean isAppendBlob, - final String eTag, - final ContextEncryptionAdapter contextEncryptionAdapter, - final TracingContext tracingContext) - throws AzureBlobFileSystemException { - final List requestHeaders = createDefaultHeaders(); - if (isFile) { - addEncryptionKeyRequestHeaders(path, requestHeaders, true, - contextEncryptionAdapter, tracingContext); - } - if (!overwrite) { - requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, AbfsHttpConstants.STAR)); - } - - if (permissions.hasPermission()) { - requestHeaders.add( - new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PERMISSIONS, - permissions.getPermission())); - } - - if (permissions.hasUmask()) { - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_UMASK, - permissions.getUmask())); - } - - if (eTag != null && !eTag.isEmpty()) { - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_MATCH, eTag)); - } - - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, isFile ? FILE : DIRECTORY); - if (isAppendBlob) { - abfsUriQueryBuilder.addQuery(QUERY_PARAM_BLOBTYPE, APPEND_BLOB_TYPE); - } - - String operation = isFile - ? SASTokenProvider.CREATE_FILE_OPERATION - : SASTokenProvider.CREATE_DIRECTORY_OPERATION; - appendSASTokenToQuery(path, operation, abfsUriQueryBuilder); - - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.CreatePath, - HTTP_METHOD_PUT, - url, - requestHeaders); - try { - op.execute(tracingContext); - } catch (AzureBlobFileSystemException ex) { - // If we have no HTTP response, throw the original exception. - if (!op.hasResult()) { - throw ex; - } - if (!isFile && op.getResult().getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { - String existingResource = - op.getResult().getResponseHeader(X_MS_EXISTING_RESOURCE_TYPE); - if (existingResource != null && existingResource.equals(DIRECTORY)) { - return op; //don't throw ex on mkdirs for existing directory - } - } - throw ex; - } - return op; - } - - public AbfsRestOperation acquireLease(final String path, int duration, TracingContext tracingContext) throws AzureBlobFileSystemException { - final List requestHeaders = createDefaultHeaders(); - - requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, ACQUIRE_LEASE_ACTION)); - requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_DURATION, Integer.toString(duration))); - requestHeaders.add(new AbfsHttpHeader(X_MS_PROPOSED_LEASE_ID, UUID.randomUUID().toString())); - - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.LeasePath, - HTTP_METHOD_POST, - url, - requestHeaders); - op.execute(tracingContext); - return op; - } - - public AbfsRestOperation renewLease(final String path, final String leaseId, - TracingContext tracingContext) throws AzureBlobFileSystemException { - final List requestHeaders = createDefaultHeaders(); - - requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, RENEW_LEASE_ACTION)); - requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId)); - - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.LeasePath, - HTTP_METHOD_POST, - url, - requestHeaders); - op.execute(tracingContext); - return op; - } - - public AbfsRestOperation releaseLease(final String path, - final String leaseId, TracingContext tracingContext) throws AzureBlobFileSystemException { - final List requestHeaders = createDefaultHeaders(); - - requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, RELEASE_LEASE_ACTION)); - requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId)); - - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.LeasePath, - HTTP_METHOD_POST, - url, - requestHeaders); - op.execute(tracingContext); - return op; - } + public abstract AbfsRestOperation createPath(String path, + boolean isFile, + boolean overwrite, + Permissions permissions, + boolean isAppendBlob, + String eTag, + ContextEncryptionAdapter contextEncryptionAdapter, + TracingContext tracingContext) throws AzureBlobFileSystemException; - public AbfsRestOperation breakLease(final String path, - TracingContext tracingContext) throws AzureBlobFileSystemException { - final List requestHeaders = createDefaultHeaders(); + public abstract AbfsRestOperation acquireLease(String path, int duration, TracingContext tracingContext) throws AzureBlobFileSystemException; - requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, BREAK_LEASE_ACTION)); - requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_BREAK_PERIOD, DEFAULT_LEASE_BREAK_PERIOD)); + public abstract AbfsRestOperation renewLease(String path, String leaseId, + TracingContext tracingContext) throws AzureBlobFileSystemException; - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + public abstract AbfsRestOperation releaseLease(String path, + String leaseId, TracingContext tracingContext) throws AzureBlobFileSystemException; - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.LeasePath, - HTTP_METHOD_POST, - url, - requestHeaders); - op.execute(tracingContext); - return op; - } + public abstract AbfsRestOperation breakLease(String path, + TracingContext tracingContext) throws AzureBlobFileSystemException; /** * Rename a file or directory. @@ -685,129 +484,20 @@ public AbfsRestOperation breakLease(final String path, * AbfsRest operation, rename recovery and incomplete metadata state failure. * @throws AzureBlobFileSystemException failure, excluding any recovery from overload failures. */ - public AbfsClientRenameResult renamePath( - final String source, - final String destination, - final String continuation, - final TracingContext tracingContext, + public abstract AbfsClientRenameResult renamePath( + String source, + String destination, + String continuation, + TracingContext tracingContext, String sourceEtag, boolean isMetadataIncompleteState, boolean isNamespaceEnabled) - throws IOException { - final List requestHeaders = createDefaultHeaders(); - - final boolean hasEtag = !isEmpty(sourceEtag); + throws IOException; - boolean shouldAttemptRecovery = renameResilience && isNamespaceEnabled; - if (!hasEtag && shouldAttemptRecovery) { - // in case eTag is already not supplied to the API - // and rename resilience is expected and it is an HNS enabled account - // fetch the source etag to be used later in recovery - try { - final AbfsRestOperation srcStatusOp = getPathStatus(source, - false, tracingContext, null); - if (srcStatusOp.hasResult()) { - final AbfsHttpOperation result = srcStatusOp.getResult(); - sourceEtag = extractEtagHeader(result); - // and update the directory status. - boolean isDir = checkIsDir(result); - shouldAttemptRecovery = !isDir; - LOG.debug("Retrieved etag of source for rename recovery: {}; isDir={}", sourceEtag, isDir); - } - } catch (AbfsRestOperationException e) { - throw new AbfsRestOperationException(e.getStatusCode(), SOURCE_PATH_NOT_FOUND.getErrorCode(), - e.getMessage(), e); - } - - } - - String encodedRenameSource = urlEncode(FORWARD_SLASH + this.getFileSystem() + source); - if (authType == AuthType.SAS) { - final AbfsUriQueryBuilder srcQueryBuilder = new AbfsUriQueryBuilder(); - appendSASTokenToQuery(source, SASTokenProvider.RENAME_SOURCE_OPERATION, srcQueryBuilder); - encodedRenameSource += srcQueryBuilder.toString(); - } - - LOG.trace("Rename source queryparam added {}", encodedRenameSource); - requestHeaders.add(new AbfsHttpHeader(X_MS_RENAME_SOURCE, encodedRenameSource)); - requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, STAR)); - - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation); - appendSASTokenToQuery(destination, SASTokenProvider.RENAME_DESTINATION_OPERATION, abfsUriQueryBuilder); - - final URL url = createRequestUrl(destination, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = createRenameRestOperation(url, requestHeaders); - try { - incrementAbfsRenamePath(); - op.execute(tracingContext); - // AbfsClientResult contains the AbfsOperation, If recovery happened or - // not, and the incompleteMetaDataState is true or false. - // If we successfully rename a path and isMetadataIncompleteState was - // true, then rename was recovered, else it didn't, this is why - // isMetadataIncompleteState is used for renameRecovery(as the 2nd param). - return new AbfsClientRenameResult(op, isMetadataIncompleteState, isMetadataIncompleteState); - } catch (AzureBlobFileSystemException e) { - // If we have no HTTP response, throw the original exception. - if (!op.hasResult()) { - throw e; - } - - // ref: HADOOP-18242. Rename failure occurring due to a rare case of - // tracking metadata being in incomplete state. - if (op.getResult().getStorageErrorCode() - .equals(RENAME_DESTINATION_PARENT_PATH_NOT_FOUND.getErrorCode()) - && !isMetadataIncompleteState) { - //Logging - ABFS_METADATA_INCOMPLETE_RENAME_FAILURE - .info("Rename Failure attempting to resolve tracking metadata state and retrying."); - // rename recovery should be attempted in this case also - shouldAttemptRecovery = true; - isMetadataIncompleteState = true; - String sourceEtagAfterFailure = sourceEtag; - if (isEmpty(sourceEtagAfterFailure)) { - // Doing a HEAD call resolves the incomplete metadata state and - // then we can retry the rename operation. - AbfsRestOperation sourceStatusOp = getPathStatus(source, false, - tracingContext, null); - isMetadataIncompleteState = true; - // Extract the sourceEtag, using the status Op, and set it - // for future rename recovery. - AbfsHttpOperation sourceStatusResult = sourceStatusOp.getResult(); - sourceEtagAfterFailure = extractEtagHeader(sourceStatusResult); - } - renamePath(source, destination, continuation, tracingContext, - sourceEtagAfterFailure, isMetadataIncompleteState, isNamespaceEnabled); - } - // if we get out of the condition without a successful rename, then - // it isn't metadata incomplete state issue. - isMetadataIncompleteState = false; - - // setting default rename recovery success to false - boolean etagCheckSucceeded = false; - if (shouldAttemptRecovery) { - etagCheckSucceeded = renameIdempotencyCheckOp( - source, - sourceEtag, op, destination, tracingContext); - } - if (!etagCheckSucceeded) { - // idempotency did not return different result - // throw back the exception - throw e; - } - return new AbfsClientRenameResult(op, true, isMetadataIncompleteState); - } - } - - private boolean checkIsDir(AbfsHttpOperation result) { - String resourceType = result.getResponseHeader( - HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); - return resourceType != null - && resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY); - } + protected abstract boolean checkIsDir(AbfsHttpOperation result); @VisibleForTesting - AbfsRestOperation createRenameRestOperation(URL url, List requestHeaders) { + protected AbfsRestOperation createRenameRestOperation(URL url, List requestHeaders) { AbfsRestOperation op = getAbfsRestOperation( AbfsRestOperationType.RenamePath, HTTP_METHOD_PUT, @@ -816,7 +506,7 @@ AbfsRestOperation createRenameRestOperation(URL url, List reques return op; } - private void incrementAbfsRenamePath() { + protected void incrementAbfsRenamePath() { abfsCounters.incrementCounter(RENAME_PATH_ATTEMPTS, 1); } @@ -886,137 +576,24 @@ boolean isSourceDestEtagEqual(String sourceEtag, AbfsHttpOperation result) { return sourceEtag.equals(extractEtagHeader(result)); } - public AbfsRestOperation append(final String path, final byte[] buffer, - AppendRequestParameters reqParams, final String cachedSasToken, + public abstract AbfsRestOperation append(String path, byte[] buffer, + AppendRequestParameters reqParams, String cachedSasToken, ContextEncryptionAdapter contextEncryptionAdapter, TracingContext tracingContext) - throws AzureBlobFileSystemException { - final List requestHeaders = createDefaultHeaders(); - addEncryptionKeyRequestHeaders(path, requestHeaders, false, - contextEncryptionAdapter, tracingContext); - if (reqParams.isExpectHeaderEnabled()) { - requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); - } - // 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. - requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, - HTTP_METHOD_PATCH)); - if (reqParams.getLeaseId() != null) { - requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, reqParams.getLeaseId())); - } - - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, APPEND_ACTION); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(reqParams.getPosition())); - - if ((reqParams.getMode() == AppendRequestParameters.Mode.FLUSH_MODE) || ( - reqParams.getMode() == AppendRequestParameters.Mode.FLUSH_CLOSE_MODE)) { - abfsUriQueryBuilder.addQuery(QUERY_PARAM_FLUSH, TRUE); - if (reqParams.getMode() == AppendRequestParameters.Mode.FLUSH_CLOSE_MODE) { - abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, TRUE); - } - } - - // Check if the retry is with "Expect: 100-continue" header being present in the previous request. - if (reqParams.isRetryDueToExpect()) { - String userAgentRetry = userAgent; - // Remove the specific marker related to "Expect: 100-continue" from the User-Agent string. - userAgentRetry = userAgentRetry.replace(HUNDRED_CONTINUE_USER_AGENT, EMPTY_STRING); - requestHeaders.removeIf(header -> header.getName().equalsIgnoreCase(USER_AGENT)); - requestHeaders.add(new AbfsHttpHeader(USER_AGENT, userAgentRetry)); - } - - // Add MD5 Hash of request content as request header if feature is enabled - if (isChecksumValidationEnabled()) { - addCheckSumHeaderForWrite(requestHeaders, reqParams, buffer); - } - - // AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance - String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.WRITE_OPERATION, - abfsUriQueryBuilder, cachedSasToken); - - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.Append, - HTTP_METHOD_PUT, - url, - requestHeaders, - buffer, - reqParams.getoffset(), - reqParams.getLength(), - sasTokenForReuse); - try { - op.execute(tracingContext); - } catch (AbfsRestOperationException e) { - /* - If the http response code indicates a user error we retry - the same append request with expect header being disabled. - When "100-continue" header is enabled but a non Http 100 response comes, - the response message might not get set correctly by the server. - So, this handling is to avoid breaking of backward compatibility - if someone has taken dependency on the exception message, - which is created using the error string present in the response header. - */ - int responseStatusCode = e.getStatusCode(); - if (checkUserError(responseStatusCode) && reqParams.isExpectHeaderEnabled()) { - LOG.debug("User error, retrying without 100 continue enabled for the given path {}", path); - reqParams.setExpectHeaderEnabled(false); - reqParams.setRetryDueToExpect(true); - return this.append(path, buffer, reqParams, cachedSasToken, - contextEncryptionAdapter, tracingContext); - } - // If we have no HTTP response, throw the original exception. - if (!op.hasResult()) { - throw e; - } - - if (isMd5ChecksumError(e)) { - throw new AbfsInvalidChecksumException(e); - } - - if (reqParams.isAppendBlob() - && appendSuccessCheckOp(op, path, - (reqParams.getPosition() + reqParams.getLength()), tracingContext)) { - final AbfsRestOperation successOp = getAbfsRestOperation( - AbfsRestOperationType.Append, - HTTP_METHOD_PUT, - url, - requestHeaders, - buffer, - reqParams.getoffset(), - reqParams.getLength(), - sasTokenForReuse); - successOp.hardSetResult(HttpURLConnection.HTTP_OK); - return successOp; - } - throw e; - } - - catch (AzureBlobFileSystemException e) { - // Any server side issue will be returned as AbfsRestOperationException and will be handled above. - LOG.debug("Append request failed with non server issues for path: {}, offset: {}, position: {}", - path, reqParams.getoffset(), reqParams.getPosition()); - throw e; - } - - return op; - } + throws AzureBlobFileSystemException; /** * Returns true if the status code lies in the range of user error. * @param responseStatusCode http response status code. * @return True or False. */ - private boolean checkUserError(int responseStatusCode) { - return (responseStatusCode >= HttpURLConnection.HTTP_BAD_REQUEST - && responseStatusCode < HttpURLConnection.HTTP_INTERNAL_ERROR); - } + public abstract boolean checkUserError(int responseStatusCode); /** * To check if the failure exception returned by server is due to MD5 Mismatch * @param e Exception returned by AbfsRestOperation * @return boolean whether exception is due to MD5Mismatch or not */ - private boolean isMd5ChecksumError(final AbfsRestOperationException e) { + protected boolean isMd5ChecksumError(final AbfsRestOperationException e) { AzureServiceErrorCode storageErrorCode = e.getErrorCode(); return storageErrorCode == AzureServiceErrorCode.MD5_MISMATCH; } @@ -1044,202 +621,44 @@ public boolean appendSuccessCheckOp(AbfsRestOperation op, final String path, return false; } - public AbfsRestOperation flush(final String path, final long position, + public abstract AbfsRestOperation flush(String path, long position, boolean retainUncommittedData, boolean isClose, - final String cachedSasToken, final String leaseId, + String cachedSasToken, String leaseId, ContextEncryptionAdapter contextEncryptionAdapter, TracingContext tracingContext) - throws AzureBlobFileSystemException { - final List requestHeaders = createDefaultHeaders(); - addEncryptionKeyRequestHeaders(path, requestHeaders, false, - contextEncryptionAdapter, tracingContext); - // 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. - requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, - HTTP_METHOD_PATCH)); - if (leaseId != null) { - requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId)); - } + throws AzureBlobFileSystemException; - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, FLUSH_ACTION); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(position)); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_RETAIN_UNCOMMITTED_DATA, String.valueOf(retainUncommittedData)); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, String.valueOf(isClose)); - - // AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance - String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.WRITE_OPERATION, - abfsUriQueryBuilder, cachedSasToken); - - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.Flush, - HTTP_METHOD_PUT, - url, - requestHeaders, sasTokenForReuse); - op.execute(tracingContext); - return op; - } - - public AbfsRestOperation setPathProperties(final String path, final String properties, - final TracingContext tracingContext, final ContextEncryptionAdapter contextEncryptionAdapter) - throws AzureBlobFileSystemException { - final List requestHeaders = createDefaultHeaders(); - addEncryptionKeyRequestHeaders(path, requestHeaders, false, - contextEncryptionAdapter, tracingContext); - // 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. - requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, - HTTP_METHOD_PATCH)); - - requestHeaders.add(new AbfsHttpHeader(X_MS_PROPERTIES, properties)); - - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, SET_PROPERTIES_ACTION); - appendSASTokenToQuery(path, SASTokenProvider.SET_PROPERTIES_OPERATION, abfsUriQueryBuilder); - - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.SetPathProperties, - HTTP_METHOD_PUT, - url, - requestHeaders); - op.execute(tracingContext); - return op; - } - - public AbfsRestOperation getPathStatus(final String path, - final boolean includeProperties, final TracingContext tracingContext, - final ContextEncryptionAdapter contextEncryptionAdapter) - throws AzureBlobFileSystemException { - final List requestHeaders = createDefaultHeaders(); - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - String operation = SASTokenProvider.GET_PROPERTIES_OPERATION; - if (!includeProperties) { - // The default action (operation) is implicitly to get properties and this action requires read permission - // because it reads user defined properties. If the action is getStatus or getAclStatus, then - // only traversal (execute) permission is required. - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.GET_STATUS); - operation = SASTokenProvider.GET_STATUS_OPERATION; - } else { - addEncryptionKeyRequestHeaders(path, requestHeaders, false, - contextEncryptionAdapter, - tracingContext); - } - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN, String.valueOf(abfsConfiguration.isUpnUsed())); - appendSASTokenToQuery(path, operation, abfsUriQueryBuilder); - - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.GetPathStatus, - HTTP_METHOD_HEAD, - url, - requestHeaders); - 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, + public abstract AbfsRestOperation flush(byte[] buffer, + String path, + boolean isClose, + String cachedSasToken, + String leaseId, + String eTag, + TracingContext tracingContext) throws AzureBlobFileSystemException; + + public abstract AbfsRestOperation setPathProperties(String path, String properties, + TracingContext tracingContext, ContextEncryptionAdapter contextEncryptionAdapter) + throws AzureBlobFileSystemException; + + public abstract AbfsRestOperation getPathStatus(String path, + boolean includeProperties, TracingContext tracingContext, + ContextEncryptionAdapter contextEncryptionAdapter) + throws AzureBlobFileSystemException; + + public abstract AbfsRestOperation read(String path, + long position, + byte[] buffer, + int bufferOffset, + int bufferLength, + String eTag, String cachedSasToken, ContextEncryptionAdapter contextEncryptionAdapter, - TracingContext tracingContext) throws AzureBlobFileSystemException { - final List requestHeaders = createDefaultHeaders(); - addEncryptionKeyRequestHeaders(path, requestHeaders, false, - contextEncryptionAdapter, tracingContext); - AbfsHttpHeader rangeHeader = new AbfsHttpHeader(RANGE, - String.format("bytes=%d-%d", position, position + bufferLength - 1)); - requestHeaders.add(rangeHeader); - requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag)); - - // Add request header to fetch MD5 Hash of data returned by server. - if (isChecksumValidationEnabled(requestHeaders, rangeHeader, bufferLength)) { - requestHeaders.add(new AbfsHttpHeader(X_MS_RANGE_GET_CONTENT_MD5, TRUE)); - } + TracingContext tracingContext) throws AzureBlobFileSystemException; - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - - // AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance - String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.READ_OPERATION, - abfsUriQueryBuilder, cachedSasToken); - - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.ReadFile, - HTTP_METHOD_GET, - url, - requestHeaders, - buffer, - bufferOffset, - bufferLength, sasTokenForReuse); - op.execute(tracingContext); - - // Verify the MD5 hash returned by server holds valid on the data received. - if (isChecksumValidationEnabled(requestHeaders, rangeHeader, bufferLength)) { - verifyCheckSumForRead(buffer, op.getResult(), bufferOffset); - } - - return op; - } - - public AbfsRestOperation deletePath(final String path, final boolean recursive, - final String continuation, + public abstract AbfsRestOperation deletePath(String path, boolean recursive, + String continuation, TracingContext tracingContext, - final boolean isNamespaceEnabled) - throws AzureBlobFileSystemException { - /* - * If Pagination is enabled and current API version is old, - * use the minimum required version for pagination. - * If Pagination is enabled and current API version is later than minimum required - * version for pagination, use current version only as azure service is backward compatible. - * If pagination is disabled, use the current API version only. - */ - final List requestHeaders = (isPaginatedDelete(recursive, - isNamespaceEnabled) && xMsVersion.compareTo(ApiVersion.AUG_03_2023) < 0) - ? createDefaultHeaders(ApiVersion.AUG_03_2023) - : createDefaultHeaders(); - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - - if (isPaginatedDelete(recursive, isNamespaceEnabled)) { - // Add paginated query parameter - abfsUriQueryBuilder.addQuery(QUERY_PARAM_PAGINATED, TRUE); - } - - abfsUriQueryBuilder.addQuery(QUERY_PARAM_RECURSIVE, String.valueOf(recursive)); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation); - String operation = recursive ? SASTokenProvider.DELETE_RECURSIVE_OPERATION : SASTokenProvider.DELETE_OPERATION; - appendSASTokenToQuery(path, operation, abfsUriQueryBuilder); - - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.DeletePath, - this, - HTTP_METHOD_DELETE, - url, - requestHeaders); - try { - op.execute(tracingContext); - } catch (AzureBlobFileSystemException e) { - // If we have no HTTP response, throw the original exception. - if (!op.hasResult()) { - throw e; - } - final AbfsRestOperation idempotencyOp = deleteIdempotencyCheckOp(op); - if (idempotencyOp.getResult().getStatusCode() - == op.getResult().getStatusCode()) { - // idempotency did not return different result - // throw back the exception - throw e; - } else { - return idempotencyOp; - } - } - - return op; - } + boolean isNamespaceEnabled) + throws AzureBlobFileSystemException; /** * Check if the delete request failure is post a retry and if delete failure @@ -1276,117 +695,30 @@ public AbfsRestOperation deleteIdempotencyCheckOp(final AbfsRestOperation op) { return op; } - public AbfsRestOperation setOwner(final String path, final String owner, final String group, + public abstract AbfsRestOperation setOwner(String path, String owner, String group, 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. - requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, - HTTP_METHOD_PATCH)); - - if (owner != null && !owner.isEmpty()) { - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_OWNER, owner)); - } - if (group != null && !group.isEmpty()) { - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_GROUP, group)); - } - - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_ACCESS_CONTROL); - appendSASTokenToQuery(path, SASTokenProvider.SET_OWNER_OPERATION, abfsUriQueryBuilder); + throws AzureBlobFileSystemException; - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.SetOwner, - AbfsHttpConstants.HTTP_METHOD_PUT, - url, - requestHeaders); - op.execute(tracingContext); - return op; - } - - public AbfsRestOperation setPermission(final String path, final String permission, + public abstract AbfsRestOperation setPermission(String path, String permission, 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. - requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, - HTTP_METHOD_PATCH)); - - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PERMISSIONS, permission)); - - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_ACCESS_CONTROL); - appendSASTokenToQuery(path, SASTokenProvider.SET_PERMISSION_OPERATION, abfsUriQueryBuilder); - - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.SetPermissions, - AbfsHttpConstants.HTTP_METHOD_PUT, - url, - requestHeaders); - op.execute(tracingContext); - return op; - } + throws AzureBlobFileSystemException; public AbfsRestOperation setAcl(final String path, final String aclSpecString, TracingContext tracingContext) throws AzureBlobFileSystemException { return setAcl(path, aclSpecString, AbfsHttpConstants.EMPTY_STRING, tracingContext); } - public AbfsRestOperation setAcl(final String path, final String aclSpecString, final String eTag, + public abstract AbfsRestOperation setAcl(String path, String aclSpecString, String eTag, 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. - requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, - HTTP_METHOD_PATCH)); - - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_ACL, aclSpecString)); - - if (eTag != null && !eTag.isEmpty()) { - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_MATCH, eTag)); - } - - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_ACCESS_CONTROL); - appendSASTokenToQuery(path, SASTokenProvider.SET_ACL_OPERATION, abfsUriQueryBuilder); - - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.SetAcl, - AbfsHttpConstants.HTTP_METHOD_PUT, - url, - requestHeaders); - op.execute(tracingContext); - return op; - } + throws AzureBlobFileSystemException; public AbfsRestOperation getAclStatus(final String path, TracingContext tracingContext) throws AzureBlobFileSystemException { return getAclStatus(path, abfsConfiguration.isUpnUsed(), tracingContext); } - public AbfsRestOperation getAclStatus(final String path, final boolean useUPN, - TracingContext tracingContext) throws AzureBlobFileSystemException { - final List requestHeaders = createDefaultHeaders(); - final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.GET_ACCESS_CONTROL); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN, String.valueOf(useUPN)); - appendSASTokenToQuery(path, SASTokenProvider.GET_ACL_OPERATION, abfsUriQueryBuilder); - - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.GetAcl, - AbfsHttpConstants.HTTP_METHOD_HEAD, - url, - requestHeaders); - op.execute(tracingContext); - return op; - } + public abstract AbfsRestOperation getAclStatus(String path, boolean useUPN, + TracingContext tracingContext) throws AzureBlobFileSystemException; /** * Talks to the server to check whether the permission specified in @@ -1398,21 +730,8 @@ 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, TracingContext tracingContext) - throws AzureBlobFileSystemException { - AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, CHECK_ACCESS); - abfsUriQueryBuilder.addQuery(QUERY_FS_ACTION, rwx); - appendSASTokenToQuery(path, SASTokenProvider.CHECK_ACCESS_OPERATION, abfsUriQueryBuilder); - URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - AbfsRestOperation op = getAbfsRestOperation( - AbfsRestOperationType.CheckAccess, - AbfsHttpConstants.HTTP_METHOD_HEAD, - url, - createDefaultHeaders()); - op.execute(tracingContext); - return op; - } + public abstract AbfsRestOperation checkAccess(String path, String rwx, TracingContext tracingContext) + throws AzureBlobFileSystemException; /** * Get the directory query parameter used by the List Paths REST API and used @@ -1435,26 +754,26 @@ public static String getDirectoryQueryParameter(final String path) { /** * If configured for SAS AuthType, appends SAS token to queryBuilder. - * @param path - * @param operation - * @param queryBuilder + * @param path for which SAS token is required. + * @param operation for which SAS token is required. + * @param queryBuilder to which SAS token is appended. * @return sasToken - returned for optional re-use. - * @throws SASTokenProviderException + * @throws SASTokenProviderException if SAS token cannot be acquired. */ - private String appendSASTokenToQuery(String path, String operation, AbfsUriQueryBuilder queryBuilder) throws SASTokenProviderException { + protected String appendSASTokenToQuery(String path, String operation, AbfsUriQueryBuilder queryBuilder) throws SASTokenProviderException { return appendSASTokenToQuery(path, operation, queryBuilder, null); } /** * If configured for SAS AuthType, appends SAS token to queryBuilder. - * @param path - * @param operation - * @param queryBuilder + * @param path for which SAS token is required. + * @param operation for which SAS token is required. + * @param queryBuilder to which SAS token is appended. * @param cachedSasToken - previously acquired SAS token to be reused. * @return sasToken - returned for optional re-use. - * @throws SASTokenProviderException + * @throws SASTokenProviderException if SAS token cannot be acquired. */ - private String appendSASTokenToQuery(String path, + protected String appendSASTokenToQuery(String path, String operation, AbfsUriQueryBuilder queryBuilder, String cachedSasToken) @@ -1491,7 +810,7 @@ private String appendSASTokenToQuery(String path, } @VisibleForTesting - private URL createRequestUrl(final String query) throws AzureBlobFileSystemException { + protected URL createRequestUrl(final String query) throws AzureBlobFileSystemException { return createRequestUrl(EMPTY_STRING, query); } @@ -1563,7 +882,7 @@ protected Boolean getIsPaginatedDeleteEnabled() { return abfsConfiguration.isPaginatedDeleteEnabled(); } - private Boolean isPaginatedDelete(boolean isRecursiveDelete, boolean isNamespaceEnabled) { + protected Boolean isPaginatedDelete(boolean isRecursiveDelete, boolean isNamespaceEnabled) { return getIsPaginatedDeleteEnabled() && isNamespaceEnabled && isRecursiveDelete; } @@ -1646,7 +965,7 @@ private void appendIfNotEmpty(StringBuilder sb, String regEx, * @param buffer for getting input data for MD5 computation * @throws AbfsRestOperationException if Md5 computation fails */ - private void addCheckSumHeaderForWrite(List requestHeaders, + protected void addCheckSumHeaderForWrite(List requestHeaders, final AppendRequestParameters reqParams, final byte[] buffer) throws AbfsRestOperationException { String md5Hash = computeMD5Hash(buffer, reqParams.getoffset(), @@ -1661,7 +980,7 @@ private void addCheckSumHeaderForWrite(List requestHeaders, * @param bufferOffset Position where data returned by server is saved in buffer. * @throws AbfsRestOperationException if Md5Mismatch. */ - private void verifyCheckSumForRead(final byte[] buffer, + protected void verifyCheckSumForRead(final byte[] buffer, final AbfsHttpOperation result, final int bufferOffset) throws AbfsRestOperationException { // Number of bytes returned by server could be less than or equal to what @@ -1685,8 +1004,7 @@ private void verifyCheckSumForRead(final byte[] buffer, /** * Conditions check for allowing checksum support for read operation. * Sending MD5 Hash in request headers. For more details see - * @see - * Path - Read Azure Storage Rest API. + * @see Path - Read Azure Storage Rest API. * 1. Range header must be present as one of the request headers. * 2. buffer length must be less than or equal to 4 MB. * @param requestHeaders to be checked for range header. @@ -1694,7 +1012,7 @@ private void verifyCheckSumForRead(final byte[] buffer, * @param bufferLength must be less than or equal to 4 MB. * @return true if all conditions are met. */ - private boolean isChecksumValidationEnabled(List requestHeaders, + protected boolean isChecksumValidationEnabled(List requestHeaders, final AbfsHttpHeader rangeHeader, final int bufferLength) { return getAbfsConfiguration().getIsChecksumValidationEnabled() && requestHeaders.contains(rangeHeader) && bufferLength <= 4 * ONE_MB; @@ -1704,11 +1022,10 @@ private boolean isChecksumValidationEnabled(List requestHeaders, * Conditions check for allowing checksum support for write operation. * Server will support this if client sends the MD5 Hash as a request header. * For azure stoage service documentation see - * @see - * Path - Update Azure Rest API. + * @see Path - Update Azure Rest API. * @return true if checksum validation enabled. */ - private boolean isChecksumValidationEnabled() { + protected boolean isChecksumValidationEnabled() { return getAbfsConfiguration().getIsChecksumValidationEnabled(); } @@ -1994,4 +1311,12 @@ AbfsRestOperation getAbfsRestOperation(final AbfsRestOperationType operationType url, requestHeaders, sasTokenForReuse); } + + protected String getUserAgent() { + return userAgent; + } + + protected boolean isRenameResilience() { + return renameResilience; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientHandler.java new file mode 100644 index 0000000000000..5c709c300104c --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientHandler.java @@ -0,0 +1,68 @@ +/** + * 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.services; + +import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType; +import org.apache.hadoop.util.Preconditions; + +/** + * AbfsClientHandler is a class that provides a way to get the AbfsClient + * based on the service type. + */ +public class AbfsClientHandler { + private final AbfsServiceType defaultServiceType; + private final AbfsDfsClient dfsAbfsClient; + private final AbfsBlobClient blobAbfsClient; + + public AbfsClientHandler(AbfsServiceType defaultServiceType, + AbfsDfsClient dfsAbfsClient, AbfsBlobClient blobAbfsClient) { + this.blobAbfsClient = blobAbfsClient; + this.dfsAbfsClient = dfsAbfsClient; + this.defaultServiceType = defaultServiceType; + } + + public AbfsClient getClient() { + if (defaultServiceType == AbfsServiceType.DFS) { + Preconditions.checkNotNull(dfsAbfsClient, "DFS client is not initialized"); + return dfsAbfsClient; + } else { + Preconditions.checkNotNull(blobAbfsClient, "Blob client is not initialized"); + return blobAbfsClient; + } + } + + public AbfsClient getClient(AbfsServiceType serviceType) { + if (serviceType == AbfsServiceType.DFS) { + Preconditions.checkNotNull(dfsAbfsClient, "DFS client is not initialized"); + return dfsAbfsClient; + } else { + Preconditions.checkNotNull(blobAbfsClient, "Blob client is not initialized"); + return blobAbfsClient; + } + } + + public AbfsDfsClient getDfsClient() { + Preconditions.checkNotNull(dfsAbfsClient, "DFS client is not initialized"); + return dfsAbfsClient; + } + + public AbfsBlobClient getBlobClient() { + Preconditions.checkNotNull(blobAbfsClient, "Blob client is not initialized"); + return blobAbfsClient; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java new file mode 100644 index 0000000000000..325abff9a9c3b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java @@ -0,0 +1,1199 @@ +/** + * 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.services; + +import java.io.Closeable; +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.List; +import java.util.UUID; + +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ApiVersion; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsInvalidChecksumException; +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; +import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; +import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider; +import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; +import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +import static org.apache.commons.lang3.StringUtils.isEmpty; +import static org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.extractEtagHeader; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ACQUIRE_LEASE_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_BLOB_TYPE; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPLICATION_JSON; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPLICATION_OCTET_STREAM; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BREAK_LEASE_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHECK_ACCESS; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COMMA; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DEFAULT_LEASE_BREAK_PERIOD; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DIRECTORY; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FILE; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FILESYSTEM; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FLUSH_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FORWARD_SLASH; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.GET_ACCESS_CONTROL; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.GET_STATUS; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_DELETE; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_GET; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_HEAD; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PATCH; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_POST; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.RELEASE_LEASE_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.RENEW_LEASE_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.SET_ACCESS_CONTROL; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.SET_PROPERTIES_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.SINGLE_WHITE_SPACE; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.STAR; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.TRUE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.ACCEPT; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.IF_MATCH; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.IF_NONE_MATCH; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.RANGE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.USER_AGENT; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_EXISTING_RESOURCE_TYPE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_BREAK_PERIOD; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_DURATION; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_ID; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_PROPERTIES; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_PROPOSED_LEASE_ID; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_RANGE_GET_CONTENT_MD5; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_RENAME_SOURCE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_FS_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_BLOBTYPE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_CLOSE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_CONTINUATION; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_DIRECTORY; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_FLUSH; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_MAXRESULTS; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_PAGINATED; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_POSITION; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_RECURSIVE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_RESOURCE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_RETAIN_UNCOMMITTED_DATA; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND; + +/** + * AbfsClient interacting with the DFS Endpoint. + */ +public class AbfsDfsClient extends AbfsClient implements Closeable { + + public AbfsDfsClient(final URL baseUrl, + final SharedKeyCredentials sharedKeyCredentials, + final AbfsConfiguration abfsConfiguration, + final AccessTokenProvider tokenProvider, + final EncryptionContextProvider encryptionContextProvider, + final AbfsClientContext abfsClientContext) throws IOException { + super(baseUrl, sharedKeyCredentials, abfsConfiguration, tokenProvider, + encryptionContextProvider, abfsClientContext); + } + + public AbfsDfsClient(final URL baseUrl, + final SharedKeyCredentials sharedKeyCredentials, + final AbfsConfiguration abfsConfiguration, + final SASTokenProvider sasTokenProvider, + final EncryptionContextProvider encryptionContextProvider, + final AbfsClientContext abfsClientContext) throws IOException { + super(baseUrl, sharedKeyCredentials, abfsConfiguration, sasTokenProvider, + encryptionContextProvider, abfsClientContext); + } + + @Override + public void close() throws IOException { + super.close(); + } + + public List createDefaultHeaders() { + return this.createDefaultHeaders(getxMsVersion()); + } + + /** + * Create request headers for Rest Operation using the specified API version. + * DFS Endpoint API responses are in JSON/Stream format. + * @param xMsVersion API version to be used. + * @return default request headers + */ + @Override + public List createDefaultHeaders(ApiVersion xMsVersion) { + List requestHeaders = super.createCommonHeaders(xMsVersion); + requestHeaders.add(new AbfsHttpHeader(ACCEPT, APPLICATION_JSON + + COMMA + SINGLE_WHITE_SPACE + APPLICATION_OCTET_STREAM)); + return requestHeaders; + } + + /** + * Get Rest Operation for API . + * Creates a filesystem. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation createFilesystem(TracingContext tracingContext) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); + + final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.CreateFileSystem, + HTTP_METHOD_PUT, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * Sets user-defined properties of the filesystem. + * @param properties comma separated list of metadata key-value pairs. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation setFilesystemProperties(final String properties, + TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + // JDK7 does not support PATCH, so to work around the issue we will use + // PUT and specify the real method in the X-Http-Method-Override header. + requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, + HTTP_METHOD_PATCH)); + requestHeaders.add(new AbfsHttpHeader(X_MS_PROPERTIES, properties)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); + + final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.SetFileSystemProperties, + HTTP_METHOD_PUT, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * Gets all the properties of the filesystem. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + * */ + @Override + public AbfsRestOperation getFilesystemProperties(TracingContext tracingContext) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); + + final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.GetFileSystemProperties, + HTTP_METHOD_HEAD, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * Deletes the current filesystem. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); + + final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.DeleteFileSystem, + HTTP_METHOD_DELETE, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * List paths and their properties in the current filesystem. + * @param relativePath to return only blobs within this directory. + * @param recursive to return all blobs in the path, including those in subdirectories. + * @param listMaxResults maximum number of blobs to return. + * @param continuation marker to specify the continuation token. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation or response parsing fails. + */ + @Override + public AbfsRestOperation listPath(final String relativePath, + final boolean recursive, + final int listMaxResults, + final String continuation, + TracingContext tracingContext) throws IOException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_DIRECTORY, + getDirectoryQueryParameter(relativePath)); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RECURSIVE, String.valueOf(recursive)); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_MAXRESULTS, + String.valueOf(listMaxResults)); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN, + String.valueOf(getAbfsConfiguration().isUpnUsed())); + appendSASTokenToQuery(relativePath, SASTokenProvider.LIST_OPERATION, + abfsUriQueryBuilder); + + final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.ListPaths, + HTTP_METHOD_GET, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * Create a path (file or directory) in the current filesystem. + * @param path to be created inside the filesystem. + * @param isFile to specify if the created path is file or directory. + * @param overwrite to specify if the path should be overwritten if it already exists. + * @param permissions to specify the permissions of the path. + * @param isAppendBlob to specify if the path to be created is an append blob. + * @param eTag to specify conditional headers. + * @param contextEncryptionAdapter to provide encryption context. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation createPath(final String path, + final boolean isFile, + final boolean overwrite, + final AzureBlobFileSystemStore.Permissions permissions, + final boolean isAppendBlob, + final String eTag, + final ContextEncryptionAdapter contextEncryptionAdapter, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + if (isFile) { + addEncryptionKeyRequestHeaders(path, requestHeaders, true, + contextEncryptionAdapter, tracingContext); + } + if (!overwrite) { + requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, STAR)); + } + + if (permissions.hasPermission()) { + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PERMISSIONS, + permissions.getPermission())); + } + + if (permissions.hasUmask()) { + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_UMASK, + permissions.getUmask())); + } + + if (eTag != null && !eTag.isEmpty()) { + requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag)); + } + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, isFile ? FILE : DIRECTORY); + if (isAppendBlob) { + abfsUriQueryBuilder.addQuery(QUERY_PARAM_BLOBTYPE, APPEND_BLOB_TYPE); + } + + String operation = isFile + ? SASTokenProvider.CREATE_FILE_OPERATION + : SASTokenProvider.CREATE_DIRECTORY_OPERATION; + appendSASTokenToQuery(path, operation, abfsUriQueryBuilder); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.CreatePath, + HTTP_METHOD_PUT, url, requestHeaders); + try { + op.execute(tracingContext); + } catch (AzureBlobFileSystemException ex) { + // If we have no HTTP response, throw the original exception. + if (!op.hasResult()) { + throw ex; + } + if (!isFile && op.getResult().getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { + String existingResource = + op.getResult().getResponseHeader(X_MS_EXISTING_RESOURCE_TYPE); + if (existingResource != null && existingResource.equals(DIRECTORY)) { + return op; //don't throw ex on mkdirs for existing directory + } + } + throw ex; + } + return op; + } + + /** + * Get Rest Operation for API . + * Acquire lease on specified path. + * @param path on which lease has to be acquired. + * @param duration for which lease has to be acquired. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation acquireLease(final String path, final int duration, + TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, ACQUIRE_LEASE_ACTION)); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_DURATION, Integer.toString(duration))); + requestHeaders.add(new AbfsHttpHeader(X_MS_PROPOSED_LEASE_ID, + UUID.randomUUID().toString())); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.LeasePath, + HTTP_METHOD_POST, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * Renew lease on specified path. + * @param path on which lease has to be renewed. + * @param leaseId of the lease to be renewed. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation renewLease(final String path, final String leaseId, + TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, RENEW_LEASE_ACTION)); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.LeasePath, + HTTP_METHOD_POST, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * Release lease on specified path. + * @param path on which lease has to be released. + * @param leaseId of the lease to be released. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation releaseLease(final String path, final String leaseId, + TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, RELEASE_LEASE_ACTION)); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.LeasePath, + HTTP_METHOD_POST, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * Break lease on specified path. + * @param path on which lease has to be broke. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation breakLease(final String path, + TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, BREAK_LEASE_ACTION)); + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_BREAK_PERIOD, + DEFAULT_LEASE_BREAK_PERIOD)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.LeasePath, + HTTP_METHOD_POST, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * @param source path to source file + * @param destination destination of rename. + * @param continuation continuation. + * @param tracingContext for tracing the server calls. + * @param sourceEtag etag of source file. may be null or empty + * @param isMetadataIncompleteState was there a rename failure due to incomplete metadata state? + * @param isNamespaceEnabled whether namespace enabled account or not + * @return executed rest operation containing response from server. + * @throws IOException if rest operation fails. + */ + @Override + public AbfsClientRenameResult renamePath( + final String source, + final String destination, + final String continuation, + final TracingContext tracingContext, + String sourceEtag, + boolean isMetadataIncompleteState, + boolean isNamespaceEnabled) throws IOException { + final List requestHeaders = createDefaultHeaders(); + + final boolean hasEtag = !isEmpty(sourceEtag); + + boolean shouldAttemptRecovery = isRenameResilience() && isNamespaceEnabled; + if (!hasEtag && shouldAttemptRecovery) { + // in case eTag is already not supplied to the API + // and rename resilience is expected and it is an HNS enabled account + // fetch the source etag to be used later in recovery + try { + final AbfsRestOperation srcStatusOp = getPathStatus(source, + false, tracingContext, null); + if (srcStatusOp.hasResult()) { + final AbfsHttpOperation result = srcStatusOp.getResult(); + sourceEtag = extractEtagHeader(result); + // and update the directory status. + boolean isDir = checkIsDir(result); + shouldAttemptRecovery = !isDir; + LOG.debug( + "Retrieved etag of source for rename recovery: {}; isDir={}", + sourceEtag, isDir); + } + } catch (AbfsRestOperationException e) { + throw new AbfsRestOperationException(e.getStatusCode(), + SOURCE_PATH_NOT_FOUND.getErrorCode(), + e.getMessage(), e); + } + + } + + String encodedRenameSource = urlEncode( + FORWARD_SLASH + this.getFileSystem() + source); + if (getAuthType() == AuthType.SAS) { + final AbfsUriQueryBuilder srcQueryBuilder = new AbfsUriQueryBuilder(); + appendSASTokenToQuery(source, SASTokenProvider.RENAME_SOURCE_OPERATION, + srcQueryBuilder); + encodedRenameSource += srcQueryBuilder.toString(); + } + + LOG.trace("Rename source queryparam added {}", encodedRenameSource); + requestHeaders.add( + new AbfsHttpHeader(X_MS_RENAME_SOURCE, encodedRenameSource)); + requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, STAR)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation); + appendSASTokenToQuery(destination, + SASTokenProvider.RENAME_DESTINATION_OPERATION, abfsUriQueryBuilder); + + final URL url = createRequestUrl(destination, + abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = createRenameRestOperation(url, requestHeaders); + try { + incrementAbfsRenamePath(); + op.execute(tracingContext); + // AbfsClientResult contains the AbfsOperation, If recovery happened or + // not, and the incompleteMetaDataState is true or false. + // If we successfully rename a path and isMetadataIncompleteState was + // true, then rename was recovered, else it didn't, this is why + // isMetadataIncompleteState is used for renameRecovery(as the 2nd param). + return new AbfsClientRenameResult(op, isMetadataIncompleteState, + isMetadataIncompleteState); + } catch (AzureBlobFileSystemException e) { + // If we have no HTTP response, throw the original exception. + if (!op.hasResult()) { + throw e; + } + + // ref: HADOOP-18242. Rename failure occurring due to a rare case of + // tracking metadata being in incomplete state. + if (op.getResult().getStorageErrorCode() + .equals(RENAME_DESTINATION_PARENT_PATH_NOT_FOUND.getErrorCode()) + && !isMetadataIncompleteState) { + //Logging + ABFS_METADATA_INCOMPLETE_RENAME_FAILURE + .info( + "Rename Failure attempting to resolve tracking metadata state and retrying."); + // rename recovery should be attempted in this case also + shouldAttemptRecovery = true; + isMetadataIncompleteState = true; + String sourceEtagAfterFailure = sourceEtag; + if (isEmpty(sourceEtagAfterFailure)) { + // Doing a HEAD call resolves the incomplete metadata state and + // then we can retry the rename operation. + AbfsRestOperation sourceStatusOp = getPathStatus(source, false, + tracingContext, null); + isMetadataIncompleteState = true; + // Extract the sourceEtag, using the status Op, and set it + // for future rename recovery. + AbfsHttpOperation sourceStatusResult = sourceStatusOp.getResult(); + sourceEtagAfterFailure = extractEtagHeader(sourceStatusResult); + } + renamePath(source, destination, continuation, tracingContext, + sourceEtagAfterFailure, isMetadataIncompleteState, + isNamespaceEnabled); + } + // if we get out of the condition without a successful rename, then + // it isn't metadata incomplete state issue. + isMetadataIncompleteState = false; + + // setting default rename recovery success to false + boolean etagCheckSucceeded = false; + if (shouldAttemptRecovery) { + etagCheckSucceeded = renameIdempotencyCheckOp( + source, + sourceEtag, op, destination, tracingContext); + } + if (!etagCheckSucceeded) { + // idempotency did not return different result + // throw back the exception + throw e; + } + return new AbfsClientRenameResult(op, true, isMetadataIncompleteState); + } + } + + /** + * Get Rest Operation for API . + * Uploads data to be appended to a file. + * @param path to which data has to be appended. + * @param buffer containing data to be appended. + * @param reqParams containing parameters for append operation like offset, length etc. + * @param cachedSasToken to be used for the authenticating operation. + * @param contextEncryptionAdapter to provide encryption context. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation append(final String path, + final byte[] buffer, + AppendRequestParameters reqParams, + final String cachedSasToken, + ContextEncryptionAdapter contextEncryptionAdapter, + TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + addEncryptionKeyRequestHeaders(path, requestHeaders, false, + contextEncryptionAdapter, tracingContext); + if (reqParams.isExpectHeaderEnabled()) { + requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); + } + requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); + if (reqParams.getLeaseId() != null) { + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, reqParams.getLeaseId())); + } + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, APPEND_ACTION); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, + Long.toString(reqParams.getPosition())); + + if ((reqParams.getMode() == AppendRequestParameters.Mode.FLUSH_MODE) || ( + reqParams.getMode() == AppendRequestParameters.Mode.FLUSH_CLOSE_MODE)) { + abfsUriQueryBuilder.addQuery(QUERY_PARAM_FLUSH, TRUE); + if (reqParams.getMode() == AppendRequestParameters.Mode.FLUSH_CLOSE_MODE) { + abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, TRUE); + } + } + + // Check if the retry is with "Expect: 100-continue" header being present in the previous request. + if (reqParams.isRetryDueToExpect()) { + String userAgentRetry = getUserAgent(); + // Remove the specific marker related to "Expect: 100-continue" from the User-Agent string. + userAgentRetry = userAgentRetry.replace(HUNDRED_CONTINUE_USER_AGENT, EMPTY_STRING); + requestHeaders.removeIf(header -> header.getName().equalsIgnoreCase(USER_AGENT)); + requestHeaders.add(new AbfsHttpHeader(USER_AGENT, userAgentRetry)); + } + + // Add MD5 Hash of request content as request header if feature is enabled + if (isChecksumValidationEnabled()) { + addCheckSumHeaderForWrite(requestHeaders, reqParams, buffer); + } + + // AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance + String sasTokenForReuse = appendSASTokenToQuery(path, + SASTokenProvider.WRITE_OPERATION, + abfsUriQueryBuilder, cachedSasToken); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.Append, + HTTP_METHOD_PUT, url, requestHeaders, + buffer, reqParams.getoffset(), reqParams.getLength(), + sasTokenForReuse); + try { + op.execute(tracingContext); + } catch (AbfsRestOperationException e) { + /* + If the http response code indicates a user error we retry + the same append request with expect header being disabled. + When "100-continue" header is enabled but a non Http 100 response comes, + the response message might not get set correctly by the server. + So, this handling is to avoid breaking of backward compatibility + if someone has taken dependency on the exception message, + which is created using the error string present in the response header. + */ + int responseStatusCode = e.getStatusCode(); + if (checkUserError(responseStatusCode) + && reqParams.isExpectHeaderEnabled()) { + LOG.debug( + "User error, retrying without 100 continue enabled for the given path {}", + path); + reqParams.setExpectHeaderEnabled(false); + reqParams.setRetryDueToExpect(true); + return this.append(path, buffer, reqParams, cachedSasToken, + contextEncryptionAdapter, tracingContext); + } + // If we have no HTTP response, throw the original exception. + if (!op.hasResult()) { + throw e; + } + + if (isMd5ChecksumError(e)) { + throw new AbfsInvalidChecksumException(e); + } + + if (reqParams.isAppendBlob() + && appendSuccessCheckOp(op, path, + (reqParams.getPosition() + reqParams.getLength()), tracingContext)) { + final AbfsRestOperation successOp = getAbfsRestOperation( + AbfsRestOperationType.Append, + HTTP_METHOD_PUT, url, requestHeaders, + buffer, reqParams.getoffset(), reqParams.getLength(), + sasTokenForReuse); + successOp.hardSetResult(HttpURLConnection.HTTP_OK); + return successOp; + } + throw e; + } catch (AzureBlobFileSystemException e) { + // Any server side issue will be returned as AbfsRestOperationException and will be handled above. + LOG.debug( + "Append request failed with non server issues for path: {}, offset: {}, position: {}", + path, reqParams.getoffset(), reqParams.getPosition()); + throw e; + } + + return op; + } + + /** + * Get Rest Operation for API . + * Flush previously uploaded data to a file. + * @param path on which data has to be flushed. + * @param position to which data has to be flushed. + * @param retainUncommittedData whether to retain uncommitted data after flush. + * @param isClose specify if this is the last flush to the file. + * @param cachedSasToken to be used for the authenticating operation. + * @param leaseId if there is an active lease on the path. + * @param contextEncryptionAdapter to provide encryption context. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + public AbfsRestOperation flush(final String path, + final long position, + boolean retainUncommittedData, + boolean isClose, + final String cachedSasToken, + final String leaseId, + ContextEncryptionAdapter contextEncryptionAdapter, + TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + addEncryptionKeyRequestHeaders(path, requestHeaders, false, + contextEncryptionAdapter, tracingContext); + requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); + if (leaseId != null) { + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId)); + } + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, FLUSH_ACTION); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(position)); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RETAIN_UNCOMMITTED_DATA, + String.valueOf(retainUncommittedData)); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, String.valueOf(isClose)); + // AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance + String sasTokenForReuse = appendSASTokenToQuery(path, + SASTokenProvider.WRITE_OPERATION, + abfsUriQueryBuilder, cachedSasToken); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.Flush, + HTTP_METHOD_PUT, url, requestHeaders, + sasTokenForReuse); + op.execute(tracingContext); + return op; + } + + @Override + public AbfsRestOperation flush(byte[] buffer, + final String path, + boolean isClose, + final String cachedSasToken, + final String leaseId, + final String eTag, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + throw new UnsupportedOperationException( + "flush with blockIds not supported on DFS Endpoint"); + } + + /** + * Get Rest Operation for API . + * Set the properties of a file or directory. + * @param path on which properties have to be set. + * @param properties comma separated list of metadata key-value pairs. + * @param tracingContext for tracing the server calls. + * @param contextEncryptionAdapter to provide encryption context. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation setPathProperties(final String path, + final String properties, + final TracingContext tracingContext, + final ContextEncryptionAdapter contextEncryptionAdapter) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + addEncryptionKeyRequestHeaders(path, requestHeaders, false, + contextEncryptionAdapter, tracingContext); + requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, + HTTP_METHOD_PATCH)); + requestHeaders.add(new AbfsHttpHeader(X_MS_PROPERTIES, properties)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, SET_PROPERTIES_ACTION); + appendSASTokenToQuery(path, SASTokenProvider.SET_PROPERTIES_OPERATION, + abfsUriQueryBuilder); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.SetPathProperties, + HTTP_METHOD_PUT, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * Get the properties of a file or directory. + * @param path of which properties have to be fetched. + * @param includeProperties to include user defined properties. + * @param tracingContext for tracing the server calls. + * @param contextEncryptionAdapter to provide encryption context. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation getPathStatus(final String path, + final boolean includeProperties, + final TracingContext tracingContext, + final ContextEncryptionAdapter contextEncryptionAdapter) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + String operation = SASTokenProvider.GET_PROPERTIES_OPERATION; + if (!includeProperties) { + // The default action (operation) is implicitly to get properties and this action requires read permission + // because it reads user defined properties. If the action is getStatus or getAclStatus, then + // only traversal (execute) permission is required. + abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, GET_STATUS); + operation = SASTokenProvider.GET_STATUS_OPERATION; + } else { + addEncryptionKeyRequestHeaders(path, requestHeaders, false, + contextEncryptionAdapter, tracingContext); + } + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN, + String.valueOf(getAbfsConfiguration().isUpnUsed())); + appendSASTokenToQuery(path, operation, abfsUriQueryBuilder); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.GetPathStatus, + HTTP_METHOD_HEAD, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * Read the contents of the file at specified path + * @param path of the file to be read. + * @param position in the file from where data has to be read. + * @param buffer to store the data read. + * @param bufferOffset offset in the buffer to start storing the data. + * @param bufferLength length of data to be read. + * @param eTag to specify conditional headers. + * @param cachedSasToken to be used for the authenticating operation. + * @param contextEncryptionAdapter to provide encryption context. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation read(final String path, + final long position, + final byte[] buffer, + final int bufferOffset, + final int bufferLength, + final String eTag, + String cachedSasToken, + ContextEncryptionAdapter contextEncryptionAdapter, + TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + addEncryptionKeyRequestHeaders(path, requestHeaders, false, + contextEncryptionAdapter, tracingContext); + AbfsHttpHeader rangeHeader = new AbfsHttpHeader(RANGE, + String.format("bytes=%d-%d", position, position + bufferLength - 1)); + requestHeaders.add(rangeHeader); + requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag)); + + // Add request header to fetch MD5 Hash of data returned by server. + if (isChecksumValidationEnabled(requestHeaders, rangeHeader, bufferLength)) { + requestHeaders.add(new AbfsHttpHeader(X_MS_RANGE_GET_CONTENT_MD5, TRUE)); + } + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + // AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance + String sasTokenForReuse = appendSASTokenToQuery(path, + SASTokenProvider.READ_OPERATION, + abfsUriQueryBuilder, cachedSasToken); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.ReadFile, + HTTP_METHOD_GET, url, requestHeaders, + buffer, bufferOffset, bufferLength, + sasTokenForReuse); + op.execute(tracingContext); + + // Verify the MD5 hash returned by server holds valid on the data received. + if (isChecksumValidationEnabled(requestHeaders, rangeHeader, bufferLength)) { + verifyCheckSumForRead(buffer, op.getResult(), bufferOffset); + } + + return op; + } + + /** + * Get Rest Operation for API . + * Delete the file or directory at specified path. + * @param path to be deleted. + * @param recursive if the path is a directory, delete recursively. + * @param continuation to specify continuation token. + * @param tracingContext for tracing the server calls. + * @param isNamespaceEnabled specify if the namespace is enabled. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation deletePath(final String path, + final boolean recursive, + final String continuation, + TracingContext tracingContext, + final boolean isNamespaceEnabled) throws AzureBlobFileSystemException { + /* + * If Pagination is enabled and current API version is old, + * use the minimum required version for pagination. + * If Pagination is enabled and current API version is later than minimum required + * version for pagination, use current version only as azure service is backward compatible. + * If pagination is disabled, use the current API version only. + */ + final List requestHeaders = (isPaginatedDelete(recursive, + isNamespaceEnabled) && getxMsVersion().compareTo( + ApiVersion.AUG_03_2023) < 0) + ? createDefaultHeaders(ApiVersion.AUG_03_2023) + : createDefaultHeaders(); + final AbfsUriQueryBuilder abfsUriQueryBuilder + = createDefaultUriQueryBuilder(); + + if (isPaginatedDelete(recursive, isNamespaceEnabled)) { + // Add paginated query parameter + abfsUriQueryBuilder.addQuery(QUERY_PARAM_PAGINATED, TRUE); + } + + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RECURSIVE, + String.valueOf(recursive)); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation); + String operation = recursive + ? SASTokenProvider.DELETE_RECURSIVE_OPERATION + : SASTokenProvider.DELETE_OPERATION; + appendSASTokenToQuery(path, operation, abfsUriQueryBuilder); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.DeletePath, this, + HTTP_METHOD_DELETE, url, requestHeaders); + try { + op.execute(tracingContext); + } catch (AzureBlobFileSystemException e) { + // If we have no HTTP response, throw the original exception. + if (!op.hasResult()) { + throw e; + } + final AbfsRestOperation idempotencyOp = deleteIdempotencyCheckOp(op); + if (idempotencyOp.getResult().getStatusCode() + == op.getResult().getStatusCode()) { + // idempotency did not return different result + // throw back the exception + throw e; + } else { + return idempotencyOp; + } + } + + return op; + } + + /** + * Get Rest Operation for API . + * @param path on which owner has to be set. + * @param owner to be set. + * @param group to be set. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation setOwner(final String path, + final String owner, + final String group, + TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); + if (owner != null && !owner.isEmpty()) { + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_OWNER, owner)); + } + if (group != null && !group.isEmpty()) { + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_GROUP, group)); + } + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, SET_ACCESS_CONTROL); + appendSASTokenToQuery(path, SASTokenProvider.SET_OWNER_OPERATION, + abfsUriQueryBuilder); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.SetOwner, + HTTP_METHOD_PUT, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * @param path on which permission has to be set. + * @param permission to be set. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation setPermission(final String path, + final String permission, + TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); + requestHeaders.add(new AbfsHttpHeader( + HttpHeaderConfigurations.X_MS_PERMISSIONS, permission)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, SET_ACCESS_CONTROL); + appendSASTokenToQuery(path, SASTokenProvider.SET_PERMISSION_OPERATION, + abfsUriQueryBuilder); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.SetPermissions, + HTTP_METHOD_PUT, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * @param path on which ACL has to be set. + * @param aclSpecString to be set. + * @param eTag to specify conditional headers. Set only if etag matches. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + public AbfsRestOperation setAcl(final String path, + final String aclSpecString, + final String eTag, + 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. + requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, + HTTP_METHOD_PATCH)); + requestHeaders.add( + new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_ACL, aclSpecString)); + if (eTag != null && !eTag.isEmpty()) { + requestHeaders.add( + new AbfsHttpHeader(IF_MATCH, eTag)); + } + + final AbfsUriQueryBuilder abfsUriQueryBuilder + = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, + SET_ACCESS_CONTROL); + appendSASTokenToQuery(path, SASTokenProvider.SET_ACL_OPERATION, + abfsUriQueryBuilder); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.SetAcl, + HTTP_METHOD_PUT, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * Retrieves the properties of blob at specified path. + * @param path of which properties have to be fetched. + * @param useUPN whether to use UPN with rest operation. + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation getAclStatus(final String path, + final boolean useUPN, + TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, GET_ACCESS_CONTROL); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN, + String.valueOf(useUPN)); + appendSASTokenToQuery(path, SASTokenProvider.GET_ACL_OPERATION, + abfsUriQueryBuilder); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.GetAcl, + HTTP_METHOD_HEAD, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Get Rest Operation for API . + * @param path Path for which access check needs to be performed + * @param rwx The permission to be checked on the path + * @param tracingContext for tracing the server calls. + * @return executed rest operation containing response from server. + * @throws AzureBlobFileSystemException if rest operation fails. + */ + @Override + public AbfsRestOperation checkAccess(String path, + String rwx, + TracingContext tracingContext) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, CHECK_ACCESS); + abfsUriQueryBuilder.addQuery(QUERY_FS_ACTION, rwx); + appendSASTokenToQuery(path, SASTokenProvider.CHECK_ACCESS_OPERATION, + abfsUriQueryBuilder); + + URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.CheckAccess, + HTTP_METHOD_HEAD, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Checks if the rest operation results indicate if the path is a directory. + * @param result executed rest operation containing response from server. + * @return True if the path is a directory, False otherwise. + */ + @Override + public boolean checkIsDir(AbfsHttpOperation result) { + String resourceType = result.getResponseHeader( + HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); + return resourceType != null + && resourceType.equalsIgnoreCase(DIRECTORY); + } + + /** + * Returns true if the status code lies in the range of user error. + * @param responseStatusCode http response status code. + * @return True or False. + */ + @Override + public boolean checkUserError(int responseStatusCode) { + return (responseStatusCode >= HttpURLConnection.HTTP_BAD_REQUEST + && responseStatusCode < HttpURLConnection.HTTP_INTERNAL_ERROR); + } +} 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 c696bd8e68639..fc41a1f77939d 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 @@ -223,6 +223,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 + * @throws AzureBlobFileSystemException if the operation fails. */ public void execute(TracingContext tracingContext) throws AzureBlobFileSystemException { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationType.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationType.java index 830297f381b91..589cbc395aa06 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationType.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationType.java @@ -42,4 +42,19 @@ public enum AbfsRestOperationType { DeletePath, CheckAccess, LeasePath, + CreateContainer, + GetContainerProperties, + SetContainerMetadata, + DeleteContainer, + ListBlobs, + PutBlob, + PutBlock, + PutBlockList, + LeaseBlob, + GetBlob, + GetBlockList, + GetBlobProperties, + SetBlobMetadata, + DeleteBlob, + CopyBlob, } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java index f1f2bc8be346f..6d80b44c8c910 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java @@ -77,6 +77,7 @@ public class ExponentialRetryPolicy extends AbfsRetryPolicy { /** * Initializes a new instance of the {@link ExponentialRetryPolicy} class. + * @param maxIoRetries The maximum number of retry attempts. */ public ExponentialRetryPolicy(final int maxIoRetries) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/CachedSASToken.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/CachedSASToken.java index 559c31dedd1a7..180e712bfb73a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/CachedSASToken.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/CachedSASToken.java @@ -54,7 +54,7 @@ public CachedSASToken() { /** * Create instance with specified minimum expiration. SAS tokens are * automatically renewed when their expiration is within this period. - * @param minExpirationInSeconds + * @param minExpirationInSeconds minimum expiration time for SAS. */ public CachedSASToken(long minExpirationInSeconds) { this.minExpirationInSeconds = minExpirationInSeconds; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/IdentityHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/IdentityHandler.java index 7f866925dfd7c..83c05fdf7bae3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/IdentityHandler.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/IdentityHandler.java @@ -30,6 +30,7 @@ public interface IdentityHandler { * Perform lookup from Service Principal's Object ID to Username. * @param originalIdentity AAD object ID. * @return User name, if no name found returns empty string. + * @throws IOException if lookup fails. * */ String lookupForLocalUserIdentity(String originalIdentity) throws IOException; @@ -37,6 +38,7 @@ public interface IdentityHandler { * Perform lookup from Security Group's Object ID to Security Group name. * @param originalIdentity AAD object ID. * @return Security group name, if no name found returns empty string. + * @throws IOException if lookup fails. * */ String lookupForLocalGroupIdentity(String originalIdentity) throws IOException; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsBlobClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsBlobClient.java new file mode 100644 index 0000000000000..f7e9bce7acf72 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsBlobClient.java @@ -0,0 +1,113 @@ +/** + * 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.assertj.core.api.Assertions; +import org.junit.Assume; +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; +import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.utils.Base64; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; + +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT; + +/** + * Test class to test AbfsBlobClient APIs. + * Todo: [FnsOverBlob] - Add more tests to cover all APIs once they are ready + */ +public class ITestAbfsBlobClient extends AbstractAbfsIntegrationTest { + + public ITestAbfsBlobClient() throws Exception { + + } + + @Test + public void testAbfsBlobClient() throws Exception { + try (AzureBlobFileSystem fs = getBlobFileSystem()) { + AbfsClient client = fs.getAbfsStore().getClient(); + Assertions.assertThat(client).isInstanceOf(AbfsBlobClient.class); + // Make sure all client.REST_API_CALLS succeed with right parameters + testClientAPIs(client, getTestTracingContext(fs)); + } catch (AzureBlobFileSystemException ex) { + // Todo: [FnsOverBlob] - Remove this block once complete Blob Endpoint Support is ready. + Assertions.assertThat(ex).isInstanceOf(InvalidConfigurationValueException.class); + } + } + + private void testClientAPIs(AbfsClient client, TracingContext tracingContext) throws Exception { + // 1. Set File System Properties + String val1 = Base64.encode("value1".getBytes()); + String val2 = Base64.encode("value2".getBytes()); + String properties = "key1=" + val1 + ",key2=" + val2; + client.setFilesystemProperties(properties, tracingContext); + + // 2. Get File System Properties + client.getFilesystemProperties(tracingContext); + + // 3. Create Path + client.createPath("/test", true, true, null, false, null, null, tracingContext); + client.createPath("/dir", false, true, null, false, null, null, tracingContext); + client.createPath("/dir/test", true, true, null, false, null, null, tracingContext); + + // 4. List Path + client.listPath("/", false, 5, null, tracingContext); + + // 5. Acquire lease + client.acquireLease("/dir/test", 5, tracingContext); + + // 6. Set Path Properties + client.setPathProperties("/test", properties, tracingContext, null); + + // 7. Get Path Status + client.getPathStatus("/test", true, tracingContext, null); + + // N. Delete File System + client.deleteFilesystem(tracingContext); + } + + private AzureBlobFileSystem getBlobFileSystem() throws Exception { + Configuration rawConfig = new Configuration(this.getFileSystem().getConf()); + + Assume.assumeFalse("Blob Endpoint Works only with FNS Accounts", + rawConfig.getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, true)); + setBlobEndpoint(rawConfig); + + return (AzureBlobFileSystem) FileSystem.newInstance(rawConfig); + } + + private void setBlobEndpoint(Configuration config) { + String defaultFs = config.get(FS_DEFAULT_NAME_KEY); + config.set(FS_DEFAULT_NAME_KEY, defaultFs.replace(".dfs.", ".blob.")); + } + + public TracingContext getTestTracingContext(AzureBlobFileSystem fs) { + String correlationId = "test-corr-id", fsId = "test-filesystem-id"; + TracingHeaderFormat format = TracingHeaderFormat.ALL_ID_FORMAT; + return new TracingContext(correlationId, fsId, FSOperationType.TEST_OP, false, format, 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 e185ab2e75e53..672a317992f32 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,6 +22,7 @@ import java.lang.reflect.Field; import java.util.List; +import org.apache.hadoop.fs.azurebfs.enums.Trilean; import org.apache.hadoop.util.Lists; import org.junit.Assume; import org.junit.Test; @@ -97,8 +98,12 @@ private void setTestUserFs() throws Exception { + getAccountName(), ClientCredsTokenProvider.class.getName()); conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, false); - conf.unset(FS_AZURE_ACCOUNT_IS_HNS_ENABLED); + // Since FS init now needs to know account type setting it before init. + conf.setBoolean(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, isHNSEnabled); this.testUserFs = FileSystem.newInstance(conf); + // Resetting the namespace enabled flag to unknown after file system init. + ((AzureBlobFileSystem)testUserFs).getAbfsStore().setNamespaceEnabled( + Trilean.UNKNOWN); } private void setTestFsConf(final String fsConfKey, @@ -306,11 +311,11 @@ public void testFsActionALL() throws Exception { } private void checkPrerequisites() throws Exception { - setTestUserFs(); Assume.assumeTrue(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false", isHNSEnabled); Assume.assumeTrue(FS_AZURE_ENABLE_CHECK_ACCESS + " is false", isCheckAccessEnabled); + setTestUserFs(); checkIfConfigIsSet(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID); checkIfConfigIsSet(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET); checkIfConfigIsSet(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID); 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 b40e317d2e32d..9e62f15ddec1a 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 @@ -133,12 +133,15 @@ private String getNonExistingUrl() { @Test public void testFailedRequestWhenFSNotExist() throws Exception { + assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT); AbfsConfiguration config = this.getConfiguration(); config.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, false); String testUri = this.getTestUrl(); String nonExistingFsUrl = getAbfsScheme() + "://" + UUID.randomUUID() + testUri.substring(testUri.indexOf("@")); + config.setBoolean(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, isUsingXNSAccount); AzureBlobFileSystem fs = this.getFileSystem(nonExistingFsUrl); + fs.getAbfsStore().setNamespaceEnabled(Trilean.UNKNOWN); intercept(FileNotFoundException.class, "\"The specified filesystem does not exist.\", 404", diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java index ca2ea92388d97..f3a1bf18891cc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java @@ -139,7 +139,8 @@ private String getUserAgentString(AbfsConfiguration config, boolean includeSSLProvider) throws IOException, URISyntaxException { AbfsCounters abfsCounters = Mockito.spy(new AbfsCountersImpl(new URI("abcd"))); AbfsClientContext abfsClientContext = new AbfsClientContextBuilder().withAbfsCounters(abfsCounters).build(); - AbfsClient client = new AbfsClient(new URL("https://azure.com"), null, + // Todo : [FnsOverBlob] Update later to work with Blob Endpoint as well. + AbfsClient client = new AbfsDfsClient(new URL("https://azure.com"), null, config, (AccessTokenProvider) null, null, abfsClientContext); String sslProviderName = null; if (includeSSLProvider) { @@ -336,7 +337,8 @@ public static AbfsClient createTestClientFromCurrentContext( .build(); // Create test AbfsClient - AbfsClient testClient = new AbfsClient( + // Todo: [FnsOverBlob] Update later to work with Blob Endpoint as well. + AbfsClient testClient = new AbfsDfsClient( baseAbfsClientInstance.getBaseUrl(), (currentAuthType == AuthType.SharedKey ? new SharedKeyCredentials( @@ -364,7 +366,8 @@ public static AbfsClient getMockAbfsClient(AbfsClient baseAbfsClientInstance, (currentAuthType == AuthType.SharedKey) || (currentAuthType == AuthType.OAuth)); - AbfsClient client = mock(AbfsClient.class); + // Todo: [FnsOverBlob] Update later to work with Blob Endpoint as well. + AbfsClient client = mock(AbfsDfsClient.class); AbfsPerfTracker tracker = new AbfsPerfTracker( "test", abfsConfig.getAccountName(),