diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index f94fdebd03e195..861a05251b9f33 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -2319,6 +2319,12 @@ The AbstractFileSystem for gs: uris. + + fs.azure.enable.readahead + true + Enabled readahead/prefetching in AbfsInputStream. + + io.seqfile.compress.blocksize 1000000 diff --git a/hadoop-tools/hadoop-azure/.gitignore b/hadoop-tools/hadoop-azure/.gitignore index 0e17efaa1eb24c..522210137ec16d 100644 --- a/hadoop-tools/hadoop-azure/.gitignore +++ b/hadoop-tools/hadoop-azure/.gitignore @@ -1,5 +1,6 @@ .checkstyle bin/ -src/test/resources/combinationConfigFiles src/test/resources/abfs-combination-test-configs.xml dev-support/testlogs +src/test/resources/accountSettings/* +!src/test/resources/accountSettings/accountName_settings.xml.template diff --git a/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/runtests.sh b/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/runtests.sh index 25d9593d573ebf..b0e4b8cf70bff5 100755 --- a/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/runtests.sh +++ b/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/runtests.sh @@ -2,7 +2,7 @@ # shellcheck disable=SC2034 # unused variables are global in nature and used in testsupport.sh - +test set -eo pipefail # Licensed to the Apache Software Foundation (ASF) under one or more @@ -22,36 +22,167 @@ set -eo pipefail # shellcheck disable=SC1091 . dev-support/testrun-scripts/testsupport.sh +init + +resourceDir=src/test/resources/ +logdir=dev-support/testlogs/ +azureTestXml=azure-auth-keys.xml +azureTestXmlPath=$resourceDir$azureTestXml +processCount=8 + +## SECTION: TEST COMBINATION METHODS -begin +runHNSOAuthTest() +{ + accountName=$(xmlstarlet sel -t -v '//property[name = "fs.azure.hnsTestAccountName"]/value' -n $azureTestXmlPath) + PROPERTIES=("fs.azure.account.auth.type") + VALUES=("OAuth") + triggerRun "HNS-OAuth" "$accountName" "$runTest" $processCount "$cleanUpTestContainers" +} -### ADD THE TEST COMBINATIONS BELOW. DO NOT EDIT THE ABOVE LINES. -### THE SCRIPT REQUIRES THE FOLLOWING UTILITIES xmlstarlet AND pcregrep. +runHNSSharedKeyTest() +{ + accountName=$(xmlstarlet sel -t -v '//property[name = "fs.azure.hnsTestAccountName"]/value' -n $azureTestXmlPath) + PROPERTIES=("fs.azure.account.auth.type") + VALUES=("SharedKey") + triggerRun "HNS-SharedKey" "$accountName" "$runTest" $processCount "$cleanUpTestContainers" +} +runNonHNSSharedKeyTest() +{ + accountName=$(xmlstarlet sel -t -v '//property[name = "fs.azure.nonHnsTestAccountName"]/value' -n $azureTestXmlPath) + PROPERTIES=("fs.azure.account.auth.type") + VALUES=("SharedKey") + triggerRun "NonHNS-SharedKey" "$accountName" "$runTest" $processCount "$cleanUpTestContainers" +} -combination=HNS-OAuth -properties=("fs.azure.abfs.account.name" "fs.azure.test.namespace.enabled" -"fs.azure.account.auth.type") -values=("{account name}.dfs.core.windows.net" "true" "OAuth") -generateconfigs +runNonHNSOAuthTest() +{ + accountName=$(xmlstarlet sel -t -v '//property[name = "fs.azure.nonHnsTestAccountName"]/value' -n $azureTestXmlPath) + PROPERTIES=("fs.azure.account.auth.type") + VALUES=("OAuth") + triggerRun "NonHNS-OAuth" "$accountName" "$runTest" $processCount "$cleanUpTestContainers" +} -combination=AppendBlob-HNS-OAuth -properties=("fs.azure.abfs.account.name" "fs.azure.test.namespace.enabled" -"fs.azure.account.auth.type" "fs.azure.test.appendblob.enabled") -values=("{account name}.dfs.core.windows.net" "true" "OAuth" "true") -generateconfigs +runAppendBlobHNSOAuthTest() +{ + accountName=$(xmlstarlet sel -t -v '//property[name = "fs.azure.hnsTestAccountName"]/value' -n $azureTestXmlPath) + PROPERTIES=("fs.azure.account.auth.type" "fs.azure.test.appendblob.enabled") + VALUES=("OAuth" "true") + triggerRun "AppendBlob-HNS-OAuth" "$accountName" "$runTest" $processCount "$cleanUpTestContainers" +} -combination=HNS-SharedKey -properties=("fs.azure.abfs.account.name" "fs.azure.test.namespace.enabled" "fs.azure.account.auth.type") -values=("{account name}.dfs.core.windows.net" "true" "SharedKey") -generateconfigs +runTest=false +cleanUpTestContainers=false +echo 'Ensure below are complete before running script:' +echo '1. Account specific settings file is present.' +echo ' Copy accountName_settings.xml.template to accountName_settings.xml' +echo ' where accountName in copied file name should be the test account name without domain' +echo ' (accountName_settings.xml.template is present in src/test/resources/accountName_settings' +echo ' folder. New account settings file to be added to same folder.)' +echo ' Follow instructions in the template to populate settings correctly for the account' +echo '2. In azure-auth-keys.xml, update properties fs.azure.hnsTestAccountName and fs.azure.nonHnsTestAccountName' +echo ' where accountNames should be the test account names without domain' +echo ' ' +echo ' ' +echo 'Choose action:' +echo '[Note - SET_ACTIVE_TEST_CONFIG will help activate the config for IDE/single test class runs]' +select scriptMode in SET_ACTIVE_TEST_CONFIG RUN_TEST CLEAN_UP_OLD_TEST_CONTAINERS SET_OR_CHANGE_TEST_ACCOUNT PRINT_LOG4J_LOG_PATHS_FROM_LAST_RUN +do + case $scriptMode in + SET_ACTIVE_TEST_CONFIG) + runTest=false + break + ;; + RUN_TEST) + runTest=true + read -r -p "Enter parallel test run process count [default - 8]: " processCount + processCount=${processCount:-8} + break + ;; + CLEAN_UP_OLD_TEST_CONTAINERS) + runTest=false + cleanUpTestContainers=true + break + ;; + SET_OR_CHANGE_TEST_ACCOUNT) + runTest=false + cleanUpTestContainers=false + accountSettingsFile="src/test/resources/azure-auth-keys.xml" + if [[ ! -f "$accountSettingsFile" ]]; + then + logOutput "No settings present. Creating new settings file ($accountSettingsFile) from template" + cp src/test/resources/azure-auth-keys.xml.template $accountSettingsFile + fi -combination=NonHNS-SharedKey -properties=("fs.azure.abfs.account.name" "fs.azure.test.namespace.enabled" "fs.azure.account.auth.type") -values=("{account name}.dfs.core.windows.net" "false" "SharedKey") -generateconfigs + vi $accountSettingsFile + exit 0 + break + ;; + PRINT_LOG4J_LOG_PATHS_FROM_LAST_RUN) + runTest=false + cleanUpTestContainers=false + logFilePaths=/tmp/logPaths + find target/ -name "*output.txt" > $logFilePaths + logOutput "$(cat $logFilePaths)" + rm $logFilePaths + exit 0 + break + ;; + *) logOutput "ERROR: Invalid selection" + ;; + esac +done +## SECTION: COMBINATION DEFINITIONS AND TRIGGER -### DO NOT EDIT THE LINES BELOW. +echo ' ' +echo 'Set the active test combination to run the action:' +select combo in HNS-OAuth HNS-SharedKey nonHNS-SharedKey NonHNS-OAuth AppendBlob-HNS-OAuth AllCombinationsTestRun Quit +do + case $combo in + HNS-OAuth) + runHNSOAuthTest + break + ;; + HNS-SharedKey) + runHNSSharedKeyTest + break + ;; + nonHNS-SharedKey) + runNonHNSSharedKeyTest + break + ;; + NonHNS-OAuth) + runNonHNSOAuthTest + break + ;; + AppendBlob-HNS-OAuth) + runAppendBlobHNSOAuthTest + break + ;; + AllCombinationsTestRun) + if [ $runTest == false ] + then + logOutput "ERROR: Invalid selection for SET_ACTIVE_TEST_CONFIG. This is applicable only for RUN_TEST." + break + fi + runHNSOAuthTest + runHNSSharedKeyTest + runNonHNSSharedKeyTest + runNonHNSOAuthTest + runAppendBlobHNSOAuthTest ## Keep this as the last run scenario always + break + ;; + Quit) + exit 0 + ;; + *) logOutput "ERROR: Invalid selection" + ;; + esac +done -runtests "$@" +if [ $runTest == true ] +then + printAggregate +fi diff --git a/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh b/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh index 5beb8b6df3a459..a54c0af4fa3052 100644 --- a/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh +++ b/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh @@ -15,117 +15,88 @@ # See the License for the specific language governing permissions and # limitations under the License. -testresourcesdir=src/test/resources -combconfsdir=$testresourcesdir/combinationConfigFiles -combtestfile=$testresourcesdir/abfs-combination-test-configs.xml +resourceDir=src/test/resources/ +accountSettingsFolderName=accountSettings +combtestfile=$resourceDir +combtestfile+=abfs-combination-test-configs.xml +logdir=dev-support/testlogs/ -logdir=dev-support/testlogs testresultsregex="Results:(\n|.)*?Tests run:" -testresultsfilename= -starttime= -threadcount= -defaultthreadcount=8 - -properties= -values= - -validate() { - if [ -z "$threadcount" ] ; then - threadcount=$defaultthreadcount - fi - numberegex='^[0-9]+$' - if ! [[ $threadcount =~ $numberegex ]] ; then - echo "Exiting. The script param (threadcount) should be a number" - exit -1 - fi - if [ -z "$combination" ]; then - echo "Exiting. combination cannot be empty" - exit -1 - fi - propertiessize=${#properties[@]} - valuessize=${#values[@]} - if [ "$propertiessize" -lt 1 ] || [ "$valuessize" -lt 1 ] || [ "$propertiessize" -ne "$valuessize" ]; then - echo "Exiting. Both properties and values arrays has to be populated and of same size. Please check for combination $combination" - exit -1 - fi - - for filename in "${combinations[@]}"; do - if [[ ! -f "$combconfsdir/$filename.xml" ]]; then - echo "Exiting. Combination config file ($combconfsdir/$combination.xml) does not exist." - exit -1 - fi - done -} - -checkdependencies() { - if ! [ "$(command -v pcregrep)" ]; then - echo "Exiting. pcregrep is required to run the script." - exit -1 - fi - if ! [ "$(command -v xmlstarlet)" ]; then - echo "Exiting. xmlstarlet is required to run the script." - exit -1 +accountConfigFileSuffix="_settings.xml" +testOutputLogFolder=$logdir +testlogfilename=combinationTestLogFile + +fullRunStartTime=$(date +%s) +STARTTIME=$(date +%s) +ENDTIME=$(date +%s) + +outputFormatOn="\033[0;95m" +outputFormatOff="\033[0m" + +triggerRun() +{ + echo ' ' + combination=$1 + accountName=$2 + runTest=$3 + processcount=$4 + cleanUpTestContainers=$5 + + if [ -z "$accountName" ]; then + logOutput "ERROR: Test account not configured. Re-run the script and choose SET_OR_CHANGE_TEST_ACCOUNT to configure the test account." + exit 1; fi -} - -cleancombinationconfigs() { - rm -rf $combconfsdir - mkdir -p $combconfsdir -} - -generateconfigs() { - combconffile="$combconfsdir/$combination.xml" - rm -rf "$combconffile" - cat > "$combconffile" << ENDOFFILE + accountConfigFile=$accountSettingsFolderName/$accountName$accountConfigFileSuffix + rm -rf $combtestfile + cat > $combtestfile << ENDOFFILE ENDOFFILE - - propertiessize=${#properties[@]} - valuessize=${#values[@]} + propertiessize=${#PROPERTIES[@]} + valuessize=${#VALUES[@]} if [ "$propertiessize" -ne "$valuessize" ]; then - echo "Exiting. Number of properties and values differ for $combination" - exit -1 + logOutput "Exiting. Number of properties and values differ for $combination" + exit 1 fi for ((i = 0; i < propertiessize; i++)); do - key=${properties[$i]} - val=${values[$i]} + key=${PROPERTIES[$i]} + val=${VALUES[$i]} + echo "Combination specific property setting: [ key=$key , value=$val ]" changeconf "$key" "$val" done - formatxml "$combconffile" -} - -formatxml() { - xmlstarlet fo -s 2 "$1" > "$1.tmp" - mv "$1.tmp" "$1" -} - -setactiveconf() { - if [[ ! -f "$combconfsdir/$combination.xml" ]]; then - echo "Exiting. Combination config file ($combconfsdir/$combination.xml) does not exist." - exit -1 - fi - rm -rf $combtestfile - cat > $combtestfile << ENDOFFILE - - - -ENDOFFILE + formatxml "$combtestfile" xmlstarlet ed -P -L -s /configuration -t elem -n include -v "" $combtestfile - xmlstarlet ed -P -L -i /configuration/include -t attr -n href -v "combinationConfigFiles/$combination.xml" $combtestfile + xmlstarlet ed -P -L -i /configuration/include -t attr -n href -v "$accountConfigFile" $combtestfile xmlstarlet ed -P -L -i /configuration/include -t attr -n xmlns -v "http://www.w3.org/2001/XInclude" $combtestfile formatxml $combtestfile -} + echo ' ' + echo "Activated [$combtestfile] - for account: $accountName for combination $combination" + testlogfilename="$testOutputLogFolder/Test-Logs-$combination.txt" + touch "$testlogfilename" -changeconf() { - xmlstarlet ed -P -L -d "/configuration/property[name='$1']" "$combconffile" - xmlstarlet ed -P -L -s /configuration -t elem -n propertyTMP -v "" -s /configuration/propertyTMP -t elem -n name -v "$1" -r /configuration/propertyTMP -v property "$combconffile" - if ! xmlstarlet ed -P -L -s "/configuration/property[name='$1']" -t elem -n value -v "$2" "$combconffile" + if [ "$runTest" == true ] + then + STARTTIME=$(date +%s) + echo "Running test for combination $combination on account $accountName [ProcessCount=$processcount]" + logOutput "Test run report can be seen in $testlogfilename" + mvn -T 1C -Dparallel-tests=abfs -Dscale -DtestsThreadCount="$processcount" verify >> "$testlogfilename" || true + ENDTIME=$(date +%s) + summary + fi + + if [ "$cleanUpTestContainers" == true ] then - echo "Exiting. Changing config property failed." - exit -1 + mvn test -Dtest=org.apache.hadoop.fs.azurebfs.utils.CleanupTestContainers >> "$testlogfilename" || true + if grep -q "There are test failures" "$testlogfilename"; + then logOutput "ERROR: All test containers could not be deleted. Detailed error cause in $testlogfilename" + pcregrep -M "$testresultsregex" "$testlogfilename" + exit 0 + fi + + logOutput "Delete test containers - complete. Test run logs in - $testlogfilename" fi + } summary() { @@ -134,109 +105,68 @@ summary() { echo "$combination" echo "========================" pcregrep -M "$testresultsregex" "$testlogfilename" - } >> "$testresultsfilename" + } >> "$aggregatedTestResult" printf "\n----- Test results -----\n" pcregrep -M "$testresultsregex" "$testlogfilename" - secondstaken=$((ENDTIME - STARTTIME)) mins=$((secondstaken / 60)) secs=$((secondstaken % 60)) printf "\nTime taken: %s mins %s secs.\n" "$mins" "$secs" - echo "Find test logs for the combination ($combination) in: $testlogfilename" - echo "Find consolidated test results in: $testresultsfilename" - echo "----------" + echo "Find test result for the combination ($combination) in: $testlogfilename" + logOutput "Consolidated test result is saved in: $aggregatedTestResult" + echo "------------------------" } -init() { - checkdependencies - if ! mvn clean install -DskipTests - then - echo "" - echo "Exiting. Build failed." - exit -1 +checkdependencies() { + if ! [ "$(command -v pcregrep)" ]; then + logOutput "Exiting. pcregrep is required to run the script." + exit 1 fi - starttime=$(date +"%Y-%m-%d_%H-%M-%S") - mkdir -p "$logdir" - testresultsfilename="$logdir/$starttime/Test-Results.txt" - if [[ -z "$combinations" ]]; then - combinations=( $( ls $combconfsdir/*.xml )) + if ! [ "$(command -v xmlstarlet)" ]; then + logOutput "Exiting. xmlstarlet is required to run the script." + exit 1 fi } -runtests() { - parseoptions "$@" - validate - if [ -z "$starttime" ]; then - init - fi - shopt -s nullglob - for combconffile in "${combinations[@]}"; do - STARTTIME=$(date +%s) - combination=$(basename "$combconffile" .xml) - mkdir -p "$logdir/$starttime" - testlogfilename="$logdir/$starttime/Test-Logs-$combination.txt" - printf "\nRunning the combination: %s..." "$combination" - setactiveconf - mvn -T 1C -Dparallel-tests=abfs -Dscale -DtestsThreadCount=$threadcount verify >> "$testlogfilename" || true - ENDTIME=$(date +%s) - summary - done +formatxml() { + xmlstarlet fo -s 2 "$1" > "$1.tmp" + mv "$1.tmp" "$1" } -begin() { - cleancombinationconfigs +changeconf() { + xmlstarlet ed -P -L -d "/configuration/property[name='$1']" "$combtestfile" + xmlstarlet ed -P -L -s /configuration -t elem -n propertyTMP -v "" -s /configuration/propertyTMP -t elem -n name -v "$1" -r /configuration/propertyTMP -v property "$combtestfile" + if ! xmlstarlet ed -P -L -s "/configuration/property[name='$1']" -t elem -n value -v "$2" "$combtestfile" + then + logOutput "Exiting. Changing config property failed." + exit 1 + fi } -parseoptions() { -runactivate=0 -runtests=0 - while getopts ":c:a:t:" option; do - case "${option}" in - a) - if [[ "$runactivate" -eq "1" ]]; then - echo "-a Option is not multivalued" - exit 1 - fi - runactivate=1 - combination=$(basename "$OPTARG" .xml) - ;; - c) - runtests=1 - combination=$(basename "$OPTARG" .xml) - combinations+=("$combination") - ;; - t) - threadcount=$OPTARG - ;; - *|?|h) - # shellcheck disable=SC2128 - if [[ -z "$combinations" ]]; then - combinations=( $( ls $combconfsdir/*.xml )) - fi - combstr="" - for combconffile in "${combinations[@]}"; do - combname=$(basename "$combconffile" .xml) - combstr="${combname}, ${combstr}" - done - combstr=${combstr:0:-2} - - echo "Usage: $0 [-n] [-a COMBINATION_NAME] [-c COMBINATION_NAME] [-t THREAD_COUNT]" - echo "" - echo "Where:" - echo " -a COMBINATION_NAME Specify the combination name which needs to be activated." - echo " Configured combinations: ${combstr}" - echo " -c COMBINATION_NAME Specify the combination name for test runs" - echo " -t THREAD_COUNT Specify the thread count" - exit 1 - ;; - esac - done - if [[ "$runactivate" -eq "1" && "$runtests" -eq "1" ]]; then - echo "Both activate (-a option) and test run combinations (-c option) cannot be specified together" +init() { + checkdependencies + if ! mvn clean install -DskipTests + then + echo "" + echo "Exiting. Build failed." exit 1 fi - if [[ "$runactivate" -eq "1" ]]; then - setactiveconf - exit 0 - fi -} + starttime=$(date +"%Y-%m-%d_%H-%M-%S") + testOutputLogFolder+=$starttime + mkdir -p "$testOutputLogFolder" + aggregatedTestResult="$testOutputLogFolder/Test-Results.txt" + } + + printAggregate() { + echo :::: AGGREGATED TEST RESULT :::: + cat "$aggregatedTestResult" + fullRunEndTime=$(date +%s) + fullRunTimeInSecs=$((fullRunEndTime - fullRunStartTime)) + mins=$((fullRunTimeInSecs / 60)) + secs=$((fullRunTimeInSecs % 60)) + printf "\nTime taken: %s mins %s secs.\n" "$mins" "$secs" + } + +logOutput() { + echo -e "$outputFormatOn" "$1" "$outputFormatOff" +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml index 546d28d5e8e624..86135f011ee669 100644 --- a/hadoop-tools/hadoop-azure/pom.xml +++ b/hadoop-tools/hadoop-azure/pom.xml @@ -158,6 +158,7 @@ com.microsoft.azure azure-storage + 7.1.0-Preview compile @@ -259,8 +260,23 @@ org.mockito mockito-core + 4.11.0 test + + + org.mockito + mockito-inline + 4.11.0 + test + + + org.mockito + mockito-core + + + + org.apache.hadoop hadoop-minikdc @@ -556,8 +572,9 @@ **/azurebfs/extensions/ITestAbfsDelegationTokens.java **/azurebfs/ITestSmallWriteOptimization.java **/azurebfs/services/ITestReadBufferManager.java + **/azurebfs/ITestAbfsMsiTokenProvider.java + **/azurebfs/ITestAbfsRestOperationException.java - @@ -597,6 +614,8 @@ **/azurebfs/extensions/ITestAbfsDelegationTokens.java **/azurebfs/ITestSmallWriteOptimization.java **/azurebfs/services/ITestReadBufferManager.java + **/azurebfs/ITestAbfsMsiTokenProvider.java + **/azurebfs/ITestAbfsRestOperationException.java diff --git a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml index 070c8c1fe827ae..2065746b766116 100644 --- a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml +++ b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml @@ -48,4 +48,11 @@ files="org[\\/]apache[\\/]hadoop[\\/]fs[\\/]azurebfs[\\/]utils[\\/]Base64.java"/> + + + + diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java index 39127712f84087..3594a45d2c357e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java @@ -18,6 +18,20 @@ package org.apache.hadoop.fs.azure; import static org.apache.hadoop.fs.azure.NativeAzureFileSystem.PATH_DELIMITER; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CUSTOM_TOKEN_FETCH_RETRY_COUNT; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN_ENDPOINT; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_USER_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_USER_PASSWORD; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_CUSTOM_TOKEN_FETCH_RETRY_COUNT; import java.io.DataOutputStream; import java.io.IOException; @@ -58,15 +72,34 @@ import org.apache.hadoop.fs.azure.metrics.BandwidthGaugeUpdater; import org.apache.hadoop.fs.azure.metrics.ErrorMetricUpdater; import org.apache.hadoop.fs.azure.metrics.ResponseReceivedMetricUpdater; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.AuthConfigurations; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException; +import org.apache.hadoop.fs.azurebfs.extensions.CustomTokenProviderAdaptee; +import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; +import org.apache.hadoop.fs.azurebfs.oauth2.AzureADToken; +import org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider; +import org.apache.hadoop.fs.azurebfs.oauth2.CustomTokenProviderAdapter; +import org.apache.hadoop.fs.azurebfs.oauth2.MsiTokenProvider; +import org.apache.hadoop.fs.azurebfs.oauth2.RefreshTokenBasedTokenProvider; +import org.apache.hadoop.fs.azurebfs.oauth2.UserPasswordTokenProvider; +import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.VersionInfo; + +import com.microsoft.azure.storage.StorageCredentialsToken; +import com.microsoft.azure.storage.blob.CloudBlobClient; import org.eclipse.jetty.util.ajax.JSON; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.kerby.kerberos.kerb.provider.TokenProvider; + import com.microsoft.azure.storage.CloudStorageAccount; import com.microsoft.azure.storage.OperationContext; import com.microsoft.azure.storage.RetryExponentialRetry; @@ -115,7 +148,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore { * Configuration for User-Agent field. */ static final String USER_AGENT_ID_KEY = "fs.azure.user.agent.prefix"; - static final String USER_AGENT_ID_DEFAULT = "unknown"; + static final String USER_AGENT_ID_DEFAULT = "version1"; public static final Logger LOG = LoggerFactory.getLogger(AzureNativeFileSystemStore.class); @@ -358,6 +391,8 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore { private TestHookOperationContext testHookOperationContext = null; + private AccessTokenProvider tokenProvider = null; + // Set if we're running against a storage emulator.. private boolean isStorageEmulator = false; @@ -418,6 +453,206 @@ private void suppressRetryPolicyInClientIfNeeded() { storageInteractionLayer.setRetryPolicyFactory(new RetryNoRetry()); } } + public String accountConf(String key) throws URISyntaxException { + return key + "." + getAccountFromAuthority(sessionUri); + } + + private > T getEnum(String name, T defaultValue) + throws URISyntaxException { + return sessionConfiguration.getEnum(accountConf(name), + sessionConfiguration.getEnum(name, defaultValue)); + } + + private String getPasswordString(String key) + throws IOException, URISyntaxException { + char[] passchars = sessionConfiguration.getPassword(accountConf(key)); + if (passchars == null) { + passchars = sessionConfiguration.getPassword(key); + } + if (passchars != null) { + return new String(passchars); + } + return null; + } + + private String getMandatoryPasswordString(String key) + throws IOException, URISyntaxException { + String value = getPasswordString(key); + if (value == null) { + throw new ConfigurationPropertyNotFoundException(key); + } + return value; + } + + private Class getAccountSpecificClass(String name, + Class defaultValue, + Class xface) throws URISyntaxException { + return sessionConfiguration.getClass(accountConf(name), + defaultValue, + xface); + } + + private > T getAccountAgnosticEnum(String name, T defaultValue) { + return sessionConfiguration.getEnum(name, defaultValue); + } + + private int getCustomTokenFetchRetryCount() { + return sessionConfiguration.getInt(AZURE_CUSTOM_TOKEN_FETCH_RETRY_COUNT, + DEFAULT_CUSTOM_TOKEN_FETCH_RETRY_COUNT); + } + + private Class getAccountAgnosticClass(String name, + Class defaultValue, + Class xface) { + return sessionConfiguration.getClass(name, defaultValue, xface); + } + + + private Class getTokenProviderClass(AuthType authType, + String name, + Class defaultValue, + Class xface) throws URISyntaxException { + Class tokenProviderClass = getAccountSpecificClass(name, defaultValue, + xface); + + // If there is none set specific for account + // fall back to generic setting if Auth Type matches + if ((tokenProviderClass == null) + && (authType == getAccountAgnosticEnum( + FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey))) { + tokenProviderClass = getAccountAgnosticClass(name, defaultValue, xface); + } + + return (tokenProviderClass == null) + ? null + : tokenProviderClass.asSubclass(xface); + } + + private String getTrimmedPasswordString(String key, String defaultValue) + throws IOException, URISyntaxException { + String value = getPasswordString(key); + if (StringUtils.isBlank(value)) { + value = defaultValue; + } + return value.trim(); + } + + private String appendSlashIfNeeded(String authority) { + if (!authority.endsWith(AbfsHttpConstants.FORWARD_SLASH)) { + authority = authority + AbfsHttpConstants.FORWARD_SLASH; + } + return authority; + } + + public AccessTokenProvider getTokenProviderStore() { + return tokenProvider; + } + + public AccessTokenProvider getTokenProvider() throws + TokenAccessProviderException, URISyntaxException { + AuthType authType = getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey); + if (authType == AuthType.OAuth) { + LOG.debug("The auth type is " + authType); + try { + Class tokenProviderClass = + getTokenProviderClass(authType, + FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME, null, + AccessTokenProvider.class); + + AccessTokenProvider tokenProvider; + LOG.debug("The token provider class is " + tokenProviderClass); + if (tokenProviderClass == ClientCredsTokenProvider.class) { + String authEndpoint = + getMandatoryPasswordString( + FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT); + String clientId = + getMandatoryPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID); + String clientSecret = + getMandatoryPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET); + tokenProvider = new ClientCredsTokenProvider(authEndpoint, clientId, + clientSecret); + LOG.debug("ClientCredsTokenProvider initialized"); + } else if (tokenProviderClass == UserPasswordTokenProvider.class) { + String authEndpoint = + getMandatoryPasswordString( + FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT); + String username = + getMandatoryPasswordString(FS_AZURE_ACCOUNT_OAUTH_USER_NAME); + String password = + getMandatoryPasswordString(FS_AZURE_ACCOUNT_OAUTH_USER_PASSWORD); + tokenProvider = new UserPasswordTokenProvider(authEndpoint, username, + password); + LOG.debug("UserPasswordTokenProvider initialized"); + } else if (tokenProviderClass == MsiTokenProvider.class) { + String authEndpoint = getTrimmedPasswordString( + FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT, + AuthConfigurations.DEFAULT_FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT); + String tenantGuid = + getMandatoryPasswordString(FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT); + String clientId = + getMandatoryPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID); + String authority = getTrimmedPasswordString( + FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY, + AuthConfigurations.DEFAULT_FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY); + authority = appendSlashIfNeeded(authority); + tokenProvider = new MsiTokenProvider(authEndpoint, tenantGuid, + clientId, authority); + LOG.debug("MsiTokenProvider initialized"); + } else if (tokenProviderClass == RefreshTokenBasedTokenProvider.class) { + String authEndpoint = getTrimmedPasswordString( + FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN_ENDPOINT, + AuthConfigurations.DEFAULT_FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN_ENDPOINT); + String refreshToken = + getMandatoryPasswordString(FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN); + String clientId = + getMandatoryPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID); + tokenProvider = new RefreshTokenBasedTokenProvider(authEndpoint, + clientId, refreshToken); + LOG.debug("RefreshTokenBasedTokenProvider initialized"); + } else { + throw new IllegalArgumentException( + "Failed to initialize " + tokenProviderClass); + } + return tokenProvider; + } catch (IllegalArgumentException e) { + LOG.debug("Exception while returning token provider ", e); + throw e; + } catch (Exception e) { + LOG.debug("Exception while returning token provider ", e); + throw new TokenAccessProviderException( + "Unable to load OAuth token provider class.", e); + } + }else if (authType == AuthType.Custom) { + LOG.debug("The auth type is " + authType); + try { + String configKey = FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME; + + Class customTokenProviderClass + = getTokenProviderClass(authType, configKey, null, + CustomTokenProviderAdaptee.class); + + if (customTokenProviderClass == null) { + LOG.debug("Exception while returning custom token provider class"); + throw new IllegalArgumentException( + String.format("The configuration value for \"%s\" is invalid.", configKey)); + } + CustomTokenProviderAdaptee azureTokenProvider = ReflectionUtils + .newInstance(customTokenProviderClass, sessionConfiguration); + if (azureTokenProvider == null) { + throw new IllegalArgumentException("Failed to initialize " + customTokenProviderClass); + } + LOG.debug("Initializing {}", customTokenProviderClass.getName()); + azureTokenProvider.initialize(sessionConfiguration, getAccountFromAuthority(sessionUri)); + LOG.debug("{} init complete", customTokenProviderClass.getName()); + return new CustomTokenProviderAdapter(azureTokenProvider, getCustomTokenFetchRetryCount()); + } catch(IllegalArgumentException e) { + throw e; + } catch (Exception e) { + throw new TokenAccessProviderException("Unable to load custom token provider class: " + e, e); + } + } + return null; + } /** * Creates a JSON serializer that can serialize a PermissionStatus object into @@ -511,13 +746,13 @@ private boolean isConcurrentOOBAppendAllowed() { */ @Override public void initialize(URI uri, Configuration conf, AzureFileSystemInstrumentation instrumentation) - throws IllegalArgumentException, AzureException, IOException { + throws IllegalArgumentException, AzureException, IOException, + URISyntaxException { if (null == instrumentation) { throw new IllegalArgumentException("Null instrumentation"); } this.instrumentation = instrumentation; - // Check that URI exists. // if (null == uri) { @@ -543,7 +778,9 @@ public void initialize(URI uri, Configuration conf, AzureFileSystemInstrumentati // sessionUri = uri; sessionConfiguration = conf; - + if (tokenProvider == null) { + tokenProvider = getTokenProvider(); + } useSecureMode = conf.getBoolean(KEY_USE_SECURE_MODE, DEFAULT_USE_SECURE_MODE); useLocalSasKeyMode = conf.getBoolean(KEY_USE_LOCAL_SAS_KEY_MODE, @@ -570,7 +807,7 @@ public void initialize(URI uri, Configuration conf, AzureFileSystemInstrumentati LOG.debug("Page blob directories: {}", setToString(pageBlobDirs)); // User-agent - userAgentId = conf.get(USER_AGENT_ID_KEY, USER_AGENT_ID_DEFAULT); + userAgentId = "wasbdriverV2.5"; // Extract the directories that should contain block blobs with compaction blockBlobWithCompationDirs = getDirectorySet( @@ -593,8 +830,7 @@ public void initialize(URI uri, Configuration conf, AzureFileSystemInstrumentati LOG.warn("Unable to initialize HBase root as an atomic rename directory."); } LOG.debug("Atomic rename directories: {} ", setToString(atomicRenameDirs)); - metadataKeyCaseSensitive = conf - .getBoolean(KEY_BLOB_METADATA_KEY_CASE_SENSITIVE, true); + metadataKeyCaseSensitive = false; if (!metadataKeyCaseSensitive) { LOG.info("{} configured as false. Blob metadata will be treated case insensitive.", KEY_BLOB_METADATA_KEY_CASE_SENSITIVE); @@ -843,8 +1079,9 @@ private void configureAzureStorageSession() throws AzureException { autoThrottlingEnabled = false; } - OperationContext.setLoggingEnabledByDefault(sessionConfiguration. - getBoolean(KEY_ENABLE_STORAGE_CLIENT_LOGGING, false)); + if (LOG.isDebugEnabled()) { + OperationContext.setLoggingEnabledByDefault(true); + } LOG.debug( "AzureNativeFileSystemStore init. Settings={},{},{},{{},{},{},{}},{{},{},{}}", @@ -907,7 +1144,7 @@ private void connectUsingAnonymousCredentials(final URI uri) private void connectUsingCredentials(String accountName, StorageCredentials credentials, String containerName) - throws URISyntaxException, StorageException, AzureException { + throws URISyntaxException, StorageException, IOException { URI blobEndPoint; if (isStorageEmulatorAccount(accountName)) { @@ -921,10 +1158,14 @@ private void connectUsingCredentials(String accountName, } suppressRetryPolicyInClientIfNeeded(); + if (tokenProvider != null) { + container = storageInteractionLayer.getContainerReference(containerName, tokenProvider); + rootDirectory = container.getDirectoryReference("", tokenProvider); + } else { + container = storageInteractionLayer.getContainerReference(containerName); + rootDirectory = container.getDirectoryReference(""); + } // Capture the container reference for debugging purposes. - container = storageInteractionLayer.getContainerReference(containerName); - rootDirectory = container.getDirectoryReference(""); - // Can only create container if using account key credentials canCreateOrModifyContainer = credentials instanceof StorageCredentialsAccountAndKey; } @@ -937,7 +1178,7 @@ private void connectUsingCredentials(String accountName, */ private void connectToAzureStorageInSecureMode(String accountName, String containerName, URI sessionUri) - throws AzureException, StorageException, URISyntaxException { + throws IOException, StorageException, URISyntaxException { LOG.debug("Connecting to Azure storage in Secure Mode"); // Assertion: storageInteractionLayer instance has to be a SecureStorageInterfaceImpl @@ -982,6 +1223,13 @@ private void connectUsingSASCredentials(final String accountName, connectUsingCredentials(accountName, credentials, containerName); } + private void connectUsingOAuthCredentials(final String accountName, + final String containerName, AzureADToken token) throws InvalidKeyException, + StorageException, IOException, URISyntaxException { + StorageCredentials credentials = new StorageCredentialsToken(accountName, token.getAccessToken()); + connectUsingCredentials(accountName, credentials, containerName); + } + private boolean isStorageEmulatorAccount(final String accountName) { return accountName.equalsIgnoreCase(sessionConfiguration.get( STORAGE_EMULATOR_ACCOUNT_NAME_PROPERTY_NAME, @@ -1093,11 +1341,16 @@ private void createAzureStorageSession() connectToAzureStorageInSecureMode(accountName, containerName, sessionUri); return; } + if (tokenProvider != null){ + connectUsingOAuthCredentials(accountName, containerName, tokenProvider.getToken()); + return; + } // Check whether we have a shared access signature for that container. String propertyValue = sessionConfiguration.get(KEY_ACCOUNT_SAS_PREFIX + containerName + "." + accountName); if (propertyValue != null) { + LOG.debug("The SAS config property value is not null"); // SAS was found. Connect using that. connectUsingSASCredentials(accountName, containerName, propertyValue); return; @@ -1106,17 +1359,18 @@ private void createAzureStorageSession() // Check whether the account is configured with an account key. propertyValue = getAccountKeyFromConfiguration(accountName, sessionConfiguration); - if (StringUtils.isNotEmpty(propertyValue)) { - // Account key was found. - // Create the Azure storage session using the account key and container. - connectUsingConnectionStringCredentials( - getAccountFromAuthority(sessionUri), - getContainerFromAuthority(sessionUri), propertyValue); - } else { - LOG.debug("The account access key is not configured for {}. " - + "Now try anonymous access.", sessionUri); - connectUsingAnonymousCredentials(sessionUri); - } + if (StringUtils.isNotEmpty(propertyValue)) { + LOG.debug("The Account Key config property value is not null"); + // Account key was found. + // Create the Azure storage session using the account key and container. + connectUsingConnectionStringCredentials( + getAccountFromAuthority(sessionUri), + getContainerFromAuthority(sessionUri), propertyValue); + } else { + LOG.debug("The account access key is not configured for {}. " + + "Now try anonymous access.", sessionUri); + connectUsingAnonymousCredentials(sessionUri); + } } catch (Exception e) { // Caught exception while attempting to initialize the Azure File // System store, re-throw the exception. @@ -1340,7 +1594,7 @@ public long getHadoopBlockSize() { * container's metadata if it's not already there. */ private ContainerState checkContainer(ContainerAccessType accessType) - throws StorageException, AzureException { + throws StorageException, IOException { synchronized (containerStateLock) { if (isOkContainerState(accessType)) { return currentKnownContainerState; @@ -1372,6 +1626,8 @@ private ContainerState checkContainer(ContainerAccessType accessType) } else { throw ex; } + } catch (IOException e) { + e.printStackTrace(); } if (currentKnownContainerState == ContainerState.DoesntExist) { @@ -1417,7 +1673,8 @@ private ContainerState checkContainer(ContainerAccessType accessType) } } - private AzureException wrongVersionException(String containerVersion) { + private AzureException wrongVersionException(String containerVersion) + throws IOException { return new AzureException("The container " + container.getName() + " is at an unsupported version: " + containerVersion + ". Current supported version: " + FIRST_WASB_VERSION); @@ -1585,7 +1842,7 @@ public DataOutputStream storefile(String keyEncoded, * to populate it from scratch with data. */ private OutputStream openOutputStream(final CloudBlobWrapper blob) - throws StorageException { + throws StorageException, IOException { if (blob instanceof CloudPageBlobWrapper){ return new PageBlobOutputStream( (CloudPageBlobWrapper) blob, getInstrumentedContext(), sessionConfiguration); @@ -1638,7 +1895,7 @@ private static PermissionStatus defaultPermissionNoBlobMetadata() { } private static void storeMetadataAttribute(CloudBlobWrapper blob, - String key, String value) { + String key, String value) throws IOException { HashMap metadata = blob.getMetadata(); if (null == metadata) { metadata = new HashMap(); @@ -1671,7 +1928,7 @@ private String getMetadataAttribute(HashMap metadata, } private static void removeMetadataAttribute(CloudBlobWrapper blob, - String key) { + String key) throws IOException { HashMap metadata = blob.getMetadata(); if (metadata != null) { metadata.remove(key); @@ -1680,14 +1937,15 @@ private static void removeMetadataAttribute(CloudBlobWrapper blob, } private static void storePermissionStatus(CloudBlobWrapper blob, - PermissionStatus permissionStatus) { + PermissionStatus permissionStatus) throws IOException { storeMetadataAttribute(blob, PERMISSION_METADATA_KEY, PERMISSION_JSON_SERIALIZER.toJSON(permissionStatus)); // Remove the old metadata key if present removeMetadataAttribute(blob, OLD_PERMISSION_METADATA_KEY); } - private PermissionStatus getPermissionStatus(CloudBlobWrapper blob) { + private PermissionStatus getPermissionStatus(CloudBlobWrapper blob) + throws IOException { String permissionMetadataValue = getMetadataAttribute(blob.getMetadata(), PERMISSION_METADATA_KEY, OLD_PERMISSION_METADATA_KEY); if (permissionMetadataValue != null) { @@ -1698,7 +1956,8 @@ private PermissionStatus getPermissionStatus(CloudBlobWrapper blob) { } } - private static void storeFolderAttribute(CloudBlobWrapper blob) { + private static void storeFolderAttribute(CloudBlobWrapper blob) + throws IOException { storeMetadataAttribute(blob, IS_FOLDER_METADATA_KEY, "true"); // Remove the old metadata key if present removeMetadataAttribute(blob, OLD_IS_FOLDER_METADATA_KEY); @@ -1726,7 +1985,7 @@ private static String ensureValidAttributeName(String attribute) { } private static void storeLinkAttribute(CloudBlobWrapper blob, - String linkTarget) throws UnsupportedEncodingException { + String linkTarget) throws IOException { String encodedLinkTarget = encodeMetadataAttribute(linkTarget); storeMetadataAttribute(blob, LINK_BACK_TO_UPLOAD_IN_PROGRESS_METADATA_KEY, @@ -1737,14 +1996,15 @@ private static void storeLinkAttribute(CloudBlobWrapper blob, } private String getLinkAttributeValue(CloudBlobWrapper blob) - throws UnsupportedEncodingException { + throws IOException { String encodedLinkTarget = getMetadataAttribute(blob.getMetadata(), LINK_BACK_TO_UPLOAD_IN_PROGRESS_METADATA_KEY, OLD_LINK_BACK_TO_UPLOAD_IN_PROGRESS_METADATA_KEY); return decodeMetadataAttribute(encodedLinkTarget); } - private boolean retrieveFolderAttribute(CloudBlobWrapper blob) { + private boolean retrieveFolderAttribute(CloudBlobWrapper blob) + throws IOException { HashMap metadata = blob.getMetadata(); if (null != metadata) { if (metadataKeyCaseSensitive) { @@ -1764,7 +2024,8 @@ private boolean retrieveFolderAttribute(CloudBlobWrapper blob) { return false; } - private static void storeVersionAttribute(CloudBlobContainerWrapper container) { + private static void storeVersionAttribute(CloudBlobContainerWrapper container) + throws IOException { HashMap metadata = container.getMetadata(); if (null == metadata) { metadata = new HashMap(); @@ -1776,7 +2037,8 @@ private static void storeVersionAttribute(CloudBlobContainerWrapper container) { container.setMetadata(metadata); } - private String retrieveVersionAttribute(CloudBlobContainerWrapper container) { + private String retrieveVersionAttribute(CloudBlobContainerWrapper container) + throws IOException { return getMetadataAttribute(container.getMetadata(), VERSION_METADATA_KEY, OLD_VERSION_METADATA_KEY); } @@ -1927,7 +2189,8 @@ private boolean isAuthenticatedAccess() throws AzureException { * */ private Iterable listRootBlobs(boolean includeMetadata, - boolean useFlatBlobListing) throws StorageException, URISyntaxException { + boolean useFlatBlobListing) + throws StorageException, URISyntaxException, IOException { return rootDirectory.listBlobs( null, useFlatBlobListing, @@ -1958,7 +2221,8 @@ private Iterable listRootBlobs(boolean includeMetadata, * */ private Iterable listRootBlobs(String aPrefix, boolean includeMetadata, - boolean useFlatBlobListing) throws StorageException, URISyntaxException { + boolean useFlatBlobListing) + throws StorageException, URISyntaxException, IOException { Iterable list = rootDirectory.listBlobs(aPrefix, useFlatBlobListing, @@ -1996,7 +2260,8 @@ private Iterable listRootBlobs(String aPrefix, boolean includeMeta */ private Iterable listRootBlobs(String aPrefix, boolean useFlatBlobListing, EnumSet listingDetails, BlobRequestOptions options, - OperationContext opContext) throws StorageException, URISyntaxException { + OperationContext opContext) + throws StorageException, URISyntaxException, IOException { CloudBlobDirectoryWrapper directory = this.container.getDirectoryReference(aPrefix); return directory.listBlobs( @@ -2021,13 +2286,21 @@ private Iterable listRootBlobs(String aPrefix, boolean useFlatBlob * */ private CloudBlobWrapper getBlobReference(String aKey) - throws StorageException, URISyntaxException { + throws StorageException, URISyntaxException, IOException { CloudBlobWrapper blob = null; if (isPageBlobKey(aKey)) { - blob = this.container.getPageBlobReference(aKey); + if (tokenProvider != null) { + blob = this.container.getPageBlobReference(aKey, tokenProvider); + } else { + blob = this.container.getPageBlobReference(aKey); + } } else { - blob = this.container.getBlockBlobReference(aKey); + if (tokenProvider != null) { + blob = this.container.getBlockBlobReference(aKey, tokenProvider); + } else { + blob = this.container.getBlockBlobReference(aKey); + } blob.setStreamMinimumReadSizeInBytes(downloadBlockSizeBytes); blob.setWriteBlockSizeInBytes(uploadBlockSizeBytes); } @@ -2666,16 +2939,16 @@ private long getDataLength(CloudBlobWrapper blob, BlobProperties properties) * @param lease Azure blob lease, or null if no lease is to be used. * @throws StorageException */ - private void safeDelete(CloudBlobWrapper blob, SelfRenewingLease lease) throws StorageException { + private void safeDelete(CloudBlobWrapper blob, SelfRenewingLease lease) throws StorageException, IOException { OperationContext operationContext = getInstrumentedContext(); try { blob.delete(operationContext, lease); } catch (StorageException e) { - if (!NativeAzureFileSystemHelper.isFileNotFoundException(e)) { - LOG.error("Encountered Storage Exception for delete on Blob: {}" - + ", Exception Details: {} Error Code: {}", - blob.getUri(), e.getMessage(), e.getErrorCode()); - } + if (!NativeAzureFileSystemHelper.isFileNotFoundException(e)) { + LOG.error("Encountered Storage Exception for delete on Blob: {}" + + ", Exception Details: {} Error Code: {}", + blob.getUri(), e.getMessage(), e.getErrorCode()); + } // On exception, check that if: // 1. It's a BlobNotFound exception AND // 2. It got there after one-or-more retries THEN @@ -2903,13 +3176,14 @@ public void rename(String srcKey, String dstKey, boolean acquireLease, } } - private void waitForCopyToComplete(CloudBlobWrapper blob, OperationContext opContext){ + private void waitForCopyToComplete(CloudBlobWrapper blob, OperationContext opContext) + throws IOException { boolean copyInProgress = true; while (copyInProgress) { try { blob.downloadAttributes(opContext); } - catch (StorageException se){ + catch (StorageException | IOException se){ } // test for null because mocked filesystem doesn't know about copystates yet. @@ -2939,7 +3213,7 @@ private void waitForCopyToComplete(CloudBlobWrapper blob, OperationContext opCon * does not exist. So this method explicitly checks for the blob. */ @Override - public boolean explicitFileExists(String key) throws AzureException { + public boolean explicitFileExists(String key) throws IOException { CloudBlobWrapper blob; try { blob = getBlobReference(key); @@ -2952,6 +3226,8 @@ public boolean explicitFileExists(String key) throws AzureException { throw new AzureException(e); } catch (URISyntaxException e) { throw new AzureException(e); + } catch (IOException e) { + throw new IOException(e); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java index e9f0e784fc1213..84dcb85216f0f3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java @@ -443,7 +443,7 @@ public FileMetadata getSourceMetadata() throws IOException { * failure. * @throws IOException Thrown when fail to renaming. */ - public void execute() throws IOException { + public void execute(BlobMaterialization srcBlobMaterialization) throws IOException { AzureFileSystemThreadTask task = new AzureFileSystemThreadTask() { @Override @@ -460,8 +460,9 @@ public boolean execute(FileMetadata file) throws IOException{ // Rename the source folder 0-byte root file itself. FileMetadata srcMetadata2 = this.getSourceMetadata(); - if (srcMetadata2.getBlobMaterialization() == - BlobMaterialization.Explicit) { + if (srcBlobMaterialization == BlobMaterialization.Explicit && + srcMetadata2.getBlobMaterialization() == + BlobMaterialization.Explicit) { // It already has a lease on it from the "prepare" phase so there's no // need to get one now. Pass in existing lease to allow file delete. @@ -762,7 +763,7 @@ public String getScheme() { /** * The default number of threads to be used for rename operation. */ - public static final int DEFAULT_AZURE_RENAME_THREADS = 0; + public static final int DEFAULT_AZURE_RENAME_THREADS = 5; /** * The configuration property to set number of threads to be used for delete operation. @@ -772,7 +773,7 @@ public String getScheme() { /** * The default number of threads to be used for delete operation. */ - public static final int DEFAULT_AZURE_DELETE_THREADS = 0; + public static final int DEFAULT_AZURE_DELETE_THREADS = 5; /** * The number of threads to be used for delete operation after reading user configuration. @@ -1420,14 +1421,18 @@ public void initialize(URI uri, Configuration conf) instrumentation); } - store.initialize(uri, conf, instrumentation); + try { + store.initialize(uri, conf, instrumentation); + } catch (URISyntaxException e) { + e.printStackTrace(); + } setConf(conf); this.ugi = UserGroupInformation.getCurrentUser(); this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority()); this.workingDir = new Path("/user", UserGroupInformation.getCurrentUser() .getShortUserName()).makeQualified(getUri(), getWorkingDirectory()); - this.appendSupportEnabled = conf.getBoolean(APPEND_SUPPORT_ENABLE_PROPERTY_NAME, false); + this.appendSupportEnabled = conf.getBoolean(APPEND_SUPPORT_ENABLE_PROPERTY_NAME, true); LOG.debug("NativeAzureFileSystem. Initializing."); LOG.debug(" blockSize = {}", store.getHadoopBlockSize()); @@ -2264,7 +2269,7 @@ && isFileNotFoundException((StorageException) innerException)) { // corresponding directory blob a) and that would implicitly delete // the directory as well, which is not correct. - if (parentPath.getParent() != null) {// Not root + if (parentPath != null && parentPath.getParent() != null) {// Not root String parentKey = pathToKey(parentPath); FileMetadata parentMetadata = null; @@ -2339,7 +2344,7 @@ && isFileNotFoundException((StorageException) innerException)) { // The path specifies a folder. Recursively delete all entries under the // folder. LOG.debug("Directory Delete encountered: {}", f); - if (parentPath.getParent() != null) { + if (parentPath != null && parentPath.getParent() != null) { String parentKey = pathToKey(parentPath); FileMetadata parentMetadata = null; @@ -3274,9 +3279,13 @@ && isFileNotFoundException((StorageException) innerException)) { "rename", absoluteDstPath); } } - FileMetadata srcMetadata = null; + FileMetadata srcMetadata; + BlobMaterialization srcBlobMaterialization = null; try { srcMetadata = store.retrieveMetadata(srcKey); + if (srcMetadata != null){ + srcBlobMaterialization = srcMetadata.getBlobMaterialization(); + } } catch (IOException ex) { Throwable innerException = NativeAzureFileSystemHelper.checkForAzureStorageException(ex); @@ -3334,7 +3343,7 @@ && isFileNotFoundException((StorageException) innerException)) { // operation to HBase log file folders, where atomic rename is required. // In the future, we could generalize it easily to all folders. renamePending = prepareAtomicFolderRename(srcKey, dstKey); - renamePending.execute(); + renamePending.execute(srcBlobMaterialization); LOG.debug("Renamed {} to {} successfully.", src, dst); renamePending.cleanup(); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java index 91aad992a1f19b..705ddb034d3c10 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.io.InputStream; import java.net.URI; +import java.net.URISyntaxException; import java.util.Date; import java.util.Optional; @@ -40,7 +41,8 @@ @InterfaceAudience.Private interface NativeFileSystemStore { - void initialize(URI uri, Configuration conf, AzureFileSystemInstrumentation instrumentation) throws IOException; + void initialize(URI uri, Configuration conf, AzureFileSystemInstrumentation instrumentation) + throws IOException, URISyntaxException; void storeEmptyFolder(String key, PermissionStatus permissionStatus) throws AzureException; @@ -144,5 +146,5 @@ void updateFolderLastModifiedTime(String key, Date lastModified, DataOutputStream retrieveAppendStream(String key, int bufferSize) throws IOException; - boolean explicitFileExists(String key) throws AzureException; + boolean explicitFileExists(String key) throws IOException; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PageBlobOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PageBlobOutputStream.java index 1e409cd908fa8c..3a3d436669a569 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PageBlobOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PageBlobOutputStream.java @@ -148,7 +148,7 @@ final class PageBlobOutputStream extends OutputStream implements Syncable, Strea */ public PageBlobOutputStream(final CloudPageBlobWrapper blob, final OperationContext opContext, - final Configuration conf) throws StorageException { + final Configuration conf) throws StorageException, IOException { this.blob = blob; this.outBuffer = new ByteArrayOutputStream(); this.opContext = opContext; @@ -270,12 +270,14 @@ public void run() { LOG.debug("before runInternal()"); runInternal(); LOG.debug("after runInternal()"); + } catch (IOException e) { + e.printStackTrace(); } finally { doneSignal.countDown(); } } - private void runInternal() { + private void runInternal() throws IOException { if (lastError != null) { // We're already in an error state, no point doing anything. return; @@ -410,7 +412,7 @@ synchronized void waitForLastFlushCompletion() throws IOException { /** * Extend the page blob file if we are close to the end. */ - private void conditionalExtendFile() { + private void conditionalExtendFile() throws IOException { // maximum allowed size of an Azure page blob (1 terabyte) final long MAX_PAGE_BLOB_SIZE = 1024L * 1024L * 1024L * 1024L; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java index f6eb75cad59c8c..e333bcd4f00710 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java @@ -56,6 +56,7 @@ import com.microsoft.azure.storage.blob.BlockEntry; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; /*** * An implementation of the StorageInterface for SAS Key mode. @@ -169,6 +170,13 @@ public CloudBlobContainerWrapper getContainerReference(String name) } } + @Override + public CloudBlobContainerWrapper getContainerReference(final String name, + final AccessTokenProvider tokenProvider) + throws URISyntaxException, StorageException { + return getContainerReference(name); + } + public void setStorageAccountName(String storageAccount) { this.storageAccount = storageAccount; } @@ -236,6 +244,14 @@ public CloudBlobDirectoryWrapper getDirectoryReference(String relativePath) return new SASCloudBlobDirectoryWrapperImpl(dir); } + @Override + public CloudBlobDirectoryWrapper getDirectoryReference(String relativePath, AccessTokenProvider tokenProvider) + throws URISyntaxException, StorageException { + + CloudBlobDirectory dir = container.getDirectoryReference(relativePath); + return new SASCloudBlobDirectoryWrapperImpl(dir); + } + @Override public CloudBlobWrapper getBlockBlobReference(String relativePath) throws URISyntaxException, StorageException { @@ -255,6 +271,13 @@ public CloudBlobWrapper getBlockBlobReference(String relativePath) } } + @Override + public CloudBlobWrapper getBlockBlobReference(final String relativePath, + final AccessTokenProvider tokenProvider) + throws URISyntaxException, StorageException, IOException { + return getBlockBlobReference(relativePath); + } + @Override public CloudBlobWrapper getPageBlobReference(String relativePath) throws URISyntaxException, StorageException { @@ -275,6 +298,13 @@ public CloudBlobWrapper getPageBlobReference(String relativePath) throw new StorageException(SAS_ERROR_CODE, errorMsg, sasEx); } } + + @Override + public CloudBlobWrapper getPageBlobReference(final String relativePath, + final AccessTokenProvider tokenProvider) + throws URISyntaxException, StorageException, IOException { + return getPageBlobReference(relativePath); + } } // @@ -508,7 +538,7 @@ public CopyState getCopyState() { @Override public void startCopyFromBlob(CloudBlobWrapper sourceBlob, BlobRequestOptions options, OperationContext opContext, boolean overwriteDestination) - throws StorageException, URISyntaxException { + throws StorageException, URISyntaxException, IOException { AccessCondition dstAccessCondition = overwriteDestination ? null @@ -526,7 +556,8 @@ public void downloadRange(long offset, long length, OutputStream outStream, } @Override - public SelfRenewingLease acquireLease() throws StorageException { + public SelfRenewingLease acquireLease() throws StorageException, + IOException { return new SelfRenewingLease(this, false); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java index 200945f6b533eb..933948f3d768b1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfRenewingLease.java @@ -28,6 +28,7 @@ import com.microsoft.azure.storage.StorageException; import com.microsoft.azure.storage.blob.CloudBlob; +import java.io.IOException; import java.util.concurrent.atomic.AtomicInteger; import static com.microsoft.azure.storage.StorageErrorCodeStrings.LEASE_ALREADY_PRESENT; @@ -69,7 +70,7 @@ public class SelfRenewingLease { static final int LEASE_ACQUIRE_RETRY_INTERVAL = 2000; public SelfRenewingLease(CloudBlobWrapper blobWrapper, boolean throwIfPresent) - throws StorageException { + throws StorageException, IOException { this.leaseFreed = false; this.blobWrapper = blobWrapper; @@ -119,7 +120,7 @@ public SelfRenewingLease(CloudBlobWrapper blobWrapper, boolean throwIfPresent) * Free the lease and stop the keep-alive thread. * @throws StorageException Thrown when fail to free the lease. */ - public void free() throws StorageException { + public void free() throws StorageException, IOException { AccessCondition accessCondition = AccessCondition.generateEmptyCondition(); accessCondition.setLeaseID(leaseID); try { @@ -137,6 +138,8 @@ public void free() throws StorageException { + " on " + blobWrapper.getStorageUri()); throw(e); } + } catch (IOException e) { + throw e; } finally { // Even if releasing the lease fails (e.g. because the file was deleted), @@ -156,7 +159,7 @@ public String getLeaseID() { return leaseID; } - public CloudBlob getCloudBlob() { + public CloudBlob getCloudBlob() throws IOException { return blobWrapper.getBlob(); } @@ -192,7 +195,7 @@ public void run() { LOG.info("Renewed lease " + leaseID + " on " + getCloudBlob().getUri()); } - } catch (StorageException e) { + } catch (StorageException | IOException e) { if (!leaseFreed) { // Free the lease so we don't leave this thread running forever. @@ -200,10 +203,14 @@ public void run() { // Normally leases should be freed and there should be no // exceptions, so log a warning. - LOG.warn("Attempt to renew lease " + leaseID + " on " - + getCloudBlob().getUri() - + " failed, but lease not yet freed. Reason: " + - e.getMessage()); + try { + LOG.warn("Attempt to renew lease " + leaseID + " on " + + getCloudBlob().getUri() + + " failed, but lease not yet freed. Reason: " + + e.getMessage()); + } catch (IOException ex) { + ex.printStackTrace(); + } } } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java index dbb38491d7f552..52f3ce34cf5bcf 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java @@ -29,6 +29,7 @@ import java.util.HashMap; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import com.microsoft.azure.storage.AccessCondition; import com.microsoft.azure.storage.CloudStorageAccount; @@ -150,6 +151,9 @@ public abstract void createBlobClient(URI baseUri, public abstract CloudBlobContainerWrapper getContainerReference(String name) throws URISyntaxException, StorageException; + public abstract CloudBlobContainerWrapper getContainerReference(String name, AccessTokenProvider tokenProvider) + throws URISyntaxException, StorageException; + /** * A thin wrapper over the * {@link com.microsoft.azure.storage.blob.CloudBlobDirectory} class @@ -206,7 +210,7 @@ public abstract static class CloudBlobDirectoryWrapper implements public abstract Iterable listBlobs(String prefix, boolean useFlatBlobListing, EnumSet listingDetails, BlobRequestOptions options, OperationContext opContext) - throws URISyntaxException, StorageException; + throws URISyntaxException, StorageException, IOException; } /** @@ -221,7 +225,7 @@ public abstract static class CloudBlobContainerWrapper { * * @return A String that represents the name of the container. */ - public abstract String getName(); + public abstract String getName() throws IOException; /** * Returns a value that indicates whether the container exists, using the @@ -240,7 +244,7 @@ public abstract static class CloudBlobContainerWrapper { * If a storage service error occurred. */ public abstract boolean exists(OperationContext opContext) - throws StorageException; + throws StorageException, IOException; /** * Returns the metadata for the container. @@ -248,7 +252,7 @@ public abstract boolean exists(OperationContext opContext) * @return A java.util.HashMap object that represents the * metadata for the container. */ - public abstract HashMap getMetadata(); + public abstract HashMap getMetadata() throws IOException; /** * Sets the metadata for the container. @@ -257,7 +261,8 @@ public abstract boolean exists(OperationContext opContext) * A java.util.HashMap object that represents the * metadata being assigned to the container. */ - public abstract void setMetadata(HashMap metadata); + public abstract void setMetadata(HashMap metadata) + throws IOException; /** * Downloads the container's attributes, which consist of metadata and @@ -273,7 +278,7 @@ public abstract boolean exists(OperationContext opContext) * If a storage service error occurred. */ public abstract void downloadAttributes(OperationContext opContext) - throws StorageException; + throws StorageException, IOException; /** * Uploads the container's metadata using the specified operation context. @@ -288,7 +293,7 @@ public abstract void downloadAttributes(OperationContext opContext) * If a storage service error occurred. */ public abstract void uploadMetadata(OperationContext opContext) - throws StorageException; + throws StorageException, IOException; /** * Creates the container using the specified operation context. @@ -303,7 +308,7 @@ public abstract void uploadMetadata(OperationContext opContext) * If a storage service error occurred. */ public abstract void create(OperationContext opContext) - throws StorageException; + throws StorageException, IOException; /** * Returns a wrapper for a CloudBlobDirectory. @@ -319,7 +324,12 @@ public abstract void create(OperationContext opContext) * If URI syntax exception occurred. */ public abstract CloudBlobDirectoryWrapper getDirectoryReference( - String relativePath) throws URISyntaxException, StorageException; + String relativePath) + throws URISyntaxException, StorageException, IOException; + + public abstract CloudBlobDirectoryWrapper getDirectoryReference( + String relativePath, AccessTokenProvider tokenProvider) + throws URISyntaxException, StorageException, IOException; /** * Returns a wrapper for a CloudBlockBlob. @@ -335,7 +345,12 @@ public abstract CloudBlobDirectoryWrapper getDirectoryReference( * If URI syntax exception occurred. */ public abstract CloudBlobWrapper getBlockBlobReference( - String relativePath) throws URISyntaxException, StorageException; + String relativePath) + throws URISyntaxException, StorageException, IOException; + + public abstract CloudBlobWrapper getBlockBlobReference( + String relativePath, AccessTokenProvider tokenProvider) + throws URISyntaxException, StorageException, IOException; /** * Returns a wrapper for a CloudPageBlob. @@ -350,7 +365,10 @@ public abstract CloudBlobWrapper getBlockBlobReference( * If URI syntax exception occurred. */ public abstract CloudBlobWrapper getPageBlobReference(String relativePath) - throws URISyntaxException, StorageException; + throws URISyntaxException, StorageException, IOException; + + public abstract CloudBlobWrapper getPageBlobReference(String relativePath, AccessTokenProvider tokenProvider) + throws URISyntaxException, StorageException, IOException; } @@ -374,7 +392,7 @@ public interface CloudBlobWrapper extends ListBlobItem { * @return A java.util.HashMap object that represents the * metadata for the blob. */ - HashMap getMetadata(); + HashMap getMetadata() throws IOException; /** * Sets the metadata for the blob. @@ -383,7 +401,7 @@ public interface CloudBlobWrapper extends ListBlobItem { * A java.util.HashMap object that contains the * metadata being assigned to the blob. */ - void setMetadata(HashMap metadata); + void setMetadata(HashMap metadata) throws IOException; /** * Copies an existing blob's contents, properties, and metadata to this instance of the CloudBlob @@ -407,7 +425,7 @@ public interface CloudBlobWrapper extends ListBlobItem { */ public abstract void startCopyFromBlob(CloudBlobWrapper sourceBlob, BlobRequestOptions options, OperationContext opContext, boolean overwriteDestination) - throws StorageException, URISyntaxException; + throws StorageException, URISyntaxException, IOException; /** * Returns the blob's copy state. @@ -415,7 +433,7 @@ public abstract void startCopyFromBlob(CloudBlobWrapper sourceBlob, * @return A {@link CopyState} object that represents the copy state of the * blob. */ - CopyState getCopyState(); + CopyState getCopyState() throws IOException; /** * Downloads a range of bytes from the blob to the given byte buffer, using the specified request options and @@ -466,7 +484,7 @@ void downloadRange(final long offset, final long length, * If a storage service error occurred. */ void delete(OperationContext opContext, SelfRenewingLease lease) - throws StorageException; + throws StorageException, IOException; /** * Checks to see if the blob exists, using the specified operation context. @@ -484,7 +502,7 @@ void delete(OperationContext opContext, SelfRenewingLease lease) * If a storage service error occurred. */ boolean exists(OperationContext opContext) - throws StorageException; + throws StorageException, IOException; /** * Populates a blob's properties and metadata using the specified operation @@ -505,7 +523,7 @@ boolean exists(OperationContext opContext) * If a storage service error occurred. */ void downloadAttributes(OperationContext opContext) - throws StorageException; + throws StorageException, IOException; /** * Returns the blob's properties. @@ -513,7 +531,7 @@ void downloadAttributes(OperationContext opContext) * @return A {@link BlobProperties} object that represents the properties of * the blob. */ - BlobProperties getProperties(); + BlobProperties getProperties() throws IOException; /** * Opens a blob input stream to download the blob using the specified @@ -535,7 +553,7 @@ void downloadAttributes(OperationContext opContext) * If a storage service error occurred. */ InputStream openInputStream(BlobRequestOptions options, - OperationContext opContext) throws StorageException; + OperationContext opContext) throws StorageException, IOException; /** * Uploads the blob's metadata to the storage service using the specified @@ -551,7 +569,7 @@ InputStream openInputStream(BlobRequestOptions options, * If a storage service error occurred. */ void uploadMetadata(OperationContext opContext) - throws StorageException; + throws StorageException, IOException; /** * Uploads the blob's metadata to the storage service using the specified @@ -575,20 +593,20 @@ void uploadMetadata(OperationContext opContext) * If a storage service error occurred. */ void uploadMetadata(AccessCondition accessCondition, BlobRequestOptions options, - OperationContext opContext) throws StorageException; + OperationContext opContext) throws StorageException, IOException; void uploadProperties(OperationContext opContext, SelfRenewingLease lease) - throws StorageException; + throws StorageException, IOException; - SelfRenewingLease acquireLease() throws StorageException; + SelfRenewingLease acquireLease() throws StorageException, IOException; /** * Gets the minimum read block size to use with this Blob. * * @return The minimum block size, in bytes, for reading from a block blob. */ - int getStreamMinimumReadSizeInBytes(); + int getStreamMinimumReadSizeInBytes() throws IOException; /** * Sets the minimum read block size to use with this Blob. @@ -599,7 +617,7 @@ void uploadProperties(OperationContext opContext, * bytes to 64 MB, inclusive. */ void setStreamMinimumReadSizeInBytes( - int minimumReadSizeBytes); + int minimumReadSizeBytes) throws IOException; /** * Sets the write block size to use with this Blob. @@ -613,9 +631,9 @@ void setStreamMinimumReadSizeInBytes( * If writeBlockSizeInBytes is less than 1 MB or * greater than 4 MB. */ - void setWriteBlockSizeInBytes(int writeBlockSizeBytes); + void setWriteBlockSizeInBytes(int writeBlockSizeBytes) throws IOException; - CloudBlob getBlob(); + CloudBlob getBlob() throws IOException; } /** @@ -641,7 +659,7 @@ public abstract interface CloudBlockBlobWrapper */ OutputStream openOutputStream( BlobRequestOptions options, - OperationContext opContext) throws StorageException; + OperationContext opContext) throws StorageException, IOException; /** * @@ -730,7 +748,7 @@ public abstract interface CloudPageBlobWrapper * If a storage service error occurred. */ void create(final long length, BlobRequestOptions options, - OperationContext opContext) throws StorageException; + OperationContext opContext) throws StorageException, IOException; /** @@ -785,9 +803,9 @@ void uploadPages(final InputStream sourceStream, final long offset, */ ArrayList downloadPageRanges(BlobRequestOptions options, - OperationContext opContext) throws StorageException; + OperationContext opContext) throws StorageException, IOException; void uploadMetadata(OperationContext opContext) - throws StorageException; + throws StorageException, IOException; } } \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java index e600f9e59da3fb..76d2f4cabeb315 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java @@ -29,12 +29,14 @@ import java.util.Iterator; import java.util.List; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import com.microsoft.azure.storage.AccessCondition; import com.microsoft.azure.storage.CloudStorageAccount; import com.microsoft.azure.storage.OperationContext; import com.microsoft.azure.storage.RetryPolicyFactory; import com.microsoft.azure.storage.StorageCredentials; +import com.microsoft.azure.storage.StorageCredentialsToken; import com.microsoft.azure.storage.StorageException; import com.microsoft.azure.storage.StorageUri; import com.microsoft.azure.storage.blob.BlobListingDetails; @@ -75,6 +77,10 @@ private void updateTimeoutInMs() { } } + CloudBlobClient getServiceClient() { + return serviceClient; + } + @Override public void setRetryPolicyFactory(final RetryPolicyFactory retryPolicyFactory) { this.retryPolicyFactory = retryPolicyFactory; @@ -118,6 +124,12 @@ public CloudBlobContainerWrapper getContainerReference(String uri) serviceClient.getContainerReference(uri)); } + public CloudBlobContainerWrapper getContainerReference(String uri, AccessTokenProvider tokenProvider) + throws URISyntaxException, StorageException { + return new CloudBlobContainerWrapperImpl( + serviceClient.getContainerReference(uri), serviceClient, tokenProvider); + } + // // WrappingIterator // @@ -174,42 +186,78 @@ public void remove() { @InterfaceAudience.Private static class CloudBlobDirectoryWrapperImpl extends CloudBlobDirectoryWrapper { private final CloudBlobDirectory directory; + private CloudBlobClient serviceClient = null; + private AccessTokenProvider tokenProvider = null; + + public CloudBlobDirectory getDirectory() throws IOException { + if (tokenProvider != null) { + String token = tokenProvider.getToken().getAccessToken(); + ((StorageCredentialsToken) serviceClient.getCredentials()).updateToken( + token); + } + return directory; + } public CloudBlobDirectoryWrapperImpl(CloudBlobDirectory directory) { this.directory = directory; } + public CloudBlobDirectoryWrapperImpl(CloudBlobDirectory directory, AccessTokenProvider tokenProvider, CloudBlobClient serviceClient) { + this.directory = directory; + this.tokenProvider = tokenProvider; + this.serviceClient = serviceClient; + } + @Override public URI getUri() { - return directory.getUri(); + try { + return getDirectory().getUri(); + } catch (IOException e) { + e.printStackTrace(); + } + return null; } @Override public Iterable listBlobs(String prefix, boolean useFlatBlobListing, EnumSet listingDetails, BlobRequestOptions options, OperationContext opContext) - throws URISyntaxException, StorageException { - return WrappingIterator.wrap(directory.listBlobs(prefix, + throws URISyntaxException, StorageException, IOException { + return WrappingIterator.wrap(getDirectory().listBlobs(prefix, useFlatBlobListing, listingDetails, options, opContext)); } @Override public CloudBlobContainer getContainer() throws URISyntaxException, StorageException { - return directory.getContainer(); + try { + return getDirectory().getContainer(); + } catch (IOException e) { + e.printStackTrace(); + } + return null; } @Override public CloudBlobDirectory getParent() throws URISyntaxException, StorageException { - return directory.getParent(); + try { + return getDirectory().getParent(); + } catch (IOException e) { + e.printStackTrace(); + } + return null; } @Override public StorageUri getStorageUri() { - return directory.getStorageUri(); + try { + return getDirectory().getStorageUri(); + } catch (IOException e) { + e.printStackTrace(); + } + return null; } - } // @@ -218,99 +266,157 @@ public StorageUri getStorageUri() { @InterfaceAudience.Private static class CloudBlobContainerWrapperImpl extends CloudBlobContainerWrapper { private final CloudBlobContainer container; + private CloudBlobClient serviceClient = null; + private AccessTokenProvider tokenProvider = null; + + public CloudBlobContainer getContainer() throws IOException { + if (tokenProvider != null) { + String token = tokenProvider.getToken().getAccessToken(); + ((StorageCredentialsToken) serviceClient.getCredentials()).updateToken( + token); + } + return container; + } public CloudBlobContainerWrapperImpl(CloudBlobContainer container) { this.container = container; } + public CloudBlobContainerWrapperImpl(CloudBlobContainer container, CloudBlobClient serviceClient, AccessTokenProvider tokenProvider) { + this.container = container; + this.serviceClient = serviceClient; + this.tokenProvider = tokenProvider; + } + @Override - public String getName() { - return container.getName(); + public String getName() throws IOException { + return getContainer().getName(); } @Override - public boolean exists(OperationContext opContext) throws StorageException { - return container.exists(AccessCondition.generateEmptyCondition(), null, + public boolean exists(OperationContext opContext) + throws StorageException, IOException { + return getContainer().exists(AccessCondition.generateEmptyCondition(), null, opContext); } @Override - public void create(OperationContext opContext) throws StorageException { - container.create(null, opContext); + public void create(OperationContext opContext) + throws StorageException, IOException { + getContainer().create(null, opContext); } @Override - public HashMap getMetadata() { - return container.getMetadata(); + public HashMap getMetadata() throws IOException { + return getContainer().getMetadata(); } @Override - public void setMetadata(HashMap metadata) { - container.setMetadata(metadata); + public void setMetadata(HashMap metadata) + throws IOException { + getContainer().setMetadata(metadata); } @Override public void downloadAttributes(OperationContext opContext) - throws StorageException { - container.downloadAttributes(AccessCondition.generateEmptyCondition(), + throws StorageException, IOException { + getContainer().downloadAttributes(AccessCondition.generateEmptyCondition(), null, opContext); } @Override public void uploadMetadata(OperationContext opContext) - throws StorageException { - container.uploadMetadata(AccessCondition.generateEmptyCondition(), null, + throws StorageException, IOException { + getContainer().uploadMetadata(AccessCondition.generateEmptyCondition(), null, opContext); } @Override public CloudBlobDirectoryWrapper getDirectoryReference(String relativePath) - throws URISyntaxException, StorageException { + throws URISyntaxException, StorageException, IOException { - CloudBlobDirectory dir = container.getDirectoryReference(relativePath); + CloudBlobDirectory dir = getContainer().getDirectoryReference(relativePath); return new CloudBlobDirectoryWrapperImpl(dir); } + @Override + public CloudBlobDirectoryWrapper getDirectoryReference(String relativePath, AccessTokenProvider tokenProvider) + throws URISyntaxException, StorageException, IOException { + + CloudBlobDirectory dir = getContainer().getDirectoryReference(relativePath); + return new CloudBlobDirectoryWrapperImpl(dir, tokenProvider, serviceClient); + } + @Override public CloudBlobWrapper getBlockBlobReference(String relativePath) - throws URISyntaxException, StorageException { + throws URISyntaxException, StorageException, IOException { + return new CloudBlockBlobWrapperImpl(getContainer().getBlockBlobReference(relativePath)); + } - return new CloudBlockBlobWrapperImpl(container.getBlockBlobReference(relativePath)); + @Override + public CloudBlobWrapper getBlockBlobReference(String relativePath, AccessTokenProvider tokenProvider) + throws URISyntaxException, StorageException, IOException { + return new CloudBlockBlobWrapperImpl(getContainer().getBlockBlobReference(relativePath), tokenProvider, serviceClient); } @Override public CloudBlobWrapper getPageBlobReference(String relativePath) - throws URISyntaxException, StorageException { + throws URISyntaxException, StorageException, IOException { return new CloudPageBlobWrapperImpl( - container.getPageBlobReference(relativePath)); + getContainer().getPageBlobReference(relativePath)); + } + + @Override + public CloudBlobWrapper getPageBlobReference(String relativePath, AccessTokenProvider tokenProvider) + throws URISyntaxException, StorageException, IOException { + return new CloudPageBlobWrapperImpl( + getContainer().getPageBlobReference(relativePath), tokenProvider, serviceClient); } } abstract static class CloudBlobWrapperImpl implements CloudBlobWrapper { private final CloudBlob blob; + private AccessTokenProvider tokenProvider = null; + private CloudBlobClient serviceClient = null; @Override - public CloudBlob getBlob() { + public CloudBlob getBlob() throws IOException { + if (tokenProvider != null) { + String token = tokenProvider.getToken().getAccessToken(); + ((StorageCredentialsToken) serviceClient.getCredentials()).updateToken( + token); + } return blob; } public URI getUri() { - return getBlob().getUri(); + try { + return getBlob().getUri(); + } catch (IOException e) { + e.printStackTrace(); + } + return null; } protected CloudBlobWrapperImpl(CloudBlob blob) { this.blob = blob; } + protected CloudBlobWrapperImpl(CloudBlob blob, AccessTokenProvider tokenProvider, CloudBlobClient serviceClient) { + this.blob = blob; + this.tokenProvider = tokenProvider; + this.serviceClient = serviceClient; + } + @Override - public HashMap getMetadata() { + public HashMap getMetadata() throws IOException { return getBlob().getMetadata(); } @Override public void delete(OperationContext opContext, SelfRenewingLease lease) - throws StorageException { + throws StorageException, IOException { getBlob().delete(DeleteSnapshotsOption.NONE, getLeaseCondition(lease), null, opContext); } @@ -329,36 +435,37 @@ private AccessCondition getLeaseCondition(SelfRenewingLease lease) { @Override public boolean exists(OperationContext opContext) - throws StorageException { + throws StorageException, IOException { return getBlob().exists(null, null, opContext); } @Override public void downloadAttributes( - OperationContext opContext) throws StorageException { + OperationContext opContext) throws StorageException, IOException { getBlob().downloadAttributes(null, null, opContext); } @Override - public BlobProperties getProperties() { + public BlobProperties getProperties() throws IOException { return getBlob().getProperties(); } @Override - public void setMetadata(HashMap metadata) { + public void setMetadata(HashMap metadata) + throws IOException { getBlob().setMetadata(metadata); } @Override public InputStream openInputStream( BlobRequestOptions options, - OperationContext opContext) throws StorageException { + OperationContext opContext) throws StorageException, IOException { return getBlob().openInputStream(null, options, opContext); } public OutputStream openOutputStream( BlobRequestOptions options, - OperationContext opContext) throws StorageException { + OperationContext opContext) throws StorageException, IOException { return ((CloudBlockBlob) getBlob()).openOutputStream(null, options, opContext); } @@ -370,63 +477,80 @@ public void upload(InputStream sourceStream, OperationContext opContext) @Override public CloudBlobContainer getContainer() throws URISyntaxException, StorageException { - return getBlob().getContainer(); + try { + return getBlob().getContainer(); + } catch (IOException e) { + e.printStackTrace(); + } + return null; } @Override public CloudBlobDirectory getParent() throws URISyntaxException, StorageException { - return getBlob().getParent(); + try { + return getBlob().getParent(); + } catch (IOException e) { + e.printStackTrace(); + } + return null; } @Override public void uploadMetadata(OperationContext opContext) - throws StorageException { + throws StorageException, IOException { uploadMetadata(null, null, opContext); } @Override public void uploadMetadata(AccessCondition accessConditions, BlobRequestOptions options, - OperationContext opContext) throws StorageException{ + OperationContext opContext) throws StorageException, IOException { getBlob().uploadMetadata(accessConditions, options, opContext); } public void uploadProperties(OperationContext opContext, SelfRenewingLease lease) - throws StorageException { + throws StorageException, IOException { // Include lease in request if lease not null. getBlob().uploadProperties(getLeaseCondition(lease), null, opContext); } @Override - public int getStreamMinimumReadSizeInBytes() { + public int getStreamMinimumReadSizeInBytes() throws IOException { return getBlob().getStreamMinimumReadSizeInBytes(); } @Override - public void setStreamMinimumReadSizeInBytes(int minimumReadSizeBytes) { + public void setStreamMinimumReadSizeInBytes(int minimumReadSizeBytes) + throws IOException { getBlob().setStreamMinimumReadSizeInBytes(minimumReadSizeBytes); } @Override - public void setWriteBlockSizeInBytes(int writeBlockSizeBytes) { + public void setWriteBlockSizeInBytes(int writeBlockSizeBytes) + throws IOException { getBlob().setStreamWriteSizeInBytes(writeBlockSizeBytes); } @Override public StorageUri getStorageUri() { - return getBlob().getStorageUri(); + try { + return getBlob().getStorageUri(); + } catch (IOException e) { + e.printStackTrace(); + } + return null; } @Override - public CopyState getCopyState() { + public CopyState getCopyState() throws IOException { return getBlob().getCopyState(); } @Override public void startCopyFromBlob(CloudBlobWrapper sourceBlob, BlobRequestOptions options, OperationContext opContext, boolean overwriteDestination) - throws StorageException, URISyntaxException { + throws StorageException, URISyntaxException, IOException { AccessCondition dstAccessCondition = overwriteDestination ? null @@ -444,7 +568,8 @@ public void downloadRange(long offset, long length, OutputStream outStream, } @Override - public SelfRenewingLease acquireLease() throws StorageException { + public SelfRenewingLease acquireLease() throws StorageException, + IOException { return new SelfRenewingLease(this, false); } } @@ -459,9 +584,13 @@ public CloudBlockBlobWrapperImpl(CloudBlockBlob blob) { super(blob); } + public CloudBlockBlobWrapperImpl(CloudBlockBlob blob, AccessTokenProvider tokenProvider, CloudBlobClient serviceClient) { + super(blob, tokenProvider, serviceClient); + } + public OutputStream openOutputStream( BlobRequestOptions options, - OperationContext opContext) throws StorageException { + OperationContext opContext) throws StorageException, IOException { return ((CloudBlockBlob) getBlob()).openOutputStream(null, options, opContext); } @@ -471,7 +600,7 @@ public void upload(InputStream sourceStream, OperationContext opContext) } public void uploadProperties(OperationContext opContext) - throws StorageException { + throws StorageException, IOException { getBlob().uploadProperties(null, null, opContext); } @@ -501,8 +630,12 @@ public CloudPageBlobWrapperImpl(CloudPageBlob blob) { super(blob); } + public CloudPageBlobWrapperImpl(CloudPageBlob blob, AccessTokenProvider tokenProvider, CloudBlobClient serviceClient) { + super(blob, tokenProvider, serviceClient); + } + public void create(final long length, BlobRequestOptions options, - OperationContext opContext) throws StorageException { + OperationContext opContext) throws StorageException, IOException { ((CloudPageBlob) getBlob()).create(length, null, options, opContext); } @@ -514,7 +647,7 @@ public void uploadPages(final InputStream sourceStream, final long offset, } public ArrayList downloadPageRanges(BlobRequestOptions options, - OperationContext opContext) throws StorageException { + OperationContext opContext) throws StorageException, IOException { return ((CloudPageBlob) getBlob()).downloadPageRanges( null, options, opContext); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 50cc57447f92b8..d127c3f60621a6 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -21,6 +21,8 @@ import java.io.IOException; import java.lang.reflect.Field; +import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -83,6 +85,7 @@ public class AbfsConfiguration{ private final Configuration rawConfig; private final String accountName; private final boolean isSecure; + private PrefixMode prefixMode; private static final Logger LOG = LoggerFactory.getLogger(AbfsConfiguration.class); @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ACCOUNT_IS_HNS_ENABLED, @@ -117,6 +120,15 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_OPTIMIZE_FOOTER_READ) private boolean optimizeFooterRead; + @BooleanConfigurationValidatorAnnotation( + ConfigurationKey = FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED, + DefaultValue = DEFAULT_FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED) + private boolean isExpectHeaderEnabled; + + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED, + DefaultValue = DEFAULT_FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED) + private boolean accountThrottlingEnabled; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_READ_BUFFER_SIZE, MinValue = MIN_BUFFER_SIZE, MaxValue = MAX_BUFFER_SIZE, @@ -213,6 +225,11 @@ public class AbfsConfiguration{ DEFAULT_FS_AZURE_ENABLE_MKDIR_OVERWRITE) private boolean mkdirOverwrite; + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = + FS_AZURE_ENABLE_BLOB_MKDIR_OVERWRITE, DefaultValue = + DEFAULT_FS_AZURE_BLOB_ENABLE_MKDIR_OVERWRITE) + private boolean blobMkdirOverwrite; + @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_APPEND_BLOB_KEY, DefaultValue = DEFAULT_FS_AZURE_APPEND_BLOB_DIRECTORIES) private String azureAppendBlobDirs; @@ -238,6 +255,18 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_READ_AHEAD_QUEUE_DEPTH) private int readAheadQueueDepth; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_BLOB_DIR_RENAME_MAX_THREAD, + DefaultValue = DEFAULT_FS_AZURE_BLOB_RENAME_THREAD) + private int blobDirRenameMaxThread; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_BLOB_DIR_DELETE_MAX_THREAD, + DefaultValue = DEFAULT_FS_AZURE_BLOB_DELETE_THREAD) + private int blobDirDeleteMaxThread; + + @LongConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_BLOB_COPY_PROGRESS_POLL_WAIT_MILLIS, + DefaultValue = 1_000L) + private long blobCopyProgressPollWaitMillis; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_READ_AHEAD_BLOCK_SIZE, MinValue = MIN_BUFFER_SIZE, MaxValue = MAX_BUFFER_SIZE, @@ -260,6 +289,14 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_ENABLE_AUTOTHROTTLING) private boolean enableAutoThrottling; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ACCOUNT_OPERATION_IDLE_TIMEOUT, + DefaultValue = DEFAULT_ACCOUNT_OPERATION_IDLE_TIMEOUT_MS) + private int accountOperationIdleTimeout; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ANALYSIS_PERIOD, + DefaultValue = DEFAULT_ANALYSIS_PERIOD_MS) + private int analysisPeriod; + @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_USER_AGENT_PREFIX_KEY, DefaultValue = DEFAULT_FS_AZURE_USER_AGENT_PREFIX) private String userAgentId; @@ -297,6 +334,11 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_ABFS_LATENCY_TRACK) private boolean trackLatency; + @BooleanConfigurationValidatorAnnotation( + ConfigurationKey = FS_AZURE_ENABLE_READAHEAD, + DefaultValue = DEFAULT_ENABLE_READAHEAD) + private boolean enabledReadAhead; + @LongConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_SAS_TOKEN_RENEW_PERIOD_FOR_STREAMS, MinValue = 0, DefaultValue = DEFAULT_SAS_TOKEN_RENEW_PERIOD_FOR_STREAMS_IN_SECONDS) @@ -306,6 +348,13 @@ public class AbfsConfiguration{ FS_AZURE_ENABLE_ABFS_LIST_ITERATOR, DefaultValue = DEFAULT_ENABLE_ABFS_LIST_ITERATOR) private boolean enableAbfsListIterator; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = + FS_AZURE_PRODUCER_QUEUE_MAX_SIZE, DefaultValue = DEFAULT_FS_AZURE_PRODUCER_QUEUE_MAX_SIZE) + private int producerQueueMaxSize; + + @BooleanConfigurationValidatorAnnotation(ConfigurationKey=FS_AZURE_LEASE_CREATE_NON_RECURSIVE, DefaultValue = DEFAULT_FS_AZURE_LEASE_CREATE_NON_RECURSIVE) + private boolean leaseOnCreateNonRecursive; + public AbfsConfiguration(final Configuration rawConfig, String accountName) throws IllegalAccessException, InvalidConfigurationValueException, IOException { this.rawConfig = ProviderUtils.excludeIncompatibleCredentialProviders( @@ -332,10 +381,76 @@ public AbfsConfiguration(final Configuration rawConfig, String accountName) } } + @BooleanConfigurationValidatorAnnotation( + ConfigurationKey = FS_AZURE_REDIRECT_RENAME, + DefaultValue = DEFAULT_FS_AZURE_REDIRECT_RENAME) + private boolean redirectRename; + + @BooleanConfigurationValidatorAnnotation( + ConfigurationKey = FS_AZURE_REDIRECT_DELETE, + DefaultValue = DEFAULT_FS_AZURE_REDIRECT_DELETE) + private boolean redirectDelete; + + public boolean shouldRedirectRename() { + return this.redirectRename; + } + + public boolean shouldRedirectDelete() { + return this.redirectDelete; + } + + public boolean isRedirection() { + return (shouldRedirectRename() || shouldRedirectDelete()); + } + public Trilean getIsNamespaceEnabledAccount() { return Trilean.getTrilean(isNamespaceEnabledAccount); } + public PrefixMode getPrefixMode() { + return prefixMode; + } + + public void setPrefixMode(final PrefixMode prefixMode) { + this.prefixMode = prefixMode; + } + + @BooleanConfigurationValidatorAnnotation( + ConfigurationKey = FS_AZURE_ENABLE_BLOB_ENDPOINT, + DefaultValue = DEFAULT_FS_AZURE_ENABLE_BLOBENDPOINT) + private boolean enableBlobEndpoint; + + public boolean shouldEnableBlobEndPoint() { + return this.enableBlobEndpoint; + } + + @BooleanConfigurationValidatorAnnotation( + ConfigurationKey = FS_AZURE_MKDIRS_FALLBACK_TO_DFS, + DefaultValue = DEFAULT_FS_AZURE_MKDIRS_FALLBACK_TO_DFS) + private boolean mkdirFallbackToDfs; + + @BooleanConfigurationValidatorAnnotation( + ConfigurationKey = FS_AZURE_INGRESS_FALLBACK_TO_DFS, + DefaultValue = DEFAULT_FS_AZURE_INGRESS_FALLBACK_TO_DFS) + private boolean ingressFallbackToDfs; + + @BooleanConfigurationValidatorAnnotation( + ConfigurationKey = FS_AZURE_READ_FALLBACK_TO_DFS, + DefaultValue = DEFAULT_AZURE_READ_FALLBACK_TO_DFS) + private boolean readFallbackToDfs; + + public boolean shouldMkdirFallbackToDfs() { + return mkdirFallbackToDfs; + } + + public boolean shouldIngressFallbackToDfs() { + return ingressFallbackToDfs; + } + + public boolean shouldReadFallbackToDfs() { + return readFallbackToDfs; + } + /** * Gets the Azure Storage account name corresponding to this instance of configuration. * @return the Azure Storage account name @@ -680,10 +795,22 @@ public boolean isEnabledMkdirOverwrite() { return mkdirOverwrite; } + public boolean isEnabledBlobMkdirOverwrite() { + return blobMkdirOverwrite; + } + public String getAppendBlobDirs() { return this.azureAppendBlobDirs; } + public boolean isExpectHeaderEnabled() { + return this.isExpectHeaderEnabled; + } + + public boolean accountThrottlingEnabled() { + return accountThrottlingEnabled; + } + public String getAzureInfiniteLeaseDirs() { return this.azureInfiniteLeaseDirs; } @@ -726,8 +853,16 @@ public boolean isAutoThrottlingEnabled() { return this.enableAutoThrottling; } + public int getAccountOperationIdleTimeout() { + return accountOperationIdleTimeout; + } + + public int getAnalysisPeriod() { + return analysisPeriod; + } + public String getCustomUserAgentPrefix() { - return this.userAgentId; + return "abfsdriverV2.5"; } public String getClusterName() { @@ -906,6 +1041,15 @@ public SASTokenProvider getSASTokenProvider() throws AzureBlobFileSystemExceptio } } + public boolean isReadAheadEnabled() { + return this.enabledReadAhead; + } + + @VisibleForTesting + void setReadAheadEnabled(final boolean enabledReadAhead) { + this.enabledReadAhead = enabledReadAhead; + } + public int getReadAheadRange() { return this.readAheadRange; } @@ -1013,6 +1157,18 @@ public String getClientProvidedEncryptionKey() { return rawConfig.get(accSpecEncKey, null); } + public int getBlobDirRenameMaxThread() { + return blobDirRenameMaxThread; + } + + public int getBlobDirDeleteMaxThread() { + return blobDirDeleteMaxThread; + } + + public long getBlobCopyProgressPollWaitMillis() { + return blobCopyProgressPollWaitMillis; + } + @VisibleForTesting void setReadBufferSize(int bufferSize) { this.readBufferSize = bufferSize; @@ -1083,4 +1239,11 @@ public void setEnableAbfsListIterator(boolean enableAbfsListIterator) { this.enableAbfsListIterator = enableAbfsListIterator; } + public int getProducerQueueMaxSize() { + return producerQueueMaxSize; + } + + public boolean isLeaseOnCreateNonRecursive() { + return leaseOnCreateNonRecursive; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 750306c4a983f4..a5d8f9af2526ff 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -26,6 +26,7 @@ import java.net.HttpURLConnection; import java.net.URI; import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; import java.nio.file.AccessDeniedException; import java.util.Hashtable; import java.util.List; @@ -42,15 +43,25 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; +import org.apache.hadoop.fs.azurebfs.services.AbfsBlobLease; +import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; +import org.apache.hadoop.fs.azurebfs.services.BlobProperty; +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.azurebfs.services.OperativeEndpoint; +import org.apache.hadoop.fs.azurebfs.services.PathInformation; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; +import org.apache.hadoop.fs.azurebfs.services.RenameAtomicityUtils; +import org.apache.hadoop.fs.azurebfs.services.RenameNonAtomicUtils; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.apache.hadoop.fs.azure.NativeAzureFileSystem; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.commons.lang3.ArrayUtils; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; -import org.apache.hadoop.fs.azurebfs.services.AbfsClientThrottlingIntercept; import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.classification.InterfaceStability; @@ -102,14 +113,26 @@ import org.apache.hadoop.util.LambdaUtils; import org.apache.hadoop.util.Progressable; +import static java.net.HttpURLConnection.HTTP_CONFLICT; import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL; import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_DEFAULT; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.*; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOB_LEASE_ONE_MINUTE_DURATION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FORWARD_SLASH; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_BLOB_ENDPOINT; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DNS_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX; +import static org.apache.hadoop.fs.azurebfs.services.RenameAtomicityUtils.SUFFIX; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.DATA_BLOCKS_BUFFER; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BLOCK_UPLOAD_ACTIVE_BLOCKS; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BLOCK_UPLOAD_BUFFER_DIR; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_UPLOAD_ACTIVE_BLOCKS_DEFAULT; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DATA_BLOCKS_BUFFER_DEFAULT; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.PATH_EXISTS; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND; +import static org.apache.hadoop.fs.azurebfs.constants.InternalConstants.CAPABILITY_SAFE_READAHEAD; +import static org.apache.hadoop.fs.azurebfs.utils.UriUtils.decodeMetadataAttribute; +import static org.apache.hadoop.fs.azurebfs.utils.UriUtils.encodeMetadataAttribute; import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtLevel; import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator; @@ -142,6 +165,9 @@ public class AzureBlobFileSystem extends FileSystem private DataBlocks.BlockFactory blockFactory; /** Maximum Active blocks per OutputStream. */ private int blockOutputActiveBlocks; + private PrefixMode prefixMode = PrefixMode.DFS; + private boolean isNamespaceEnabled; + private NativeAzureFileSystem nativeFs; @Override public void initialize(URI uri, Configuration configuration) @@ -190,9 +216,31 @@ public void initialize(URI uri, Configuration configuration) tracingHeaderFormat = abfsConfiguration.getTracingHeaderFormat(); this.setWorkingDirectory(this.getHomeDirectory()); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, listener); + try { + isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); + } catch (AbfsRestOperationException ex) { + /* since the filesystem has not been created. The API for HNS account would + * return 404 status. + */ + if(ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { + isNamespaceEnabled = true; + } else { + throw ex; + } + } + // CPK is not supported over blob endpoint and hence initialization should fail if key is not null. + if (!isNamespaceEnabled && (abfsConfiguration.shouldEnableBlobEndPoint() || + uri.toString().contains(FileSystemUriSchemes.WASB_DNS_PREFIX))) { + if (abfsConfiguration.getClientProvidedEncryptionKey() == null) { + this.prefixMode = PrefixMode.BLOB; + } else { + throw new InvalidConfigurationValueException("CPK is not supported over blob endpoint " + uri); + } + } + abfsConfiguration.setPrefixMode(this.prefixMode); if (abfsConfiguration.getCreateRemoteFileSystemDuringInitialization()) { - TracingContext tracingContext = new TracingContext(clientCorrelationId, - fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, listener); if (this.tryGetFileStatus(new Path(AbfsHttpConstants.ROOT_PATH), tracingContext) == null) { try { this.createFileSystem(tracingContext); @@ -214,8 +262,25 @@ public void initialize(URI uri, Configuration configuration) } } - AbfsClientThrottlingIntercept.initializeSingleton(abfsConfiguration.isAutoThrottlingEnabled()); - + boolean isRedirect = abfsConfiguration.isRedirection(); + if (isRedirect) { + String abfsUrl = uri.toString(); + URI wasbUri = null; + try { + wasbUri = new URI(abfsUrlToWasbUrl(abfsUrl, + abfsStore.getAbfsConfiguration().isHttpsAlwaysUsed())); + } catch (URISyntaxException e) { + e.printStackTrace(); + } + nativeFs = new NativeAzureFileSystem(); + Configuration config = getConf(); + try { + nativeFs.initialize(wasbUri, config); + } catch (IOException e) { + LOG.debug("Initializing NativeAzureBlobFileSystem failed ", e); + throw e; + } + } LOG.debug("Initializing AzureBlobFileSystem for {} complete", uri); } @@ -226,6 +291,7 @@ public String toString() { sb.append("uri=").append(uri); sb.append(", user='").append(abfsStore.getUser()).append('\''); sb.append(", primaryUserGroup='").append(abfsStore.getPrimaryGroup()).append('\''); + sb.append("[" + CAPABILITY_SAFE_READAHEAD + "]"); sb.append('}'); return sb.toString(); } @@ -243,6 +309,45 @@ public void registerListener(Listener listener1) { listener = listener1; } + private static String convertTestUrls( + final String url, + final String fromNonSecureScheme, + final String fromSecureScheme, + final String fromDnsPrefix, + final String toNonSecureScheme, + final String toSecureScheme, + final String toDnsPrefix, + final boolean isAlwaysHttpsUsed) { + String data = null; + if (url.startsWith(fromNonSecureScheme + "://") && isAlwaysHttpsUsed) { + data = url.replace(fromNonSecureScheme + "://", toSecureScheme + "://"); + } else if (url.startsWith(fromNonSecureScheme + "://")) { + data = url.replace(fromNonSecureScheme + "://", toNonSecureScheme + "://"); + } else if (url.startsWith(fromSecureScheme + "://")) { + data = url.replace(fromSecureScheme + "://", toSecureScheme + "://"); + } + + if (data != null) { + data = data.replace(fromDnsPrefix , toDnsPrefix); + } + return data; + } + + protected static String abfsUrlToWasbUrl(final String abfsUrl, final boolean isAlwaysHttpsUsed) { + return convertTestUrls( + abfsUrl, FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME, FileSystemUriSchemes.ABFS_DNS_PREFIX, + FileSystemUriSchemes.WASB_SCHEME, FileSystemUriSchemes.WASB_SECURE_SCHEME, FileSystemUriSchemes.WASB_DNS_PREFIX, isAlwaysHttpsUsed); + } + + private URI changePrefixFromDfsToBlob(URI uri) throws InvalidUriException { + try { + String uriString = uri.toString().replace(ABFS_DNS_PREFIX, WASB_DNS_PREFIX); + return new URI(uriString); + } catch (URISyntaxException ex) { + throw new InvalidUriException(uri.toString()); + } + } + @Override public FSDataInputStream open(final Path path, final int bufferSize) throws IOException { LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize); @@ -251,7 +356,7 @@ public FSDataInputStream open(final Path path, final int bufferSize) throws IOEx } private FSDataInputStream open(final Path path, - final Optional options) throws IOException { + final Optional parameters) throws IOException { statIncrement(CALL_OPEN); Path qualifiedPath = makeQualified(path); @@ -259,8 +364,8 @@ private FSDataInputStream open(final Path path, TracingContext tracingContext = new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.OPEN, tracingHeaderFormat, listener); - InputStream inputStream = abfsStore.openFileForRead(qualifiedPath, - options, statistics, tracingContext); + InputStream inputStream = getAbfsStore().openFileForRead(qualifiedPath, + parameters, statistics, tracingContext); return new FSDataInputStream(inputStream); } catch(AzureBlobFileSystemException ex) { checkException(path, ex); @@ -268,6 +373,15 @@ private FSDataInputStream open(final Path path, } } + /** + * Takes config and other options through + * {@link org.apache.hadoop.fs.impl.OpenFileParameters}. Ensure that + * FileStatus entered is up-to-date, as it will be used to create the + * InputStream (with info such as contentLength, eTag) + * @param path The location of file to be opened + * @param parameters OpenFileParameters instance; can hold FileStatus, + * Configuration, bufferSize and mandatoryKeys + */ @Override protected CompletableFuture openFileWithOptions( final Path path, final OpenFileParameters parameters) throws IOException { @@ -278,12 +392,51 @@ protected CompletableFuture openFileWithOptions( "for " + path); return LambdaUtils.eval( new CompletableFuture<>(), () -> - open(path, Optional.of(parameters.getOptions()))); + open(path, Optional.of(parameters))); + } + + private boolean shouldRedirect(FSOperationType type, TracingContext context) + throws AzureBlobFileSystemException { + if (getIsNamespaceEnabled(context)) { + return false; + } + switch (type) { + case DELETE: + return abfsStore.getAbfsConfiguration().shouldRedirectDelete(); + case RENAME: + return abfsStore.getAbfsConfiguration().shouldRedirectRename(); + } + + return false; } + // Fallback plan : default to v1 create flow which will hit dfs endpoint. Config to enable: "fs.azure.ingress.fallback.to.dfs". @Override public FSDataOutputStream create(final Path f, final FsPermission permission, final boolean overwrite, final int bufferSize, final short replication, final long blockSize, final Progressable progress) throws IOException { + return create(f, permission, overwrite, bufferSize, replication, blockSize, + progress, false); + } + + /** + * Creates a file in the file system with the specified parameters. + * @param f the path of the file to create + * @param permission the permission of the file + * @param overwrite whether to overwrite the existing file if any + * @param bufferSize the size of the buffer to be used + * @param replication the number of replicas for the file + * @param blockSize the size of the block for the file + * @param progress the progress indicator for the file creation + * @param blobParentDirPresentChecked whether the presence of parent directory + * been checked + * @return a FSDataOutputStream object that can be used to write to the file + * @throws IOException if an error occurs while creating the file + */ + private FSDataOutputStream create(final Path f, + final FsPermission permission, + final boolean overwrite, final int bufferSize, + final short replication, + final long blockSize, final Progressable progress, final Boolean blobParentDirPresentChecked) throws IOException { LOG.debug("AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}", f, permission, @@ -292,15 +445,40 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi statIncrement(CALL_CREATE); trailingPeriodCheck(f); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.CREATE, overwrite, tracingHeaderFormat, listener); Path qualifiedPath = makeQualified(f); + FileStatus fileStatus = tryGetFileStatus(qualifiedPath, tracingContext); + // This fix is needed for create idempotency, should throw error if overwrite is false and file status is not null. + boolean fileOverwrite = overwrite; + if (!fileOverwrite) { + if (fileStatus != null) { + // path references a file and overwrite is disabled + throw new FileAlreadyExistsException(f + " already exists"); + } + fileOverwrite = true; + } + + if (prefixMode == PrefixMode.BLOB) { + if (fileStatus != null && fileStatus.isDirectory()) { + throw new AbfsRestOperationException(HTTP_CONFLICT, + AzureServiceErrorCode.PATH_CONFLICT.getErrorCode(), + PATH_EXISTS, + null); + } + if (!blobParentDirPresentChecked) { + Path parent = qualifiedPath.getParent(); + if (parent != null && !parent.isRoot()) { + mkdirs(parent); + } + } + } try { - TracingContext tracingContext = new TracingContext(clientCorrelationId, - fileSystemId, FSOperationType.CREATE, overwrite, tracingHeaderFormat, listener); - OutputStream outputStream = abfsStore.createFile(qualifiedPath, statistics, overwrite, + OutputStream outputStream = getAbfsStore().createFile(qualifiedPath, statistics, fileOverwrite, permission == null ? FsPermission.getFileDefault() : permission, - FsPermission.getUMask(getConf()), tracingContext); + FsPermission.getUMask(getConf()), tracingContext, null); statIncrement(FILES_CREATED); return new FSDataOutputStream(outputStream, statistics); } catch(AzureBlobFileSystemException ex) { @@ -316,18 +494,49 @@ public FSDataOutputStream createNonRecursive(final Path f, final FsPermission pe final Progressable progress) throws IOException { statIncrement(CALL_CREATE_NON_RECURSIVE); + + if (f.isRoot()) { + // Throwing same exception as create() on root. + throw new AbfsRestOperationException(HTTP_CONFLICT, + AzureServiceErrorCode.PATH_CONFLICT.getErrorCode(), + PATH_EXISTS, + null); + } + final Path parent = f.getParent(); TracingContext tracingContext = new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.CREATE_NON_RECURSIVE, tracingHeaderFormat, listener); + /* + * Get exclusive access to folder if this is a directory designated for atomic + * rename. The primary use case of the HBase write-ahead log file management. + */ + AbfsBlobLease abfsBlobLease = null; + String parentPath = parent.toUri().getPath(); + if (getAbfsStore().getPrefixMode() == PrefixMode.BLOB + && getAbfsStore().isAtomicRenameKey(parentPath)) { + if (getAbfsStore().getAbfsConfiguration().isLeaseOnCreateNonRecursive()) { + abfsBlobLease = new AbfsBlobLease(getAbfsClient(), + parentPath, BLOB_LEASE_ONE_MINUTE_DURATION, tracingContext); + } + } final FileStatus parentFileStatus = tryGetFileStatus(parent, tracingContext); - if (parentFileStatus == null) { + if (parentFileStatus == null || !parentFileStatus.isDirectory()) { + if (abfsBlobLease != null) { + abfsBlobLease.free(); + } throw new FileNotFoundException("Cannot create file " - + f.getName() + " because parent folder does not exist."); + + f.getName() + + " because parent folder does not exist or is a file."); } - return create(f, permission, overwrite, bufferSize, replication, blockSize, progress); + final FSDataOutputStream outputStream = create(f, permission, overwrite, + bufferSize, replication, blockSize, progress, true); + if (abfsBlobLease != null) { + abfsBlobLease.free(); + } + return outputStream; } @Override @@ -356,6 +565,7 @@ public FSDataOutputStream createNonRecursive(final Path f, overwrite, bufferSize, replication, blockSize, progress); } + // Fallback plan : default to v1 append flow which will hit dfs endpoint. Config to enable: "fs.azure.ingress.fallback.to.dfs". @Override public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress) throws IOException { LOG.debug( @@ -369,7 +579,7 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr TracingContext tracingContext = new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.APPEND, tracingHeaderFormat, listener); - OutputStream outputStream = abfsStore + OutputStream outputStream = getAbfsStore() .openFileForWrite(qualifiedPath, statistics, false, tracingContext); return new FSDataOutputStream(outputStream, statistics); } catch(AzureBlobFileSystemException ex) { @@ -378,10 +588,134 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr } } + /** + * Checks if the paths are allowable for executing rename steps over Blob endpoint. + * + * @param src source which has to be renamed + * @param dst destination which will contain the renamed blobs. + * @param fnsDstPathInformation pathInformation for the given dst + * @param tracingContext tracingContext for tracing the server calls + * + * @return true if all checks pass. + * @throws IOException exceptions received in the server calls. Or the exceptions + * thrown from the client which are equal to the exceptions that would be thrown + * from the DFS endpoint server for certain scenarios. + */ + private Boolean applyBlobRenameChecks(final Path src, + final Path dst, + final PathInformation fnsDstPathInformation, + final TracingContext tracingContext) + throws IOException { + if (containsColon(dst)) { + throw new IOException("Cannot rename to file " + dst + + " that has colons in the name through blob endpoint"); + } + + Path qualifiedSrcPath = makeQualified(src); + Path qualifiedDstPath = makeQualified(dst); + Path nestedDstParent = dst.getParent(); + + /* + * Special case 1: + * For blob endpoint with non-HNS account, client has to ensure that destination + * is not a sub-directory of source. + */ + LOG.debug("Check if the destination is subDirectory"); + if (nestedDstParent != null && makeQualified(nestedDstParent).toUri() + .getPath() + .indexOf(qualifiedSrcPath.toUri().getPath()) == 0) { + LOG.info("Rename src: {} dst: {} failed as dst is subDir of src", + qualifiedSrcPath, qualifiedDstPath); + return false; + } + + if (fnsDstPathInformation.getPathExists()) { + if (fnsDstPathInformation.getIsDirectory()) { + /* + * For blob-endpoint with nonHNS account, check if the qualifiedDstPath + * exist in backend. If yes, HTTP_CONFLICT exception has to be thrown. + */ + + String sourceFileName = src.getName(); + Path adjustedDst = new Path(dst, sourceFileName); + qualifiedDstPath = makeQualified(adjustedDst); + + final PathInformation qualifiedDstPathInformation + = getPathInformation(qualifiedDstPath, tracingContext + ); + final Boolean isQualifiedDstExists + = qualifiedDstPathInformation.getPathExists(); + if (isQualifiedDstExists) { + //destination already there. Rename should not be overwriting. + LOG.info( + "Rename src: {} dst: {} failed as qualifiedDst already exists", + qualifiedSrcPath, qualifiedDstPath); + throw new AbfsRestOperationException( + HttpURLConnection.HTTP_CONFLICT, + AzureServiceErrorCode.PATH_ALREADY_EXISTS.getErrorCode(), null, + null); + } + } + } else { + /* + * If the destination doesn't exist, check if parent of destination exists. + */ + Path adjustedDst = dst; + qualifiedDstPath = makeQualified(adjustedDst); + Path parent = qualifiedDstPath.getParent(); + if (parent != null && !parent.isRoot()) { + PathInformation dstParentPathInformation = getPathInformation(parent, + tracingContext + ); + final Boolean dstParentPathExists + = dstParentPathInformation.getPathExists(); + final Boolean isDstParentPathDirectory + = dstParentPathInformation.getIsDirectory(); + if (!dstParentPathExists || !isDstParentPathDirectory) { + LOG.info("parent of {} is {} is not directory. Failing rename", + adjustedDst, parent); + throw new AbfsRestOperationException( + HttpURLConnection.HTTP_NOT_FOUND, + RENAME_DESTINATION_PARENT_PATH_NOT_FOUND.getErrorCode(), null, + new Exception( + RENAME_DESTINATION_PARENT_PATH_NOT_FOUND.getErrorCode())); + } + } + } + return true; + } + public boolean rename(final Path src, final Path dst) throws IOException { - LOG.debug("AzureBlobFileSystem.rename src: {} dst: {}", src, dst); + LOG.debug("AzureBlobFileSystem.rename src: {} dst: {} via {} endpoint", src, dst, + getAbfsStore().getAbfsConfiguration().getPrefixMode()); statIncrement(CALL_RENAME); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.RENAME, true, tracingHeaderFormat, + listener); + + if (shouldRedirect(FSOperationType.RENAME, tracingContext)) { + LOG.debug("Rename redirected for the given src {} and the given destination {}", src, dst); + Path wasbSrc = src; + Path wasbDest = dst; + if (FileSystemUriSchemes.ABFS_SCHEME.equalsIgnoreCase(src.toUri().getScheme()) + || FileSystemUriSchemes.ABFS_SECURE_SCHEME.equalsIgnoreCase(src.toUri().getScheme())) { + wasbSrc = new Path(abfsUrlToWasbUrl(src.toString(), + abfsStore.getAbfsConfiguration().isHttpsAlwaysUsed())); + } + if (FileSystemUriSchemes.ABFS_SCHEME.equalsIgnoreCase(dst.toUri().getScheme()) + || FileSystemUriSchemes.ABFS_SECURE_SCHEME.equalsIgnoreCase(dst.toUri().getScheme())) { + wasbDest = new Path(abfsUrlToWasbUrl(dst.toString(), + abfsStore.getAbfsConfiguration().isHttpsAlwaysUsed())); + } + try { + return getNativeFs().rename(wasbSrc, wasbDest); + } catch (IOException e) { + LOG.debug("Rename redirection failed for the given src {} and the given destination {}", src, dst); + throw e; + } + } + trailingPeriodCheck(dst); Path parentFolder = src.getParent(); @@ -391,48 +725,81 @@ public boolean rename(final Path src, final Path dst) throws IOException { Path qualifiedSrcPath = makeQualified(src); Path qualifiedDstPath = makeQualified(dst); - TracingContext tracingContext = new TracingContext(clientCorrelationId, - fileSystemId, FSOperationType.RENAME, true, tracingHeaderFormat, - listener); + // special case 2: // rename under same folder; - if(makeQualified(parentFolder).equals(qualifiedDstPath)) { - return tryGetFileStatus(qualifiedSrcPath, tracingContext) != null; + if (makeQualified(parentFolder).equals(qualifiedDstPath)) { + PathInformation pathInformation = getPathInformation(qualifiedSrcPath, + tracingContext); + return pathInformation.getPathExists(); } - FileStatus dstFileStatus = null; + //special case 3: if (qualifiedSrcPath.equals(qualifiedDstPath)) { // rename to itself // - if it doesn't exist, return false // - if it is file, return true // - if it is dir, return false. - dstFileStatus = tryGetFileStatus(qualifiedDstPath, tracingContext); - if (dstFileStatus == null) { + + final PathInformation pathInformation = getPathInformation( + qualifiedDstPath, tracingContext + ); + final Boolean isDstExists = pathInformation.getPathExists(); + final Boolean isDstDirectory = pathInformation.getIsDirectory(); + if (!isDstExists) { return false; } - return dstFileStatus.isDirectory() ? false : true; + return isDstDirectory ? false : true; } + // special case 4: // Non-HNS account need to check dst status on driver side. - if (!abfsStore.getIsNamespaceEnabled(tracingContext) && dstFileStatus == null) { - dstFileStatus = tryGetFileStatus(qualifiedDstPath, tracingContext); + PathInformation fnsPathInformation = null; + if (!abfsStore.getIsNamespaceEnabled(tracingContext)) { + fnsPathInformation = getPathInformation(qualifiedDstPath, tracingContext + ); } try { + final Boolean isFnsDstExists, isFnsDstDirectory; + if (fnsPathInformation != null) { + isFnsDstDirectory = fnsPathInformation.getIsDirectory(); + isFnsDstExists = fnsPathInformation.getPathExists(); + } else { + isFnsDstExists = false; + isFnsDstDirectory = false; + } String sourceFileName = src.getName(); Path adjustedDst = dst; - if (dstFileStatus != null) { - if (!dstFileStatus.isDirectory()) { + if (isFnsDstExists) { + if (!isFnsDstDirectory) { return qualifiedSrcPath.equals(qualifiedDstPath); } adjustedDst = new Path(dst, sourceFileName); } - qualifiedDstPath = makeQualified(adjustedDst); - - abfsStore.rename(qualifiedSrcPath, qualifiedDstPath, tracingContext); + LOG.debug("Qualified dst path: {}", qualifiedDstPath); + + final RenameAtomicityUtils renameAtomicityUtils; + if (getAbfsStore().getAbfsConfiguration().getPrefixMode() + == PrefixMode.BLOB && + abfsStore.isAtomicRenameKey(qualifiedSrcPath.toUri().getPath())) { + renameAtomicityUtils = new RenameAtomicityUtils(this, + qualifiedSrcPath, qualifiedDstPath, tracingContext); + } else { + renameAtomicityUtils = new RenameNonAtomicUtils(this, + qualifiedSrcPath, qualifiedDstPath, tracingContext); + } + if(getAbfsStore().getAbfsConfiguration().getPrefixMode() == PrefixMode.BLOB) { + if (!applyBlobRenameChecks(src, dst, fnsPathInformation, + tracingContext)) { + return false; + } + } + getAbfsStore().rename(qualifiedSrcPath, qualifiedDstPath, renameAtomicityUtils, + tracingContext); return true; - } catch(AzureBlobFileSystemException ex) { + } catch (AzureBlobFileSystemException ex) { LOG.debug("Rename operation failed. ", ex); checkException( src, @@ -445,7 +812,58 @@ public boolean rename(final Path src, final Path dst) throws IOException { AzureServiceErrorCode.INTERNAL_OPERATION_ABORT); return false; } + } + /** + * Defines if the given path exists and if is it a directory.
+ * If the path check is for a blob (non-HNS over blob-endpoint). First it will + * call the listBlob API on the given path. If it returns list of object, then + * it can be defined that it exist and it is a directory. Else, it will call + * getBlobProperties API on the path. If it returns an object, it can be defined + * that the path exists. If the object contains the metadata + * {@link org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations#X_MS_META_HDI_ISFOLDER}. + * If yes, the path can be defined as directory.
+ * If the path check is for a path on dfs-endpoint, getPathStatus API for the path + * shall be called. If the response returned an object, the path can be defined + * as existing. If the response's metadata contains it is directory, the path + * can be defined as a directory. + * + * @param path path for which information is requried. + * @param tracingContext tracingContext for the operations. + * + * @return pathInformation containing if path exists and is a directory. + * + * @throws AzureBlobFileSystemException exceptions caught from the server calls. + */ + private PathInformation getPathInformation(final Path path, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + if (getAbfsStore().getAbfsConfiguration().getPrefixMode() + == PrefixMode.BLOB) { + List blobProperties = getAbfsStore() + .getListBlobs(path, null, null, tracingContext, 2, true); + if (blobProperties.size() > 0) { + return new PathInformation(true, true); + } + BlobProperty blobProperty; + try { + blobProperty = getAbfsStore().getBlobProperty(path, tracingContext); + } catch (AbfsRestOperationException ex) { + if (ex.getStatusCode() != HttpURLConnection.HTTP_NOT_FOUND) { + throw ex; + } + blobProperty = null; + } + if (blobProperty != null) { + return new PathInformation(true, blobProperty.getIsDirectory()); + } + } else { + final FileStatus fileStatus = tryGetFileStatus(path, + tracingContext); + if (fileStatus != null) { + return new PathInformation(true, fileStatus.isDirectory()); + } + } + return new PathInformation(false, false); } @Override @@ -454,20 +872,43 @@ public boolean delete(final Path f, final boolean recursive) throws IOException "AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), recursive); statIncrement(CALL_DELETE); Path qualifiedPath = makeQualified(f); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.DELETE, true, tracingHeaderFormat, + listener); + + if (shouldRedirect(FSOperationType.DELETE, tracingContext)) { + LOG.debug("Delete redirected for the given path {} ", qualifiedPath); + Path wasbPath = f; + if (FileSystemUriSchemes.ABFS_SCHEME.equalsIgnoreCase(wasbPath.toUri().getScheme()) + || FileSystemUriSchemes.ABFS_SECURE_SCHEME.equalsIgnoreCase(wasbPath.toUri().getScheme())) { + wasbPath = new Path(abfsUrlToWasbUrl(wasbPath.toString(), + abfsStore.getAbfsConfiguration().isHttpsAlwaysUsed())); + } + try { + return getNativeFs().delete(wasbPath, recursive); + } catch (IOException e) { + LOG.debug("Delete redirection failed for the given path {} ", qualifiedPath); + throw e; + } + } if (f.isRoot()) { if (!recursive) { return false; } - return deleteRoot(); + /* + * For DFS root directory deletion, direct DELETE Path API on root doesn't + * work, the server returns: "The request URI is invalid.", 400, DELETE. + * Hence, there is a special handling for root deletion on DFS endpoint + */ + if (getAbfsStore().getPrefixMode() == PrefixMode.DFS) { + return deleteRoot(); + } } try { - TracingContext tracingContext = new TracingContext(clientCorrelationId, - fileSystemId, FSOperationType.DELETE, tracingHeaderFormat, - listener); - abfsStore.delete(qualifiedPath, recursive, tracingContext); + getAbfsStore().delete(qualifiedPath, recursive, tracingContext); return true; } catch (AzureBlobFileSystemException ex) { checkException(f, ex, AzureServiceErrorCode.PATH_NOT_FOUND); @@ -487,7 +928,42 @@ public FileStatus[] listStatus(final Path f) throws IOException { TracingContext tracingContext = new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.LISTSTATUS, true, tracingHeaderFormat, listener); - FileStatus[] result = abfsStore.listStatus(qualifiedPath, tracingContext); + FileStatus[] result = getAbfsStore().listStatus(qualifiedPath, + tracingContext); + if (getAbfsStore().getAbfsConfiguration().getPrefixMode() + == PrefixMode.BLOB && getAbfsStore().isAtomicRenameKey( + qualifiedPath.toUri().getPath() + FORWARD_SLASH)) { + Pair renamePendingJsonAndSrcFileStatusPair + = getAbfsStore().getRenamePendingFileStatus(result); + FileStatus renamePendingSrcFileStatus + = renamePendingJsonAndSrcFileStatusPair.getRight(); + FileStatus renamePendingJsonFileStatus + = renamePendingJsonAndSrcFileStatusPair.getLeft(); + if (renamePendingJsonFileStatus != null) { + final Boolean isRedone; + if (renamePendingSrcFileStatus != null) { + RenameAtomicityUtils renameAtomicityUtils = + getRenameAtomicityUtilsForRedo( + renamePendingJsonFileStatus.getPath(), + tracingContext, + ((AzureBlobFileSystemStore.VersionedFileStatus) renamePendingSrcFileStatus).getEtag(), + getRenamePendingJsonInputStream( + renamePendingJsonFileStatus, tracingContext)); + renameAtomicityUtils.cleanup(renamePendingJsonFileStatus.getPath()); + isRedone = renameAtomicityUtils.isRedone(); + } else { + isRedone = false; + getAbfsStore().delete(renamePendingJsonFileStatus.getPath(), true, + tracingContext); + } + if (isRedone) { + result = getAbfsStore().listStatus(qualifiedPath, tracingContext); + } else { + result = ArrayUtils.removeElement(result, + renamePendingJsonFileStatus); + } + } + } return result; } catch (AzureBlobFileSystemException ex) { checkException(f, ex); @@ -495,6 +971,15 @@ public FileStatus[] listStatus(final Path f) throws IOException { } } + RenameAtomicityUtils getRenameAtomicityUtilsForRedo(final Path renamePendingJsonPath, + final TracingContext tracingContext, final String srcEtag, + final AbfsInputStream renamePendingJsonInputStream) throws IOException { + return new RenameAtomicityUtils(this, + renamePendingJsonPath, + getAbfsStore().getRedoRenameInvocation(tracingContext), srcEtag, + renamePendingJsonInputStream); + } + /** * Increment of an Abfs statistic. * @@ -540,6 +1025,7 @@ private void trailingPeriodCheck(Path path) throws IllegalArgumentException { } } + // Fallback plan : default to v1 Mkdir flow which will hit dfs endpoint. Config to enable: "fs.azure.mkdirs.fallback.to.dfs". @Override public boolean mkdirs(final Path f, final FsPermission permission) throws IOException { LOG.debug( @@ -557,11 +1043,11 @@ public boolean mkdirs(final Path f, final FsPermission permission) throws IOExce try { TracingContext tracingContext = new TracingContext(clientCorrelationId, - fileSystemId, FSOperationType.MKDIR, false, tracingHeaderFormat, - listener); - abfsStore.createDirectory(qualifiedPath, - permission == null ? FsPermission.getDirDefault() : permission, - FsPermission.getUMask(getConf()), tracingContext); + fileSystemId, FSOperationType.MKDIR, false, tracingHeaderFormat, + listener); + getAbfsStore().createDirectory(qualifiedPath, statistics, + permission == null ? FsPermission.getDirDefault() : permission, + FsPermission.getUMask(getConf()), true, tracingContext); statIncrement(DIRECTORIES_CREATED); return true; } catch (AzureBlobFileSystemException ex) { @@ -570,6 +1056,7 @@ public boolean mkdirs(final Path f, final FsPermission permission) throws IOExce } } + @Override public synchronized void close() throws IOException { if (isClosed) { @@ -594,7 +1081,7 @@ public synchronized void close() throws IOException { @Override public FileStatus getFileStatus(final Path f) throws IOException { TracingContext tracingContext = new TracingContext(clientCorrelationId, - fileSystemId, FSOperationType.GET_FILESTATUS, tracingHeaderFormat, + fileSystemId, FSOperationType.GET_FILESTATUS, true, tracingHeaderFormat, listener); return getFileStatus(f, tracingContext); } @@ -604,15 +1091,71 @@ private FileStatus getFileStatus(final Path path, LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", path); statIncrement(CALL_GET_FILE_STATUS); Path qualifiedPath = makeQualified(path); + PrefixMode prefixMode = getAbfsStore().getPrefixMode(); + AbfsConfiguration abfsConfiguration = getAbfsStore().getAbfsConfiguration(); + boolean useBlobEndpoint = !(OperativeEndpoint.isIngressEnabledOnDFS(prefixMode, abfsConfiguration) || + OperativeEndpoint.isMkdirEnabledOnDFS(abfsConfiguration) || + OperativeEndpoint.isReadEnabledOnDFS(abfsConfiguration)); try { - return abfsStore.getFileStatus(qualifiedPath, tracingContext); - } catch(AzureBlobFileSystemException ex) { + /** + * Get File Status over Blob Endpoint will Have an additional call + * to check if directory is implicit. + */ + final FileStatus fileStatus = getAbfsStore().getFileStatus(qualifiedPath, + tracingContext, useBlobEndpoint); + final String filePathStr = qualifiedPath.toUri().getPath(); + if (getAbfsStore().getPrefixMode() == PrefixMode.BLOB + && fileStatus != null && fileStatus.isDirectory() + && getAbfsStore().isAtomicRenameKey(filePathStr)) { + FileStatus renamePendingJsonFileStatus; + try { + renamePendingJsonFileStatus = getAbfsStore().getPathProperty( + makeQualified( + new Path(filePathStr + SUFFIX)), + tracingContext, true); + } catch (AbfsRestOperationException ex) { + if(ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { + renamePendingJsonFileStatus = null; + } else { + throw ex; + } + } + + if (renamePendingJsonFileStatus != null) { + RenameAtomicityUtils renameAtomicityUtils + = getRenameAtomicityUtilsForRedo( + renamePendingJsonFileStatus.getPath(), + tracingContext, + ((AzureBlobFileSystemStore.VersionedFileStatus) fileStatus).getEtag(), + getRenamePendingJsonInputStream(renamePendingJsonFileStatus, tracingContext)); + renameAtomicityUtils.cleanup(renamePendingJsonFileStatus.getPath()); + if (renameAtomicityUtils.isRedone()) { + throw new AbfsRestOperationException( + HttpURLConnection.HTTP_NOT_FOUND, + AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), null, + new FileNotFoundException( + qualifiedPath + ": No such file or directory.")); + } + } + } + return fileStatus; + } catch (AzureBlobFileSystemException ex) { checkException(path, ex); return null; } } + private AbfsInputStream getRenamePendingJsonInputStream(final FileStatus renamePendingJsonFileStatus, + final TracingContext tracingContext) + throws IOException { + Path qualifiedPath = makeQualified(renamePendingJsonFileStatus.getPath()); + return getAbfsStore().openFileForRead(qualifiedPath, + Optional.of( + new OpenFileParameters().withStatus(renamePendingJsonFileStatus)), + statistics, tracingContext); + } + /** * Break the current lease on an ABFS file if it exists. A lease that is broken cannot be * renewed. A new lease may be obtained on the file immediately. @@ -853,13 +1396,38 @@ public void setXAttr(final Path path, final String name, final byte[] value, fin TracingContext tracingContext = new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.SET_ATTR, true, tracingHeaderFormat, listener); - Hashtable properties = abfsStore - .getPathStatus(qualifiedPath, tracingContext); + Hashtable properties; String xAttrName = ensureValidAttributeName(name); + String xAttrValue; + + if (abfsStore.getPrefixMode() == PrefixMode.BLOB) { + if (qualifiedPath.isRoot()) { + properties = abfsStore.getContainerMetadata(tracingContext); + } else { + properties = abfsStore.getBlobMetadata(qualifiedPath, tracingContext); + } + + boolean xAttrExists = properties.containsKey(xAttrName); + XAttrSetFlag.validate(name, xAttrExists, flag); + + // On Blob Endpoint metadata are passed as HTTP Request Headers + // Values in UTF_8 needed to be URL encoded after decoding into String + xAttrValue = encodeMetadataAttribute(new String(value, StandardCharsets.UTF_8)); + properties.put(xAttrName, xAttrValue); + if (qualifiedPath.isRoot()) { + abfsStore.setContainerMetadata(properties, tracingContext); + } else { + abfsStore.setBlobMetadata(qualifiedPath, properties, tracingContext); + } + + return; + } + + properties = abfsStore.getPathStatus(qualifiedPath, tracingContext); boolean xAttrExists = properties.containsKey(xAttrName); XAttrSetFlag.validate(name, xAttrExists, flag); - String xAttrValue = abfsStore.decodeAttribute(value); + xAttrValue = abfsStore.decodeAttribute(value); properties.put(xAttrName, xAttrValue); abfsStore.setPathProperties(qualifiedPath, properties, tracingContext); } catch (AzureBlobFileSystemException ex) { @@ -893,9 +1461,26 @@ public byte[] getXAttr(final Path path, final String name) TracingContext tracingContext = new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.GET_ATTR, true, tracingHeaderFormat, listener); - Hashtable properties = abfsStore - .getPathStatus(qualifiedPath, tracingContext); + Hashtable properties; String xAttrName = ensureValidAttributeName(name); + + if (abfsStore.getPrefixMode() == PrefixMode.BLOB) { + if (qualifiedPath.isRoot()) { + properties = abfsStore.getContainerMetadata(tracingContext); + } else { + properties = abfsStore.getBlobMetadata(qualifiedPath, tracingContext); + } + + if (properties.containsKey(xAttrName)) { + String xAttrValue = properties.get(xAttrName); + value = decodeMetadataAttribute(xAttrValue).getBytes( + StandardCharsets.UTF_8); + } + return value; + } + + properties = abfsStore.getPathStatus(qualifiedPath, tracingContext); + if (properties.containsKey(xAttrName)) { String xAttrValue = properties.get(xAttrName); value = abfsStore.encodeAttribute(xAttrValue); @@ -1258,7 +1843,12 @@ private void createFileSystem(TracingContext tracingContext) throws IOException LOG.debug( "AzureBlobFileSystem.createFileSystem uri: {}", uri); try { - abfsStore.createFilesystem(tracingContext); + PrefixMode prefixMode = getAbfsStore().getPrefixMode(); + AbfsConfiguration abfsConfiguration = getAbfsStore().getAbfsConfiguration(); + boolean useBlobEndpoint = !(OperativeEndpoint.isIngressEnabledOnDFS(prefixMode, abfsConfiguration) || + OperativeEndpoint.isMkdirEnabledOnDFS(abfsConfiguration) || + OperativeEndpoint.isReadEnabledOnDFS(abfsConfiguration)); + abfsStore.createFilesystem(tracingContext, useBlobEndpoint); } catch (AzureBlobFileSystemException ex) { checkException(null, ex); } @@ -1417,6 +2007,10 @@ private Throwable getRootCause(Throwable throwable) { return result; } + private boolean containsColon(Path p) { + return p.toUri().getPath().contains(":"); + } + /** * Get a delegation token from remote service endpoint if * 'fs.azure.enable.kerberos.support' is set to 'true', and @@ -1496,6 +2090,11 @@ boolean getIsNamespaceEnabled(TracingContext tracingContext) return abfsStore.getIsNamespaceEnabled(tracingContext); } + @VisibleForTesting + NativeAzureFileSystem getNativeFs() { + return nativeFs; + } + /** * Returns the counter() map in IOStatistics containing all the counters * and their values. @@ -1526,13 +2125,18 @@ public boolean hasPathCapability(final Path path, final String capability) case CommonPathCapabilities.FS_PERMISSIONS: case CommonPathCapabilities.FS_APPEND: case CommonPathCapabilities.ETAGS_AVAILABLE: - case CommonPathCapabilities.ETAGS_PRESERVED_IN_RENAME: return true; + case CommonPathCapabilities.ETAGS_PRESERVED_IN_RENAME: case CommonPathCapabilities.FS_ACLS: return getIsNamespaceEnabled( new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.HAS_PATH_CAPABILITY, tracingHeaderFormat, listener)); + + // probe for presence of the HADOOP-18546 readahead fix. + case CAPABILITY_SAFE_READAHEAD: + return true; + default: return super.hasPathCapability(p, capability); } @@ -1547,4 +2151,5 @@ public boolean hasPathCapability(final Path path, final String capability) public IOStatistics getIOStatistics() { return abfsCounters != null ? abfsCounters.getIOStatistics() : null; } + } 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 d86a3d96846171..b48b77be09743a 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 @@ -49,16 +49,37 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.TreeMap; import java.util.WeakHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.classification.VisibleForTesting; + +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; +import org.apache.hadoop.fs.azurebfs.enums.BlobCopyProgress; +import org.apache.hadoop.fs.azurebfs.services.AbfsBlobLease; +import org.apache.hadoop.fs.azurebfs.services.AbfsDfsLease; +import org.apache.hadoop.fs.azurebfs.services.ListBlobConsumer; +import org.apache.hadoop.fs.azurebfs.services.ListBlobProducer; +import org.apache.hadoop.fs.azurebfs.services.ListBlobQueue; +import org.apache.hadoop.fs.azurebfs.services.OperativeEndpoint; +import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; +import org.apache.hadoop.fs.azurebfs.services.BlobList; +import org.apache.hadoop.fs.azurebfs.services.BlobProperty; + +import org.apache.hadoop.fs.azurebfs.services.RenameAtomicityUtils; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableFuture; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -118,6 +139,7 @@ import org.apache.hadoop.fs.azurebfs.utils.DateTimeUtils; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.UriUtils; +import org.apache.hadoop.fs.impl.OpenFileParameters; import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.FsAction; @@ -130,18 +152,43 @@ import org.apache.hadoop.util.concurrent.HadoopExecutors; import org.apache.http.client.utils.URIBuilder; +import static java.net.HttpURLConnection.HTTP_CONFLICT; +import static java.net.HttpURLConnection.HTTP_NOT_FOUND; + +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOB_LEASE_ONE_MINUTE_DURATION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FORWARD_SLASH; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HDI_ISFOLDER; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_METADATA_PREFIX; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_EQUALS; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_FORWARD_SLASH; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_HYPHEN; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_PLUS; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_STAR; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_UNDERSCORE; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_ABORTED; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_FAILED; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_SUCCESS; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DIRECTORY; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FILE; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ROOT_PATH; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.SINGLE_WHITE_SPACE; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.TOKEN_VERSION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.TRUE; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_ABFS_ENDPOINT; 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_IDENTITY_TRANSFORM_CLASS; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.HBASE_ROOT; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.CONTENT_LENGTH; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_COPY_ID; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_COPY_SOURCE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_COPY_STATUS; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_COPY_STATUS_DESCRIPTION; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_META_HDI_ISFOLDER; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.COPY_BLOB_ABORTED; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.COPY_BLOB_FAILED; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.PATH_EXISTS; +import static org.apache.hadoop.fs.azurebfs.services.RenameAtomicityUtils.SUFFIX; /** * Provides the bridging logic between Hadoop's abstract filesystem and Azure Storage. @@ -170,6 +217,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport { private final IdentityTransformerInterface identityTransformer; private final AbfsPerfTracker abfsPerfTracker; private final AbfsCounters abfsCounters; + private PrefixMode prefixMode; /** * The set of directories where we should store files as append blobs. @@ -227,6 +275,7 @@ public AzureBlobFileSystemStore( this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList( abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA))); + this.azureAtomicRenameDirSet.add(HBASE_ROOT); updateInfiniteLeaseDirs(); this.authType = abfsConfiguration.getAuthType(accountName); boolean usingOauth = (authType == AuthType.OAuth); @@ -284,6 +333,13 @@ public String getPrimaryGroup() { return this.primaryUserGroup; } + public PrefixMode getPrefixMode() { + if (prefixMode == null) { + prefixMode = abfsConfiguration.getPrefixMode(); + } + return prefixMode; + } + @Override public void close() throws IOException { List> futures = new ArrayList<>(); @@ -416,16 +472,19 @@ public Hashtable getFilesystemProperties( client.getFileSystem()); final Hashtable parsedXmsProperties; + final AbfsRestOperation op; - final AbfsRestOperation op = client - .getFilesystemProperties(tracingContext); - perfInfo.registerResult(op.getResult()); + if (getPrefixMode() == PrefixMode.BLOB) { + parsedXmsProperties = getAndParseContainerMetadata(tracingContext, perfInfo); + + return parsedXmsProperties; + } + op = client.getFilesystemProperties(tracingContext); final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); parsedXmsProperties = parseCommaSeparatedXmsProperties(xMsProperties); - perfInfo.registerSuccess(true); - + perfInfo.registerResult(op.getResult()).registerSuccess(true); return parsedXmsProperties; } } @@ -444,6 +503,10 @@ public void setFilesystemProperties( try (AbfsPerfInfo perfInfo = startTracking("setFilesystemProperties", "setFilesystemProperties")) { + if (getPrefixMode() == PrefixMode.BLOB) { + parseAndSetContainerMetadata(properties, tracingContext, perfInfo); + } + final String commaSeparatedProperties; try { commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties); @@ -484,9 +547,9 @@ public void setPathProperties(final Path path, throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("setPathProperties", "setPathProperties")){ LOG.debug("setFilesystemProperties for filesystem: {} path: {} with properties: {}", - client.getFileSystem(), - path, - properties); + client.getFileSystem(), + path, + properties); final String commaSeparatedProperties; try { @@ -501,13 +564,397 @@ public void setPathProperties(final Path path, } } - public void createFilesystem(TracingContext tracingContext) + /** + * Orchestrates the copying of blob from given source to a given destination. + * @param srcPath source path + * @param dstPath destination path + * @param copySrcLeaseId leaseId on the source + * @param tracingContext object of TracingContext used for the tracing of the + * server calls. + * + * @throws AzureBlobFileSystemException exception thrown from the server calls, + * or if it is discovered that the copying is failed or aborted. + */ + @VisibleForTesting + void copyBlob(Path srcPath, + Path dstPath, + final String copySrcLeaseId, TracingContext tracingContext) throws AzureBlobFileSystemException { + AbfsRestOperation copyOp = null; + try { + copyOp = client.copyBlob(srcPath, dstPath, + copySrcLeaseId, tracingContext); + } catch (AbfsRestOperationException ex) { + if (ex.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { + final BlobProperty dstBlobProperty = getBlobProperty(dstPath, + tracingContext); + try { + if (dstBlobProperty.getCopySourceUrl() != null && + (ROOT_PATH + client.getFileSystem() + srcPath.toUri() + .getPath()).equals( + new URL(dstBlobProperty.getCopySourceUrl()).toURI() + .getPath())) { + return; + } + } catch (URISyntaxException | MalformedURLException e) { + throw new RuntimeException(e); + } + } + throw ex; + } + final String progress = copyOp.getResult() + .getResponseHeader(X_MS_COPY_STATUS); + if (COPY_STATUS_SUCCESS.equalsIgnoreCase(progress)) { + return; + } + final String copyId = copyOp.getResult().getResponseHeader(X_MS_COPY_ID); + final long pollWait = abfsConfiguration.getBlobCopyProgressPollWaitMillis(); + while (handleCopyInProgress(dstPath, tracingContext, copyId) + == BlobCopyProgress.PENDING) { + try { + Thread.sleep(pollWait); + } catch (Exception e) { + + } + } + } + + /** + * Verifies if the blob copy is success or a failure or still in progress. + * + * @param dstPath path of the destination for the copying + * @param tracingContext object of tracingContext used for the tracing of the + * server calls. + * @param copyId id returned by server on the copy server-call. This id gets + * attached to blob and is returned by GetBlobProperties API on the destination. + * + * @return true if copying is success, false if it is still in progress. + * + * @throws AzureBlobFileSystemException exception returned in making server call + * for GetBlobProperties on the path. It can be thrown if the copyStatus is failure + * or is aborted. + */ + @VisibleForTesting + BlobCopyProgress handleCopyInProgress(final Path dstPath, + final TracingContext tracingContext, + final String copyId) throws AzureBlobFileSystemException { + BlobProperty blobProperty = getBlobProperty(dstPath, + tracingContext); + if (blobProperty != null && copyId.equals(blobProperty.getCopyId())) { + if (COPY_STATUS_SUCCESS.equalsIgnoreCase(blobProperty.getCopyStatus())) { + return BlobCopyProgress.SUCCESS; + } + if (COPY_STATUS_FAILED.equalsIgnoreCase(blobProperty.getCopyStatus())) { + throw new AbfsRestOperationException( + COPY_BLOB_FAILED.getStatusCode(), COPY_BLOB_FAILED.getErrorCode(), + String.format("copy to path %s failed due to: %s", + dstPath.toUri().getPath(), blobProperty.getCopyStatusDescription()), + new Exception(COPY_BLOB_FAILED.getErrorCode())); + } + if (COPY_STATUS_ABORTED.equalsIgnoreCase(blobProperty.getCopyStatus())) { + throw new AbfsRestOperationException( + COPY_BLOB_ABORTED.getStatusCode(), COPY_BLOB_ABORTED.getErrorCode(), + String.format("copy to path %s aborted", dstPath.toUri().getPath()), + new Exception(COPY_BLOB_ABORTED.getErrorCode())); + } + } + return BlobCopyProgress.PENDING; + } + + /** + * Gets the property for the blob over Blob Endpoint. + * + * @param blobPath blobPath for which property information is required + * @param tracingContext object of TracingContext required for tracing server calls. + * @return BlobProperty for the given path + * @throws AzureBlobFileSystemException exception thrown from + * {@link AbfsClient#getBlobProperty(Path, TracingContext)} call + */ + BlobProperty getBlobProperty(Path blobPath, + TracingContext tracingContext) throws AzureBlobFileSystemException { + AbfsRestOperation op = client.getBlobProperty(blobPath, tracingContext); + BlobProperty blobProperty = new BlobProperty(); + final AbfsHttpOperation opResult = op.getResult(); + blobProperty.setIsDirectory(opResult + .getResponseHeader(X_MS_META_HDI_ISFOLDER) != null); + blobProperty.setUrl(op.getUrl().toString()); + blobProperty.setCopyId(opResult.getResponseHeader(X_MS_COPY_ID)); + blobProperty.setPath(blobPath); + blobProperty.setCopySourceUrl(opResult.getResponseHeader(X_MS_COPY_SOURCE)); + blobProperty.setCopyStatusDescription( + opResult.getResponseHeader(X_MS_COPY_STATUS_DESCRIPTION)); + blobProperty.setCopyStatus(opResult.getResponseHeader(X_MS_COPY_STATUS)); + blobProperty.setContentLength( + Long.parseLong(opResult.getResponseHeader(CONTENT_LENGTH))); + blobProperty.setETag(extractEtagHeader(opResult)); + return blobProperty; + } + + /** + * Gets the property for the container(filesystem) over Blob Endpoint. + * + * @param tracingContext object of TracingContext required for tracing server calls. + * @return BlobProperty for the given path + * @throws AzureBlobFileSystemException exception thrown from + * {@link AbfsClient#getBlobProperty(Path, TracingContext)} call + */ + BlobProperty getContainerProperty(TracingContext tracingContext) + throws AzureBlobFileSystemException { + try (AbfsPerfInfo perfInfo = startTracking("getContainerProperty", "getContainerProperty")) { + LOG.debug("getContainerProperty for filesystem: {}", + client.getFileSystem()); + + AbfsRestOperation op = client.getContainerProperty(tracingContext); + perfInfo.registerResult(op.getResult()).registerSuccess(true); + + BlobProperty blobProperty = new BlobProperty(); + blobProperty.setIsDirectory(true); + blobProperty.setPath(new Path(FORWARD_SLASH)); + + return blobProperty; + } + } + + /** + * Gets user-defined properties(metadata) of the blob over blob endpoint. + * @param path + * @param tracingContext + * @return hashmap containing key value pairs for blob metadata + * @throws AzureBlobFileSystemException + */ + public Hashtable getBlobMetadata(final Path path, + TracingContext tracingContext) throws AzureBlobFileSystemException { + try (AbfsPerfInfo perfInfo = startTracking("getBlobMetadata", "getBlobMetadata")) { + LOG.debug("getBlobMetadata for filesystem: {} path: {}", + client.getFileSystem(), + path); + + final AbfsRestOperation op = client.getBlobMetadata(path, tracingContext); + perfInfo.registerResult(op.getResult()).registerSuccess(true); + + final Hashtable metadata = parseResponseHeadersToHashTable(op.getResult()); + return metadata; + } + catch (AbfsRestOperationException ex) { + // The path does not exist explicitly. + // Check here if the path is an implicit dir + if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND && !path.isRoot()) { + List blobProperties = getListBlobs(path, null, null, + tracingContext, 2, true); + if (blobProperties.size() == 0) { + throw ex; + } + else { + // Path exists as implicit directory. + // Return empty hashmap for properties + return new Hashtable<>(); + } + } + else { + throw ex; + } + } + } + + /** + * Sets user-defined properties(metadata) of the blob over blob endpoint. + * @param path on which metadata is to be set + * @param metadata set of user-defined properties to be set + * @param tracingContext + * @throws AzureBlobFileSystemException + */ + public void setBlobMetadata(final Path path, + final Hashtable metadata, TracingContext tracingContext) + throws IOException { + try (AbfsPerfInfo perfInfo = startTracking("setBlobMetadata", "setBlobMetadata")) { + LOG.debug("setBlobMetadata for filesystem: {} path: {} with properties: {}", + client.getFileSystem(), + path, + metadata); + + List metadataRequestHeaders = getRequestHeadersForMetadata(metadata); + + try { + final AbfsRestOperation op = client.setBlobMetadata(path, metadataRequestHeaders, tracingContext); + perfInfo.registerResult(op.getResult()).registerSuccess(true); + } catch (AbfsRestOperationException ex) { + // The path does not exist explicitly. + // Check here if the path is an implicit dir + if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { + List blobProperties = getListBlobs( + path,null, null, tracingContext, 2, true); + if (blobProperties.size() == 0) { + throw ex; + } + else { + // The path was an implicit blob. Create Marker and set metadata on it + createDirectory(path, null, FsPermission.getDirDefault(), + FsPermission.getUMask( + getAbfsConfiguration().getRawConfiguration()), + false, tracingContext); + + boolean xAttrExists = metadata.containsKey(HDI_ISFOLDER); + if (!xAttrExists) { + metadata.put(HDI_ISFOLDER, TRUE); + metadataRequestHeaders = getRequestHeadersForMetadata(metadata); + } + + final AbfsRestOperation op = client.setBlobMetadata(path, metadataRequestHeaders, tracingContext); + perfInfo.registerResult(op.getResult()).registerSuccess(true); + } + } + else { + throw ex; + } + } + } + } + + /** + * Gets user-defined properties(metadata) of the container over blob endpoint. + * @param tracingContext + * @return hashmap containing key value pairs for container metadata + * @throws AzureBlobFileSystemException + */ + public Hashtable getContainerMetadata(TracingContext tracingContext) + throws AzureBlobFileSystemException { + try (AbfsPerfInfo perfInfo = startTracking("getContainerMetadata", "getContainerMetadata")) { + LOG.debug("getContainerMetadata for filesystem: {}", client.getFileSystem()); + + return getAndParseContainerMetadata(tracingContext, perfInfo); + } + } + + /** + * Sets user-defined properties(metadata) of the container over blob endpoint. + * @param metadata set of user-defined properties to be set + * @param tracingContext + * @throws AzureBlobFileSystemException + */ + public void setContainerMetadata(final Hashtable metadata, + TracingContext tracingContext) throws AzureBlobFileSystemException { + try (AbfsPerfInfo perfInfo = startTracking("setContainerMetadata", "setContainerMetadata")) { + LOG.debug("setContainerMetadata for filesystem: {} with properties: {}", + client.getFileSystem(), + metadata); + + parseAndSetContainerMetadata(metadata, tracingContext, perfInfo); + } + } + + /** + * User-Defined Properties over blob endpoint are actually response headers + * with prefix "x-ms-meta-". Each property is a different response header. + * This parses all the headers, removes the prefix and create a hashmap. + * @param result AbfsHttpOperation result containing response headers. + * @return Hashmap defining user defined metadata. + */ + private Hashtable parseResponseHeadersToHashTable( + AbfsHttpOperation result) { + final Hashtable metadata = new Hashtable<>(); + String name, value; + + final Map> responseHeaders = result.getResponseHeaders(); + for (Map.Entry> entry : responseHeaders.entrySet()) { + name = entry.getKey(); + + if (name != null && name.startsWith(X_MS_METADATA_PREFIX)) { + value = entry.getValue().get(0); + metadata.put(name.substring(X_MS_METADATA_PREFIX.length()), value); + } + } + return metadata; + } + + /** + * User-defined properties over blob endpoint are required to be set + * as request header with prefix "x-ms-meta-". Each property need to be made + * into a different request header. This parses all the properties, add prefix + * and create request headers. + * @param metadata Hashmap + * @return List of request headers to be passed with API call. + */ + private List getRequestHeadersForMetadata(Hashtable metadata) { + final List headers = new ArrayList(); + + for(Map.Entry entry : metadata.entrySet()) { + headers.add(new AbfsHttpHeader(X_MS_METADATA_PREFIX + entry.getKey(), entry.getValue())); + } + return headers; + } + + private Hashtable getAndParseContainerMetadata(TracingContext tracingContext, + AbfsPerfInfo perfInfo) throws AzureBlobFileSystemException{ + final AbfsRestOperation op = client.getContainerMetadata(tracingContext); + perfInfo.registerResult(op.getResult()).registerSuccess(true); + + return parseResponseHeadersToHashTable(op.getResult()); + } + + private void parseAndSetContainerMetadata(final Hashtable metadata, TracingContext tracingContext, + AbfsPerfInfo perfInfo) throws AzureBlobFileSystemException{ + final List metadataRequestHeaders = getRequestHeadersForMetadata(metadata); + final AbfsRestOperation op = client.setContainerMetadata(metadataRequestHeaders, tracingContext); + + perfInfo.registerResult(op.getResult()).registerSuccess(true); + } + + /** + * Get the list of a blob on a give path, or blob starting with the given prefix. + * + * @param sourceDirBlobPath path from where the list of blob is required. + * @param prefix Optional value to be provided. If provided, API call would have + * prefix = given value. If not provided, the API call would have prefix = + * sourceDirBlobPath. + * @param tracingContext object of {@link TracingContext} + * @param maxResult defines maximum blobs the method should process + * @param isDefinitiveDirSearch defines if (true) it is blobList search on a + * definitive directory, if (false) it is blobList search on a prefix. + * + * @return List of blobProperties + * + * @throws AbfsRestOperationException exception from server-calls / xml-parsing + */ + public List getListBlobs(Path sourceDirBlobPath, + String prefix, String delimiter, TracingContext tracingContext, + final Integer maxResult, final Boolean isDefinitiveDirSearch) + throws AzureBlobFileSystemException { + List blobProperties = new ArrayList<>(); + String nextMarker = null; + if (prefix == null) { + prefix = (!sourceDirBlobPath.isRoot() + ? sourceDirBlobPath.toUri().getPath() + : EMPTY_STRING) + (isDefinitiveDirSearch + ? ROOT_PATH + : EMPTY_STRING); + } + if (delimiter == null) { + delimiter = ""; + } + do { + AbfsRestOperation op = getClient().getListBlobs( + nextMarker, prefix, delimiter, maxResult, tracingContext + ); + BlobList blobList = op.getResult().getBlobList(); + nextMarker = blobList.getNextMarker(); + blobProperties.addAll(blobList.getBlobPropertyList()); + if (maxResult != null && blobProperties.size() >= maxResult) { + break; + } + } while (nextMarker != null); + return blobProperties; + } + + public void createFilesystem(TracingContext tracingContext, final boolean useBlobEndpoint) throws AzureBlobFileSystemException { try (AbfsPerfInfo perfInfo = startTracking("createFilesystem", "createFilesystem")){ LOG.debug("createFilesystem for filesystem: {}", client.getFileSystem()); - - final AbfsRestOperation op = client.createFilesystem(tracingContext); + final AbfsRestOperation op; + if (useBlobEndpoint) { + op = client.createContainer(tracingContext); + } else { + op = client.createFilesystem(tracingContext); + } perfInfo.registerResult(op.getResult()).registerSuccess(true); } } @@ -518,15 +965,76 @@ public void deleteFilesystem(TracingContext tracingContext) LOG.debug("deleteFilesystem for filesystem: {}", client.getFileSystem()); - final AbfsRestOperation op = client.deleteFilesystem(tracingContext); + final AbfsRestOperation op; + if (getPrefixMode() == PrefixMode.BLOB) { + op = client.deleteContainer(tracingContext); + } else { + op = client.deleteFilesystem(tracingContext); + } perfInfo.registerResult(op.getResult()).registerSuccess(true); } } - public OutputStream createFile(final Path path, - final FileSystem.Statistics statistics, final boolean overwrite, + /** + * Checks if we are creating a normal blob or markerFile. + * @param metadata takes metadata as param. + * @return true or false. + */ + private boolean checkIsBlobOrMarker(HashMap metadata) { + return metadata != null && TRUE.equalsIgnoreCase(metadata.get(X_MS_META_HDI_ISFOLDER)); + } + + private AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite, + final String permission, final String umask, + final boolean isAppendBlob, final String eTag, + TracingContext tracingContext) throws AzureBlobFileSystemException { + return client.createPath(path, isFile, overwrite, permission, umask, isAppendBlob, eTag, tracingContext); + } + + private AbfsRestOperation createPathBlob(final String path, final boolean isFile, final boolean overwrite, + final HashMap metadata, + final String eTag, + TracingContext tracingContext) throws AzureBlobFileSystemException { + return client.createPathBlob(path, isFile, overwrite, metadata, eTag, tracingContext); + } + + private AbfsRestOperation createFileOrMarker(boolean isNormalBlob, String relativePath, boolean isNamespaceEnabled, + boolean overwrite, HashMap metadata, TracingContext tracingContext, + FsPermission permission, FsPermission umask, boolean isAppendBlob, String eTag) throws AzureBlobFileSystemException { + AbfsRestOperation op; + if (!isNormalBlob) { + // Marker blob creation flow. + if (OperativeEndpoint.isMkdirEnabledOnDFS(abfsConfiguration)) { + LOG.debug("DFS fallback enabled and incorrect mkdir flow is hit for path {} and config value {} ", + relativePath, abfsConfiguration.shouldMkdirFallbackToDfs()); + // Marker blob creation is not possible with dfs endpoint. + throw new InvalidConfigurationValueException("Incorrect flow for create directory for dfs is hit " + + relativePath); + } else { + LOG.debug("Path created via blob for mkdir call for path {} and config value {} ", + relativePath, abfsConfiguration.shouldMkdirFallbackToDfs()); + op = createPathBlob(relativePath, false, overwrite, metadata, eTag, tracingContext); + } + } else { + // Normal blob creation flow. If config for fallback is not enabled and prefix mode is blob go to blob, else go to dfs. + if (!OperativeEndpoint.isIngressEnabledOnDFS(getPrefixMode(), abfsConfiguration)) { + LOG.debug("Path created via blob endpoint for path {} and config value {} ", + relativePath, abfsConfiguration.shouldIngressFallbackToDfs()); + op = createPathBlob(relativePath, true, overwrite, metadata, eTag, tracingContext); + } else { + LOG.debug("Path created via dfs endpoint for path {} and config value {} ", + relativePath, abfsConfiguration.shouldIngressFallbackToDfs()); + op = createPath(relativePath, true, overwrite, isNamespaceEnabled ? getOctalNotation(permission) : null, + isNamespaceEnabled ? getOctalNotation(umask) : null, isAppendBlob, eTag, tracingContext); + } + } + return op; + } + + // Fallback plan : default to v1 create flow which will hit dfs endpoint. Config to enable: "fs.azure.ingress.fallback.to.dfs". + public AbfsOutputStream createFile(final Path path, final FileSystem.Statistics statistics, final boolean overwrite, final FsPermission permission, final FsPermission umask, - TracingContext tracingContext) throws IOException { + TracingContext tracingContext, HashMap metadata) throws IOException { try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}", @@ -556,25 +1064,24 @@ public OutputStream createFile(final Path path, if (triggerConditionalCreateOverwrite) { op = conditionalCreateOverwriteFile(relativePath, statistics, - isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null, + isNamespaceEnabled, + permission, + umask, isAppendBlob, + metadata, tracingContext ); } else { - op = client.createPath(relativePath, true, - overwrite, - isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null, - isAppendBlob, - null, - tracingContext); - + boolean isNormalBlob = !checkIsBlobOrMarker(metadata); + op = createFileOrMarker(isNormalBlob, relativePath, isNamespaceEnabled, overwrite, metadata, + tracingContext, permission, umask, isAppendBlob, null); } perfInfo.registerResult(op.getResult()).registerSuccess(true); AbfsLease lease = maybeCreateLease(relativePath, tracingContext); + String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + checkAppendSmallWrite(isAppendBlob); return new AbfsOutputStream( populateAbfsOutputStreamContext( @@ -584,6 +1091,7 @@ public OutputStream createFile(final Path path, statistics, relativePath, 0, + eTag, tracingContext)); } } @@ -601,24 +1109,33 @@ public OutputStream createFile(final Path path, */ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePath, final FileSystem.Statistics statistics, - final String permission, - final String umask, + boolean isNamespaceEnabled, + final FsPermission permission, + final FsPermission umask, final boolean isAppendBlob, - TracingContext tracingContext) throws AzureBlobFileSystemException { + HashMap metadata, + TracingContext tracingContext) throws IOException { AbfsRestOperation op; - + VersionedFileStatus fileStatus; try { // Trigger a create with overwrite=false first so that eTag fetch can be // avoided for cases when no pre-existing file is present (major portion // of create file traffic falls into the case of no pre-existing file). - op = client.createPath(relativePath, true, false, permission, umask, - isAppendBlob, null, tracingContext); - + boolean isNormalBlob = !checkIsBlobOrMarker(metadata); + op = createFileOrMarker(isNormalBlob, relativePath, isNamespaceEnabled, false, metadata, + tracingContext, permission, umask, isAppendBlob, null); } catch (AbfsRestOperationException e) { - if (e.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { + if (e.getStatusCode() == HTTP_CONFLICT) { // File pre-exists, fetch eTag try { - op = client.getPathStatus(relativePath, false, tracingContext); + boolean useBlobEndpoint = getPrefixMode() == PrefixMode.BLOB; + if (OperativeEndpoint.isIngressEnabledOnDFS( + getAbfsConfiguration().getPrefixMode(), getAbfsConfiguration())) { + LOG.debug("GetFileStatus over DFS for create for ingress config value {} for path {} ", + abfsConfiguration.shouldIngressFallbackToDfs(), relativePath); + useBlobEndpoint = false; + } + fileStatus = (VersionedFileStatus) getFileStatus(new Path(relativePath), tracingContext, useBlobEndpoint); } catch (AbfsRestOperationException ex) { if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { // Is a parallel access case, as file which was found to be @@ -631,13 +1148,13 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa } } - String eTag = op.getResult() - .getResponseHeader(HttpHeaderConfigurations.ETAG); + String eTag = fileStatus.getEtag(); try { - // overwrite only if eTag matches with the file properties fetched befpre - op = client.createPath(relativePath, true, true, permission, umask, - isAppendBlob, eTag, tracingContext); + // overwrite only if eTag matches with the file properties fetched before. + boolean isNormalBlob = !checkIsBlobOrMarker(metadata); + op = createFileOrMarker(isNormalBlob, relativePath, isNamespaceEnabled, true, metadata, + tracingContext, permission, umask, isAppendBlob, eTag); } catch (AbfsRestOperationException ex) { if (ex.getStatusCode() == HttpURLConnection.HTTP_PRECON_FAILED) { // Is a parallel access case, as file with eTag was just queried @@ -680,6 +1197,7 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext( FileSystem.Statistics statistics, String path, long position, + String eTag, TracingContext tracingContext) { int bufferSize = abfsConfiguration.getWriteBufferSize(); if (isAppendBlob && bufferSize > FileSystemConfigurations.APPENDBLOB_MAX_WRITE_BUFFER_SIZE) { @@ -687,6 +1205,7 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext( } return new AbfsOutputStreamContext(abfsConfiguration.getSasTokenRenewPeriodForStreamsInSeconds()) .withWriteBufferSize(bufferSize) + .enableExpectHeader(abfsConfiguration.isExpectHeaderEnabled()) .enableFlush(abfsConfiguration.isFlushEnabled()) .enableSmallWriteOptimization(abfsConfiguration.isSmallWriteOptimizationEnabled()) .disableOutputStreamFlush(abfsConfiguration.isOutputStreamFlushDisabled()) @@ -701,17 +1220,40 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext( .withPosition(position) .withFsStatistics(statistics) .withPath(path) + .withETag(eTag) .withExecutorService(new SemaphoredDelegatingExecutor(boundedThreadPool, blockOutputActiveBlocks, true)) .withTracingContext(tracingContext) .build(); } - public void createDirectory(final Path path, final FsPermission permission, - final FsPermission umask, TracingContext tracingContext) - throws AzureBlobFileSystemException { + public String createDirectory(final Path path, final FileSystem.Statistics statistics, final FsPermission permission, + final FsPermission umask, + final Boolean checkParentChain, + TracingContext tracingContext) + throws IOException { try (AbfsPerfInfo perfInfo = startTracking("createDirectory", "createPath")) { + if (!OperativeEndpoint.isMkdirEnabledOnDFS(abfsConfiguration)) { + LOG.debug("Mkdir created via blob endpoint for the given path {} and config value {} ", + path, abfsConfiguration.shouldMkdirFallbackToDfs()); + ArrayList keysToCreateAsFolder = new ArrayList<>(); + if (checkParentChain) { + checkParentChainForFile(path, tracingContext, keysToCreateAsFolder); + } + boolean blobOverwrite = abfsConfiguration.isEnabledBlobMkdirOverwrite(); + + AbfsOutputStream pathDirectoryOutputStream = createDirectoryMarkerBlob( + path, statistics, permission, umask, tracingContext, + blobOverwrite); + for (Path pathToCreate: keysToCreateAsFolder) { + createDirectoryMarkerBlob(pathToCreate, statistics, permission, umask, + tracingContext, blobOverwrite); + } + return pathDirectoryOutputStream.getETag(); + } boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); + LOG.debug("Mkdir created via dfs endpoint for the given path {} and config value {} ", + path, abfsConfiguration.shouldMkdirFallbackToDfs()); LOG.debug("createDirectory filesystem: {} path: {} permission: {} umask: {} isNamespaceEnabled: {}", client.getFileSystem(), path, @@ -720,56 +1262,131 @@ public void createDirectory(final Path path, final FsPermission permission, isNamespaceEnabled); boolean overwrite = - !isNamespaceEnabled || abfsConfiguration.isEnabledMkdirOverwrite(); + !isNamespaceEnabled || abfsConfiguration.isEnabledMkdirOverwrite(); final AbfsRestOperation op = client.createPath(getRelativePath(path), - false, overwrite, + false, overwrite, isNamespaceEnabled ? getOctalNotation(permission) : null, isNamespaceEnabled ? getOctalNotation(umask) : null, false, null, tracingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); + return op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + } + } + + private AbfsOutputStream createDirectoryMarkerBlob(final Path path, + final FileSystem.Statistics statistics, + final FsPermission permission, + final FsPermission umask, + final TracingContext tracingContext, + final boolean blobOverwrite) throws IOException { + HashMap metadata = new HashMap<>(); + metadata.put(X_MS_META_HDI_ISFOLDER, TRUE); + return createFile(path, statistics, blobOverwrite, + permission, umask, tracingContext, metadata); + } + + /** + * Checks for the entire parent hierarchy and returns if any directory exists and + * throws an exception if any file exists. + * @param path path to check the hierarchy for. + * @param tracingContext the tracingcontext. + */ + private void checkParentChainForFile(Path path, TracingContext tracingContext, + List keysToCreateAsFolder) throws IOException { + + FileStatus fileStatus = tryGetPathProperty(path, tracingContext, true); + Boolean isDirectory = fileStatus != null ? fileStatus.isDirectory() : false; + if (fileStatus != null && !isDirectory) { + throw new AbfsRestOperationException(HTTP_CONFLICT, + AzureServiceErrorCode.PATH_CONFLICT.getErrorCode(), + PATH_EXISTS, + null); + } + if (isDirectory) { + return; + } + Path current = path.getParent(); + while (current != null && !current.isRoot()) { + fileStatus = tryGetPathProperty(current, tracingContext, true); + isDirectory = fileStatus != null ? fileStatus.isDirectory() : false; + if (fileStatus != null && !isDirectory) { + throw new AbfsRestOperationException(HTTP_CONFLICT, + AzureServiceErrorCode.PATH_CONFLICT.getErrorCode(), + PATH_EXISTS, + null); + } + if (isDirectory) { + break; + } + keysToCreateAsFolder.add(current); + current = current.getParent(); } } public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statistics statistics, TracingContext tracingContext) - throws AzureBlobFileSystemException { - return openFileForRead(path, Optional.empty(), statistics, tracingContext); + throws IOException { + return openFileForRead(path, Optional.empty(), statistics, + tracingContext); } - public AbfsInputStream openFileForRead(final Path path, - final Optional options, + public AbfsInputStream openFileForRead(Path path, + final Optional parameters, final FileSystem.Statistics statistics, TracingContext tracingContext) - throws AzureBlobFileSystemException { - try (AbfsPerfInfo perfInfo = startTracking("openFileForRead", "getPathStatus")) { + throws IOException { + try (AbfsPerfInfo perfInfo = startTracking("openFileForRead", + "getPathStatus")) { LOG.debug("openFileForRead filesystem: {} path: {}", - client.getFileSystem(), - path); + client.getFileSystem(), path); + FileStatus parameterFileStatus = parameters.map(OpenFileParameters::getStatus) + .orElse(null); String relativePath = getRelativePath(path); - final AbfsRestOperation op = client - .getPathStatus(relativePath, false, tracingContext); - perfInfo.registerResult(op.getResult()); + final VersionedFileStatus fileStatus; + if (parameterFileStatus instanceof VersionedFileStatus) { + Preconditions.checkArgument(parameterFileStatus.getPath() + .equals(path.makeQualified(this.uri, path)), + String.format( + "Filestatus path [%s] does not match with given path [%s]", + parameterFileStatus.getPath(), path)); + fileStatus = (VersionedFileStatus) parameterFileStatus; + } else { + if (parameterFileStatus != null) { + LOG.warn( + "Fallback to getPathStatus REST call as provided filestatus " + + "is not of type VersionedFileStatus"); + } + boolean useBlobEndpoint = getPrefixMode() == PrefixMode.BLOB; + if (OperativeEndpoint.isReadEnabledOnDFS(getAbfsConfiguration())) { + LOG.debug("GetFileStatus over DFS for open file for read for read config value {} for path {} ", + abfsConfiguration.shouldReadFallbackToDfs(), path); + useBlobEndpoint = false; + } + fileStatus = (VersionedFileStatus) getFileStatus(path, tracingContext, useBlobEndpoint); + } - final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); - final long contentLength = Long.parseLong(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); - final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + boolean isDirectory = fileStatus.isDirectory(); + + final long contentLength = fileStatus.getLen(); + final String eTag = fileStatus.getEtag(); - if (parseIsDirectory(resourceType)) { + if (isDirectory) { throw new AbfsRestOperationException( AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), - "openFileForRead must be used with files and not directories", + "openFileForRead must be used with files and not directories." + + "Attempt made for read on explicit directory.", null); } perfInfo.registerSuccess(true); // Add statistics for InputStream - return new AbfsInputStream(client, statistics, - relativePath, contentLength, - populateAbfsInputStreamContext(options), - eTag, tracingContext); + return new AbfsInputStream(client, statistics, relativePath, + contentLength, populateAbfsInputStreamContext( + parameters.map(OpenFileParameters::getOptions)), + eTag, tracingContext); } } @@ -782,6 +1399,7 @@ private AbfsInputStreamContext populateAbfsInputStreamContext( .withReadBufferSize(abfsConfiguration.getReadBufferSize()) .withReadAheadQueueDepth(abfsConfiguration.getReadAheadQueueDepth()) .withTolerateOobAppends(abfsConfiguration.getTolerateOobAppends()) + .isReadAheadEnabled(abfsConfiguration.isReadAheadEnabled()) .withReadSmallFilesCompletely(abfsConfiguration.readSmallFilesCompletely()) .withOptimizeFooterRead(abfsConfiguration.optimizeFooterRead()) .withReadAheadRange(abfsConfiguration.getReadAheadRange()) @@ -803,19 +1421,24 @@ public OutputStream openFileForWrite(final Path path, overwrite); String relativePath = getRelativePath(path); + boolean useBlobEndpoint = getPrefixMode() == PrefixMode.BLOB; + if (OperativeEndpoint.isIngressEnabledOnDFS( + getAbfsConfiguration().getPrefixMode(), getAbfsConfiguration())) { + LOG.debug("GetFileStatus over DFS for open file for write for ingress config value {} for path {} ", + abfsConfiguration.shouldIngressFallbackToDfs(), path); + useBlobEndpoint = false; + } + VersionedFileStatus fileStatus; + fileStatus = (VersionedFileStatus) getFileStatus(path, tracingContext, useBlobEndpoint); - final AbfsRestOperation op = client - .getPathStatus(relativePath, false, tracingContext); - perfInfo.registerResult(op.getResult()); - - final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); - final Long contentLength = Long.valueOf(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); + final Long contentLength = fileStatus.getLen(); - if (parseIsDirectory(resourceType)) { + boolean isDirectory = fileStatus.isDirectory(); + if (isDirectory) { throw new AbfsRestOperationException( AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), - "openFileForRead must be used with files and not directories", + "openFileForWrite must be used with files and not directories", null); } @@ -829,6 +1452,8 @@ public OutputStream openFileForWrite(final Path path, } AbfsLease lease = maybeCreateLease(relativePath, tracingContext); + final String eTag = fileStatus.getEtag(); + checkAppendSmallWrite(isAppendBlob); return new AbfsOutputStream( populateAbfsOutputStreamContext( @@ -838,10 +1463,22 @@ public OutputStream openFileForWrite(final Path path, statistics, relativePath, offset, + eTag, tracingContext)); } } + public void checkAppendSmallWrite(boolean isAppendBlob) throws IOException { + if (getAbfsConfiguration().getPrefixMode() == PrefixMode.BLOB) { + if (isAppendBlob) { + throw new IOException("AppendBlob is not supported for blob endpoint."); + } + if (abfsConfiguration.isSmallWriteOptimizationEnabled()) { + throw new IOException("Small write optimization is not supported for blob endpoint."); + } + } + } + /** * Break any current lease on an ABFS file. * @@ -855,12 +1492,74 @@ public void breakLease(final Path path, final TracingContext tracingContext) thr client.breakLease(getRelativePath(path), tracingContext); } - public void rename(final Path source, final Path destination, TracingContext tracingContext) throws - AzureBlobFileSystemException { + public void rename(final Path source, final Path destination, + final RenameAtomicityUtils renameAtomicityUtils, TracingContext tracingContext) throws + IOException { final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue; + if (getAbfsConfiguration().getPrefixMode() == PrefixMode.BLOB) { + LOG.debug("Rename for src: {} dst: {} for non-HNS blob-endpoint", + source, destination); + /* + * Fetch the list of blobs in the given sourcePath. + */ + StringBuilder listSrcBuilder = new StringBuilder( + source.toUri().getPath()); + if (!source.isRoot()) { + listSrcBuilder.append(FORWARD_SLASH); + } + String listSrc = listSrcBuilder.toString(); + BlobList blobList = client.getListBlobs(null, listSrc, null, null, + tracingContext).getResult() + .getBlobList(); + List srcBlobProperties = blobList.getBlobPropertyList(); + + if (srcBlobProperties.size() > 0) { + orchestrateBlobRenameDir(source, destination, renameAtomicityUtils, + tracingContext, listSrc, blobList); + } else { + /* + * Source doesn't have any hierarchy. It can either be marker or non-marker blob. + * Or there can be no blob on the path. + * Rename procedure will start. If its a file or a marker file, it will be renamed. + * In case there is no blob on the path, server will return exception. + */ + LOG.debug("source {} doesn't have any blob in its hierarchy. " + + "Starting rename process on the source.", source); + + AbfsLease lease = null; + try { + if (isAtomicRenameKey(source.toUri().getPath())) { + lease = getBlobLease(source.toUri().getPath(), + BLOB_LEASE_ONE_MINUTE_DURATION, tracingContext); + } + renameBlob(source, destination, lease, tracingContext); + } catch (AzureBlobFileSystemException ex) { + if (lease != null) { + lease.free(); + } + LOG.error( + String.format("Rename of path from %s to %s failed", + source, destination), ex); + if (ex instanceof AbfsRestOperationException + && ((AbfsRestOperationException) ex).getStatusCode() + == HTTP_NOT_FOUND) { + AbfsRestOperationException ex1 = (AbfsRestOperationException) ex; + throw new AbfsRestOperationException( + ex1.getStatusCode(), + AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND.getErrorCode(), + ex1.getErrorMessage(), ex1); + } + throw ex; + } + } + LOG.info("Rename from source {} to destination {} done", source, + destination); + return; + } + if (isAtomicRenameKey(source.getName())) { LOG.warn("The atomic rename feature is not supported by the ABFS scheme; however rename," +" create and delete operations are atomic if Namespace is enabled for your Azure Storage account."); @@ -894,17 +1593,256 @@ public void rename(final Path source, final Path destination, TracingContext tra } while (shouldContinue); } + private void orchestrateBlobRenameDir(final Path source, + final Path destination, + final RenameAtomicityUtils renameAtomicityUtils, + final TracingContext tracingContext, + final String listSrc, + final BlobList blobList) throws IOException { + ListBlobQueue listBlobQueue = new ListBlobQueue( + blobList.getBlobPropertyList(), + getAbfsConfiguration().getProducerQueueMaxSize(), + getAbfsConfiguration().getBlobDirRenameMaxThread()); + + if (blobList.getNextMarker() != null) { + getListBlobProducer(listSrc, listBlobQueue, blobList.getNextMarker(), + tracingContext); + } else { + listBlobQueue.complete(); + } + LOG.debug("src {} exists and is a directory", source); + /* + * Fetch if there is a marker-blob for the source blob. + */ + BlobProperty blobPropOnSrcNullable; + try { + blobPropOnSrcNullable = getBlobProperty(source, tracingContext); + } catch (AbfsRestOperationException ex) { + if (ex.getStatusCode() != HttpURLConnection.HTTP_NOT_FOUND) { + throw ex; + } + blobPropOnSrcNullable = null; + } + + final String srcDirETag; + if (blobPropOnSrcNullable == null) { + /* + * There is no marker-blob, the client has to create marker blob before + * starting the rename. + */ + LOG.debug("Source {} is a directory but there is no marker-blob", + source); + srcDirETag = createDirectory(source, null, FsPermission.getDirDefault(), + FsPermission.getUMask( + getAbfsConfiguration().getRawConfiguration()), + true, tracingContext); + } else { + LOG.debug("Source {} is a directory but there is a marker-blob", + source); + srcDirETag = blobPropOnSrcNullable.getETag(); + } + /* + * If source is a directory, all the blobs in the directory have to be + * individually copied and then deleted at the source. + */ + LOG.debug("source {} is a directory", source); + final AbfsBlobLease srcDirLease; + final Boolean isAtomicRename; + if (isAtomicRenameKey(source.toUri().getPath())) { + LOG.debug("source dir {} is an atomicRenameKey", + source.toUri().getPath()); + srcDirLease = getBlobLease(source.toUri().getPath(), + BLOB_LEASE_ONE_MINUTE_DURATION, + tracingContext); + renameAtomicityUtils.preRename( + isCreateOperationOnBlobEndpoint(), srcDirETag); + isAtomicRename = true; + } else { + srcDirLease = null; + isAtomicRename = false; + LOG.debug("source dir {} is not an atomicRenameKey", + source.toUri().getPath()); + } + + renameBlobDir(source, destination, tracingContext, listBlobQueue, + srcDirLease, isAtomicRename); + + if (isAtomicRename) { + renameAtomicityUtils.cleanup(); + } + } + + @VisibleForTesting + ListBlobProducer getListBlobProducer(final String listSrc, + final ListBlobQueue listBlobQueue, + final String initNextMarker, + final TracingContext tracingContext) { + return new ListBlobProducer(listSrc, + client, listBlobQueue, initNextMarker, tracingContext); + } + + @VisibleForTesting + AbfsBlobLease getBlobLease(final String source, + final Integer blobLeaseOneMinuteDuration, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + return new AbfsBlobLease(client, source, blobLeaseOneMinuteDuration, + tracingContext); + } + + private void renameBlobDir(final Path source, + final Path destination, + final TracingContext tracingContext, + final ListBlobQueue listBlobQueue, + final AbfsBlobLease srcDirBlobLease, + final Boolean isAtomicRename) throws AzureBlobFileSystemException { + List blobList; + ListBlobConsumer listBlobConsumer = new ListBlobConsumer(listBlobQueue); + final ExecutorService renameBlobExecutorService + = Executors.newFixedThreadPool( + getAbfsConfiguration().getBlobDirRenameMaxThread()); + AtomicInteger renamedBlob = new AtomicInteger(0); + while(!listBlobConsumer.isCompleted()) { + blobList = listBlobConsumer.consume(); + if(blobList == null) { + continue; + } + List futures = new ArrayList<>(); + for (BlobProperty blobProperty : blobList) { + futures.add(renameBlobExecutorService.submit(() -> { + try { + AbfsBlobLease blobLease = null; + if (isAtomicRename) { + /* + * Conditionally get a lease on the source blob to prevent other writers + * from changing it. This is used for correctness in HBase when log files + * are renamed. It generally should do no harm other than take a little + * more time for other rename scenarios. When the HBase master renames a + * log file folder, the lease locks out other writers. This + * prevents a region server that the master thinks is dead, but is still + * alive, from committing additional updates. This is different than + * when HBase runs on HDFS, where the region server recovers the lease + * on a log file, to gain exclusive access to it, before it splits it. + */ + blobLease = getBlobLease(blobProperty.getPath().toUri().getPath(), + BLOB_LEASE_ONE_MINUTE_DURATION, tracingContext); + } + renameBlob( + blobProperty.getPath(), + createDestinationPathForBlobPartOfRenameSrcDir(destination, + blobProperty.getPath(), source), + blobLease, + tracingContext); + renamedBlob.incrementAndGet(); + } catch (AzureBlobFileSystemException e) { + LOG.error(String.format("rename from %s to %s for blob %s failed", + source, destination, blobProperty.getPath()), e); + throw new RuntimeException(e); + } + })); + } + for (Future future : futures) { + try { + future.get(); + } catch (InterruptedException | ExecutionException e) { + LOG.error(String.format("rename from %s to %s failed", source, + destination), e); + listBlobConsumer.fail(); + renameBlobExecutorService.shutdown(); + if (srcDirBlobLease != null) { + srcDirBlobLease.free(); + } + throw new RuntimeException(e); + } + } + } + renameBlobExecutorService.shutdown(); + + tracingContext.setOperatedBlobCount(renamedBlob.get() + 1); + renameBlob( + source, createDestinationPathForBlobPartOfRenameSrcDir(destination, + source, source), + srcDirBlobLease, + tracingContext); + tracingContext.setOperatedBlobCount(null); + } + + private Boolean isCreateOperationOnBlobEndpoint() { + return !OperativeEndpoint.isIngressEnabledOnDFS(getPrefixMode(), abfsConfiguration); + } + + /** + * Translates the destination path for a blob part of a source directory getting + * renamed. + * + * @param destinationDir destination directory for the rename operation + * @param blobPath path of blob inside sourceDir being renamed. + * @param sourceDir source directory for the rename operation + * + * @return translated path for the blob + */ + private Path createDestinationPathForBlobPartOfRenameSrcDir(final Path destinationDir, + final Path blobPath, final Path sourceDir) { + String destinationPathStr = destinationDir.toUri().getPath(); + String sourcePathStr = sourceDir.toUri().getPath(); + String srcBlobPropertyPathStr = blobPath.toUri().getPath(); + if (sourcePathStr.equals(srcBlobPropertyPathStr)) { + return destinationDir; + } + return new Path(destinationPathStr + ROOT_PATH + srcBlobPropertyPathStr.substring( + sourcePathStr.length())); + } + + /** + * Renames blob. + * It copies the source blob to the destination. After copy is succesful, it + * deletes the source blob + * + * @param sourcePath source path which gets copied to the destination + * @param destination destination path to which the source has to be moved + * @param lease lease of the srcBlob + * @param tracingContext tracingContext for tracing the API calls + * + * @throws AzureBlobFileSystemException exception in making server calls + */ + private void renameBlob(final Path sourcePath, final Path destination, + final AbfsLease lease, final TracingContext tracingContext) + throws AzureBlobFileSystemException { + copyBlob(sourcePath, destination, lease != null ? lease.getLeaseID() : null, + tracingContext); + deleteBlob(sourcePath, lease, tracingContext); + } + + private void deleteBlob(final Path sourcePath, + final AbfsLease lease, final TracingContext tracingContext) + throws AzureBlobFileSystemException { + try { + client.deleteBlobPath(sourcePath, + lease != null ? lease.getLeaseID() : null, tracingContext); + if (lease != null) { + lease.cancelTimer(); + } + } catch (AbfsRestOperationException ex) { + if (ex.getStatusCode() != HttpURLConnection.HTTP_NOT_FOUND) { + throw ex; + } + } + } + public void delete(final Path path, final boolean recursive, - TracingContext tracingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) throws IOException { + LOG.debug("delete filesystem: {} path: {} recursive: {}", + client.getFileSystem(), + path, + String.valueOf(recursive)); + if (getPrefixMode() == PrefixMode.BLOB) { + deleteBlobPath(path, recursive, tracingContext); + return; + } + final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue = true; - LOG.debug("delete filesystem: {} path: {} recursive: {}", - client.getFileSystem(), - path, - String.valueOf(recursive)); - String continuation = null; String relativePath = getRelativePath(path); @@ -926,16 +1864,293 @@ public void delete(final Path path, final boolean recursive, } while (shouldContinue); } - public FileStatus getFileStatus(final Path path, - TracingContext tracingContext) throws IOException { - try (AbfsPerfInfo perfInfo = startTracking("getFileStatus", "undetermined")) { + /** + * Handles deletion of a path over Blob Endpoint. + * @param path path to be deleted + * @param recursive defines deletion to be recursive or not + * @param tracingContext object to trace the API flows + * + * @throws IOException exception from server call or if recursive used on + * non-empty directory + */ + private void deleteBlobPath(final Path path, + final boolean recursive, + final TracingContext tracingContext) throws IOException { + StringBuilder listSrcBuilder = new StringBuilder(); + String srcPathStr = path.toUri().getPath(); + listSrcBuilder.append(srcPathStr); + if (!path.isRoot()) { + listSrcBuilder.append(FORWARD_SLASH); + } + String listSrc = listSrcBuilder.toString(); + BlobList blobList = client.getListBlobs(null, listSrc, null, null, + tracingContext).getResult().getBlobList(); + if (blobList.getBlobPropertyList().size() > 0) { + orchestrateBlobDirDeletion(path, recursive, listSrc, blobList, + tracingContext); + } else { + LOG.debug(String.format("Path %s doesn't have child-blobs", srcPathStr)); + if (!path.isRoot()) { + try { + client.deleteBlobPath(path, null, tracingContext); + } catch (AbfsRestOperationException ex) { + if (ex.getStatusCode() == HTTP_NOT_FOUND) { + LOG.error(String.format("Path %s doesn't exist", srcPathStr), ex); + throw new AbfsRestOperationException( + ex.getStatusCode(), + AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), + ex.getErrorMessage(), ex); + } + LOG.error(String.format("Deletion failed for path %s", srcPathStr), ex); + throw ex; + } + } + } + /* + * Src Path here would be either a non-empty directory, marker-blob or a + * normal-blob. Path can not be non-existing at this point. + * If parent blob is implicit directory and in case the blob deleted was the + * only blob in the directory, it will render path as non-existing. To prevent + * that happening, it is needed to create marker-based directory on the + * parentPath. This is inspired from WASB implementation. + */ + createParentDirectory(path, tracingContext); + LOG.debug(String.format("Deletion of Path %s completed", srcPathStr)); + } + + private void orchestrateBlobDirDeletion(final Path path, + final boolean recursive, + final String listSrc, + final BlobList blobList, + final TracingContext tracingContext) throws IOException { + final String srcPathStr = path.toUri().getPath(); + LOG.debug(String.format("Path %s has child-blobs", srcPathStr)); + if (!recursive) { + LOG.error(String.format("Non-recursive delete of non-empty directory %s", + srcPathStr)); + throw new IOException( + "Non-recursive delete of non-empty directory"); + } + ListBlobQueue queue = new ListBlobQueue(blobList.getBlobPropertyList(), + getAbfsConfiguration().getProducerQueueMaxSize(), + getAbfsConfiguration().getBlobDirDeleteMaxThread()); + if (blobList.getNextMarker() != null) { + getListBlobProducer(listSrc, queue, blobList.getNextMarker(), + tracingContext); + } else { + queue.complete(); + } + ListBlobConsumer consumer = new ListBlobConsumer(queue); + deleteOnConsumedBlobs(path, consumer, tracingContext); + } + + /** + * If parent blob is implicit directory and in case the blob deleted was the + * only blob in the directory, it will render path as non-existing. To prevent + * that happening, client create marker-based directory on the parentPath. + * + * @param path path getting deleted + * @param tracingContext tracingContext to trace the API flow + * @throws IOException + */ + private void createParentDirectory(final Path path, + final TracingContext tracingContext) throws IOException { + if (path.isRoot()) { + return; + } + Path parentPath = path.getParent(); + if (parentPath.isRoot()) { + return; + } + + String srcPathStr = path.toUri().getPath(); + String srcParentPathSrc = parentPath.toUri().getPath(); + LOG.debug(String.format( + "Creating Parent of Path %s : %s", srcPathStr, srcParentPathSrc)); + createDirectory(parentPath, null, FsPermission.getDirDefault(), + FsPermission.getUMask( + getAbfsConfiguration().getRawConfiguration()), false, + tracingContext); + LOG.debug(String.format("Directory for parent of Path %s : %s created", + srcPathStr, srcParentPathSrc)); + } + + /** + * Consumes list of blob over the consumer. Deletes the blob listed. + * The deletion of the consumed blobs is executed over parallel threads spawned + * by an ExecutorService with threads equal to {@link AbfsConfiguration#getBlobDirDeleteMaxThread()}. + * Once a list of blobs that were consumed are deleted, the next batch of blobs + * are consumed. + * + * @param srcPath path of the directory which has to be deleted + * @param consumer {@link ListBlobConsumer} object through which batches of blob + * are consumed. + * @param tracingContext object for tracing the API flow + * + * @throws AzureBlobFileSystemException exception received from server call. + */ + private void deleteOnConsumedBlobs(final Path srcPath, + final ListBlobConsumer consumer, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + AtomicInteger deletedBlobCount = new AtomicInteger(0); + String srcPathStr = srcPath.toUri().getPath(); + ExecutorService deleteBlobExecutorService = Executors.newFixedThreadPool( + getAbfsConfiguration().getBlobDirDeleteMaxThread()); + try { + while (!consumer.isCompleted()) { + final List blobList = consumer.consume(); + if (blobList == null) { + continue; + } + List futureList = new ArrayList<>(); + for (BlobProperty blobProperty : blobList) { + futureList.add(deleteBlobExecutorService.submit(() -> { + String blobPropertyPathStr = blobProperty.getPath().toUri() + .getPath(); + try { + client.deleteBlobPath(blobProperty.getPath(), null, + tracingContext); + deletedBlobCount.incrementAndGet(); + } catch (AzureBlobFileSystemException ex) { + if (ex instanceof AbfsRestOperationException + && ((AbfsRestOperationException) ex).getStatusCode() + == HttpURLConnection.HTTP_NOT_FOUND) { + return; + } + LOG.error(String.format("Deleting Path %s failed", + blobPropertyPathStr), ex); + consumer.fail(); + throw new RuntimeException(ex); + } + })); + } + + for (Future future : futureList) { + try { + future.get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + } + } + LOG.debug(String.format( + "Deletion of child blobs in hierarchy of Path %s is done", + srcPathStr)); + } finally { + deleteBlobExecutorService.shutdown(); + } + + tracingContext.setOperatedBlobCount(deletedBlobCount.get() + 1); + if (!srcPath.isRoot()) { + try { + LOG.debug(String.format("Deleting Path %s", srcPathStr)); + client.deleteBlobPath(srcPath, null, tracingContext); + LOG.debug(String.format("Deleted Path %s", srcPathStr)); + } catch (AbfsRestOperationException ex) { + if (ex.getStatusCode() != HttpURLConnection.HTTP_NOT_FOUND) { + LOG.error(String.format("Deleting Path %s failed", srcPathStr), ex); + throw ex; + } + LOG.debug( + String.format("Path %s is an implicit directory", srcPathStr)); + } + } + tracingContext.setOperatedBlobCount(null); + } + + public FileStatus getFileStatus(Path path, TracingContext tracingContext, boolean useBlobEndpoint) throws IOException { + try { + AbfsPerfInfo perfInfo = startTracking("getFileStatus", "undetermined"); boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); LOG.debug("getFileStatus filesystem: {} path: {} isNamespaceEnabled: {}", client.getFileSystem(), path, isNamespaceEnabled); + perfInfo.registerCallee("getPathProperty"); + return getPathProperty(path, tracingContext, useBlobEndpoint); - final AbfsRestOperation op; + } catch (AbfsRestOperationException ex) { + if (useBlobEndpoint && ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND && !path.isRoot()) { + List blobProperties = getListBlobs(path, null, null, tracingContext, 2, true); + if (blobProperties.size() == 0) { + throw ex; + } + else { + // TODO: return properties of first child blob here like in wasb after listFileStatus is implemented over blob + return new VersionedFileStatus( + userName, + primaryUserGroup, + new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL), + false, + 0L, + true, + 1, + abfsConfiguration.getAzureBlockSize(), + DateTimeUtils.parseLastModifiedTime(null), + path, + null); + } + } else { + throw ex; + } + } + } + + /** + * Method that deals with error cases of calling getPathProperty directly. + * getPathProperty itself does not do the error handling, as it is intended to be + * one part of the calls constituting getFileStatus. This additional method + * would help when getPathProperty has to be called in a direct flow and needs a check for this error. + * @param path Current Path + * @param tracingContext current tracing context + * @param useBlobEndpoint whether to use blob endpoint + * @return FileStatus or null if blob does not exist or is not explicit + * @throws IOException + */ + FileStatus tryGetPathProperty(Path path, TracingContext tracingContext, Boolean useBlobEndpoint) throws IOException { + try { + return getPathProperty(path, tracingContext, useBlobEndpoint); + } catch(AbfsRestOperationException ex) { + if (ex.getStatusCode() == HTTP_NOT_FOUND) { + LOG.debug("No explicit directory/path found: {}", path); + return null; + } + throw ex; + } + } + + /** + * Method to make a call to get path property based on various configs- + * like whether to go over blob/dfs endpoint, whether path provided is root etc. + * This does not segregate between implicit and explicit paths. + * @param path Path to call the downstream get property method on + * @param tracingContext Current tracing context for the call + * @param useBlobEndpoint Flag indicating whether to use blob endpoint + * @return VersionedFileStatus object for given path + * @throws IOException + */ + FileStatus getPathProperty(Path path, TracingContext tracingContext, Boolean useBlobEndpoint) throws IOException { + AbfsPerfInfo perfInfo = startTracking("getPathProperty", "undetermined"); + final AbfsRestOperation op; + Boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); + if (useBlobEndpoint) { + LOG.debug("getPathProperty filesystem call over blob endpoint: {} path: {} isNamespaceEnabled: {}", + client.getFileSystem(), + path, + isNamespaceEnabled); + + if (path.isRoot()) { + perfInfo.registerCallee("getContainerProperties"); + op = client.getContainerProperty(tracingContext); + } else { + perfInfo.registerCallee("getBlobProperty"); + op = client.getBlobProperty(path, tracingContext); + } + } else { + LOG.debug("getPathProperty filesystem call over dfs endpoint: {} path: {} isNamespaceEnabled: {}", + client.getFileSystem(), + path, + isNamespaceEnabled); if (path.isRoot()) { if (isNamespaceEnabled) { perfInfo.registerCallee("getAclStatus"); @@ -948,52 +2163,56 @@ public FileStatus getFileStatus(final Path path, perfInfo.registerCallee("getPathStatus"); op = client.getPathStatus(getRelativePath(path), false, tracingContext); } + } - perfInfo.registerResult(op.getResult()); - final long blockSize = abfsConfiguration.getAzureBlockSize(); - final AbfsHttpOperation result = op.getResult(); + perfInfo.registerResult(op.getResult()); + final long blockSize = abfsConfiguration.getAzureBlockSize(); + final AbfsHttpOperation result = op.getResult(); - String eTag = extractEtagHeader(result); - final String lastModified = result.getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED); - final String permissions = result.getResponseHeader((HttpHeaderConfigurations.X_MS_PERMISSIONS)); - final boolean hasAcl = AbfsPermission.isExtendedAcl(permissions); - final long contentLength; - final boolean resourceIsDir; + String eTag = extractEtagHeader(result); + final String lastModified = result.getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED); + final String permissions = result.getResponseHeader((HttpHeaderConfigurations.X_MS_PERMISSIONS)); + final boolean hasAcl = AbfsPermission.isExtendedAcl(permissions); + final long contentLength; + final boolean resourceIsDir; - if (path.isRoot()) { - contentLength = 0; - resourceIsDir = true; + if (path.isRoot()) { + contentLength = 0; + resourceIsDir = true; + } else { + contentLength = parseContentLength(result.getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); + if (useBlobEndpoint) { + resourceIsDir = result.getResponseHeader(X_MS_META_HDI_ISFOLDER) != null; } else { - contentLength = parseContentLength(result.getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); resourceIsDir = parseIsDirectory(result.getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE)); } - - final String transformedOwner = identityTransformer.transformIdentityForGetRequest( - result.getResponseHeader(HttpHeaderConfigurations.X_MS_OWNER), - true, - userName); - - final String transformedGroup = identityTransformer.transformIdentityForGetRequest( - result.getResponseHeader(HttpHeaderConfigurations.X_MS_GROUP), - false, - primaryUserGroup); - - perfInfo.registerSuccess(true); - - return new VersionedFileStatus( - transformedOwner, - transformedGroup, - permissions == null ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL) - : AbfsPermission.valueOf(permissions), - hasAcl, - contentLength, - resourceIsDir, - 1, - blockSize, - DateTimeUtils.parseLastModifiedTime(lastModified), - path, - eTag); } + + final String transformedOwner = identityTransformer.transformIdentityForGetRequest( + result.getResponseHeader(HttpHeaderConfigurations.X_MS_OWNER), + true, + userName); + + final String transformedGroup = identityTransformer.transformIdentityForGetRequest( + result.getResponseHeader(HttpHeaderConfigurations.X_MS_GROUP), + false, + primaryUserGroup); + + perfInfo.registerSuccess(true); + + return new VersionedFileStatus( + transformedOwner, + transformedGroup, + permissions == null ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL) + : AbfsPermission.valueOf(permissions), + hasAcl, + contentLength, + resourceIsDir, + 1, + blockSize, + DateTimeUtils.parseLastModifiedTime(lastModified), + path, + eTag); } /** @@ -1039,16 +2258,80 @@ public String listStatus(final Path path, final String startFrom, startFrom); final String relativePath = getRelativePath(path); + boolean useBlobEndpointListing = getPrefixMode() == PrefixMode.BLOB; if (continuation == null || continuation.isEmpty()) { // generate continuation token if a valid startFrom is provided. if (startFrom != null && !startFrom.isEmpty()) { + // In case startFrom is passed, fallback to DFS for now + // TODO: Support startFrom for List Blobs on Blob Endpoint + useBlobEndpointListing = false; continuation = getIsNamespaceEnabled(tracingContext) ? generateContinuationTokenForXns(startFrom) : generateContinuationTokenForNonXns(relativePath, startFrom); } } + if (useBlobEndpointListing) { + // For blob endpoint continuation will be used as nextMarker. + String prefix = relativePath + ROOT_PATH; + String delimiter = ROOT_PATH; + if (path.isRoot()) { + prefix = null; + } + + TreeMap fileMetadata = new TreeMap<>(); + long objectCountReturnedByServer = 0; + + do { + /* + * List Blob calls will be made with delimiter "/". This will ensure + * that all the children of a folder not listed out separately. Instead, + * a single entry corresponding to the directory name will be returned as BlobPrefix. + */ + try (AbfsPerfInfo perfInfo = startTracking("listStatus", "getListBlobs")) { + AbfsRestOperation op = client.getListBlobs( + continuation, prefix, delimiter, abfsConfiguration.getListMaxResults(), + tracingContext + ); + perfInfo.registerResult(op.getResult()); + BlobList blobList = op.getResult().getBlobList(); + int blobListSize = blobList.getBlobPropertyList().size(); + LOG.debug("List Blob Call on filesystem: {} path: {} marker: {} delimiter: {} returned {} objects", + client.getFileSystem(), prefix, continuation, + delimiter, blobListSize); + + continuation = blobList.getNextMarker(); + objectCountReturnedByServer += blobListSize; + + addBlobListAsFileStatus(blobList, fileMetadata); + + perfInfo.registerSuccess(true); + countAggregate++; + shouldContinue = + fetchAll && continuation != null && !continuation.isEmpty(); + + if (!shouldContinue) { + perfInfo.registerAggregates(startAggregate, countAggregate); + } + } + } while (shouldContinue); + + fileStatuses.addAll(fileMetadata.values()); + + if (fileStatuses.size() == 0) { + FileStatus status = getPathProperty(path, tracingContext, true); + if (status.isFile()) { + fileStatuses.add(status); + } + } + + LOG.debug("List Status on Blob Endpoint on filesystem: {} path: {} received {} objects from server and returned {} objects to user", + client.getFileSystem(), path, objectCountReturnedByServer, fileStatuses.size()); + + return continuation; + } + do { try (AbfsPerfInfo perfInfo = startTracking("listStatus", "listPath")) { AbfsRestOperation op = client.listPath(relativePath, false, @@ -1115,6 +2398,67 @@ public String listStatus(final Path path, final String startFrom, return continuation; } + private void addBlobListAsFileStatus(final BlobList blobList, + TreeMap fileMetadata) throws IOException { + + /* + * Here before adding the data we might have to remove the duplicates. + * List Blobs call over blob endpoint returns two types of entries: Blob + * and BlobPrefix. In the case where ABFS generated the data, + * there will be a marker blob for each "directory" created by driver, + * and we will receive them as a Blob. If there are also files within this + * "directory", we will also receive a BlobPrefix. To further + * complicate matters, the data may not be generated by ABFS Driver, in + * which case we may not have a marker blob for each "directory". In this + * the only way to know there is a directory is using BlobPrefix entry. + * So, sometimes we receive both a Blob and a BlobPrefix for directories, + * and sometimes we receive only BlobPrefix as directory. We remove duplicates + * but prefer Blob over BlobPrefix. + */ + List blobProperties = blobList.getBlobPropertyList(); + + for (BlobProperty entry: blobProperties) { + String blobKey = entry.getName(); + final String owner = identityTransformer.transformIdentityForGetRequest( + entry.getOwner(), true, userName); + final String group = identityTransformer.transformIdentityForGetRequest( + entry.getGroup(), false, primaryUserGroup); + final FsPermission fsPermission = entry.getPermission() == null + ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL) + : AbfsPermission.valueOf(entry.getPermission()); + final boolean hasAcl = entry.getAcl() != null; + long blockSize = abfsConfiguration.getAzureBlockSize(); + + Path entryPath = entry.getPath(); + entryPath = entryPath.makeQualified(this.uri, entryPath); + + FileStatus fileStatus = new VersionedFileStatus( + owner, + group, + fsPermission, + hasAcl, + entry.getContentLength(), + entry.getIsDirectory(), + 1, + blockSize, + entry.getLastModifiedTime(), + entryPath, + entry.getETag()); + + if (entry.getETag() != null) { + // This is a blob entry. It is either a file or a marker blob. + // In both cases we will add this. + fileMetadata.put(blobKey, fileStatus); + } else { + // This is a BlobPrefix entry. It is a directory with file inside + // This might have already been added as a marker blob. + if (!fileMetadata.containsKey(blobKey)) { + fileMetadata.put(blobKey, fileStatus); + } + } + } + } + // generate continuation token for xns account private String generateContinuationTokenForXns(final String firstEntryName) { Preconditions.checkArgument(!Strings.isNullOrEmpty(firstEntryName) @@ -1505,6 +2849,46 @@ public boolean isAtomicRenameKey(String key) { return isKeyForDirectorySet(key, azureAtomicRenameDirSet); } + /** + * Provides a standard implementation of + * {@link RenameAtomicityUtils.RedoRenameInvocation}. + */ + RenameAtomicityUtils.RedoRenameInvocation getRedoRenameInvocation(final TracingContext tracingContext) { + return new RenameAtomicityUtils.RedoRenameInvocation() { + @Override + public void redo(final Path destination, final Path src) + throws AzureBlobFileSystemException { + + ListBlobQueue listBlobQueue = new ListBlobQueue( + getAbfsConfiguration().getProducerQueueMaxSize(), + getAbfsConfiguration().getBlobDirRenameMaxThread()); + StringBuilder listSrcBuilder = new StringBuilder(src.toUri().getPath()); + if (!src.isRoot()) { + listSrcBuilder.append(FORWARD_SLASH); + } + String listSrc = listSrcBuilder.toString(); + getListBlobProducer(listSrc, listBlobQueue, null, tracingContext); + final AbfsBlobLease abfsBlobLease; + try { + abfsBlobLease = getBlobLease(src.toUri().getPath(), + BLOB_LEASE_ONE_MINUTE_DURATION, tracingContext); + } catch (AbfsRestOperationException ex) { + /* + * The required blob might be deleted in between the last check (from + * GetFileStatus or ListStatus) and the leaseAcquire. Hence, catching + * HTTP_NOT_FOUND error. + */ + if (ex.getStatusCode() == HTTP_NOT_FOUND) { + return; + } + throw ex; + } + renameBlobDir(src, destination, tracingContext, listBlobQueue, + abfsBlobLease, true); + } + }; + } + public boolean isInfiniteLeaseKey(String key) { if (azureInfiniteLeaseDirSet.isEmpty()) { return false; @@ -1530,9 +2914,7 @@ private void initializeClient(URI uri, String fileSystemName, return; } - final URIBuilder uriBuilder = getURIBuilder(accountName, isSecure); - - final String url = uriBuilder.toString() + AbfsHttpConstants.FORWARD_SLASH + fileSystemName; + final String url = getBaseUrlString(fileSystemName, accountName, isSecure); URL baseUrl; try { @@ -1581,6 +2963,16 @@ private void initializeClient(URI uri, String fileSystemName, LOG.trace("AbfsClient init complete"); } + private String getBaseUrlString(final String fileSystemName, + final String accountName, + final boolean isSecure) { + final URIBuilder uriBuilder = getURIBuilder(accountName, isSecure); + + final String url = uriBuilder.toString() + AbfsHttpConstants.FORWARD_SLASH + + fileSystemName; + return url; + } + /** * Populate a new AbfsClientContext instance with the desired properties. * @@ -1711,12 +3103,37 @@ private AbfsPerfInfo startTracking(String callerName, String calleeName) { return new AbfsPerfInfo(abfsPerfTracker, callerName, calleeName); } + /** + * Returns a pair of fileStatuses for the renamePendingJSON file and the renameSource file. + * @param fileStatuses array of fileStatus from which pair has to be searched. + * @return Pair of FileStatus. Left of the pair is fileStatus of renamePendingJson file. + * Right of the pair is fileStatus of renameSource file. + */ + public Pair getRenamePendingFileStatus(final FileStatus[] fileStatuses) { + Map fileStatusMap = new HashMap<>(); + FileStatus renamePendingJsonFileStatus = null; + String requiredRenameSrcPath = null; + for (FileStatus fileStatus : fileStatuses) { + String path = fileStatus.getPath().toUri().getPath(); + if (path.equals(requiredRenameSrcPath)) { + return Pair.of(renamePendingJsonFileStatus, fileStatus); + } + fileStatusMap.put(path, fileStatus); + if (path.endsWith(SUFFIX)) { + renamePendingJsonFileStatus = fileStatus; + requiredRenameSrcPath = path.split(SUFFIX)[0]; + } + } + return Pair.of(renamePendingJsonFileStatus, + fileStatusMap.get(requiredRenameSrcPath)); + } + /** * A File status with version info extracted from the etag value returned * in a LIST or HEAD request. * The etag is included in the java serialization. */ - private static final class VersionedFileStatus extends FileStatus + static final class VersionedFileStatus extends FileStatus implements EtagSource { /** @@ -1888,7 +3305,12 @@ private AbfsLease maybeCreateLease(String relativePath, TracingContext tracingCo if (!enableInfiniteLease) { return null; } - AbfsLease lease = new AbfsLease(client, relativePath, tracingContext); + final AbfsLease lease; + if (getPrefixMode() == PrefixMode.DFS) { + lease = new AbfsDfsLease(client, relativePath, null, tracingContext); + } else { + lease = getBlobLease(relativePath, null, tracingContext); + } leaseRefs.put(lease, null); return lease; } @@ -1909,7 +3331,7 @@ boolean areLeasesFreed() { * @param result response to process. * @return the quote-unwrapped etag. */ - private static String extractEtagHeader(AbfsHttpOperation result) { + static String extractEtagHeader(AbfsHttpOperation result) { String etag = result.getResponseHeader(HttpHeaderConfigurations.ETAG); if (etag != null) { // strip out any wrapper "" quotes which come back, for consistency with 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 5cf7ec565b59e5..541f249658757a 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 @@ -33,6 +33,12 @@ public final class AbfsHttpConstants { public static final String FILE = "file"; public static final String DIRECTORY = "directory"; public static final String APPEND_ACTION = "append"; + public static final String BLOCK = "block"; + public static final String BLOCKLIST = "blocklist"; + + public static final String COMP_LIST = "comp=list"; + public static final String COMMITTED = "committed"; + public static final String UNCOMMITTED = "uncommitted"; public static final String FLUSH_ACTION = "flush"; public static final String SET_PROPERTIES_ACTION = "setProperties"; public static final String SET_ACCESS_CONTROL = "setAccessControl"; @@ -46,7 +52,11 @@ public final class AbfsHttpConstants { public static final String DEFAULT_LEASE_BREAK_PERIOD = "0"; public static final String DEFAULT_TIMEOUT = "90"; public static final String APPEND_BLOB_TYPE = "appendblob"; + public static final String BLOCK_BLOB_TYPE = "BlockBlob"; public static final String TOKEN_VERSION = "2"; + public static final String COMMITTED_BLOCKS = "CommittedBlocks"; + public static final String BLOCK_NAME = "Block"; + public static final String NAME = "Name"; public static final String JAVA_VENDOR = "java.vendor"; public static final String JAVA_VERSION = "java.version"; @@ -64,6 +74,11 @@ public final class AbfsHttpConstants { public static final String HTTP_METHOD_PATCH = "PATCH"; public static final String HTTP_METHOD_POST = "POST"; public static final String HTTP_METHOD_PUT = "PUT"; + /** + * All status codes less than http 100 signify error + * and should qualify for retry. + */ + public static final int HTTP_CONTINUE = 100; // Abfs generic constants public static final String SINGLE_WHITE_SPACE = " "; @@ -82,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"; @@ -90,6 +106,12 @@ 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 PUT_BLOCK_LIST = "PutBlockList"; + public static final String XML_VERSION = "\n"; + public static final String BLOCK_LIST_START_TAG = "\n"; + public static final String BLOCK_LIST_END_TAG = "\n"; + public static final String LATEST_BLOCK_FORMAT = "%s\n"; public static final String ROOT_PATH = "/"; public static final String ACCESS_MASK = "mask:"; @@ -103,6 +125,9 @@ public final class AbfsHttpConstants { public static final String DEFAULT_SCOPE = "default:"; public static final String PERMISSION_FORMAT = "%04d"; public static final String SUPER_USER = "$superuser"; + // The HTTP 100 Continue informational status response code indicates that everything so far + // is OK and that the client should continue with the request or ignore it if it is already finished. + public static final String HUNDRED_CONTINUE = "100-continue"; public static final char CHAR_FORWARD_SLASH = '/'; public static final char CHAR_EXCLAMATION_POINT = '!'; @@ -111,6 +136,50 @@ public final class AbfsHttpConstants { public static final char CHAR_EQUALS = '='; public static final char CHAR_STAR = '*'; public static final char CHAR_PLUS = '+'; + /** + * Value that differentiates categories of the http_status. + *
+   * 100 - 199 : Informational responses
+   * 200 - 299 : Successful responses
+   * 300 - 399 : Redirection messages
+   * 400 - 499 : Client error responses
+   * 500 - 599 : Server error responses
+   * 
+ */ + public static final Integer HTTP_STATUS_CATEGORY_QUOTIENT = 100; + + public static final String BLOB = "Blob"; + private static final String PREFIX = "Prefix"; + public static final String NEXT_MARKER = "NextMarker"; + public static final String METADATA = "Metadata"; + public static final String PROPERTIES = "Properties"; + public static final String BLOB_PREFIX = "BlobPrefix"; + public static final String CONTENT_LEN = "Content-Length"; + public static final String RESOURCE_TYPE = "ResourceType"; + public static final String INVALID_XML = "Invalid XML"; + public static final String COPY_STATUS_SUCCESS = "success"; + public static final String COPY_STATUS_PENDING = "pending"; + public static final String COPY_STATUS_ABORTED = "aborted"; + public static final String COPY_STATUS_FAILED = "failed"; + public static final String HDI_ISFOLDER = "hdi_isfolder"; + public static final Integer BLOB_LEASE_ONE_MINUTE_DURATION = 60; + public static final String ETAG = "Etag"; + public static final String LAST_MODIFIED_TIME = "Last-Modified"; + public static final String CREATION_TIME = "Creation-Time"; + public static final String OWNER = "Owner"; + public static final String GROUP = "Group"; + public static final String PERMISSIONS = "Permissions"; + public static final String ACL = "Acl"; + public static final String COPY_ID = "CopyId"; + public static final String COPY_STATUS = "CopyStatus"; + public static final String COPY_SOURCE = "CopySource"; + public static final String COPY_PROGRESS = "CopyProgress"; + public static final String COPY_COMPLETION_TIME = "CopyCompletionTime"; + public static final String COPY_STATUS_DESCRIPTION = "CopyStatusDescription"; + public static final String BLOB_ERROR_CODE_START_XML = ""; + public static final String BLOB_ERROR_CODE_END_XML = ""; + public static final String BLOB_ERROR_MESSAGE_START_XML = ""; + public static final String BLOB_ERROR_MESSAGE_END_XML = ""; private AbfsHttpConstants() {} } 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 12beb5a9bbabe8..7cf95261b9dfdc 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,9 +35,15 @@ public final class ConfigurationKeys { * path to determine HNS status. */ public static final String FS_AZURE_ACCOUNT_IS_HNS_ENABLED = "fs.azure.account.hns.enabled"; + /** + * Enable or disable expect hundred continue header. + * Value: {@value}. + */ + public static final String FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED = "fs.azure.account.expect.header.enabled"; public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME = "fs.azure.account.key"; public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX = "fs\\.azure\\.account\\.key\\.(.*)"; public static final String FS_AZURE_SECURE_MODE = "fs.azure.secure.mode"; + public static final String FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED = "fs.azure.account.throttling.enabled"; // Retry strategy defined by the user public static final String AZURE_MIN_BACKOFF_INTERVAL = "fs.azure.io.retry.min.backoff.interval"; @@ -79,7 +85,7 @@ public final class ConfigurationKeys { /** * What data block buffer to use. *
- * Options include: "disk"(Default), "array", and "bytebuffer". + * Options include: "disk", "array", and "bytebuffer"(Default). *
* Default is {@link FileSystemConfigurations#DATA_BLOCKS_BUFFER_DEFAULT}. * Value: {@value} @@ -116,6 +122,8 @@ public final class ConfigurationKeys { public static final String AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION = "fs.azure.createRemoteFileSystemDuringInitialization"; public static final String AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION = "fs.azure.skipUserGroupMetadataDuringInitialization"; public static final String FS_AZURE_ENABLE_AUTOTHROTTLING = "fs.azure.enable.autothrottling"; + public static final String FS_AZURE_ACCOUNT_OPERATION_IDLE_TIMEOUT = "fs.azure.account.operation.idle.timeout"; + public static final String FS_AZURE_ANALYSIS_PERIOD = "fs.azure.analysis.period"; public static final String FS_AZURE_ALWAYS_USE_HTTPS = "fs.azure.always.use.https"; public static final String FS_AZURE_ATOMIC_RENAME_KEY = "fs.azure.atomic.rename.key"; /** This config ensures that during create overwrite an existing file will be @@ -123,6 +131,7 @@ public final class ConfigurationKeys { */ public static final String FS_AZURE_ENABLE_CONDITIONAL_CREATE_OVERWRITE = "fs.azure.enable.conditional.create.overwrite"; public static final String FS_AZURE_ENABLE_MKDIR_OVERWRITE = "fs.azure.enable.mkdir.overwrite"; + public static final String FS_AZURE_ENABLE_BLOB_MKDIR_OVERWRITE = "fs.azure.enable.blob.mkdir.overwrite"; /** Provides a config to provide comma separated path prefixes on which Appendblob based files are created * Default is empty. **/ public static final String FS_AZURE_APPEND_BLOB_KEY = "fs.azure.appendblob.directories"; @@ -186,6 +195,13 @@ public final class ConfigurationKeys { public static final String FS_AZURE_SKIP_SUPER_USER_REPLACEMENT = "fs.azure.identity.transformer.skip.superuser.replacement"; public static final String AZURE_KEY_ACCOUNT_KEYPROVIDER = "fs.azure.account.keyprovider"; public static final String AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT = "fs.azure.shellkeyprovider.script"; + + /** + * Enable or disable readahead buffer in AbfsInputStream. + * Value: {@value}. + */ + public static final String FS_AZURE_ENABLE_READAHEAD = "fs.azure.enable.readahead"; + /** Setting this true will make the driver use it's own RemoteIterator implementation */ public static final String FS_AZURE_ENABLE_ABFS_LIST_ITERATOR = "fs.azure.enable.abfslistiterator"; /** Server side encryption key */ @@ -229,6 +245,8 @@ public static String accountProperty(String property, String account) { /** Key for SAS token provider **/ public static final String FS_AZURE_SAS_TOKEN_PROVIDER_TYPE = "fs.azure.sas.token.provider.type"; + /** Key for fixed SAS token **/ + public static final String FS_AZURE_SAS_FIXED_TOKEN = "fs.azure.sas.fixed.token"; /** For performance, AbfsInputStream/AbfsOutputStream re-use SAS tokens until the expiry is within this number of seconds. **/ public static final String FS_AZURE_SAS_TOKEN_RENEW_PERIOD_FOR_STREAMS = "fs.azure.sas.token.renew.period.for.streams"; @@ -246,5 +264,19 @@ public static String accountProperty(String property, String account) { * @see FileSystem#openFile(org.apache.hadoop.fs.Path) */ public static final String FS_AZURE_BUFFERED_PREAD_DISABLE = "fs.azure.buffered.pread.disable"; + public static final String FS_AZURE_BLOB_DIR_RENAME_MAX_THREAD = "fs.azure.blob.dir.rename.max.thread"; + public static final String FS_AZURE_BLOB_DIR_DELETE_MAX_THREAD = "fs.azure.blob.dir.delete.max.thread"; + public static final String FS_AZURE_BLOB_COPY_PROGRESS_POLL_WAIT_MILLIS = "fs.azure.blob.copy.progress.poll.wait.millis"; + + public static final String FS_AZURE_ENABLE_BLOB_ENDPOINT = "fs.azure.enable.blob.endpoint"; + public static final String FS_AZURE_MKDIRS_FALLBACK_TO_DFS = "fs.azure.mkdirs.fallback.to.dfs"; + public static final String FS_AZURE_INGRESS_FALLBACK_TO_DFS = "fs.azure.ingress.fallback.to.dfs"; + public static final String FS_AZURE_READ_FALLBACK_TO_DFS = "fs.azure.read.fallback.to.dfs"; + + public static final String FS_AZURE_REDIRECT_DELETE = "fs.azure.redirect.delete"; + public static final String FS_AZURE_REDIRECT_RENAME = "fs.azure.redirect.rename"; + public static final String FS_AZURE_PRODUCER_QUEUE_MAX_SIZE = "fs.azure.blob.dir.list.producer.queue.max.size"; + public static final String FS_AZURE_LEASE_CREATE_NON_RECURSIVE = "fs.azure.lease.create.non.recursive"; + private ConfigurationKeys() {} } 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 f58c61e8908a69..8b96a7a6acd931 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,7 +32,7 @@ public final class FileSystemConfigurations { public static final String DEFAULT_FS_AZURE_ACCOUNT_IS_HNS_ENABLED = ""; - + public static final boolean DEFAULT_FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED = true; public static final String USER_HOME_DIRECTORY_PREFIX = "/user"; private static final int SIXTY_SECONDS = 60 * 1000; @@ -78,8 +78,10 @@ public final class FileSystemConfigurations { public static final boolean DEFAULT_AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION = false; public static final String DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES = "/hbase"; + public static final String HBASE_ROOT = "/hbase"; public static final boolean DEFAULT_FS_AZURE_ENABLE_CONDITIONAL_CREATE_OVERWRITE = true; public static final boolean DEFAULT_FS_AZURE_ENABLE_MKDIR_OVERWRITE = true; + public static final boolean DEFAULT_FS_AZURE_BLOB_ENABLE_MKDIR_OVERWRITE = false; public static final String DEFAULT_FS_AZURE_APPEND_BLOB_DIRECTORIES = ""; public static final String DEFAULT_FS_AZURE_INFINITE_LEASE_DIRECTORIES = ""; public static final int DEFAULT_LEASE_THREADS = 0; @@ -91,9 +93,14 @@ public final class FileSystemConfigurations { public static final int DEFAULT_READ_AHEAD_QUEUE_DEPTH = 2; + public static final String IS_FOLDER_METADATA_KEY = "hdi_isfolder"; + public static final boolean DEFAULT_ENABLE_FLUSH = true; public static final boolean DEFAULT_DISABLE_OUTPUTSTREAM_FLUSH = true; public static final boolean DEFAULT_ENABLE_AUTOTHROTTLING = true; + public static final boolean DEFAULT_FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED = true; + public static final int DEFAULT_ACCOUNT_OPERATION_IDLE_TIMEOUT_MS = 60_000; + public static final int DEFAULT_ANALYSIS_PERIOD_MS = 10_000; public static final DelegatingSSLSocketFactory.SSLChannelMode DEFAULT_FS_AZURE_SSL_CHANNEL_MODE = DelegatingSSLSocketFactory.SSLChannelMode.Default; @@ -106,6 +113,7 @@ public final class FileSystemConfigurations { public static final boolean DEFAULT_ABFS_LATENCY_TRACK = false; public static final long DEFAULT_SAS_TOKEN_RENEW_PERIOD_FOR_STREAMS_IN_SECONDS = 120; + public static final boolean DEFAULT_ENABLE_READAHEAD = true; public static final String DEFAULT_FS_AZURE_USER_AGENT_PREFIX = EMPTY_STRING; public static final String DEFAULT_VALUE_UNKNOWN = "UNKNOWN"; @@ -115,6 +123,19 @@ public final class FileSystemConfigurations { public static final int STREAM_ID_LEN = 12; public static final boolean DEFAULT_ENABLE_ABFS_LIST_ITERATOR = true; + public static final boolean DEFAULT_FS_AZURE_ENABLE_BLOBENDPOINT = false; + public static final boolean DEFAULT_FS_AZURE_MKDIRS_FALLBACK_TO_DFS = false; + public static final boolean DEFAULT_FS_AZURE_INGRESS_FALLBACK_TO_DFS = false; + public static final boolean DEFAULT_AZURE_READ_FALLBACK_TO_DFS = false; + + // To have functionality similar to drop1 delete is going to wasb by default for now. + public static final boolean DEFAULT_FS_AZURE_REDIRECT_RENAME = false; + public static final boolean DEFAULT_FS_AZURE_REDIRECT_DELETE = false; + public static final int DEFAULT_FS_AZURE_PRODUCER_QUEUE_MAX_SIZE = 10000; + public static final boolean DEFAULT_FS_AZURE_LEASE_CREATE_NON_RECURSIVE = false; + public static final int DEFAULT_FS_AZURE_BLOB_RENAME_THREAD = 5; + public static final int DEFAULT_FS_AZURE_BLOB_DELETE_THREAD = 5; + /** * Limit of queued block upload operations before writes * block for an OutputStream. Value: {@value} @@ -127,11 +148,16 @@ public final class FileSystemConfigurations { */ public static final String DATA_BLOCKS_BUFFER_DISK = "disk"; + public static final String DATA_BLOCKS_BYTEBUFFER = "bytebuffer"; + /** * Default buffer option: {@value}. */ public static final String DATA_BLOCKS_BUFFER_DEFAULT = - DATA_BLOCKS_BUFFER_DISK; + DATA_BLOCKS_BYTEBUFFER; + + /** The blockId of each block should be of the same length. */ + public static final int BLOCK_ID_LENGTH = 60; private FileSystemConfigurations() {} } 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 c7a0cdad605ab2..573c819fc29f2f 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 @@ -29,14 +29,14 @@ public final class FileSystemUriSchemes { public static final String ABFS_SCHEME = "abfs"; public static final String ABFS_SECURE_SCHEME = "abfss"; - public static final String ABFS_DNS_PREFIX = "dfs"; + public static final String ABFS_DNS_PREFIX = ".dfs."; public static final String HTTP_SCHEME = "http"; public static final String HTTPS_SCHEME = "https"; public static final String WASB_SCHEME = "wasb"; public static final String WASB_SECURE_SCHEME = "wasbs"; - public static final String WASB_DNS_PREFIX = "blob"; + public static final String WASB_DNS_PREFIX = ".blob."; 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 d4065ac2836d02..cd3c321b4395d0 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 @@ -68,8 +68,17 @@ public final class HttpHeaderConfigurations { public static final String X_MS_LEASE_ACTION = "x-ms-lease-action"; public static final String X_MS_LEASE_DURATION = "x-ms-lease-duration"; public static final String X_MS_LEASE_ID = "x-ms-lease-id"; + public static final String X_MS_SOURCE_LEASE_ID = "x-ms-source-lease-id"; public static final String X_MS_PROPOSED_LEASE_ID = "x-ms-proposed-lease-id"; public static final String X_MS_LEASE_BREAK_PERIOD = "x-ms-lease-break-period"; + 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_COPY_ID = "x-ms-copy-id"; + public static final String X_MS_COPY_SOURCE = "x-ms-copy-source"; + public static final String X_MS_COPY_STATUS_DESCRIPTION = "x-ms-copy-status-description"; + public static final String X_MS_COPY_STATUS = "x-ms-copy-status"; + public static final String EXPECT = "Expect"; + public static final String X_MS_METADATA_PREFIX = "x-ms-meta-"; 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 e9bb95cad21cd6..1b493e590b9b76 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 @@ -27,6 +27,14 @@ @InterfaceStability.Evolving public final class HttpQueryParams { public static final String QUERY_PARAM_RESOURCE = "resource"; + public static final String QUERY_PARAM_RESTYPE = "restype"; + public static final String QUERY_PARAM_COMP = "comp"; + public static final String QUERY_PARAM_COMP_LEASE_VALUE = "lease"; + public static final String QUERY_PARAM_COMP_VALUE_LIST = "list"; + public static final String QUERY_PARAM_PREFIX = "prefix"; + public static final String QUERY_PARAM_MARKER = "marker"; + public static final String QUERY_PARAM_MAXRESULT = "maxresults"; + public static final String QUERY_PARAM_BLOCKLISTTYPE = "blocklisttype"; public static final String QUERY_PARAM_DIRECTORY = "directory"; public static final String QUERY_PARAM_CONTINUATION = "continuation"; public static final String QUERY_PARAM_RECURSIVE = "recursive"; @@ -40,12 +48,17 @@ public final class HttpQueryParams { public static final String QUERY_PARAM_CLOSE = "close"; public static final String QUERY_PARAM_UPN = "upn"; public static final String QUERY_PARAM_BLOBTYPE = "blobtype"; + public static final String QUERY_PARAM_BLOCKID = "blockid"; + public static final String QUERY_PARAM_DELIMITER = "delimiter"; //query params for SAS public static final String QUERY_PARAM_SAOID = "saoid"; public static final String QUERY_PARAM_SKOID = "skoid"; public static final String QUERY_PARAM_SUOID = "suoid"; public static final String QUERY_PARAM_SIGNATURE = "sig"; + public static final String QUERY_PARAM_INCLUDE = "include"; + public static final String QUERY_PARAM_INCLUDE_VALUE_METADATA = "metadata"; + public static final String CONTAINER = "container"; private HttpQueryParams() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/InternalConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/InternalConstants.java new file mode 100644 index 00000000000000..12d4f14d92a00b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/InternalConstants.java @@ -0,0 +1,46 @@ +/* + * 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; + +import org.apache.hadoop.classification.InterfaceAudience; + +/** + * Constants which are used internally and which don't fit into the other + * classes. + * For use within the {@code hadoop-azure} module only. + */ +@InterfaceAudience.Private +public final class InternalConstants { + + private InternalConstants() { + } + + /** + * Does this version of the store have safe readahead? + * Possible combinations of this and the probe + * {@code "fs.capability.etags.available"}. + *
    + *
  1. {@value}: store is safe
  2. + *
  3. !etags: store is safe
  4. + *
  5. etags && !{@value}: store is UNSAFE
  6. + *
+ */ + public static final String CAPABILITY_SAFE_READAHEAD = + "fs.azure.capability.readahead.safe"; +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsRestOperationException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsRestOperationException.java index 6c53762363840a..2ac325ba3181e8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsRestOperationException.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsRestOperationException.java @@ -84,18 +84,20 @@ private static String formatMessage(final AbfsHttpOperation abfsHttpOperation) { // HEAD request response doesn't have StorageErrorCode, StorageErrorMessage. if (abfsHttpOperation.getMethod().equals("HEAD")) { return String.format( - "Operation failed: \"%1$s\", %2$s, HEAD, %3$s", + "Operation failed: \"%1$s\", %2$s, HEAD, %3$s, rId: %4$s", abfsHttpOperation.getStatusDescription(), abfsHttpOperation.getStatusCode(), - abfsHttpOperation.getMaskedUrl()); + abfsHttpOperation.getMaskedUrl(), + abfsHttpOperation.getRequestId()); } return String.format( - "Operation failed: \"%1$s\", %2$s, %3$s, %4$s, %5$s, \"%6$s\"", + "Operation failed: \"%1$s\", %2$s, %3$s, %4$s, rId: %5$s, %6$s, \"%7$s\"", abfsHttpOperation.getStatusDescription(), abfsHttpOperation.getStatusCode(), abfsHttpOperation.getMethod(), abfsHttpOperation.getMaskedUrl(), + abfsHttpOperation.getRequestId(), abfsHttpOperation.getStorageErrorCode(), // Remove break line to ensure the request id and timestamp can be shown in console. abfsHttpOperation.getStorageErrorMessage().replaceAll("\\n", " ")); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java index aba1d8c1efa2b3..147cb6d83cb0ba 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java @@ -29,12 +29,33 @@ @InterfaceAudience.Public @InterfaceStability.Evolving public class InvalidAbfsRestOperationException extends AbfsRestOperationException { + + private static final String ERROR_MESSAGE = "InvalidAbfsRestOperationException"; + public InvalidAbfsRestOperationException( final Exception innerException) { super( AzureServiceErrorCode.UNKNOWN.getStatusCode(), AzureServiceErrorCode.UNKNOWN.getErrorCode(), - "InvalidAbfsRestOperationException", + innerException != null + ? innerException.toString() + : ERROR_MESSAGE, innerException); } + + /** + * Adds the retry count along with the exception. + * @param innerException The inner exception which is originally caught. + * @param retryCount The retry count when the exception was thrown. + */ + public InvalidAbfsRestOperationException( + final Exception innerException, int retryCount) { + super( + AzureServiceErrorCode.UNKNOWN.getStatusCode(), + AzureServiceErrorCode.UNKNOWN.getErrorCode(), + innerException != null + ? innerException.toString() + : ERROR_MESSAGE + " RetryCount: " + retryCount, + innerException); + } } 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 7369bfaf56422c..170a331df74f15 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 @@ -34,19 +34,24 @@ public enum Mode { private final Mode mode; private final boolean isAppendBlob; private final String leaseId; + private boolean isExpectHeaderEnabled; + private boolean isRetryDueToExpect; public AppendRequestParameters(final long position, final int offset, final int length, final Mode mode, final boolean isAppendBlob, - final String leaseId) { + final String leaseId, + final boolean isExpectHeaderEnabled) { this.position = position; this.offset = offset; this.length = length; this.mode = mode; this.isAppendBlob = isAppendBlob; this.leaseId = leaseId; + this.isExpectHeaderEnabled = isExpectHeaderEnabled; + this.isRetryDueToExpect = false; } public long getPosition() { @@ -72,4 +77,20 @@ public boolean isAppendBlob() { public String getLeaseId() { return this.leaseId; } + + public boolean isExpectHeaderEnabled() { + return isExpectHeaderEnabled; + } + + public boolean isRetryDueToExpect() { + return isRetryDueToExpect; + } + + public void setExpectHeaderEnabled(boolean expectHeaderEnabled) { + isExpectHeaderEnabled = expectHeaderEnabled; + } + + public void setRetryDueToExpect(boolean retryDueToExpect) { + isRetryDueToExpect = retryDueToExpect; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java index 8bc31c4f92b2a3..ef4f7dc1773ae3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java @@ -47,6 +47,8 @@ public enum AzureServiceErrorCode { INVALID_QUERY_PARAMETER_VALUE("InvalidQueryParameterValue", HttpURLConnection.HTTP_BAD_REQUEST, null), AUTHORIZATION_PERMISSION_MISS_MATCH("AuthorizationPermissionMismatch", HttpURLConnection.HTTP_FORBIDDEN, null), ACCOUNT_REQUIRES_HTTPS("AccountRequiresHttps", HttpURLConnection.HTTP_BAD_REQUEST, null), + COPY_BLOB_FAILED("COPY_BLOB_FAILED", HttpURLConnection.HTTP_INTERNAL_ERROR, null), + COPY_BLOB_ABORTED("COPY_BLOB_ABORTED", HttpURLConnection.HTTP_INTERNAL_ERROR, null), UNKNOWN(null, -1, null); private final String errorCode; @@ -66,6 +68,10 @@ public String getErrorCode() { return this.errorCode; } + public String getErrorMessage() { + return this.errorMessage; + } + public static List getAzureServiceCode(int httpStatusCode) { List errorCodes = new ArrayList<>(); if (httpStatusCode == UNKNOWN.httpStatusCode) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/enums/BlobCopyProgress.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/enums/BlobCopyProgress.java new file mode 100644 index 00000000000000..16ab2a0ac02628 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/enums/BlobCopyProgress.java @@ -0,0 +1,8 @@ +package org.apache.hadoop.fs.azurebfs.enums; + +public enum BlobCopyProgress { + SUCCESS, + FAILURE, + ABORTED, + PENDING; +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/SASTokenProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/SASTokenProvider.java index a2cd292b0b2309..98846e7bb8cfd9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/SASTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/SASTokenProvider.java @@ -36,11 +36,22 @@ public interface SASTokenProvider { String CREATE_DIRECTORY_OPERATION = "create-directory"; String CREATE_FILE_OPERATION = "create-file"; String DELETE_OPERATION = "delete"; + String DELETE_BLOB_OPERATION = "delete-blob"; + String DELETE_CONTAINER_OPERATION = "delete-container"; + String CREATE_CONTAINER_OPERATION = "create-container"; String DELETE_RECURSIVE_OPERATION = "delete-recursive"; String GET_ACL_OPERATION = "get-acl"; String GET_STATUS_OPERATION = "get-status"; + String GET_BLOCK_LIST = "get-block-list"; String GET_PROPERTIES_OPERATION = "get-properties"; + String GET_BLOB_PROPERTIES_OPERATION = "get-blob-properties"; + String GET_CONTAINER_PROPERTIES_OPERATION = "get-container-properties"; + String GET_BLOB_METADATA_OPERATION = "get-blob-metadata"; + String GET_CONTAINER_METADATA_OPERATION = "get-container-metadata"; String LIST_OPERATION = "list"; + String LIST_BLOB_OPERATION = "list-blob"; + String COPY_BLOB_DESTINATION = "copy-blob-dst"; + String COPY_BLOB_SOURCE = "copy-blob-src"; String READ_OPERATION = "read"; String RENAME_SOURCE_OPERATION = "rename-source"; String RENAME_DESTINATION_OPERATION = "rename-destination"; @@ -48,8 +59,12 @@ public interface SASTokenProvider { String SET_OWNER_OPERATION = "set-owner"; String SET_PERMISSION_OPERATION = "set-permission"; String SET_PROPERTIES_OPERATION = "set-properties"; + String SET_BLOB_METADATA_OPERATION = "set-blob-metadata"; + String SET_CONTAINER_METADATA_OPERATION = "set-container-metadata"; String WRITE_OPERATION = "write"; + String LEASE_OPERATION = "lease"; + /** * Initialize authorizer for Azure Blob File System. * @param configuration Configuration object 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 49f90feb22e95c..fa77d09f546c05 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 @@ -212,7 +212,7 @@ public String getRequestId() { return this.requestId; } - protected HttpException( + public HttpException( final int httpErrorCode, final String requestId, final String message, @@ -340,7 +340,7 @@ private static boolean isRecoverableFailure(IOException e) { || e instanceof FileNotFoundException); } - private static AzureADToken getTokenSingleCall(String authEndpoint, + public static AzureADToken getTokenSingleCall(String authEndpoint, String payload, Hashtable headers, String httpMethod, boolean isMsi) throws IOException { @@ -386,6 +386,7 @@ private static AzureADToken getTokenSingleCall(String authEndpoint, long responseContentLength = conn.getHeaderFieldLong("Content-Length", 0); requestId = requestId == null ? "" : requestId; + LOG.debug("The res " + responseContentType); if (httpResponseCode == HttpURLConnection.HTTP_OK && responseContentType.startsWith("application/json") && responseContentLength > 0) { InputStream httpResponseStream = conn.getInputStream(); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADToken.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADToken.java index daa5a93bf6cdce..a3f15ca27c3d33 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADToken.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADToken.java @@ -18,6 +18,10 @@ package org.apache.hadoop.fs.azurebfs.oauth2; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; import java.util.Date; @@ -25,10 +29,15 @@ * Object representing the AAD access token to use when making HTTP requests to Azure Data Lake Storage. */ public class AzureADToken { + private static final Logger LOG = LoggerFactory.getLogger(AzureADAuthenticator.class); private String accessToken; private Date expiry; - public String getAccessToken() { + public String getAccessToken() throws IOException { + if (accessToken == null || accessToken.length() == 0) { + LOG.debug("The access token value obtained is empty"); + throw new IOException("The token value obtained is empty"); + } return this.accessToken; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdapter.java index 14914101e5cca5..737ccacbfd4fc7 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdapter.java @@ -72,6 +72,10 @@ protected AzureADToken refreshToken() throws IOException { ex = null; try { accessToken = adaptee.getAccessToken(); + if (accessToken == null || accessToken.length() == 0) { + LOG.debug("CustomTokenProvider Access token value obtained is empty"); + throw new IOException("The CustomTokenProvider Access token is empty"); + } LOG.trace("CustomTokenProvider Access token fetch was successful with retry count {}", (fetchTokenRetryCount - retryCount)); } catch (Exception e) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobLease.java new file mode 100644 index 00000000000000..e254ff98f6444c --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobLease.java @@ -0,0 +1,69 @@ +/** + * 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.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_ID; + +public class AbfsBlobLease extends AbfsLease { + + public AbfsBlobLease(final AbfsClient client, + final String path, + final Integer leaseDuration, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + super(client, path, leaseDuration, tracingContext); + } + + public AbfsBlobLease(final AbfsClient client, + final String path, + final int acquireMaxRetries, + final int acquireRetryInterval, + final Integer leaseDuration, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + super(client, path, acquireMaxRetries, acquireRetryInterval, leaseDuration, + tracingContext); + } + + @Override + String callRenewLeaseAPI(final String path, + final String leaseId, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + return extractLeaseInfo(client.renewBlobLease(path, leaseId, tracingContext)); + } + + @Override + AbfsRestOperation callAcquireLeaseAPI(final String path, + final Integer leaseDuration, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + return client.acquireBlobLease(path, leaseDuration, tracingContext); + } + + @Override + void callReleaseLeaseAPI(final String path, + final String leaseID, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + client.releaseBlobLease(path, leaseID, tracingContext); + } + + private String extractLeaseInfo(final AbfsRestOperation op) { + return op.getResult().getResponseHeader(X_MS_LEASE_ID); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlock.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlock.java new file mode 100644 index 00000000000000..025dfd4f76c69e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlock.java @@ -0,0 +1,129 @@ +/** + * 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.commons.codec.binary.Base64; +import org.apache.hadoop.fs.store.DataBlocks; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_ID_LENGTH; + +/** + * Return activeBlock with blockId. + */ +public class AbfsBlock { + + DataBlocks.DataBlock activeBlock; + String blockId; + AbfsOutputStream outputStream; + + /** + * Gets the activeBlock and the blockId. + * @param outputStream AbfsOutputStream Instance. + * @param offset Used to generate blockId based on offset. + * @throws IOException + */ + AbfsBlock(AbfsOutputStream outputStream, long offset) throws IOException { + this.outputStream = outputStream; + DataBlocks.BlockFactory blockFactory = outputStream.getBlockFactory(); + long blockCount = outputStream.getBlockCount(); + int blockSize = outputStream.getBlockSize(); + AbfsOutputStreamStatistics outputStreamStatistics = outputStream.getOutputStreamStatistics(); + this.activeBlock = blockFactory.create(blockCount, blockSize, outputStreamStatistics); + this.blockId = generateBlockId(offset); + } + + /** + * Helper method that generates blockId. + * @param position The offset needed to generate blockId. + * @return String representing the block ID generated. + */ + private String generateBlockId(long position) { + String streamId = this.outputStream.getStreamID(); + String streamIdHash = Integer.toString(streamId.hashCode()); + String blockId = String.format("%d_%s", position, streamIdHash); + byte[] blockIdByteArray = new byte[BLOCK_ID_LENGTH]; + System.arraycopy(blockId.getBytes(), 0, blockIdByteArray, 0, Math.min(BLOCK_ID_LENGTH, blockId.length())); + return new String(Base64.encodeBase64(blockIdByteArray), StandardCharsets.UTF_8); + } + + /** + * Returns activeBlock. + * @return activeBlock. + */ + public DataBlocks.DataBlock getActiveBlock() { + return activeBlock; + } + + /** + * Returns blockId for the block. + * @return blockId. + */ + public String getBlockId() { + return blockId; + } + + /** + * Returns datasize for the block. + * @return datasize. + */ + public int dataSize() { + return activeBlock.dataSize(); + } + + /** + * Return instance of BlockUploadData. + * @return instance of BlockUploadData. + * @throws IOException + */ + public DataBlocks.BlockUploadData startUpload() throws IOException { + return activeBlock.startUpload(); + } + + /** + * Return the block has data or not. + * @return block has data or not. + */ + public boolean hasData() { + return activeBlock.hasData(); + } + + /** + * Write a series of bytes from the buffer, from the offset. Returns the number of bytes written. + * Only valid in the state Writing. Base class verifies the state but does no writing. + * @param buffer buffer. + * @param offset offset. + * @param length length. + * @return number of bytes written. + * @throws IOException + */ + public int write(byte[] buffer, int offset, int length) throws IOException { + return activeBlock.write(buffer, offset, length); + } + + /** + * Returns remainingCapacity. + * @return remainingCapacity. + */ + public int remainingCapacity() { + return activeBlock.remainingCapacity(); + } +} 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 69ef0d01c7823f..8bdca8c3b5a155 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 @@ -30,14 +30,18 @@ import java.security.NoSuchAlgorithmException; import java.util.ArrayList; import java.util.Base64; +import java.util.HashMap; import java.util.List; import java.util.Locale; +import java.util.Map; import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; -import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.Path; + import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback; @@ -67,18 +71,23 @@ import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.apache.hadoop.util.concurrent.HadoopExecutors; +import static java.net.HttpURLConnection.HTTP_CONFLICT; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_DELETE_CONSIDERED_IDEMPOTENT; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.SERVER_SIDE_ENCRYPTION_ALGORITHM; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DNS_PREFIX; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*; import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.*; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; /** * AbfsClient. */ public 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; private final URL baseUrl; private final SharedKeyCredentials sharedKeyCredentials; @@ -96,6 +105,7 @@ public class AbfsClient implements Closeable { private AccessTokenProvider tokenProvider; private SASTokenProvider sasTokenProvider; private final AbfsCounters abfsCounters; + private final AbfsThrottlingIntercept intercept; private final ListeningScheduledExecutorService executorService; @@ -111,6 +121,7 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden this.retryPolicy = abfsClientContext.getExponentialRetryPolicy(); this.accountName = abfsConfiguration.getAccountName().substring(0, abfsConfiguration.getAccountName().indexOf(AbfsHttpConstants.DOT)); this.authType = abfsConfiguration.getAuthType(accountName); + this.intercept = AbfsThrottlingInterceptFactory.getInstance(accountName, abfsConfiguration); String encryptionKey = this.abfsConfiguration .getClientProvidedEncryptionKey(); @@ -175,6 +186,28 @@ private byte[] getSHA256Hash(String key) throws IOException { } } + private URL changePrefixFromBlobtoDfs(URL url) throws InvalidUriException { + try { + url = new URL(url.toString().replace(WASB_DNS_PREFIX, ABFS_DNS_PREFIX)); + } catch (MalformedURLException ex) { + throw new InvalidUriException(url.toString()); + } + return url; + } + + private URL changePrefixFromDfsToBlob(URL url) throws InvalidUriException { + if (url.toString().contains(WASB_DNS_PREFIX) + || getAbfsConfiguration().getPrefixMode() == PrefixMode.DFS) { + return url; + } + try { + url = new URL(url.toString().replace(ABFS_DNS_PREFIX, WASB_DNS_PREFIX)); + } catch (MalformedURLException ex) { + throw new InvalidUriException(url.toString()); + } + return url; + } + private String getBase64EncodedString(String key) { return getBase64EncodedString(key.getBytes(StandardCharsets.UTF_8)); } @@ -207,11 +240,16 @@ SharedKeyCredentials getSharedKeyCredentials() { return sharedKeyCredentials; } + AbfsThrottlingIntercept getIntercept() { + return intercept; + } + List createDefaultHeaders() { final List requestHeaders = new ArrayList(); requestHeaders.add(new AbfsHttpHeader(X_MS_VERSION, xMsVersion)); requestHeaders.add(new AbfsHttpHeader(ACCEPT, APPLICATION_JSON - + COMMA + SINGLE_WHITE_SPACE + APPLICATION_OCTET_STREAM)); + + COMMA + SINGLE_WHITE_SPACE + APPLICATION_OCTET_STREAM + + COMMA + SINGLE_WHITE_SPACE + APPLICATION_XML)); requestHeaders.add(new AbfsHttpHeader(ACCEPT_CHARSET, UTF_8)); requestHeaders.add(new AbfsHttpHeader(CONTENT_TYPE, EMPTY_STRING)); @@ -243,13 +281,37 @@ public AbfsRestOperation createFilesystem(TracingContext tracingContext) throws final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); - final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.CreateFileSystem, - this, - HTTP_METHOD_PUT, - url, - requestHeaders); + URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.CreateFileSystem, HTTP_METHOD_PUT, url, + requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Caller of + * @param tracingContext + * @return Creates the Container acting as current filesystem + * @throws AzureBlobFileSystemException + */ + public AbfsRestOperation createContainer(TracingContext tracingContext) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESTYPE, CONTAINER); + + appendSASTokenToQuery("", + SASTokenProvider.CREATE_CONTAINER_OPERATION, abfsUriQueryBuilder); + final URL url = createBlobRequestUrl(abfsUriQueryBuilder); + + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.CreateContainer, HTTP_METHOD_PUT, url, + requestHeaders); op.execute(tracingContext); return op; } @@ -267,13 +329,13 @@ public AbfsRestOperation setFilesystemProperties(final String properties, Tracin final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); - final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.SetFileSystemProperties, - this, - HTTP_METHOD_PUT, - url, - requestHeaders); + URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.SetFileSystemProperties, HTTP_METHOD_PUT, url, + requestHeaders); op.execute(tracingContext); return op; } @@ -292,13 +354,12 @@ public AbfsRestOperation listPath(final String relativePath, final boolean recur 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 = new AbfsRestOperation( - AbfsRestOperationType.ListPaths, - this, - HTTP_METHOD_GET, - url, - requestHeaders); + URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.ListPaths, HTTP_METHOD_GET, url, requestHeaders); op.execute(tracingContext); return op; } @@ -309,13 +370,13 @@ public AbfsRestOperation getFilesystemProperties(TracingContext tracingContext) final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); - final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.GetFileSystemProperties, - this, - HTTP_METHOD_HEAD, - url, - requestHeaders); + URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.GetFileSystemProperties, HTTP_METHOD_HEAD, url, + requestHeaders); op.execute(tracingContext); return op; } @@ -326,13 +387,37 @@ public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); - final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.DeleteFileSystem, - this, - HTTP_METHOD_DELETE, - url, - requestHeaders); + URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.DeleteFileSystem, HTTP_METHOD_DELETE, url, + requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Caller of + * @param tracingContext + * @return Deletes the Container acting as current filesystem + * @throws AzureBlobFileSystemException + */ + public AbfsRestOperation deleteContainer(TracingContext tracingContext) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESTYPE, CONTAINER); + + appendSASTokenToQuery("", + SASTokenProvider.DELETE_CONTAINER_OPERATION, abfsUriQueryBuilder); + final URL url = createBlobRequestUrl(abfsUriQueryBuilder); + + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.DeleteContainer, HTTP_METHOD_DELETE, url, + requestHeaders); op.execute(tracingContext); return op; } @@ -372,13 +457,12 @@ public AbfsRestOperation createPath(final String path, final boolean isFile, fin : SASTokenProvider.CREATE_DIRECTORY_OPERATION; appendSASTokenToQuery(path, operation, abfsUriQueryBuilder); - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.CreatePath, - this, - HTTP_METHOD_PUT, - url, - requestHeaders); + URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.CreatePath, HTTP_METHOD_PUT, url, requestHeaders); try { op.execute(tracingContext); } catch (AzureBlobFileSystemException ex) { @@ -386,7 +470,7 @@ public AbfsRestOperation createPath(final String path, final boolean isFile, fin if (!op.hasResult()) { throw ex; } - if (!isFile && op.getResult().getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { + if (!isFile && op.getResult().getStatusCode() == HTTP_CONFLICT) { String existingResource = op.getResult().getResponseHeader(X_MS_EXISTING_RESOURCE_TYPE); if (existingResource != null && existingResource.equals(DIRECTORY)) { @@ -398,6 +482,56 @@ public AbfsRestOperation createPath(final String path, final boolean isFile, fin return op; } + public AbfsRestOperation createPathBlob(final String path, final boolean isFile, final boolean overwrite, + final HashMap metadata, + final String eTag, + TracingContext tracingContext) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + if (!overwrite) { + requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, AbfsHttpConstants.STAR)); + } + if (eTag != null && !eTag.isEmpty()) { + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_MATCH, eTag)); + } + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + + String operation = SASTokenProvider.CREATE_FILE_OPERATION; + appendSASTokenToQuery(path, operation, abfsUriQueryBuilder); + + final URL url = createBlobRequestUrl(path, abfsUriQueryBuilder); + if (metadata != null && !metadata.isEmpty()) { + for (Map.Entry entry : metadata.entrySet()) { + requestHeaders.add(new AbfsHttpHeader(entry.getKey(), entry.getValue())); + } + } + requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, ZERO)); + requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_TYPE, BLOCK_BLOB_TYPE)); + 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. + tracingContext.setFallbackDFSAppend(tracingContext.getFallbackDFSAppend() + "M"); + AbfsRestOperation blobProperty = getBlobProperty(new Path(path), tracingContext); + final AbfsHttpOperation opResult = blobProperty.getResult(); + boolean isDirectory = (opResult.getResponseHeader(X_MS_META_HDI_ISFOLDER) != null); + if (isDirectory) { + return op; + } + } + throw ex; + } + return op; + } + public AbfsRestOperation acquireLease(final String path, int duration, TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); @@ -407,13 +541,30 @@ public AbfsRestOperation acquireLease(final String path, int duration, TracingCo final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.LeasePath, - this, - HTTP_METHOD_POST, - url, - requestHeaders); + URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.LeasePath, HTTP_METHOD_POST, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + public AbfsRestOperation acquireBlobLease(final String path, final int duration, final 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, QUERY_PARAM_COMP_LEASE_VALUE); + appendSASTokenToQuery(path, SASTokenProvider.LEASE_OPERATION, abfsUriQueryBuilder); + + + final URL url = createBlobRequestUrl(path, abfsUriQueryBuilder); + + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.LeaseBlob, HTTP_METHOD_PUT, url, requestHeaders); op.execute(tracingContext); return op; } @@ -427,13 +578,31 @@ public AbfsRestOperation renewLease(final String path, final String leaseId, final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.LeasePath, - this, - HTTP_METHOD_POST, - url, - requestHeaders); + URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.LeasePath, HTTP_METHOD_POST, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + public AbfsRestOperation renewBlobLease(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, QUERY_PARAM_COMP_LEASE_VALUE); + appendSASTokenToQuery(path, SASTokenProvider.LEASE_OPERATION, abfsUriQueryBuilder); + + final URL url = createBlobRequestUrl(path, abfsUriQueryBuilder); + + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.LeaseBlob, HTTP_METHOD_PUT, url, requestHeaders); op.execute(tracingContext); return op; } @@ -447,13 +616,31 @@ public AbfsRestOperation releaseLease(final String path, final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.LeasePath, - this, - HTTP_METHOD_POST, - url, - requestHeaders); + URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.LeasePath, HTTP_METHOD_POST, url, requestHeaders); + op.execute(tracingContext); + return op; + } + + public AbfsRestOperation releaseBlobLease(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, QUERY_PARAM_COMP_LEASE_VALUE); + appendSASTokenToQuery(path, SASTokenProvider.LEASE_OPERATION, abfsUriQueryBuilder); + + final URL url = createBlobRequestUrl(path, abfsUriQueryBuilder); + + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.LeasePath, HTTP_METHOD_PUT, url, requestHeaders); op.execute(tracingContext); return op; } @@ -467,13 +654,12 @@ public AbfsRestOperation breakLease(final String path, final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.LeasePath, - this, - HTTP_METHOD_POST, - url, - requestHeaders); + URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.LeasePath, HTTP_METHOD_POST, url, requestHeaders); op.execute(tracingContext); return op; } @@ -498,13 +684,12 @@ public AbfsRestOperation renamePath(String source, final String destination, abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation); appendSASTokenToQuery(destination, SASTokenProvider.RENAME_DESTINATION_OPERATION, abfsUriQueryBuilder); - final URL url = createRequestUrl(destination, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.RenamePath, - this, - HTTP_METHOD_PUT, - url, - requestHeaders); + URL url = createRequestUrl(destination, abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.RenamePath, HTTP_METHOD_PUT, url, requestHeaders); // no attempt at recovery using timestamps as it was not reliable. op.execute(tracingContext); return op; @@ -515,6 +700,9 @@ public AbfsRestOperation append(final String path, final byte[] buffer, throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); addCustomerProvidedKeyHeaders(requestHeaders); + 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, @@ -534,25 +722,49 @@ public AbfsRestOperation append(final String path, final byte[] buffer, abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, TRUE); } } + if (reqParams.isRetryDueToExpect()) { + String userAgentRetry = userAgent; + userAgentRetry = userAgentRetry.replace(HUNDRED_CONTINUE_USER_AGENT, EMPTY_STRING); + requestHeaders.removeIf(header -> header.getName().equalsIgnoreCase(USER_AGENT)); + requestHeaders.add(new AbfsHttpHeader(USER_AGENT, userAgentRetry)); + } // 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 = new AbfsRestOperation( - AbfsRestOperationType.Append, - this, - HTTP_METHOD_PUT, - url, - requestHeaders, - buffer, - reqParams.getoffset(), - reqParams.getLength(), - sasTokenForReuse); + URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperationForAppend(AbfsRestOperationType.Append, + 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, + tracingContext); + } // If we have no HTTP response, throw the original exception. if (!op.hasResult()) { throw e; @@ -560,16 +772,15 @@ public AbfsRestOperation append(final String path, final byte[] buffer, if (reqParams.isAppendBlob() && appendSuccessCheckOp(op, path, (reqParams.getPosition() + reqParams.getLength()), tracingContext)) { - final AbfsRestOperation successOp = new AbfsRestOperation( - AbfsRestOperationType.Append, - this, - HTTP_METHOD_PUT, - url, - requestHeaders, - buffer, - reqParams.getoffset(), - reqParams.getLength(), - sasTokenForReuse); + final AbfsRestOperation successOp = getAbfsRestOperationForAppend( + AbfsRestOperationType.Append, + HTTP_METHOD_PUT, + url, + requestHeaders, + buffer, + reqParams.getoffset(), + reqParams.getLength(), + sasTokenForReuse); successOp.hardSetResult(HttpURLConnection.HTTP_OK); return successOp; } @@ -579,6 +790,184 @@ && appendSuccessCheckOp(op, path, return op; } + /** + * Append operation for blob endpoint which takes block id as a param. + * @param blockId The blockId of the block to be appended. + * @param path The path at which the block is to be appended. + * @param buffer The buffer which has the data to be appended. + * @param reqParams The request params. + * @param cachedSasToken The cachedSasToken if available. + * @param tracingContext Tracing context of the operation. + * @param eTag Etag of the blob to prevent parallel writer situations. + * @return AbfsRestOperation op. + * @throws AzureBlobFileSystemException + */ + public AbfsRestOperation append(final String blockId, final String path, final byte[] buffer, + AppendRequestParameters reqParams, final String cachedSasToken, + TracingContext tracingContext, String eTag) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + if (reqParams.getLeaseId() != null) { + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, reqParams.getLeaseId())); + } + if (reqParams.isExpectHeaderEnabled()) { + requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); + } + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCK); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_BLOCKID, blockId); + requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, String.valueOf(buffer.length))); + requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag)); + + if (reqParams.isRetryDueToExpect()) { + String userAgentRetry = userAgent; + userAgentRetry = userAgentRetry.replace(HUNDRED_CONTINUE_USER_AGENT, EMPTY_STRING); + requestHeaders.removeIf(header -> header.getName().equalsIgnoreCase(USER_AGENT)); + requestHeaders.add(new AbfsHttpHeader(USER_AGENT, userAgentRetry)); + } + + // AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance + String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.WRITE_OPERATION, + abfsUriQueryBuilder, cachedSasToken); + + final URL url = createBlobRequestUrl(path, abfsUriQueryBuilder); + final AbfsRestOperation op = getPutBlockOperation(buffer, reqParams, requestHeaders, + sasTokenForReuse, url); + 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 (checkUserErrorBlob(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(blockId, path, buffer, reqParams, cachedSasToken, + tracingContext, eTag); + } + else { + throw e; + } + } + return op; + } + + @VisibleForTesting + AbfsRestOperation getPutBlockOperation(final byte[] buffer, + final AppendRequestParameters reqParams, + final List requestHeaders, + final String sasTokenForReuse, + final URL url) { + return getAbfsRestOperation(AbfsRestOperationType.PutBlock, HTTP_METHOD_PUT, + url, + requestHeaders, buffer, reqParams.getoffset(), reqParams.getLength(), + sasTokenForReuse); + } + + /** + * 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 Tracing context for the operation. + * @return AbfsRestOperation op. + * @throws IOException + */ + public AbfsRestOperation flush(byte[] buffer, final String path, boolean isClose, + final String cachedSasToken, final String leaseId, String eTag, + TracingContext tracingContext) throws IOException { + final List requestHeaders = createDefaultHeaders(); + if (leaseId != null) { + requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId)); + } + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST); + 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)); + 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 = createBlobRequestUrl(path, abfsUriQueryBuilder); + final AbfsRestOperation op = getPutBlockListOperation(buffer, + requestHeaders, sasTokenForReuse, + url); + op.execute(tracingContext); + return op; + } + + @VisibleForTesting + AbfsRestOperation getPutBlockListOperation(final byte[] buffer, + final List requestHeaders, + final String sasTokenForReuse, + final URL url) { + return getAbfsRestOperation(AbfsRestOperationType.PutBlockList, + HTTP_METHOD_PUT, url, requestHeaders, buffer, 0, buffer.length, + sasTokenForReuse); + } + + /* + * Returns the rest operation for append. + * @param operationType The AbfsRestOperationType. + * @param httpMethod specifies the httpMethod. + * @param url specifies the url. + * @param requestHeaders This includes the list of request headers. + * @param buffer The buffer to write into. + * @param bufferOffset The buffer offset. + * @param bufferLength The buffer Length. + * @param sasTokenForReuse The sasToken. + * @return AbfsRestOperation op. + */ + @VisibleForTesting + AbfsRestOperation getAbfsRestOperationForAppend(final AbfsRestOperationType operationType, + final String httpMethod, + final URL url, + final List requestHeaders, + final byte[] buffer, + final int bufferOffset, + final int bufferLength, + final String sasTokenForReuse) { + return getAbfsRestOperation(operationType, httpMethod, url, requestHeaders, + buffer, bufferOffset, bufferLength, sasTokenForReuse); + } + + /** + * 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); + } + + /** + * Returns true if the status code lies in the range of user error. + * In the case of HTTP_CONFLICT for PutBlockList we fallback to DFS and hence + * this retry handling is not needed. + * @param responseStatusCode http response status code. + * @return True or False. + */ + private boolean checkUserErrorBlob(int responseStatusCode) { + return (responseStatusCode >= HttpURLConnection.HTTP_BAD_REQUEST + && responseStatusCode < HttpURLConnection.HTTP_INTERNAL_ERROR + && responseStatusCode != HttpURLConnection.HTTP_CONFLICT); + } + // For AppendBlob its possible that the append succeeded in the backend but the request failed. // However a retry would fail with an InvalidQueryParameterValue // (as the current offset would be unacceptable). @@ -624,13 +1013,13 @@ public AbfsRestOperation flush(final String path, final long position, String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.WRITE_OPERATION, abfsUriQueryBuilder, cachedSasToken); - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.Flush, - this, - HTTP_METHOD_PUT, - url, - requestHeaders, sasTokenForReuse); + URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.Flush, HTTP_METHOD_PUT, url, requestHeaders, + sasTokenForReuse); op.execute(tracingContext); return op; } @@ -651,13 +1040,13 @@ public AbfsRestOperation setPathProperties(final String path, final String prope 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 = new AbfsRestOperation( - AbfsRestOperationType.SetPathProperties, - this, - HTTP_METHOD_PUT, - url, - requestHeaders); + URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.SetPathProperties, HTTP_METHOD_PUT, url, + requestHeaders); op.execute(tracingContext); return op; } @@ -680,13 +1069,38 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN, String.valueOf(abfsConfiguration.isUpnUsed())); appendSASTokenToQuery(path, operation, abfsUriQueryBuilder); - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.GetPathStatus, - this, - HTTP_METHOD_HEAD, - url, - requestHeaders); + URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.GetPathStatus, HTTP_METHOD_HEAD, url, + requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * GetBlockList call to the backend to get the list of committed blockId's. + * @param path The path to get the list of blockId's. + * @param tracingContext The tracing context for the operation. + * @return AbfsRestOperation op. + * @throws AzureBlobFileSystemException + */ + public AbfsRestOperation getBlockList(final String path, TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + String operation = SASTokenProvider.GET_BLOCK_LIST; + appendSASTokenToQuery(path, operation, abfsUriQueryBuilder); + + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_BLOCKLISTTYPE, COMMITTED); + final URL url = createBlobRequestUrl(path, abfsUriQueryBuilder); + + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.GetBlockList, HTTP_METHOD_GET, url, + requestHeaders); op.execute(tracingContext); return op; } @@ -705,16 +1119,22 @@ public AbfsRestOperation read(final String path, final long position, final byte String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.READ_OPERATION, abfsUriQueryBuilder, cachedSasToken); - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.ReadFile, - this, - HTTP_METHOD_GET, - url, - requestHeaders, - buffer, - bufferOffset, - bufferLength, sasTokenForReuse); + URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperationType opType; + if (!OperativeEndpoint.isReadEnabledOnDFS(getAbfsConfiguration())) { + LOG.debug("Read over Blob for read config value {} for path {} ", + abfsConfiguration.shouldReadFallbackToDfs(), path); + opType = AbfsRestOperationType.GetBlob; + url = changePrefixFromDfsToBlob(url); + } else { + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + opType = AbfsRestOperationType.ReadFile; + } + final AbfsRestOperation op = getAbfsRestOperation(opType, HTTP_METHOD_GET, + url, requestHeaders, buffer, bufferOffset, bufferLength, + sasTokenForReuse); op.execute(tracingContext); return op; @@ -731,13 +1151,13 @@ public AbfsRestOperation deletePath(final String path, final boolean recursive, 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); + URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.DeletePath, HTTP_METHOD_DELETE, url, + requestHeaders); try { op.execute(tracingContext); } catch (AzureBlobFileSystemException e) { @@ -781,11 +1201,8 @@ public AbfsRestOperation deleteIdempotencyCheckOp(final AbfsRestOperation op) { && DEFAULT_DELETE_CONSIDERED_IDEMPOTENT) { // Server has returned HTTP 404, which means path no longer // exists. Assuming delete result to be idempotent, return success. - final AbfsRestOperation successOp = new AbfsRestOperation( - AbfsRestOperationType.DeletePath, - this, - HTTP_METHOD_DELETE, - op.getUrl(), + final AbfsRestOperation successOp = getAbfsRestOperation( + AbfsRestOperationType.DeletePath, HTTP_METHOD_DELETE, op.getUrl(), op.getRequestHeaders()); successOp.hardSetResult(HttpURLConnection.HTTP_OK); LOG.debug("Returning success response from delete idempotency logic"); @@ -815,12 +1232,12 @@ public AbfsRestOperation setOwner(final String path, final String owner, final S abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_ACCESS_CONTROL); appendSASTokenToQuery(path, SASTokenProvider.SET_OWNER_OPERATION, abfsUriQueryBuilder); - final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); - final AbfsRestOperation op = new AbfsRestOperation( - AbfsRestOperationType.SetOwner, - this, - AbfsHttpConstants.HTTP_METHOD_PUT, - url, + URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.SetOwner, AbfsHttpConstants.HTTP_METHOD_PUT, url, requestHeaders); op.execute(tracingContext); return op; @@ -841,13 +1258,13 @@ public AbfsRestOperation setPermission(final String path, final String permissio 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 = new AbfsRestOperation( - AbfsRestOperationType.SetPermissions, - this, - AbfsHttpConstants.HTTP_METHOD_PUT, - url, - requestHeaders); + URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.SetPermissions, AbfsHttpConstants.HTTP_METHOD_PUT, + url, requestHeaders); op.execute(tracingContext); return op; } @@ -876,12 +1293,12 @@ public AbfsRestOperation setAcl(final String path, final String aclSpecString, f 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 = new AbfsRestOperation( - AbfsRestOperationType.SetAcl, - this, - AbfsHttpConstants.HTTP_METHOD_PUT, - url, + URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.SetAcl, AbfsHttpConstants.HTTP_METHOD_PUT, url, requestHeaders); op.execute(tracingContext); return op; @@ -901,12 +1318,12 @@ public AbfsRestOperation getAclStatus(final String path, final boolean useUPN, 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 = new AbfsRestOperation( - AbfsRestOperationType.GetAcl, - this, - AbfsHttpConstants.HTTP_METHOD_HEAD, - url, + URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.GetAcl, AbfsHttpConstants.HTTP_METHOD_HEAD, url, requestHeaders); op.execute(tracingContext); return op; @@ -928,14 +1345,365 @@ public AbfsRestOperation checkAccess(String path, String rwx, TracingContext tra 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 = new AbfsRestOperation( - AbfsRestOperationType.CheckAccess, this, - AbfsHttpConstants.HTTP_METHOD_HEAD, url, createDefaultHeaders()); + if (url.toString().contains(WASB_DNS_PREFIX)) { + url = changePrefixFromBlobtoDfs(url); + } + + AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.CheckAccess, AbfsHttpConstants.HTTP_METHOD_HEAD, + url, createDefaultHeaders()); op.execute(tracingContext); return op; } + /** + * Caller of + * copyBlob 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 + * @param tracingContext tracingContext object + * + * @return AbfsRestOperation abfsRestOperation which contains the response from the server. + * This method owns the logic of triggereing 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 exception recevied while making server call. + */ + 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.COPY_BLOB_DESTINATION, abfsUriQueryBuilderDst); + appendSASTokenToQuery(srcBlobRelativePath, + SASTokenProvider.COPY_BLOB_SOURCE, abfsUriQueryBuilderSrc); + final URL url = createBlobRequestUrl(dstBlobRelativePath, + abfsUriQueryBuilderDst); + final String sourcePathUrl = createBlobRequestUrl(srcBlobRelativePath, + abfsUriQueryBuilderSrc).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 = getCopyBlobOperation(url, requestHeaders); + op.execute(tracingContext); + + return op; + } + + @VisibleForTesting + AbfsRestOperation getCopyBlobOperation(final URL url, + final List requestHeaders) { + return getAbfsRestOperation(AbfsRestOperationType.CopyBlob, HTTP_METHOD_PUT, + url, requestHeaders); + } + + @VisibleForTesting + AbfsRestOperation getListBlobOperation(final URL url, + final List requestHeaders) { + return getAbfsRestOperation(AbfsRestOperationType.GetListBlobProperties, HTTP_METHOD_GET, + url, requestHeaders); + } + + /** + * @return the blob properties returned from server. + * @throws AzureBlobFileSystemException in case it is not a 404 error or some other exception + * which was not able to be retried. + * */ + public AbfsRestOperation getBlobProperty(Path blobPath, + TracingContext tracingContext) throws AzureBlobFileSystemException { + AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + String blobRelativePath = blobPath.toUri().getPath(); + appendSASTokenToQuery(blobRelativePath, + SASTokenProvider.GET_BLOB_PROPERTIES_OPERATION, abfsUriQueryBuilder); + final URL url = createBlobRequestUrl(blobRelativePath, abfsUriQueryBuilder); + final List requestHeaders = createDefaultHeaders(); + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.GetBlobProperties, HTTP_METHOD_HEAD, url, + requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Caller of + * @return the container properties returned from server. + * @throws AzureBlobFileSystemException in case it is not a 404 error or some other exception + * which was not able to be retried. + * */ + public AbfsRestOperation getContainerProperty(TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESTYPE, CONTAINER); + + appendSASTokenToQuery("", + SASTokenProvider.GET_CONTAINER_PROPERTIES_OPERATION, abfsUriQueryBuilder); + + final URL url = createBlobRequestUrl(abfsUriQueryBuilder); + + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.GetContainerProperties, HTTP_METHOD_HEAD, url, + requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Gets user-defined properties(metadata) of the blob over blob endpoint. + * @param blobPath + * @param tracingContext + * @return the user-defined properties on blob path + * @throws AzureBlobFileSystemException + */ + public AbfsRestOperation getBlobMetadata(Path blobPath, + TracingContext tracingContext) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, QUERY_PARAM_INCLUDE_VALUE_METADATA); + + String blobRelativePath = blobPath.toUri().getPath(); + appendSASTokenToQuery(blobRelativePath, + SASTokenProvider.GET_BLOB_METADATA_OPERATION, abfsUriQueryBuilder); + + final URL url = createBlobRequestUrl(blobRelativePath, abfsUriQueryBuilder); + + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.GetBlobMetadata, HTTP_METHOD_HEAD, url, + requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Sets user-defined properties(metadata) of the blob over blob endpoint. + * @param blobPath + * @param metadataRequestHeaders + * @param tracingContext + * @throws AzureBlobFileSystemException + */ + public AbfsRestOperation setBlobMetadata(Path blobPath, List metadataRequestHeaders, + TracingContext tracingContext) throws AzureBlobFileSystemException { + // Request Header for this call will also contain metadata headers + final List defaultRequestHeaders = createDefaultHeaders(); + final List requestHeaders = new ArrayList(); + requestHeaders.addAll(defaultRequestHeaders); + requestHeaders.addAll(metadataRequestHeaders); + + AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, QUERY_PARAM_INCLUDE_VALUE_METADATA); + + String blobRelativePath = blobPath.toUri().getPath(); + appendSASTokenToQuery(blobRelativePath, + SASTokenProvider.SET_BLOB_METADATA_OPERATION, abfsUriQueryBuilder); + + final URL url = createBlobRequestUrl(blobRelativePath, abfsUriQueryBuilder); + + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.SetBlobMetadata, HTTP_METHOD_PUT, url, + requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Caller of + * Gets user-defined properties(metadata) of the container(filesystem) over blob endpoint. + * @param tracingContext + * @return the user-defined properties on container path + * @throws AzureBlobFileSystemException + */ + public AbfsRestOperation getContainerMetadata(TracingContext tracingContext) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESTYPE, CONTAINER); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, QUERY_PARAM_INCLUDE_VALUE_METADATA); + + appendSASTokenToQuery("", + SASTokenProvider.GET_CONTAINER_METADATA_OPERATION, abfsUriQueryBuilder); + + final URL url = createBlobRequestUrl(abfsUriQueryBuilder); + + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.GetContainerMetadata, HTTP_METHOD_HEAD, url, + requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Caller of + * Sets user-defined properties(metadata) of the container(filesystem) over blob endpoint. + * @param metadataRequestHeaders + * @param tracingContext + * @throws AzureBlobFileSystemException + */ + public AbfsRestOperation setContainerMetadata(List metadataRequestHeaders, + TracingContext tracingContext) throws AzureBlobFileSystemException { + // Request Header for this call will also contain metadata headers + final List defaultRequestHeaders = createDefaultHeaders(); + final List requestHeaders = new ArrayList(); + requestHeaders.addAll(defaultRequestHeaders); + requestHeaders.addAll(metadataRequestHeaders); + + AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESTYPE, CONTAINER); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, QUERY_PARAM_INCLUDE_VALUE_METADATA); + + appendSASTokenToQuery("", + SASTokenProvider.SET_CONTAINER_METADATA_OPERATION, abfsUriQueryBuilder); + + final URL url = createBlobRequestUrl(abfsUriQueryBuilder); + + final AbfsRestOperation op = getAbfsRestOperation( + AbfsRestOperationType.SetContainerMetadata, HTTP_METHOD_PUT, url, + requestHeaders); + op.execute(tracingContext); + return op; + } + + /** + * Call server API BlobList. + * + * @param marker optional value. To be sent in case this method call in a non-first + * iteration to the blobList API. Value has to be equal to the field NextMarker in the response + * of previous iteration for the same operation. + * @param prefix optional value. Filters the results to return only blobs + * with names that begin with the specified prefix + * @param delimiter Optional. When the request includes this parameter, + * the operation returns a BlobPrefix element in the response body. + * This element acts as a placeholder for all blobs with names that begin + * with the same substring, up to the appearance of the delimiter character. + * @param maxResult define how many blobs can client handle in server response. + * In case maxResult <= 5000, server sends number of blobs equal to the value. In + * case maxResult > 5000, server sends maximum 5000 blobs. + * @param tracingContext object of {@link TracingContext} + * + * @return abfsRestOperation which contain list of {@link BlobProperty} + * via {@link AbfsRestOperation#getResult()}.{@link AbfsHttpOperation#getBlobList()} + * + * @throws AzureBlobFileSystemException thrown from server-call / xml-parsing + */ + public AbfsRestOperation getListBlobs(String marker, + String prefix, + String delimiter, + Integer maxResult, + TracingContext tracingContext) + throws AzureBlobFileSystemException { + AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESTYPE, CONTAINER); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, QUERY_PARAM_COMP_VALUE_LIST); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_DELIMITER, delimiter); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_INCLUDE, + QUERY_PARAM_INCLUDE_VALUE_METADATA); + prefix = getDirectoryQueryParameter(prefix); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_PREFIX, prefix); + if (marker != null) { + abfsUriQueryBuilder.addQuery(QUERY_PARAM_MARKER, marker); + } + if (maxResult != null) { + abfsUriQueryBuilder.addQuery(QUERY_PARAM_MAXRESULT, maxResult.toString()); + } + appendSASTokenToQuery(null, SASTokenProvider.LIST_BLOB_OPERATION, + abfsUriQueryBuilder); + final URL url = createBlobRequestUrl(abfsUriQueryBuilder); + final List requestHeaders = createDefaultHeaders(); + final AbfsRestOperation op = getListBlobOperation(url, requestHeaders); + + op.execute(tracingContext); + return op; + } + + /** + * Deletes the blob for which the path is given. + * + * @param blobPath path on which blob has to be deleted. + * @param leaseId + * @param tracingContext tracingContext object for tracing the server calls. + * + * @return abfsRestOpertion + * + * @throws AzureBlobFileSystemException exception thrown from server or due to + * network issue. + */ + 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_BLOB_OPERATION, abfsUriQueryBuilder); + final URL url = createBlobRequestUrl(blobRelativePath, abfsUriQueryBuilder); + 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; + } + + AbfsRestOperation getAbfsRestOperation(final AbfsRestOperationType operationType, + final String httpMethod, + final URL url, + final List requestHeaders, + final byte[] buffer, + final int bufferOffset, + final int bufferLength, + final String sasTokenForReuse) { + return new AbfsRestOperation( + operationType, + this, + httpMethod, + url, + requestHeaders, + buffer, + bufferOffset, + bufferLength, + sasTokenForReuse); + } + + AbfsRestOperation getAbfsRestOperation(final AbfsRestOperationType operationType, + final String httpMethod, + final URL url, + final List requestHeaders) { + return new AbfsRestOperation( + operationType, + this, + httpMethod, + url, + requestHeaders + ); + } + + AbfsRestOperation getAbfsRestOperation(final AbfsRestOperationType operationType, + final String httpMethod, + final URL url, + final List requestHeaders, + final String sasTokenForReuse) { + return new AbfsRestOperation( + operationType, + this, + httpMethod, + url, + requestHeaders, sasTokenForReuse); + } + /** * Get the directory query parameter used by the List Paths REST API and used * as the path in the continuation token. If the input path is null or the @@ -1007,6 +1775,19 @@ private String appendSASTokenToQuery(String path, return sasToken; } + private URL createBlobRequestUrl(final AbfsUriQueryBuilder abfsUriQueryBuilder) + throws AzureBlobFileSystemException { + return changePrefixFromDfsToBlob( + createRequestUrl(abfsUriQueryBuilder.toString())); + } + + private URL createBlobRequestUrl(final String path, + final AbfsUriQueryBuilder abfsUriQueryBuilder) + throws AzureBlobFileSystemException { + return changePrefixFromDfsToBlob( + createRequestUrl(path, abfsUriQueryBuilder.toString())); + } + private URL createRequestUrl(final String query) throws AzureBlobFileSystemException { return createRequestUrl(EMPTY_STRING, query); } @@ -1096,6 +1877,12 @@ String initializeUserAgent(final AbfsConfiguration abfsConfiguration, appendIfNotEmpty(sb, ExtensionHelper.getUserAgentSuffix(tokenProvider, EMPTY_STRING), true); + if (abfsConfiguration.isExpectHeaderEnabled()) { + sb.append(SINGLE_WHITE_SPACE); + sb.append(HUNDRED_CONTINUE); + sb.append(SEMICOLON); + } + sb.append(SINGLE_WHITE_SPACE); sb.append(abfsConfiguration.getClusterName()); sb.append(FORWARD_SLASH); @@ -1154,4 +1941,13 @@ public ListenableFuture submit(Runnable runnable) { public void addCallback(ListenableFuture future, FutureCallback callback) { Futures.addCallback(future, callback, executorService); } + + AbfsConfiguration getAbfsConfiguration() { + return abfsConfiguration; + } + + @VisibleForTesting + protected AccessTokenProvider getTokenProvider() { + return tokenProvider; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingAnalyzer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingAnalyzer.java index a55c924dd81524..2060de6f14a97e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingAnalyzer.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingAnalyzer.java @@ -20,27 +20,30 @@ import java.util.Timer; import java.util.TimerTask; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.apache.hadoop.util.Preconditions; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.util.Time.now; + class AbfsClientThrottlingAnalyzer { private static final Logger LOG = LoggerFactory.getLogger( - AbfsClientThrottlingAnalyzer.class); - private static final int DEFAULT_ANALYSIS_PERIOD_MS = 10 * 1000; + AbfsClientThrottlingAnalyzer.class); private static final int MIN_ANALYSIS_PERIOD_MS = 1000; private static final int MAX_ANALYSIS_PERIOD_MS = 30000; private static final double MIN_ACCEPTABLE_ERROR_PERCENTAGE = .1; private static final double MAX_EQUILIBRIUM_ERROR_PERCENTAGE = 1; private static final double RAPID_SLEEP_DECREASE_FACTOR = .75; private static final double RAPID_SLEEP_DECREASE_TRANSITION_PERIOD_MS = 150 - * 1000; + * 1000; private static final double SLEEP_DECREASE_FACTOR = .975; private static final double SLEEP_INCREASE_FACTOR = 1.05; private int analysisPeriodMs; @@ -50,49 +53,86 @@ class AbfsClientThrottlingAnalyzer { private String name = null; private Timer timer = null; private AtomicReference blobMetrics = null; + private AtomicLong lastExecutionTime = null; + private final AtomicBoolean isOperationOnAccountIdle = new AtomicBoolean(false); + private AbfsConfiguration abfsConfiguration = null; + private boolean accountLevelThrottlingEnabled = true; private AbfsClientThrottlingAnalyzer() { // hide default constructor } - /** - * Creates an instance of the AbfsClientThrottlingAnalyzer class with - * the specified name. - * - * @param name a name used to identify this instance. - * @throws IllegalArgumentException if name is null or empty. - */ - AbfsClientThrottlingAnalyzer(String name) throws IllegalArgumentException { - this(name, DEFAULT_ANALYSIS_PERIOD_MS); - } - /** * Creates an instance of the AbfsClientThrottlingAnalyzer class with * the specified name and period. * * @param name A name used to identify this instance. - * @param period The frequency, in milliseconds, at which metrics are - * analyzed. + * @param abfsConfiguration The configuration set. * @throws IllegalArgumentException If name is null or empty. * If period is less than 1000 or greater than 30000 milliseconds. */ - AbfsClientThrottlingAnalyzer(String name, int period) - throws IllegalArgumentException { + AbfsClientThrottlingAnalyzer(String name, AbfsConfiguration abfsConfiguration) + throws IllegalArgumentException { Preconditions.checkArgument( - StringUtils.isNotEmpty(name), - "The argument 'name' cannot be null or empty."); + StringUtils.isNotEmpty(name), + "The argument 'name' cannot be null or empty."); + int period = abfsConfiguration.getAnalysisPeriod(); Preconditions.checkArgument( - period >= MIN_ANALYSIS_PERIOD_MS && period <= MAX_ANALYSIS_PERIOD_MS, - "The argument 'period' must be between 1000 and 30000."); + period >= MIN_ANALYSIS_PERIOD_MS && period <= MAX_ANALYSIS_PERIOD_MS, + "The argument 'period' must be between 1000 and 30000."); this.name = name; - this.analysisPeriodMs = period; + this.abfsConfiguration = abfsConfiguration; + this.accountLevelThrottlingEnabled = abfsConfiguration.accountThrottlingEnabled(); + this.analysisPeriodMs = abfsConfiguration.getAnalysisPeriod(); + this.lastExecutionTime = new AtomicLong(now()); this.blobMetrics = new AtomicReference( - new AbfsOperationMetrics(System.currentTimeMillis())); + new AbfsOperationMetrics(System.currentTimeMillis())); this.timer = new Timer( - String.format("abfs-timer-client-throttling-analyzer-%s", name), true); + String.format("abfs-timer-client-throttling-analyzer-%s", name), true); this.timer.schedule(new TimerTaskImpl(), - analysisPeriodMs, - analysisPeriodMs); + analysisPeriodMs, + analysisPeriodMs); + } + + /** + * Resumes the timer if it was stopped. + */ + private void resumeTimer() { + blobMetrics = new AtomicReference( + new AbfsOperationMetrics(System.currentTimeMillis())); + timer.schedule(new TimerTaskImpl(), + analysisPeriodMs, + analysisPeriodMs); + isOperationOnAccountIdle.set(false); + } + + /** + * Synchronized method to suspend or resume timer. + * @param timerFunctionality resume or suspend. + * @param timerTask The timertask object. + * @return true or false. + */ + private synchronized boolean timerOrchestrator(TimerFunctionality timerFunctionality, + TimerTask timerTask) { + switch (timerFunctionality) { + case RESUME: + if (isOperationOnAccountIdle.get()) { + resumeTimer(); + } + break; + case SUSPEND: + if (accountLevelThrottlingEnabled && (System.currentTimeMillis() + - lastExecutionTime.get() >= getOperationIdleTimeout())) { + isOperationOnAccountIdle.set(true); + timerTask.cancel(); + timer.purge(); + return true; + } + break; + default: + break; + } + return false; } /** @@ -104,12 +144,13 @@ private AbfsClientThrottlingAnalyzer() { public void addBytesTransferred(long count, boolean isFailedOperation) { AbfsOperationMetrics metrics = blobMetrics.get(); if (isFailedOperation) { - metrics.bytesFailed.addAndGet(count); - metrics.operationsFailed.incrementAndGet(); + metrics.addBytesFailed(count); + metrics.incrementOperationsFailed(); } else { - metrics.bytesSuccessful.addAndGet(count); - metrics.operationsSuccessful.incrementAndGet(); + metrics.addBytesSuccessful(count); + metrics.incrementOperationsSuccessful(); } + blobMetrics.set(metrics); } /** @@ -117,6 +158,8 @@ public void addBytesTransferred(long count, boolean isFailedOperation) { * @return true if Thread sleeps(Throttling occurs) else false. */ public boolean suspendIfNecessary() { + lastExecutionTime.set(now()); + timerOrchestrator(TimerFunctionality.RESUME, null); int duration = sleepDuration; if (duration > 0) { try { @@ -134,19 +177,27 @@ int getSleepDuration() { return sleepDuration; } + int getOperationIdleTimeout() { + return abfsConfiguration.getAccountOperationIdleTimeout(); + } + + AtomicBoolean getIsOperationOnAccountIdle() { + return isOperationOnAccountIdle; + } + private int analyzeMetricsAndUpdateSleepDuration(AbfsOperationMetrics metrics, int sleepDuration) { final double percentageConversionFactor = 100; - double bytesFailed = metrics.bytesFailed.get(); - double bytesSuccessful = metrics.bytesSuccessful.get(); - double operationsFailed = metrics.operationsFailed.get(); - double operationsSuccessful = metrics.operationsSuccessful.get(); + double bytesFailed = metrics.getBytesFailed().get(); + double bytesSuccessful = metrics.getBytesSuccessful().get(); + double operationsFailed = metrics.getOperationsFailed().get(); + double operationsSuccessful = metrics.getOperationsSuccessful().get(); double errorPercentage = (bytesFailed <= 0) - ? 0 - : (percentageConversionFactor - * bytesFailed - / (bytesFailed + bytesSuccessful)); - long periodMs = metrics.endTime - metrics.startTime; + ? 0 + : (percentageConversionFactor + * bytesFailed + / (bytesFailed + bytesSuccessful)); + long periodMs = metrics.getEndTime() - metrics.getStartTime(); double newSleepDuration; @@ -154,10 +205,10 @@ private int analyzeMetricsAndUpdateSleepDuration(AbfsOperationMetrics metrics, ++consecutiveNoErrorCount; // Decrease sleepDuration in order to increase throughput. double reductionFactor = - (consecutiveNoErrorCount * analysisPeriodMs - >= RAPID_SLEEP_DECREASE_TRANSITION_PERIOD_MS) - ? RAPID_SLEEP_DECREASE_FACTOR - : SLEEP_DECREASE_FACTOR; + (consecutiveNoErrorCount * analysisPeriodMs + >= RAPID_SLEEP_DECREASE_TRANSITION_PERIOD_MS) + ? RAPID_SLEEP_DECREASE_FACTOR + : SLEEP_DECREASE_FACTOR; newSleepDuration = sleepDuration * reductionFactor; } else if (errorPercentage < MAX_EQUILIBRIUM_ERROR_PERCENTAGE) { @@ -176,15 +227,15 @@ private int analyzeMetricsAndUpdateSleepDuration(AbfsOperationMetrics metrics, double additionalDelayNeeded = 5 * analysisPeriodMs; if (bytesSuccessful > 0) { additionalDelayNeeded = (bytesSuccessful + bytesFailed) - * periodMs - / bytesSuccessful - - periodMs; + * periodMs + / bytesSuccessful + - periodMs; } // amortize the additional delay needed across the estimated number of // requests during the next period newSleepDuration = additionalDelayNeeded - / (operationsFailed + operationsSuccessful); + / (operationsFailed + operationsSuccessful); final double maxSleepDuration = analysisPeriodMs; final double minSleepDuration = sleepDuration * SLEEP_INCREASE_FACTOR; @@ -201,16 +252,16 @@ private int analyzeMetricsAndUpdateSleepDuration(AbfsOperationMetrics metrics, if (LOG.isDebugEnabled()) { LOG.debug(String.format( - "%5.5s, %10d, %10d, %10d, %10d, %6.2f, %5d, %5d, %5d", - name, - (int) bytesFailed, - (int) bytesSuccessful, - (int) operationsFailed, - (int) operationsSuccessful, - errorPercentage, - periodMs, - (int) sleepDuration, - (int) newSleepDuration)); + "%5.5s, %10d, %10d, %10d, %10d, %6.2f, %5d, %5d, %5d", + name, + (int) bytesFailed, + (int) bytesSuccessful, + (int) operationsFailed, + (int) operationsSuccessful, + errorPercentage, + periodMs, + (int) sleepDuration, + (int) newSleepDuration)); } return (int) newSleepDuration; @@ -238,12 +289,15 @@ public void run() { } long now = System.currentTimeMillis(); - if (now - blobMetrics.get().startTime >= analysisPeriodMs) { + if (timerOrchestrator(TimerFunctionality.SUSPEND, this)) { + return; + } + if (now - blobMetrics.get().getStartTime() >= analysisPeriodMs) { AbfsOperationMetrics oldMetrics = blobMetrics.getAndSet( - new AbfsOperationMetrics(now)); - oldMetrics.endTime = now; + new AbfsOperationMetrics(now)); + oldMetrics.setEndTime(now); sleepDuration = analyzeMetricsAndUpdateSleepDuration(oldMetrics, - sleepDuration); + sleepDuration); } } finally { if (doWork) { @@ -252,24 +306,4 @@ public void run() { } } } - - /** - * Stores Abfs operation metrics during each analysis period. - */ - static class AbfsOperationMetrics { - private AtomicLong bytesFailed; - private AtomicLong bytesSuccessful; - private AtomicLong operationsFailed; - private AtomicLong operationsSuccessful; - private long endTime; - private long startTime; - - AbfsOperationMetrics(long startTime) { - this.startTime = startTime; - this.bytesFailed = new AtomicLong(); - this.bytesSuccessful = new AtomicLong(); - this.operationsFailed = new AtomicLong(); - this.operationsSuccessful = new AtomicLong(); - } - } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java index 7303e833418db7..900c9c39fe6763 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java @@ -19,13 +19,17 @@ package org.apache.hadoop.fs.azurebfs.services; import java.net.HttpURLConnection; +import java.util.concurrent.locks.ReentrantLock; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.AbfsStatistic; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import static java.net.HttpURLConnection.HTTP_UNAVAILABLE; + /** * Throttles Azure Blob File System read and write operations to achieve maximum * throughput by minimizing errors. The errors occur when the account ingress @@ -38,35 +42,101 @@ * and sleeps just enough to minimize errors, allowing optimal ingress and/or * egress throughput. */ -public final class AbfsClientThrottlingIntercept { +public final class AbfsClientThrottlingIntercept implements AbfsThrottlingIntercept { private static final Logger LOG = LoggerFactory.getLogger( AbfsClientThrottlingIntercept.class); private static final String RANGE_PREFIX = "bytes="; - private static AbfsClientThrottlingIntercept singleton = null; - private AbfsClientThrottlingAnalyzer readThrottler = null; - private AbfsClientThrottlingAnalyzer writeThrottler = null; - private static boolean isAutoThrottlingEnabled = false; + private static AbfsClientThrottlingIntercept singleton; // singleton, initialized in static initialization block + private static final ReentrantLock LOCK = new ReentrantLock(); + private final AbfsClientThrottlingAnalyzer readThrottler; + private final AbfsClientThrottlingAnalyzer writeThrottler; + private final String accountName; // Hide default constructor - private AbfsClientThrottlingIntercept() { - readThrottler = new AbfsClientThrottlingAnalyzer("read"); - writeThrottler = new AbfsClientThrottlingAnalyzer("write"); + public AbfsClientThrottlingIntercept(String accountName, AbfsConfiguration abfsConfiguration) { + this.accountName = accountName; + this.readThrottler = setAnalyzer("read " + accountName, abfsConfiguration); + this.writeThrottler = setAnalyzer("write " + accountName, abfsConfiguration); + LOG.debug("Client-side throttling is enabled for the ABFS file system for the account : {}", accountName); } - public static synchronized void initializeSingleton(boolean enableAutoThrottling) { - if (!enableAutoThrottling) { - return; - } + // Hide default constructor + private AbfsClientThrottlingIntercept(AbfsConfiguration abfsConfiguration) { + // Account name is kept as empty as same instance is shared across all accounts. + this.accountName = ""; + this.readThrottler = setAnalyzer("read", abfsConfiguration); + this.writeThrottler = setAnalyzer("write", abfsConfiguration); + LOG.debug("Client-side throttling is enabled for the ABFS file system using singleton intercept"); + } + + /** + * Sets the analyzer for the intercept. + * @param name Name of the analyzer. + * @param abfsConfiguration The configuration. + * @return AbfsClientThrottlingAnalyzer instance. + */ + private AbfsClientThrottlingAnalyzer setAnalyzer(String name, AbfsConfiguration abfsConfiguration) { + return new AbfsClientThrottlingAnalyzer(name, abfsConfiguration); + } + + /** + * Returns the analyzer for read operations. + * @return AbfsClientThrottlingAnalyzer for read. + */ + AbfsClientThrottlingAnalyzer getReadThrottler() { + return readThrottler; + } + + /** + * Returns the analyzer for write operations. + * @return AbfsClientThrottlingAnalyzer for write. + */ + AbfsClientThrottlingAnalyzer getWriteThrottler() { + return writeThrottler; + } + + /** + * Creates a singleton object of the AbfsClientThrottlingIntercept. + * which is shared across all filesystem instances. + * @param abfsConfiguration configuration set. + * @return singleton object of intercept. + */ + static AbfsClientThrottlingIntercept initializeSingleton(AbfsConfiguration abfsConfiguration) { if (singleton == null) { - singleton = new AbfsClientThrottlingIntercept(); - isAutoThrottlingEnabled = true; - LOG.debug("Client-side throttling is enabled for the ABFS file system."); + LOCK.lock(); + try { + if (singleton == null) { + singleton = new AbfsClientThrottlingIntercept(abfsConfiguration); + LOG.debug("Client-side throttling is enabled for the ABFS file system."); + } + } finally { + LOCK.unlock(); + } } + return singleton; + } + + /** + * Updates the metrics for the case when response code signifies throttling + * but there are some expected bytes to be sent. + * @param isThrottledOperation returns true if status code is HTTP_UNAVAILABLE + * @param abfsHttpOperation Used for status code and data transferred. + * @return true if the operation is throttled and has some bytes to transfer. + */ + private boolean updateBytesTransferred(boolean isThrottledOperation, + AbfsHttpOperation abfsHttpOperation) { + return isThrottledOperation && abfsHttpOperation.getExpectedBytesToBeSent() > 0; } - static void updateMetrics(AbfsRestOperationType operationType, - AbfsHttpOperation abfsHttpOperation) { - if (!isAutoThrottlingEnabled || abfsHttpOperation == null) { + /** + * Updates the metrics for successful and failed read and write operations. + * @param operationType Only applicable for read and write operations. + * @param abfsHttpOperation Used for status code and data transferred. + */ + @Override + public void updateMetrics(AbfsRestOperationType operationType, + AbfsHttpOperation abfsHttpOperation) { + if (abfsHttpOperation == null) { return; } @@ -78,19 +148,34 @@ static void updateMetrics(AbfsRestOperationType operationType, boolean isFailedOperation = (status < HttpURLConnection.HTTP_OK || status >= HttpURLConnection.HTTP_INTERNAL_ERROR); + // If status code is 503, it is considered as a throttled operation. + boolean isThrottledOperation = (status == HTTP_UNAVAILABLE); + switch (operationType) { case Append: + case PutBlock: contentLength = abfsHttpOperation.getBytesSent(); + if (contentLength == 0) { + /* + Signifies the case where we could not update the bytesSent due to + throttling but there were some expectedBytesToBeSent. + */ + if (updateBytesTransferred(isThrottledOperation, abfsHttpOperation)) { + LOG.debug("Updating metrics due to throttling for path {}", abfsHttpOperation.getConnUrl().getPath()); + contentLength = abfsHttpOperation.getExpectedBytesToBeSent(); + } + } if (contentLength > 0) { - singleton.writeThrottler.addBytesTransferred(contentLength, + writeThrottler.addBytesTransferred(contentLength, isFailedOperation); } break; case ReadFile: + case GetBlob: String range = abfsHttpOperation.getConnection().getRequestProperty(HttpHeaderConfigurations.RANGE); contentLength = getContentLengthIfKnown(range); if (contentLength > 0) { - singleton.readThrottler.addBytesTransferred(contentLength, + readThrottler.addBytesTransferred(contentLength, isFailedOperation); } break; @@ -104,21 +189,20 @@ static void updateMetrics(AbfsRestOperationType operationType, * uses this to suspend the request, if necessary, to minimize errors and * maximize throughput. */ - static void sendingRequest(AbfsRestOperationType operationType, + @Override + public void sendingRequest(AbfsRestOperationType operationType, AbfsCounters abfsCounters) { - if (!isAutoThrottlingEnabled) { - return; - } - switch (operationType) { case ReadFile: - if (singleton.readThrottler.suspendIfNecessary() + case GetBlob: + if (readThrottler.suspendIfNecessary() && abfsCounters != null) { abfsCounters.incrementCounter(AbfsStatistic.READ_THROTTLES, 1); } break; case Append: - if (singleton.writeThrottler.suspendIfNecessary() + case PutBlock: + if (writeThrottler.suspendIfNecessary() && abfsCounters != null) { abfsCounters.incrementCounter(AbfsStatistic.WRITE_THROTTLES, 1); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsLease.java new file mode 100644 index 00000000000000..f72658fb789aa5 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsLease.java @@ -0,0 +1,66 @@ +/** + * 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.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +public class AbfsDfsLease extends AbfsLease { + + public AbfsDfsLease(final AbfsClient client, + final String path, + final int acquireMaxRetries, + final int acquireRetryInterval, + final Integer leaseDuration, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + super(client, path, acquireMaxRetries, acquireRetryInterval, leaseDuration, + tracingContext); + } + + public AbfsDfsLease(final AbfsClient client, + final String path, + final Integer leaseDuration, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + super(client, path, leaseDuration, tracingContext); + } + + @Override + String callRenewLeaseAPI(final String path, + final String leaseId, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + AbfsRestOperation op = client.renewLease(path, leaseId, tracingContext); + return op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_LEASE_ID); + } + + @Override + AbfsRestOperation callAcquireLeaseAPI(final String path, final Integer leaseDuration, + final TracingContext tracingContext) + throws AzureBlobFileSystemException { + return client.acquireLease(path, + leaseDuration, tracingContext); + } + + @Override + void callReleaseLeaseAPI(final String path, final String leaseID, final TracingContext tracingContext) + throws AzureBlobFileSystemException { + client.releaseLease(path, leaseID, tracingContext); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java index e15795efee68d0..1799f73f7307b2 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java @@ -29,10 +29,12 @@ @InterfaceStability.Evolving public final class AbfsErrors { public static final String ERR_WRITE_WITHOUT_LEASE = "Attempted to write to file without lease"; - public static final String ERR_LEASE_EXPIRED = "A lease ID was specified, but the lease for the" - + " resource has expired"; + public static final String ERR_LEASE_EXPIRED = "A lease ID was specified, but the lease for the blob has expired."; + public static final String ERR_LEASE_EXPIRED_DFS = "A lease ID was specified, but the lease for the resource has expired."; public static final String ERR_NO_LEASE_ID_SPECIFIED = "There is currently a lease on the " + "resource and no lease ID was specified in the request"; + public static final String ERR_NO_LEASE_ID_SPECIFIED_BLOB = "There is currently a lease on the " + + "blob and no lease ID was specified in the request"; public static final String ERR_PARALLEL_ACCESS_DETECTED = "Parallel access to the create path " + "detected. Failing request to honor single writer semantics"; public static final String ERR_ACQUIRING_LEASE = "Unable to acquire lease"; @@ -48,6 +50,11 @@ public final class AbfsErrors { + "operation"; public static final String ERR_NO_LEASE_THREADS = "Lease desired but no lease threads " + "configured, set " + FS_AZURE_LEASE_THREADS; + public static final String PATH_EXISTS = "The specified path, or an element of the path, " + + "exists and its resource type is invalid for this operation."; + public static final String BLOB_OPERATION_NOT_SUPPORTED = "Blob operation is not supported."; + public static final String CONDITION_NOT_MET = "The condition specified using " + + "HTTP conditional header(s) is not met."; private AbfsErrors() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index 0a3d773ece5418..2d38fa29845ccb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -23,13 +23,24 @@ import java.io.OutputStream; import java.net.HttpURLConnection; import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.List; +import java.util.Map; import javax.net.ssl.HttpsURLConnection; import javax.net.ssl.SSLSocketFactory; +import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.parsers.ParserConfigurationException; +import javax.xml.parsers.SAXParser; +import javax.xml.parsers.SAXParserFactory; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.fs.azurebfs.utils.UriUtils; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; + import org.codehaus.jackson.JsonFactory; import org.codehaus.jackson.JsonParser; import org.codehaus.jackson.JsonToken; @@ -37,11 +48,31 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.xml.sax.SAXException; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsPerfLoggable; import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; +import org.w3c.dom.Document; +import org.w3c.dom.Node; +import org.w3c.dom.NodeList; + +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCKLIST; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOB_ERROR_CODE_END_XML; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOB_ERROR_CODE_START_XML; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COMMITTED_BLOCKS; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EQUAL; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOB_ERROR_MESSAGE_END_XML; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOB_ERROR_MESSAGE_START_XML; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.NAME; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_COMP; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COMP_LIST; + +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT; /** * Represents an HTTP operation. @@ -56,7 +87,6 @@ public class AbfsHttpOperation implements AbfsPerfLoggable { private static final int ONE_THOUSAND = 1000; private static final int ONE_MILLION = ONE_THOUSAND * ONE_THOUSAND; - private final String method; private final URL url; private String maskedUrl; @@ -73,6 +103,7 @@ public class AbfsHttpOperation implements AbfsPerfLoggable { // metrics private int bytesSent; + private int expectedBytesToBeSent; private long bytesReceived; // optional trace enabled metrics @@ -81,6 +112,24 @@ public class AbfsHttpOperation implements AbfsPerfLoggable { private long sendRequestTimeMs; private long recvResponseTimeMs; private boolean shouldMask = false; + private List blockIdList = new ArrayList<>(); + private BlobList blobList; + + private static final ThreadLocal saxParserThreadLocal + = new ThreadLocal() { + @Override + public SAXParser initialValue() { + SAXParserFactory factory = SAXParserFactory.newInstance(); + factory.setNamespaceAware(true); + try { + return factory.newSAXParser(); + } catch (SAXException e) { + throw new RuntimeException("Unable to create SAXParser", e); + } catch (ParserConfigurationException e) { + throw new RuntimeException("Check parser configuration", e); + } + } + }; public static AbfsHttpOperation getAbfsHttpOperationWithFixedResult( final URL url, @@ -100,7 +149,7 @@ public static AbfsHttpOperation getAbfsHttpOperationWithFixedResult( protected AbfsHttpOperation(final URL url, final String method, final int httpStatus) { - this.isTraceEnabled = LOG.isTraceEnabled(); + this.isTraceEnabled = true; this.url = url; this.method = method; this.statusCode = httpStatus; @@ -139,6 +188,10 @@ public String getClientRequestId() { .getRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); } + public String getRequestHeaderValue(String requestHeader) { + return connection.getRequestProperty(requestHeader); + } + public String getExpectedAppendPos() { return expectedAppendPos; } @@ -155,6 +208,10 @@ public int getBytesSent() { return bytesSent; } + public int getExpectedBytesToBeSent() { + return expectedBytesToBeSent; + } + public long getBytesReceived() { return bytesReceived; } @@ -167,6 +224,18 @@ public String getResponseHeader(String httpHeader) { return connection.getHeaderField(httpHeader); } + public Map> getResponseHeaders() { + return connection.getHeaderFields(); + } + + public List getBlockIdList() { + return blockIdList; + } + + public BlobList getBlobList() { + return blobList; + } + // Returns a trace message for the request @Override public String toString() { @@ -263,7 +332,7 @@ public String getMaskedEncodedUrl() { */ public AbfsHttpOperation(final URL url, final String method, final List requestHeaders) throws IOException { - this.isTraceEnabled = LOG.isTraceEnabled(); + this.isTraceEnabled = true; this.url = url; this.method = method; @@ -282,7 +351,7 @@ public AbfsHttpOperation(final URL url, final String method, final List= HttpURLConnection.HTTP_BAD_REQUEST) { - processStorageErrorResponse(); + processServerErrorResponse(); if (this.isTraceEnabled) { this.recvResponseTimeMs += elapsedTimeMs(startTime); } @@ -388,7 +488,13 @@ public void processResponse(final byte[] buffer, final int offset, final int len // this is a list operation and need to retrieve the data // need a better solution if (AbfsHttpConstants.HTTP_METHOD_GET.equals(this.method) && buffer == null) { - parseListFilesResponse(stream); + if (url.toString().contains(QUERY_PARAM_COMP + EQUAL + BLOCKLIST)) { + parseBlockListResponse(stream); + } else if (url.toString().contains(COMP_LIST)) { + parseListBlobResponse(stream); + } else { + parseListFilesResponse(stream); + } } else { if (buffer != null) { while (totalBytesRead < length) { @@ -412,6 +518,10 @@ public void processResponse(final byte[] buffer, final int offset, final int len } catch (IOException ex) { LOG.error("UnexpectedError: ", ex); throw ex; + } catch (ParserConfigurationException e) { + throw new RuntimeException("Check parser configuration", e); + } catch (SAXException e) { + throw new RuntimeException("SAX parser exception", e); } finally { if (this.isTraceEnabled) { this.recvResponseTimeMs += elapsedTimeMs(startTime); @@ -421,10 +531,92 @@ public void processResponse(final byte[] buffer, final int offset, final int len } } + @VisibleForTesting + void processServerErrorResponse() throws IOException { + if (getBaseUrl().contains(WASB_DNS_PREFIX)) { + processBlobStorageErrorResponse(); + } else { + processDfsStorageErrorResponse(); + } + } + + /** + * Parse the stream from the response and set {@link #blobList} field of this + * class. + * + * @param stream inputStream from the server-response. + */ + private void parseListBlobResponse(final InputStream stream) { + try { + final SAXParser saxParser = saxParserThreadLocal.get(); + saxParser.reset(); + BlobList blobList = new BlobList(); + saxParser.parse(stream, new BlobListXmlParser(blobList, getBaseUrl())); + this.blobList = blobList; + } catch (SAXException | IOException e) { + throw new RuntimeException(e); + } + } + + private String getBaseUrl() { + String urlStr = url.toString(); + int queryParamStart = urlStr.indexOf("?"); + if (queryParamStart == -1) { + return urlStr; + } + return urlStr.substring(0, queryParamStart); + } + public void setRequestProperty(String key, String value) { this.connection.setRequestProperty(key, value); } + @VisibleForTesting + void setConnection(HttpURLConnection connection) { + this.connection = connection; + } + + /** + * Parses the get block list response and returns list of committed blocks. + * + * @param stream InputStream contains the list results. + * @throws IOException, ParserConfigurationException, SAXException + */ + private void parseBlockListResponse(final InputStream stream) throws IOException, ParserConfigurationException, SAXException { + if (stream == null) { + return; + } + + if (blockIdList.size() != 0) { + // already parsed the response + return; + } + + // Convert the input stream to a Document object + DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance(); + Document doc = factory.newDocumentBuilder().parse(stream); + +// Find the CommittedBlocks element and extract the list of block IDs + NodeList committedBlocksList = doc.getElementsByTagName(COMMITTED_BLOCKS); + if (committedBlocksList.getLength() > 0) { + Node committedBlocks = committedBlocksList.item(0); + NodeList blockList = committedBlocks.getChildNodes(); + for (int i = 0; i < blockList.getLength(); i++) { + Node block = blockList.item(i); + if (block.getNodeName().equals(BLOCK_NAME)) { + NodeList nameList = block.getChildNodes(); + for (int j = 0; j < nameList.getLength(); j++) { + Node name = nameList.item(j); + if (name.getNodeName().equals(NAME)) { + String blockId = name.getTextContent(); + blockIdList.add(blockId); + } + } + } + } + } + } + /** * Open the HTTP connection. * @@ -459,8 +651,8 @@ private HttpURLConnection openConnection() throws IOException { * } * */ - private void processStorageErrorResponse() { - try (InputStream stream = connection.getErrorStream()) { + private void processDfsStorageErrorResponse() { + try (InputStream stream = getConnectionErrorStream()) { if (stream == null) { return; } @@ -501,6 +693,48 @@ private void processStorageErrorResponse() { } } + /** + * Extract errorCode and errorMessage from errorStream populated by server. + * Error-message in the form of: + *
+   *   {@code
+   *   
+   *   
+   *      string-value
+   *      string-value
+   *   
+   * }
+   * 
+ * + * Reference + */ + private void processBlobStorageErrorResponse() throws IOException { + InputStream errorStream = getConnectionErrorStream(); + if (errorStream == null) { + return; + } + final String data = IOUtils.toString(errorStream, StandardCharsets.UTF_8); + + int codeStartFirstInstance = data.indexOf(BLOB_ERROR_CODE_START_XML); + int codeEndFirstInstance = data.indexOf(BLOB_ERROR_CODE_END_XML); + if (codeEndFirstInstance != -1 && codeStartFirstInstance != -1) { + storageErrorCode = data.substring(codeStartFirstInstance, + codeEndFirstInstance).replace(BLOB_ERROR_CODE_START_XML, ""); + } + + int msgStartFirstInstance = data.indexOf(BLOB_ERROR_MESSAGE_START_XML); + int msgEndFirstInstance = data.indexOf(BLOB_ERROR_MESSAGE_END_XML); + if (msgEndFirstInstance != -1 && msgStartFirstInstance != -1) { + storageErrorMessage = data.substring(msgStartFirstInstance, + msgEndFirstInstance).replace(BLOB_ERROR_MESSAGE_START_XML, ""); + } + } + + @VisibleForTesting + InputStream getConnectionErrorStream() { + return connection.getErrorStream(); + } + /** * Returns the elapsed time in milliseconds. */ @@ -541,6 +775,58 @@ private boolean isNullInputStream(InputStream stream) { return stream == null ? true : false; } + /** + * Gets the connection request property for a key. + * @param key The request property key. + * @return request peoperty value. + */ + String getConnProperty(String key) { + return connection.getRequestProperty(key); + } + + /** + * Gets the connection url. + * @return url. + */ + URL getConnUrl() { + return connection.getURL(); + } + + /** + * Gets the connection request method. + * @return request method. + */ + String getConnRequestMethod() { + return connection.getRequestMethod(); + } + + /** + * Gets the connection response code. + * @return response code. + * @throws IOException + */ + Integer getConnResponseCode() throws IOException { + return connection.getResponseCode(); + } + + /** + * Gets the connection output stream. + * @return output stream. + * @throws IOException + */ + OutputStream getConnOutputStream() throws IOException { + return connection.getOutputStream(); + } + + /** + * Gets the connection response message. + * @return response message. + * @throws IOException + */ + String getConnResponseMessage() throws IOException { + return connection.getResponseMessage(); + } + public static class AbfsHttpOperationWithFixedResult extends AbfsHttpOperation { /** * Creates an instance to represent fixed results. diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 7033ae9a4a0399..e17db10c91cd1d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -50,6 +50,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_KB; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.STREAM_ID_LEN; +import static org.apache.hadoop.fs.azurebfs.constants.InternalConstants.CAPABILITY_SAFE_READAHEAD; import static org.apache.hadoop.util.StringUtils.toLowerCase; /** @@ -137,7 +138,7 @@ public AbfsInputStream( this.tolerateOobAppends = abfsInputStreamContext.isTolerateOobAppends(); this.eTag = eTag; this.readAheadRange = abfsInputStreamContext.getReadAheadRange(); - this.readAheadEnabled = true; + this.readAheadEnabled = abfsInputStreamContext.isReadAheadEnabled(); this.alwaysReadBufferSize = abfsInputStreamContext.shouldReadBufferSizeAlways(); this.bufferedPreadDisabled = abfsInputStreamContext @@ -147,7 +148,16 @@ public AbfsInputStream( this.streamStatistics = abfsInputStreamContext.getStreamStatistics(); this.inputStreamId = createInputStreamId(); this.tracingContext = new TracingContext(tracingContext); - this.tracingContext.setOperation(FSOperationType.READ); + /* + * If this inputStream is getting opened in listStatus or GetFileStatus, it would + * be in the flow of rename-resume. It required that all operations have the + * same primaryId and opType as that of the listStatus or getFileStatus which + * is invoking the rename-resume. + */ + if (tracingContext.getOpType() != FSOperationType.LISTSTATUS + && this.tracingContext.getOpType() != FSOperationType.GET_FILESTATUS) { + this.tracingContext.setOperation(FSOperationType.READ); + } this.tracingContext.setStreamID(inputStreamId); this.context = abfsInputStreamContext; readAheadBlockSize = abfsInputStreamContext.getReadAheadBlockSize(); @@ -745,6 +755,11 @@ byte[] getBuffer() { return buffer; } + @VisibleForTesting + public boolean isReadAheadEnabled() { + return readAheadEnabled; + } + @VisibleForTesting public int getReadAheadRange() { return readAheadRange; @@ -823,11 +838,12 @@ public IOStatistics getIOStatistics() { @Override public String toString() { final StringBuilder sb = new StringBuilder(super.toString()); + sb.append("AbfsInputStream@(").append(this.hashCode()).append("){"); + sb.append("[" + CAPABILITY_SAFE_READAHEAD + "]"); if (streamStatistics != null) { - sb.append("AbfsInputStream@(").append(this.hashCode()).append("){"); - sb.append(streamStatistics.toString()); - sb.append("}"); + sb.append(", ").append(streamStatistics); } + sb.append("}"); return sb.toString(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java index 55f01bf15bcf71..05afc7b9858da8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java @@ -35,6 +35,8 @@ public class AbfsInputStreamContext extends AbfsStreamContext { private boolean tolerateOobAppends; + private boolean isReadAheadEnabled = true; + private boolean alwaysReadBufferSize; private int readAheadBlockSize; @@ -72,6 +74,12 @@ public AbfsInputStreamContext withTolerateOobAppends( return this; } + public AbfsInputStreamContext isReadAheadEnabled( + final boolean isReadAheadEnabled) { + this.isReadAheadEnabled = isReadAheadEnabled; + return this; + } + public AbfsInputStreamContext withReadAheadRange( final int readAheadRange) { this.readAheadRange = readAheadRange; @@ -141,6 +149,10 @@ public boolean isTolerateOobAppends() { return tolerateOobAppends; } + public boolean isReadAheadEnabled() { + return isReadAheadEnabled; + } + public int getReadAheadRange() { return readAheadRange; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java index 2e97598ef04f35..4c3c3e0d16bf81 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java @@ -19,7 +19,10 @@ package org.apache.hadoop.fs.azurebfs.services; import java.io.IOException; +import java.util.Timer; +import java.util.TimerTask; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback; @@ -52,7 +55,7 @@ * Call free() to release the Lease. If the holder process dies, AzureBlobFileSystem breakLease * will need to be called before another client will be able to write to the file. */ -public final class AbfsLease { +public abstract class AbfsLease { private static final Logger LOG = LoggerFactory.getLogger(AbfsLease.class); // Number of retries for acquiring lease @@ -60,16 +63,19 @@ public final class AbfsLease { // Retry interval for acquiring lease in secs static final int DEFAULT_LEASE_ACQUIRE_RETRY_INTERVAL = 10; - private final AbfsClient client; + protected final AbfsClient client; private final String path; private final TracingContext tracingContext; // Lease status variables private volatile boolean leaseFreed; - private volatile String leaseID = null; + private AtomicReference leaseID = new AtomicReference<>(); private volatile Throwable exception = null; private volatile int acquireRetryCount = 0; private volatile ListenableScheduledFuture future = null; + private final Integer leaseDuration; + + private Timer timer = null; public static class LeaseException extends AzureBlobFileSystemException { public LeaseException(Throwable t) { @@ -81,20 +87,36 @@ public LeaseException(String s) { } } - public AbfsLease(AbfsClient client, String path, TracingContext tracingContext) throws AzureBlobFileSystemException { + /** + * @param client client object for making server calls + * @param path path on which lease has to be acquired, renewed and freed in future + * @param leaseDuration duration for which lease to be taken in seconds. If given + * null, it will be taken as infinte-lease. + * @param tracingContext for tracing server calls + * + * @throws AzureBlobFileSystemException exception while calling acquireLease API + */ + public AbfsLease(AbfsClient client, String path, + final Integer leaseDuration, + TracingContext tracingContext) throws AzureBlobFileSystemException { this(client, path, DEFAULT_LEASE_ACQUIRE_MAX_RETRIES, - DEFAULT_LEASE_ACQUIRE_RETRY_INTERVAL, tracingContext); + DEFAULT_LEASE_ACQUIRE_RETRY_INTERVAL, leaseDuration, tracingContext); } @VisibleForTesting public AbfsLease(AbfsClient client, String path, int acquireMaxRetries, - int acquireRetryInterval, TracingContext tracingContext) throws AzureBlobFileSystemException { + int acquireRetryInterval, final Integer leaseDuration, TracingContext tracingContext) throws AzureBlobFileSystemException { this.leaseFreed = false; this.client = client; this.path = path; this.tracingContext = tracingContext; + this.leaseDuration = leaseDuration; - if (client.getNumLeaseThreads() < 1) { + /* + * If the number of threads to use for lease operations for infinite lease directories + * and the object is created for infinite-lease (leaseDuration == null). + */ + if (client.getNumLeaseThreads() < 1 && leaseDuration == null) { throw new LeaseException(ERR_NO_LEASE_THREADS); } @@ -104,7 +126,7 @@ public AbfsLease(AbfsClient client, String path, int acquireMaxRetries, acquireLease(retryPolicy, 0, acquireRetryInterval, 0, new TracingContext(tracingContext)); - while (leaseID == null && exception == null) { + while (leaseID.get() == null && exception == null) { try { future.get(); } catch (Exception e) { @@ -122,18 +144,25 @@ public AbfsLease(AbfsClient client, String path, int acquireMaxRetries, private void acquireLease(RetryPolicy retryPolicy, int numRetries, int retryInterval, long delay, TracingContext tracingContext) - throws LeaseException { + throws AzureBlobFileSystemException { LOG.debug("Attempting to acquire lease on {}, retry {}", path, numRetries); if (future != null && !future.isDone()) { throw new LeaseException(ERR_LEASE_FUTURE_EXISTS); } - future = client.schedule(() -> client.acquireLease(path, - INFINITE_LEASE_DURATION, tracingContext), + if (leaseDuration != null) { + leaseID.set( + callAcquireLeaseAPI(path, leaseDuration, tracingContext).getResult() + .getResponseHeader(HttpHeaderConfigurations.X_MS_LEASE_ID)); + spawnLeaseRenewTimer(path, leaseDuration * 1000); + return; + } + future = client.schedule(() -> callAcquireLeaseAPI(path, + INFINITE_LEASE_DURATION, tracingContext), delay, TimeUnit.SECONDS); client.addCallback(future, new FutureCallback() { @Override public void onSuccess(@Nullable AbfsRestOperation op) { - leaseID = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_LEASE_ID); + leaseID.set(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_LEASE_ID)); LOG.debug("Acquired lease {} on {}", leaseID, path); } @@ -156,6 +185,28 @@ public void onFailure(Throwable throwable) { }); } + private void spawnLeaseRenewTimer(String path, Integer leaseDuration) { + timer = new Timer(); + timer.schedule(new TimerTask() { + @Override + public void run() { + try { + leaseID.set(callRenewLeaseAPI(path, leaseID.get(), tracingContext)); + } catch (AzureBlobFileSystemException ignored) { + } + } + }, leaseDuration / 2, leaseDuration / 2); + } + + abstract String callRenewLeaseAPI(final String path, + final String s, + final TracingContext tracingContext) throws AzureBlobFileSystemException; + + abstract AbfsRestOperation callAcquireLeaseAPI(final String path, + final Integer leaseDuration, + final TracingContext tracingContext) + throws AzureBlobFileSystemException; + /** * Cancel future and free the lease. If an exception occurs while releasing the lease, the error * will be logged. If the lease cannot be released, AzureBlobFileSystem breakLease will need to @@ -170,9 +221,10 @@ public void free() { if (future != null && !future.isDone()) { future.cancel(true); } + cancelTimer(); TracingContext tracingContext = new TracingContext(this.tracingContext); tracingContext.setOperation(FSOperationType.RELEASE_LEASE); - client.releaseLease(path, leaseID, tracingContext); + callReleaseLeaseAPI(path, leaseID.get(), tracingContext); } catch (IOException e) { LOG.warn("Exception when trying to release lease {} on {}. Lease will need to be broken: {}", leaseID, path, e.getMessage()); @@ -184,12 +236,21 @@ public void free() { } } + public void cancelTimer() { + if (timer != null) { + timer.cancel(); + } + } + + abstract void callReleaseLeaseAPI(final String path, final String leaseID, final TracingContext tracingContext) + throws AzureBlobFileSystemException; + public boolean isFreed() { return leaseFreed; } public String getLeaseID() { - return leaseID; + return leaseID.get(); } @VisibleForTesting diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsNoOpThrottlingIntercept.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsNoOpThrottlingIntercept.java new file mode 100644 index 00000000000000..b88f4a05d337b4 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsNoOpThrottlingIntercept.java @@ -0,0 +1,37 @@ +/** + * 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; + +final class AbfsNoOpThrottlingIntercept implements AbfsThrottlingIntercept { + + public static final AbfsNoOpThrottlingIntercept INSTANCE = new AbfsNoOpThrottlingIntercept(); + + private AbfsNoOpThrottlingIntercept() { + } + + @Override + public void updateMetrics(final AbfsRestOperationType operationType, + final AbfsHttpOperation abfsHttpOperation) { + } + + @Override + public void sendingRequest(final AbfsRestOperationType operationType, + final AbfsCounters abfsCounters) { + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOperationMetrics.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOperationMetrics.java new file mode 100644 index 00000000000000..d19c69354a2d33 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOperationMetrics.java @@ -0,0 +1,139 @@ +/** + * 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.util.concurrent.atomic.AtomicLong; + +/** + * Stores Abfs operation metrics during each analysis period. + */ +class AbfsOperationMetrics { + + /** + * No of bytes which could not be transferred due to a failed operation. + */ + private final AtomicLong bytesFailed; + + /** + * No of bytes successfully transferred during a successful operation. + */ + private final AtomicLong bytesSuccessful; + + /** + * Total no of failed operations. + */ + private final AtomicLong operationsFailed; + + /** + * Total no of successful operations. + */ + private final AtomicLong operationsSuccessful; + + /** + * Time when collection of metrics ended. + */ + private long endTime; + + /** + * Time when the collection of metrics started. + */ + private final long startTime; + + AbfsOperationMetrics(long startTime) { + this.startTime = startTime; + this.bytesFailed = new AtomicLong(); + this.bytesSuccessful = new AtomicLong(); + this.operationsFailed = new AtomicLong(); + this.operationsSuccessful = new AtomicLong(); + } + + /** + * + * @return bytes failed to transfer. + */ + AtomicLong getBytesFailed() { + return bytesFailed; + } + + /** + * + * @return bytes successfully transferred. + */ + AtomicLong getBytesSuccessful() { + return bytesSuccessful; + } + + /** + * + * @return no of operations failed. + */ + AtomicLong getOperationsFailed() { + return operationsFailed; + } + + /** + * + * @return no of successful operations. + */ + AtomicLong getOperationsSuccessful() { + return operationsSuccessful; + } + + /** + * + * @return end time of metric collection. + */ + long getEndTime() { + return endTime; + } + + /** + * + * @param endTime sets the end time. + */ + void setEndTime(final long endTime) { + this.endTime = endTime; + } + + /** + * + * @return start time of metric collection. + */ + long getStartTime() { + return startTime; + } + + void addBytesFailed(long bytes) { + this.getBytesFailed().addAndGet(bytes); + } + + void addBytesSuccessful(long bytes) { + this.getBytesSuccessful().addAndGet(bytes); + } + + void incrementOperationsFailed() { + this.getOperationsFailed().incrementAndGet(); + } + + void incrementOperationsSuccessful() { + this.getOperationsSuccessful().incrementAndGet(); + } + +} + diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 61160f92bdf391..24f09c58a449df 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -22,10 +22,19 @@ import java.io.IOException; import java.io.OutputStream; import java.net.HttpURLConnection; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.Future; import java.util.UUID; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService; @@ -51,7 +60,13 @@ import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.Syncable; +import static java.net.HttpURLConnection.HTTP_CONFLICT; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK_LIST_END_TAG; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK_LIST_START_TAG; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.LATEST_BLOCK_FORMAT; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_VERSION; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.STREAM_ID_LEN; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.BLOB_OPERATION_NOT_SUPPORTED; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_WRITE_WITHOUT_LEASE; import static org.apache.hadoop.fs.impl.StoreImplementationUtils.isProbeForSyncable; import static org.apache.hadoop.io.IOUtils.wrapException; @@ -80,6 +95,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable, private boolean disableOutputStreamFlush; private boolean enableSmallWriteOptimization; private boolean isAppendBlob; + private boolean isExpectHeaderEnabled; private volatile IOException lastError; private long lastFlushOffset; @@ -114,7 +130,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable, private final DataBlocks.BlockFactory blockFactory; /** Current data block. Null means none currently active. */ - private DataBlocks.DataBlock activeBlock; + private AbfsBlock activeBlock; /** Count of blocks uploaded. */ private long blockCount = 0; @@ -122,9 +138,30 @@ public class AbfsOutputStream extends OutputStream implements Syncable, /** The size of a single block. */ private final int blockSize; + /** The map to store blockId and Status **/ + private final LinkedHashMap map = new LinkedHashMap<>(); + + /** The list of already committed blocks is stored in this list. */ + private List committedBlockEntries = new ArrayList<>(); + + /** The list of all blockId's for putBlockList. */ + private final Set blockIdList = new LinkedHashSet<>(); + + /** The prefix mode for decision on BLOB or DFS endpoint. */ + private PrefixMode prefixMode; + + /** The etag of the blob. */ + private String eTag; + /** Executor service to carry out the parallel upload requests. */ private final ListeningExecutorService executorService; + /** List to validate order. */ + private final UniqueArrayList orderedBlockList = new UniqueArrayList<>(); + + /** Retry fallback for append on DFS */ + private static boolean fallbackDFSAppend = false; + public AbfsOutputStream(AbfsOutputStreamContext abfsOutputStreamContext) throws IOException { this.client = abfsOutputStreamContext.getClient(); @@ -133,6 +170,7 @@ public AbfsOutputStream(AbfsOutputStreamContext abfsOutputStreamContext) this.position = abfsOutputStreamContext.getPosition(); this.closed = false; this.supportFlush = abfsOutputStreamContext.isEnableFlush(); + this.isExpectHeaderEnabled = abfsOutputStreamContext.isExpectHeaderEnabled(); this.disableOutputStreamFlush = abfsOutputStreamContext .isDisableOutputStreamFlush(); this.enableSmallWriteOptimization @@ -145,6 +183,7 @@ public AbfsOutputStream(AbfsOutputStreamContext abfsOutputStreamContext) this.numOfAppendsToServerSinceLastFlush = 0; this.writeOperations = new ConcurrentLinkedDeque<>(); this.outputStreamStatistics = abfsOutputStreamContext.getStreamStatistics(); + this.eTag = abfsOutputStreamContext.getETag(); if (this.isAppendBlob) { this.maxConcurrentRequestCount = 1; @@ -163,14 +202,93 @@ public AbfsOutputStream(AbfsOutputStreamContext abfsOutputStreamContext) abfsOutputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds()); this.outputStreamId = createOutputStreamId(); this.tracingContext = new TracingContext(abfsOutputStreamContext.getTracingContext()); + this.prefixMode = client.getAbfsConfiguration().getPrefixMode(); + this.blockFactory = abfsOutputStreamContext.getBlockFactory(); + this.blockSize = bufferSize; + if (prefixMode == PrefixMode.BLOB && abfsOutputStreamContext.getPosition() > 0) { + // Get the list of all the committed blocks for the given path. + this.committedBlockEntries = getBlockList(path, tracingContext); + } else { + // create that first block. This guarantees that an open + close sequence + // writes a 0-byte entry. + createBlockIfNeeded(position); + } this.tracingContext.setStreamID(outputStreamId); this.tracingContext.setOperation(FSOperationType.WRITE); this.ioStatistics = outputStreamStatistics.getIOStatistics(); - this.blockFactory = abfsOutputStreamContext.getBlockFactory(); - this.blockSize = bufferSize; - // create that first block. This guarantees that an open + close sequence - // writes a 0-byte entry. - createBlockIfNeeded(); + } + + private final Lock lock = new ReentrantLock(); + + private final ReentrantLock mapLock = new ReentrantLock(); + + public LinkedHashMap getMap() { + return map; + } + + /** + * Set the eTag of the blob. + * + * @param eTag eTag. + */ + void setETag(String eTag) { + lock.lock(); + try { + this.eTag = eTag; + } finally { + lock.unlock(); + } + } + + /** + * Get eTag value of blob. + * + * @return eTag. + */ + @VisibleForTesting + public String getETag() { + lock.lock(); + try { + return eTag; + } finally { + lock.unlock(); + } + } + + public DataBlocks.BlockFactory getBlockFactory() { + return blockFactory; + } + + public long getBlockCount() { + return blockCount; + } + + public int getBlockSize() { + return blockSize; + } + + public static class UniqueArrayList extends ArrayList { + @Override + public boolean add(T element) { + if (!super.contains(element)) { + return super.add(element); + } + return false; + } + } + + /** + * Returns block id's which are committed for the blob. + * @param path The blob path. + * @param tracingContext Tracing context object. + * @return list of committed block id's. + * @throws AzureBlobFileSystemException + */ + private List getBlockList(final String path, TracingContext tracingContext) throws AzureBlobFileSystemException { + List committedBlockIdList; + final AbfsRestOperation op = client.getBlockList(path, tracingContext); + committedBlockIdList = op.getResult().getBlockIdList(); + return committedBlockIdList; } private String createOutputStreamId() { @@ -207,6 +325,10 @@ public void write(final int byteVal) throws IOException { * Writes length bytes from the specified byte array starting at off to * this output stream. * + * Before writing, this method checks for any previous write failures by + * calling {@link #maybeThrowLastError()}. If a failure occurred, + * it throws the exception that caused it. + * * @param data the byte array to write. * @param off the start off in the data. * @param length the number of bytes to write. @@ -216,6 +338,9 @@ public void write(final int byteVal) throws IOException { @Override public synchronized void write(final byte[] data, final int off, final int length) throws IOException { + if (closed) { + throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + } // validate if data is not null and index out of bounds. DataBlocks.validateWriteArgs(data, off, length); maybeThrowLastError(); @@ -227,7 +352,20 @@ public synchronized void write(final byte[] data, final int off, final int lengt if (hasLease() && isLeaseFreed()) { throw new PathIOException(path, ERR_WRITE_WITHOUT_LEASE); } - DataBlocks.DataBlock block = createBlockIfNeeded(); + + if (length == 0) { + return; + } + + AbfsBlock block = createBlockIfNeeded(position); + // Put entry in map with status as NEW which is changed to SUCCESS when successfully appended. + try { + mapLock.lock(); + map.put(block.getBlockId(), BlockStatus.NEW); + orderedBlockList.add(block.getBlockId()); + } finally { + mapLock.unlock(); + } int written = block.write(data, off, length); int remainingCapacity = block.remainingCapacity(); @@ -254,12 +392,11 @@ public synchronized void write(final byte[] data, final int off, final int lengt * @return the active block; null if there isn't one. * @throws IOException on any failure to create */ - private synchronized DataBlocks.DataBlock createBlockIfNeeded() + private synchronized AbfsBlock createBlockIfNeeded(long offset) throws IOException { if (activeBlock == null) { blockCount++; - activeBlock = blockFactory - .create(blockCount, this.blockSize, outputStreamStatistics); + activeBlock = new AbfsBlock(this, offset); } return activeBlock; } @@ -288,7 +425,7 @@ private synchronized void uploadCurrentBlock() throws IOException { * @param blockToUpload block to upload. * @throws IOException upload failure */ - private void uploadBlockAsync(DataBlocks.DataBlock blockToUpload, + private void uploadBlockAsync(AbfsBlock blockToUpload, boolean isFlush, boolean isClose) throws IOException { if (this.isAppendBlob) { @@ -327,10 +464,52 @@ private void uploadBlockAsync(DataBlocks.DataBlock blockToUpload, * leaseId - The AbfsLeaseId for this request. */ AppendRequestParameters reqParams = new AppendRequestParameters( - offset, 0, bytesLength, mode, false, leaseId); - AbfsRestOperation op = - client.append(path, blockUploadData.toByteArray(), reqParams, - cachedSasToken.get(), new TracingContext(tracingContext)); + offset, 0, bytesLength, mode, false, leaseId, isExpectHeaderEnabled); + AbfsRestOperation op; + if (!OperativeEndpoint.isIngressEnabledOnDFS(prefixMode, client.getAbfsConfiguration())) { + try { + LOG.debug("Append over Blob for ingress config value {} for path {} ", + client.getAbfsConfiguration().shouldIngressFallbackToDfs(), path); + TracingContext tracingContextBlobAppend = new TracingContext(tracingContext); + tracingContextBlobAppend.setFallbackDFSAppend("B " + offset); + op = client.append(blockToUpload.getBlockId(), path, blockUploadData.toByteArray(), reqParams, + cachedSasToken.get(), tracingContextBlobAppend, getETag()); + String key = blockToUpload.getBlockId(); + try { + mapLock.lock(); + if (!getMap().containsKey(key)) { + throw new Exception("Block is missing with blockId " + blockToUpload.getBlockId() + + " for offset " + offset + " for path" + path + " with streamId " + outputStreamId); + } else { + map.put(blockToUpload.getBlockId(), BlockStatus.SUCCESS); + } + } finally { + mapLock.unlock(); + } + } catch (AbfsRestOperationException ex) { + // The mechanism to fall back to DFS endpoint if blob operation is not supported. + if (ex.getStatusCode() == HTTP_CONFLICT && ex.getMessage().contains(BLOB_OPERATION_NOT_SUPPORTED)) { + prefixMode = PrefixMode.DFS; + LOG.debug("Retrying append due to fallback for path {} ", path); + TracingContext tracingContextAppend = new TracingContext(tracingContext); + tracingContextAppend.setFallbackDFSAppend("D"); + fallbackDFSAppend = true; + op = client.append(path, blockUploadData.toByteArray(), reqParams, + cachedSasToken.get(), tracingContextAppend); + } else { + throw ex; + } + } + } else { + LOG.debug("Append over DFS for ingress config value {} for path {} ", + client.getAbfsConfiguration().shouldIngressFallbackToDfs(), path); + TracingContext tracingContextAppend = new TracingContext(tracingContext); + if (fallbackDFSAppend) { + tracingContextAppend.setFallbackDFSAppend("D"); + } + op = client.append(path, blockUploadData.toByteArray(), reqParams, + cachedSasToken.get(), tracingContextAppend); + } cachedSasToken.update(op.getSasToken()); perfInfo.registerResult(op.getResult()); perfInfo.registerSuccess(true); @@ -371,7 +550,7 @@ private void failureWhileSubmit(Exception ex) throws IOException { * * @return the active block; null if there isn't one. */ - private synchronized DataBlocks.DataBlock getActiveBlock() { + private synchronized AbfsBlock getActiveBlock() { return activeBlock; } @@ -432,6 +611,8 @@ private void maybeThrowLastError() throws IOException { * written out. If any data remains in the payload it is committed to the * service. Data is queued for writing and forced out to the service * before the call returns. + * + * @throws IOException if error occurs in {@link #flushInternal(boolean)} call */ @Override public void flush() throws IOException { @@ -442,7 +623,7 @@ public void flush() throws IOException { /** Similar to posix fsync, flush out the data in client's user buffer * all the way to the disk device (but the disk may have it in its cache). - * @throws IOException if error occurs + * @throws IOException if error occurs in {@link #flushInternal(boolean)} call */ @Override public void hsync() throws IOException { @@ -453,7 +634,8 @@ public void hsync() throws IOException { /** Flush out the data in client's user buffer. After the return of * this call, new readers will see the data. - * @throws IOException if any error occurs + * @throws IOException if any error occurs in {@link #flushInternal(boolean)} + * call */ @Override public void hflush() throws IOException { @@ -510,6 +692,17 @@ public synchronized void close() throws IOException { LOG.debug("Closing AbfsOutputStream : {}", this); } + /** + * Flushes the internal buffer and uploads the current block to the service. + * Before flushing, checks if there has been failure in previous writes. If there + * is some data which has not been written to server, that data is written by + * {@link #uploadCurrentBlock()}. Post async upload has been invoked, + * {@link #flushWrittenBytesToService(boolean)} is called. + * + * @param isClose stream is closed after this flush call + * @throws IOException exception in any of the write operation or server exception + * on flush operation + */ private synchronized void flushInternal(boolean isClose) throws IOException { maybeThrowLastError(); @@ -540,6 +733,13 @@ private synchronized void smallWriteOptimizedflushInternal(boolean isClose) thro numOfAppendsToServerSinceLastFlush = 0; } + /** + * Check if previous write failed. If there is a block left to upload, it will + * be uploaded. Before flushing, it waits for the async writes to get completed. + * + * @throws IOException exception received in previous writes, or in async-writes, + * or exceptions from server on Flush operation. + */ private synchronized void flushInternalAsync() throws IOException { maybeThrowLastError(); if (hasActiveBlockDataToUpload()) { @@ -556,7 +756,7 @@ private synchronized void flushInternalAsync() throws IOException { * the dataBlock or while closing the BlockUploadData. */ private void writeAppendBlobCurrentBufferToService() throws IOException { - DataBlocks.DataBlock activeBlock = getActiveBlock(); + AbfsBlock activeBlock = getActiveBlock(); // No data, return. if (!hasActiveBlockDataToUpload()) { return; @@ -573,7 +773,7 @@ private void writeAppendBlobCurrentBufferToService() throws IOException { try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "writeCurrentBufferToService", "append")) { AppendRequestParameters reqParams = new AppendRequestParameters(offset, 0, - bytesLength, APPEND_MODE, true, leaseId); + bytesLength, APPEND_MODE, true, leaseId, isExpectHeaderEnabled); AbfsRestOperation op = client.append(path, uploadData.toByteArray(), reqParams, cachedSasToken.get(), new TracingContext(tracingContext)); cachedSasToken.update(op.getSasToken()); @@ -590,6 +790,11 @@ private void writeAppendBlobCurrentBufferToService() throws IOException { } } + /** + * Waits for all the ongoing async {@link #writeOperations} to complete. + * + * @throws IOException exception from failed async write operation + */ private synchronized void waitForAppendsToComplete() throws IOException { for (WriteOperation writeOperation : writeOperations) { try { @@ -611,6 +816,14 @@ private synchronized void waitForAppendsToComplete() throws IOException { } } + /** + * Wait for the previous writes to complete. Once the previous writes are + * successfully completed, the written data is flushed. + * + * @param isClose stream is closed after this flush + * @throws IOException exception from write failure or server exception on flush + * operation + */ private synchronized void flushWrittenBytesToService(boolean isClose) throws IOException { waitForAppendsToComplete(); flushWrittenBytesToServiceInternal(position, false, isClose); @@ -635,8 +848,69 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset, AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "flushWrittenBytesToServiceInternal", "flush")) { - AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, isClose, - cachedSasToken.get(), leaseId, new TracingContext(tracingContext)); + AbfsRestOperation op; + if (!OperativeEndpoint.isIngressEnabledOnDFS(prefixMode, client.getAbfsConfiguration())) { + LOG.debug("Flush over Blob for ingress config value {} for path {} ", + client.getAbfsConfiguration().shouldIngressFallbackToDfs(), path); + // Adds all the committed blocks if available to the list of blocks to be added in putBlockList. + blockIdList.addAll(committedBlockEntries); + boolean successValue = true; + String failedBlockId = ""; + BlockStatus success = BlockStatus.SUCCESS; + + // No network calls needed for empty map. + if (map.isEmpty()) { + return; + } + + int mapEntry = 0; + // If any of the entry in the map doesn't have the status of SUCCESS, fail the flush. + for (Map.Entry entry : getMap().entrySet()) { + if (!success.equals(entry.getValue())) { + successValue = false; + failedBlockId = entry.getKey(); + break; + } else { + if (!entry.getKey().equals(orderedBlockList.get(mapEntry))) { + LOG.debug("The order for the given offset {} with blockId {} and streamId {} " + + " for the path {} was not successful", offset, entry.getKey(), outputStreamId, path); + throw new IOException("The ordering in map is incorrect for blockId " + + entry.getKey() + " and offset " + offset + " for path" + path + " with streamId " + outputStreamId); + } + blockIdList.add(entry.getKey()); + mapEntry++; + } + } + if (!successValue) { + LOG.debug("A past append for the given offset {} with blockId {} and streamId {}" + + " for the path {} was not successful", offset, failedBlockId, outputStreamId, path); + throw new IOException("A past append was not successful for blockId " + + failedBlockId + " and offset " + offset + " for path" + path + " with streamId " + outputStreamId); + } + // Generate the xml with the list of blockId's to generate putBlockList call. + String blockListXml = generateBlockListXml(blockIdList); + TracingContext tracingContextBlobFlush = new TracingContext(tracingContext); + tracingContextBlobFlush.setFallbackDFSAppend("B " + position); + op = client.flush(blockListXml.getBytes(), path, + isClose, cachedSasToken.get(), leaseId, getETag(), tracingContextBlobFlush); + setETag(op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG)); + try { + mapLock.lock(); + getMap().clear(); + orderedBlockList.clear(); + } finally { + mapLock.unlock(); + } + } else { + LOG.debug("Flush over DFS for ingress config value {} for path {} ", + client.getAbfsConfiguration().shouldIngressFallbackToDfs(), path); + TracingContext tracingContextFlush = new TracingContext(tracingContext); + if (fallbackDFSAppend) { + tracingContextFlush.setFallbackDFSAppend("D"); + } + op = client.flush(path, offset, retainUncommitedData, isClose, + cachedSasToken.get(), leaseId, tracingContextFlush); + } cachedSasToken.update(op.getSasToken()); perfInfo.registerResult(op.getResult()).registerSuccess(true); } catch (AzureBlobFileSystemException ex) { @@ -651,6 +925,22 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset, this.lastFlushOffset = offset; } + /** + * Helper method to generate the xml with list of blockId's. + * @param blockIds The set of blockId's to be pushed to the backend. + * @return xml in string format. + */ + private static String generateBlockListXml(Set blockIds) { + StringBuilder stringBuilder = new StringBuilder(); + stringBuilder.append(XML_VERSION); + stringBuilder.append(BLOCK_LIST_START_TAG); + for (String blockId : blockIds) { + stringBuilder.append(String.format(LATEST_BLOCK_FORMAT, blockId)); + } + stringBuilder.append(BLOCK_LIST_END_TAG); + return stringBuilder.toString(); + } + /** * Try to remove the completed write operations from the beginning of write * operation FIFO queue. @@ -750,6 +1040,16 @@ public boolean hasLease() { return lease != null; } + @VisibleForTesting + public Boolean areWriteOperationsTasksDone() { + for (WriteOperation writeOperation : writeOperations) { + if (!writeOperation.task.isDone()) { + return false; + } + } + return true; + } + /** * Appending AbfsOutputStream statistics to base toString(). * @@ -764,4 +1064,9 @@ public String toString() { sb.append("}"); return sb.toString(); } + + @VisibleForTesting + public String getLeaseId() { + return leaseId; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java index ad303823e0c68b..94a62abbe99812 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java @@ -33,6 +33,8 @@ public class AbfsOutputStreamContext extends AbfsStreamContext { private boolean enableFlush; + private boolean enableExpectHeader; + private boolean enableSmallWriteOptimization; private boolean disableOutputStreamFlush; @@ -63,6 +65,8 @@ public class AbfsOutputStreamContext extends AbfsStreamContext { private TracingContext tracingContext; + private String eTag; + public AbfsOutputStreamContext(final long sasTokenRenewPeriodForStreamsInSeconds) { super(sasTokenRenewPeriodForStreamsInSeconds); } @@ -78,6 +82,11 @@ public AbfsOutputStreamContext enableFlush(final boolean enableFlush) { return this; } + public AbfsOutputStreamContext enableExpectHeader(final boolean enableExpectHeader) { + this.enableExpectHeader = enableExpectHeader; + return this; + } + public AbfsOutputStreamContext enableSmallWriteOptimization(final boolean enableSmallWriteOptimization) { this.enableSmallWriteOptimization = enableSmallWriteOptimization; return this; @@ -150,6 +159,12 @@ public AbfsOutputStreamContext withTracingContext( return this; } + public AbfsOutputStreamContext withETag( + final String eTag) { + this.eTag = eTag; + return this; + } + public AbfsOutputStreamContext build() { // Validation of parameters to be done here. if (streamStatistics == null) { @@ -184,6 +199,10 @@ public boolean isEnableFlush() { return enableFlush; } + public boolean isExpectHeaderEnabled() { + return enableExpectHeader; + } + public boolean isDisableOutputStreamFlush() { return disableOutputStreamFlush; } @@ -250,4 +269,8 @@ public ExecutorService getExecutorService() { public TracingContext getTracingContext() { return tracingContext; } + + public String getETag() { + return eTag; + } } 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 c2a80f8177094b..5ed8d5c5acdaf4 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 @@ -25,6 +25,7 @@ import java.net.UnknownHostException; import java.util.List; +import org.apache.hadoop.classification.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,6 +38,9 @@ import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.PUT_BLOCK_LIST; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_CONTINUE; + /** * The AbfsRestOperation for Rest AbfsClient. */ @@ -45,6 +49,8 @@ public class AbfsRestOperation { private final AbfsRestOperationType operationType; // Blob FS client, which has the credentials, retry policy, and logs. private final AbfsClient client; + // Return intercept instance + private final AbfsThrottlingIntercept intercept; // the HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE) private final String method; // full URL including query parameters @@ -71,6 +77,17 @@ public class AbfsRestOperation { private AbfsHttpOperation result; private AbfsCounters abfsCounters; + /** + * This variable contains the reason of last API call within the same + * AbfsRestOperation object. + */ + private String failureReason; + + /** + * This variable stores the tracing context used for last Rest Operation made + */ + private TracingContext lastTracingContext; + /** * Checks if there is non-null HTTP response. * @return true if there is a non-null HTTP response from the ABFS call. @@ -145,6 +162,7 @@ String getSasToken() { || AbfsHttpConstants.HTTP_METHOD_PATCH.equals(method)); this.sasToken = sasToken; this.abfsCounters = client.getAbfsCounters(); + this.intercept = client.getIntercept(); } /** @@ -185,10 +203,14 @@ String getSasToken() { public void execute(TracingContext tracingContext) throws AzureBlobFileSystemException { + // Since this might be a sub-sequential call triggered by a single + // file system call, a new tracing context should be used. + final TracingContext newTracingContext = createNewTracingContext(tracingContext); + try { IOStatisticsBinding.trackDurationOfInvocation(abfsCounters, AbfsStatistic.getStatNameFromHttpCall(method), - () -> completeExecute(tracingContext)); + () -> completeExecute(newTracingContext)); } catch (AzureBlobFileSystemException aze) { throw aze; } catch (IOException e) { @@ -202,10 +224,10 @@ public void execute(TracingContext tracingContext) * HTTP operations. * @param tracingContext TracingContext instance to track correlation IDs */ - private void completeExecute(TracingContext tracingContext) + void completeExecute(TracingContext tracingContext) throws AzureBlobFileSystemException { // see if we have latency reports from the previous requests - String latencyHeader = this.client.getAbfsPerfTracker().getClientLatency(); + String latencyHeader = getClientLatency(); if (latencyHeader != null && !latencyHeader.isEmpty()) { AbfsHttpHeader httpHeader = new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_ABFS_CLIENT_LATENCY, latencyHeader); @@ -226,14 +248,62 @@ private void completeExecute(TracingContext tracingContext) } } - if (result.getStatusCode() >= HttpURLConnection.HTTP_BAD_REQUEST) { + int status = result.getStatusCode(); + /* + If even after exhausting all retries, the http status code has an + invalid value it qualifies for InvalidAbfsRestOperationException. + All http status code less than 1xx range are considered as invalid + status codes. + */ + if (status < HTTP_CONTINUE) { + throw new InvalidAbfsRestOperationException(null, retryCount); + } + + if (status >= HttpURLConnection.HTTP_BAD_REQUEST) { throw new AbfsRestOperationException(result.getStatusCode(), result.getStorageErrorCode(), result.getStorageErrorMessage(), null, result); } - LOG.trace("{} REST operation complete", operationType); } + @VisibleForTesting + String getClientLatency() { + return client.getAbfsPerfTracker().getClientLatency(); + } + + /** + * Sign an operation. + * @param httpOperation operation to sign + * @param bytesToSign how many bytes to sign for shared key auth. + * @throws IOException failure + */ + @VisibleForTesting + public void signRequest(final AbfsHttpOperation httpOperation, + int bytesToSign) throws IOException { + switch (client.getAuthType()) { + case Custom: + case OAuth: + LOG.debug("Authenticating request with OAuth2 access token"); + httpOperation.getConnection() + .setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION, + client.getAccessToken()); + break; + case SAS: + // do nothing; the SAS token should already be appended to the query string + httpOperation.setMaskForSAS(); //mask sig/oid from url for logs + break; + case SharedKey: + default: + // sign the HTTP request + LOG.debug("Signing request with shared key"); + // sign the HTTP request + client.getSharedKeyCredentials().signRequest( + httpOperation.getConnection(), + bytesToSign); + break; + } + } + /** * Executes a single HTTP operation to complete the REST operation. If it * fails, there may be a retry. The retryCount is incremented with each @@ -241,33 +311,14 @@ private void completeExecute(TracingContext tracingContext) */ private boolean executeHttpOperation(final int retryCount, TracingContext tracingContext) throws AzureBlobFileSystemException { - AbfsHttpOperation httpOperation = null; + AbfsHttpOperation httpOperation; try { // initialize the HTTP request and open the connection - httpOperation = new AbfsHttpOperation(url, method, requestHeaders); + httpOperation = createHttpOperation(); incrementCounter(AbfsStatistic.CONNECTIONS_MADE, 1); - tracingContext.constructHeader(httpOperation); + tracingContext.constructHeader(httpOperation, failureReason); - switch(client.getAuthType()) { - case Custom: - case OAuth: - LOG.debug("Authenticating request with OAuth2 access token"); - httpOperation.getConnection().setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION, - client.getAccessToken()); - break; - case SAS: - // do nothing; the SAS token should already be appended to the query string - httpOperation.setMaskForSAS(); //mask sig/oid from url for logs - break; - case SharedKey: - // sign the HTTP request - LOG.debug("Signing request with shared key"); - // sign the HTTP request - client.getSharedKeyCredentials().signRequest( - httpOperation.getConnection(), - hasRequestBody ? bufferLength : 0); - break; - } + signRequest(httpOperation, hasRequestBody ? bufferLength : 0); } catch (IOException e) { LOG.debug("Auth failure: {}, {}", method, url); throw new AbfsRestOperationException(-1, null, @@ -278,13 +329,14 @@ private boolean executeHttpOperation(final int retryCount, // dump the headers AbfsIoUtils.dumpHeadersToDebugLog("Request Headers", httpOperation.getConnection().getRequestProperties()); - AbfsClientThrottlingIntercept.sendingRequest(operationType, abfsCounters); - + intercept.sendingRequest(operationType, abfsCounters); if (hasRequestBody) { // HttpUrlConnection requires httpOperation.sendRequest(buffer, bufferOffset, bufferLength); incrementCounter(AbfsStatistic.SEND_REQUESTS, 1); - incrementCounter(AbfsStatistic.BYTES_SENT, bufferLength); + if (!(operationType.name().equals(PUT_BLOCK_LIST))) { + incrementCounter(AbfsStatistic.BYTES_SENT, bufferLength); + } } httpOperation.processResponse(buffer, bufferOffset, bufferLength); @@ -300,10 +352,11 @@ private boolean executeHttpOperation(final int retryCount, } catch (UnknownHostException ex) { String hostname = null; hostname = httpOperation.getHost(); - LOG.warn("Unknown host name: %s. Retrying to resolve the host name...", + failureReason = RetryReason.getAbbreviation(ex, null, null); + LOG.warn("Unknown host name: {}. Retrying to resolve the host name...", hostname); if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) { - throw new InvalidAbfsRestOperationException(ex); + throw new InvalidAbfsRestOperationException(ex, retryCount); } return false; } catch (IOException ex) { @@ -311,18 +364,35 @@ private boolean executeHttpOperation(final int retryCount, LOG.debug("HttpRequestFailure: {}, {}", httpOperation.toString(), ex); } + failureReason = RetryReason.getAbbreviation(ex, -1, ""); + if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) { - throw new InvalidAbfsRestOperationException(ex); + throw new InvalidAbfsRestOperationException(ex, retryCount); } return false; } finally { - AbfsClientThrottlingIntercept.updateMetrics(operationType, httpOperation); + int status = httpOperation.getStatusCode(); + /* + A status less than 300 (2xx range) or greater than or equal + to 500 (5xx range) should contribute to throttling metrics being updated. + Less than 200 or greater than or equal to 500 show failed operations. 2xx + range contributes to successful operations. 3xx range is for redirects + and 4xx range is for user errors. These should not be a part of + throttling backoff computation. + */ + boolean updateMetricsResponseCode = (status < HttpURLConnection.HTTP_MULT_CHOICE + || status >= HttpURLConnection.HTTP_INTERNAL_ERROR); + if (updateMetricsResponseCode) { + intercept.updateMetrics(operationType, httpOperation); + } } LOG.debug("HttpRequest: {}: {}", operationType, httpOperation.toString()); if (client.getRetryPolicy().shouldRetry(retryCount, httpOperation.getStatusCode())) { + int status = httpOperation.getStatusCode(); + failureReason = RetryReason.getAbbreviation(null, status, httpOperation.getStorageErrorMessage()); return false; } @@ -331,6 +401,47 @@ private boolean executeHttpOperation(final int retryCount, return true; } + @VisibleForTesting + String getMethod() { + return method; + } + + @VisibleForTesting + void setResult(AbfsHttpOperation result) { + this.result = result; + } + + /** + * Creates new object of {@link AbfsHttpOperation} with the url, method, and + * requestHeaders fields of the AbfsRestOperation object. + */ + @VisibleForTesting + AbfsHttpOperation createHttpOperation() throws IOException { + return new AbfsHttpOperation(url, method, requestHeaders); + } + + /** + * Creates a new Tracing context before entering the retry loop of a rest operation + * This will ensure all rest operations have unique + * tracing context that will be used for all the retries + * @param tracingContext + * @return tracingContext + */ + @VisibleForTesting + final TracingContext createNewTracingContext(final TracingContext tracingContext) { + lastTracingContext = new TracingContext(tracingContext); + return lastTracingContext; + } + + /** + * Returns the tracing contest used for last rest operation made + * @return + */ + @VisibleForTesting + public final TracingContext getLastTracingContext() { + return lastTracingContext; + } + /** * Incrementing Abfs counters with a long value. * 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 830297f381b91d..1bff1e5665081a 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 @@ -36,10 +36,27 @@ public enum AbfsRestOperationType { SetOwner, SetPathProperties, SetPermissions, + SetBlobMetadata, + SetContainerMetadata, Append, Flush, ReadFile, DeletePath, CheckAccess, LeasePath, + LeaseBlob, + PutBlob, + GetBlobProperties, + GetContainerProperties, + GetBlobMetadata, + GetContainerMetadata, + PutBlock, + PutBlockList, + GetBlockList, + DeleteBlob, + CreateContainer, + DeleteContainer, + GetListBlobProperties, + CopyBlob, + GetBlob } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingIntercept.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingIntercept.java new file mode 100644 index 00000000000000..0ceb4335fcef44 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingIntercept.java @@ -0,0 +1,49 @@ +/** + * 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.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * An interface for Abfs Throttling Interface. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public interface AbfsThrottlingIntercept { + + /** + * Updates the metrics for successful and failed read and write operations. + * @param operationType Only applicable for read and write operations. + * @param abfsHttpOperation Used for status code and data transferred. + */ + void updateMetrics(AbfsRestOperationType operationType, + AbfsHttpOperation abfsHttpOperation); + + /** + * Called before the request is sent. Client-side throttling + * uses this to suspend the request, if necessary, to minimize errors and + * maximize throughput. + * @param operationType Only applicable for read and write operations. + * @param abfsCounters Used for counters. + */ + void sendingRequest(AbfsRestOperationType operationType, + AbfsCounters abfsCounters); + +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingInterceptFactory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingInterceptFactory.java new file mode 100644 index 00000000000000..0eabe18872d4f9 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingInterceptFactory.java @@ -0,0 +1,102 @@ +/** + * 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.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; + +import org.apache.hadoop.fs.azurebfs.utils.WeakReferenceMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Class to get an instance of throttling intercept class per account. + */ +final class AbfsThrottlingInterceptFactory { + + private AbfsThrottlingInterceptFactory() { + } + + private static AbfsConfiguration abfsConfig; + + /** + * List of references notified of loss. + */ + private static List lostReferences = new ArrayList<>(); + + private static final Logger LOG = LoggerFactory.getLogger( + AbfsThrottlingInterceptFactory.class); + + /** + * Map which stores instance of ThrottlingIntercept class per account. + */ + private static WeakReferenceMap + interceptMap = new WeakReferenceMap<>( + AbfsThrottlingInterceptFactory::factory, + AbfsThrottlingInterceptFactory::referenceLost); + + /** + * Returns instance of throttling intercept. + * @param accountName Account name. + * @return instance of throttling intercept. + */ + private static AbfsClientThrottlingIntercept factory(final String accountName) { + return new AbfsClientThrottlingIntercept(accountName, abfsConfig); + } + + /** + * Reference lost callback. + * @param accountName key lost. + */ + private static void referenceLost(String accountName) { + lostReferences.add(accountName); + } + + /** + * Returns an instance of AbfsThrottlingIntercept. + * + * @param accountName The account for which we need instance of throttling intercept. + @param abfsConfiguration The object of abfsconfiguration class. + * @return Instance of AbfsThrottlingIntercept. + */ + static synchronized AbfsThrottlingIntercept getInstance(String accountName, + AbfsConfiguration abfsConfiguration) { + abfsConfig = abfsConfiguration; + AbfsThrottlingIntercept intercept; + if (!abfsConfiguration.isAutoThrottlingEnabled()) { + return AbfsNoOpThrottlingIntercept.INSTANCE; + } + // If singleton is enabled use a static instance of the intercept class for all accounts + if (!abfsConfiguration.accountThrottlingEnabled()) { + intercept = AbfsClientThrottlingIntercept.initializeSingleton( + abfsConfiguration); + } else { + // Return the instance from the map + intercept = interceptMap.get(accountName); + if (intercept == null) { + intercept = new AbfsClientThrottlingIntercept(accountName, + abfsConfiguration); + interceptMap.put(accountName, intercept); + } + } + return intercept; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobList.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobList.java new file mode 100644 index 00000000000000..fdac6559507fd6 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobList.java @@ -0,0 +1,62 @@ +/** + * 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.util.ArrayList; +import java.util.List; + +/** + * Contains result of GetBlobList API in processable state. + */ +public class BlobList { + + /** + * List of {@link BlobProperty} returned by server + */ + private final List blobProperties = new ArrayList<>(); + + /** + * Since there could be many blobs which can be returned by server on the + * GetBlobList API on a path and server wants to return only limited number of + * blob-information in one go. The expectation from the server is to use a token + * called as nextMarker and call the GetBlobList API again for the + * same path. + */ + private String nextMarker; + + void addBlobProperty(final BlobProperty blobProperty) { + blobProperties.add(blobProperty); + } + + void setNextMarker(String nextMarker) { + this.nextMarker = nextMarker; + } + + public List getBlobPropertyList() { + return blobProperties; + } + + public String getNextMarker() { + return nextMarker; + } + + public BlobList() { + + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobListXmlParser.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobListXmlParser.java new file mode 100644 index 00000000000000..d00675201a4cfa --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobListXmlParser.java @@ -0,0 +1,300 @@ +/** + * 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.util.Stack; + +import org.xml.sax.Attributes; +import org.xml.sax.SAXException; +import org.xml.sax.helpers.DefaultHandler; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.utils.DateTimeUtils; + +/** + * Parses the response inputSteam and populates an object of {@link BlobList}. Parsing + * creates a list of {@link BlobProperty}.
+ * + * BlobList API XML response example + *
+ *   {@code
+ *   
+ *    string-value
+ *    string-value
+ *    int-value
+ *    string-value
+ *    
+ *       
+ *       
+ *    
+ *    
+ *   
+ *   }
+ * 
+ */ + + +public class BlobListXmlParser extends DefaultHandler { + /** + * Object that contains the parsed response. + */ + private final BlobList blobList; + private final String url; + /** + * {@link BlobProperty} for which at a given moment, the parsing is going on. + * + * Following XML elements will be parsed and added in currentBlobProperty. + * 1. Blob: for explicit files and directories + * + * blob-name + * + * + * value + * + * + * + * 2. BlobPrefix: for directories both explicit and implicit + * + * blob-prefix + * + */ + private BlobProperty currentBlobProperty; + /** + * Maintains the value in a given XML-element. + */ + private StringBuilder bld = new StringBuilder(); + /** + * Maintains the stack of XML-elements in memory at a given moment. + */ + private final Stack elements = new Stack<>(); + + /** + * Set an object of {@link BlobList} to populate from the parsing. + * Set the url for which GetBlobList API is called. + */ + public BlobListXmlParser(final BlobList blobList, final String url) { + this.blobList = blobList; + this.url = url; + } + + /** + *
Receive notification of the start of an element.
+ * If the xml start tag is "Blob", it defines that a new BlobProperty information + * is going to be parsed. + */ + @Override + public void startElement(final String uri, + final String localName, + final String qName, + final Attributes attributes) throws SAXException { + elements.push(localName); + if (AbfsHttpConstants.BLOB.equals(localName) || AbfsHttpConstants.BLOB_PREFIX.equals(localName)) { + currentBlobProperty = new BlobProperty(); + } + } + + /** + *
Receive notification of the end of an element.
+ * Whenever an XML-tag is closed, the parent-tag and current-tag shall be + * checked and correct property shall be set in the active {@link #currentBlobProperty}. + * If the current-tag is "Blob", it means that there are no more properties to + * be set in the the active {@link #currentBlobProperty}, and it shall be the + * {@link #blobList}. + */ + @Override + public void endElement(final String uri, + final String localName, + final String qName) + throws SAXException { + String currentNode = elements.pop(); + + // Check if the ending tag is correct to the starting tag in the stack. + if (!currentNode.equals(localName)) { + throw new SAXException(AbfsHttpConstants.INVALID_XML); + } + + String parentNode = ""; + if (elements.size() > 0) { + parentNode = elements.peek(); + } + + String value = bld.toString(); + if (value.isEmpty()) { + value = null; + } + + /* + * If the closing tag is Blob, there are no more properties to be set in + * currentBlobProperty. + */ + if (AbfsHttpConstants.BLOB.equals(currentNode)) { + blobList.addBlobProperty(currentBlobProperty); + currentBlobProperty = null; + } + + /* + * If the closing tag is BlobPrefix, there are no more properties to be set in + * currentBlobProperty and this is a directory (implicit or explicit) + * If implicit, it will be added only once/ + * If explicit it will be added with Blob Tag as well. + */ + if (AbfsHttpConstants.BLOB_PREFIX.equals(currentNode)) { + currentBlobProperty.setIsDirectory(true); + blobList.addBlobProperty(currentBlobProperty); + currentBlobProperty = null; + } + + /* + * If the closing tag is Next Marker, it needs to be saved with the + * list of blobs currently fetched + */ + if (AbfsHttpConstants.NEXT_MARKER.equals(currentNode)) { + blobList.setNextMarker(value); + } + + /* + * If the closing tag is Name, then it is either for a blob + * or for a blob prefix denoting a directory. We will save this + * in current BlobProperty for both + */ + if (currentNode.equals(AbfsHttpConstants.NAME) + && (parentNode.equals(AbfsHttpConstants.BLOB) + || parentNode.equals(AbfsHttpConstants.BLOB_PREFIX))) { + // In case of BlobPrefix Name will have a slash at the end + // Remove the "/" at the end of name + if (value.endsWith(AbfsHttpConstants.FORWARD_SLASH)) { + value = value.substring(0, value.length() - 1); + } + + currentBlobProperty.setName(value); + currentBlobProperty.setPath(new Path(AbfsHttpConstants.ROOT_PATH + value)); + currentBlobProperty.setUrl(url + AbfsHttpConstants.ROOT_PATH + value); + } + + /* + * For case: + * + * ... + * + * value + * value + * + * ... + * + * ParentNode will be Metadata for all key1, key2, ... , keyN. + */ + if (parentNode.equals(AbfsHttpConstants.METADATA)) { + currentBlobProperty.addMetadata(currentNode, value); + // For Marker blobs hdi_isFolder will be present as metadata + if (AbfsHttpConstants.HDI_ISFOLDER.equals(currentNode)) { + currentBlobProperty.setIsDirectory(Boolean.valueOf(value)); + } + } + + /* + * For case: + * + * ... + * + * date-time-value + * date-time-value + * Etag + * owner user id + * owning group id + * permission string + * access control list + * file | directory + * size-in-bytes + * id + * pending | success | aborted | failed + * source url + * bytes copied/bytes total + * datetime + * error string + * + * ... + * + * ParentNode will be Properties for Content-Length, ResourceType. + */ + if (parentNode.equals(AbfsHttpConstants.PROPERTIES)) { + if (currentNode.equals(AbfsHttpConstants.CREATION_TIME)) { + currentBlobProperty.setCreationTime(DateTimeUtils.parseLastModifiedTime(value)); + } + if (currentNode.equals(AbfsHttpConstants.LAST_MODIFIED_TIME)) { + currentBlobProperty.setLastModifiedTime(DateTimeUtils.parseLastModifiedTime(value)); + } + if (currentNode.equals(AbfsHttpConstants.ETAG)) { + currentBlobProperty.setETag(value); + } + if (currentNode.equals(AbfsHttpConstants.OWNER)) { + currentBlobProperty.setOwner(value); + } + if (currentNode.equals(AbfsHttpConstants.GROUP)) { + currentBlobProperty.setGroup(value); + } + if (currentNode.equals(AbfsHttpConstants.PERMISSIONS)) { + currentBlobProperty.setPermission(value); + } + if (currentNode.equals(AbfsHttpConstants.ACL)) { + currentBlobProperty.setAcl(value); + } + if (currentNode.equals(AbfsHttpConstants.RESOURCE_TYPE)) { + if (AbfsHttpConstants.DIRECTORY.equals(value)) { + currentBlobProperty.setIsDirectory(true); + } + } + if (currentNode.equals(AbfsHttpConstants.CONTENT_LEN)) { + currentBlobProperty.setContentLength(Long.parseLong(value)); + } + if (currentNode.equals(AbfsHttpConstants.COPY_ID)) { + currentBlobProperty.setCopyId(value); + } + if (currentNode.equals(AbfsHttpConstants.COPY_STATUS)) { + currentBlobProperty.setCopyStatus(value); + } + if (currentNode.equals(AbfsHttpConstants.COPY_SOURCE)) { + currentBlobProperty.setCopySourceUrl(value); + } + if (currentNode.equals(AbfsHttpConstants.COPY_PROGRESS)) { + currentBlobProperty.setCopyProgress(value); + } + if (currentNode.equals(AbfsHttpConstants.COPY_COMPLETION_TIME)) { + currentBlobProperty.setCopyCompletionTime(DateTimeUtils.parseLastModifiedTime(value)); + } + if (currentNode.equals(AbfsHttpConstants.COPY_STATUS_DESCRIPTION)) { + currentBlobProperty.setCopyStatusDescription(value); + } + } + /* + * refresh bld for the next XML-tag value + */ + bld = new StringBuilder(); + } + + /** + * Receive notification of character data inside an element. No heuristics to + * apply. Just append the {@link #bld}. + */ + @Override + public void characters(final char[] ch, final int start, final int length) + throws SAXException { + bld.append(ch, start, length); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobProperty.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobProperty.java new file mode 100644 index 00000000000000..ea7351d2120a8e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobProperty.java @@ -0,0 +1,217 @@ +/** + * 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.util.HashMap; +import java.util.Map; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; + +/** + * Encapsulates all the information related to a Blob as fetched + * on blob endpoint APIs + */ +public class BlobProperty { + private String name; + private Path path; + private String url; + private Boolean isDirectory = false; + private String eTag; + private long lastModifiedTime; + private long creationTime; + private String owner; + private String group; + private String permission; + private String acl; + private Long contentLength = 0L; + private String copyId; + private String copyStatus; + private String copySourceUrl; + private String copyProgress; + private String copyStatusDescription; + private long copyCompletionTime; + private Map metadata = new HashMap<>(); + + private AzureBlobFileSystemException ex; + + public BlobProperty() { + + } + + public String getName() { + return name; + } + + public Path getPath() { + return path; + } + + public String getUrl() { + return url; + } + + public Boolean getIsDirectory() { + return isDirectory; + } + + public String getETag() { + return eTag; + } + + public long getLastModifiedTime() { + return lastModifiedTime; + } + + public long getCreationTime() { + return creationTime; + } + + public String getOwner() { + return owner; + } + + public String getGroup() { + return group; + } + + public String getPermission() { + return permission; + } + + public String getAcl() { + return acl; + } + + public Long getContentLength() { + return contentLength; + } + + public String getCopyId() { + return copyId; + } + + public String getCopyStatus() { + return copyStatus; + } + + public String getCopySourceUrl() { + return copySourceUrl; + } + + public String getCopyProgress() { + return copyProgress; + } + + public String getCopyStatusDescription() { + return copyStatusDescription; + } + + public long getCopyCompletionTime() { + return copyCompletionTime; + } + + public Map getMetadata() { + return metadata; + } + + public AzureBlobFileSystemException getFailureException() { + return ex; + } + + public Path getBlobDstPath(Path dstBlobPath) { + return null; + } + + public void setName(final String name) { + this.name = name; + } + + public void setPath(final Path path) { + this.path = path; + } + + public void setUrl(final String url) { + this.url = url; + } + + public void setIsDirectory(final Boolean isDirectory) { + this.isDirectory = isDirectory; + } + + public void setETag(final String eTag) { + this.eTag = eTag; + } + + public void setLastModifiedTime(final long lastModifiedTime) { + this.lastModifiedTime = lastModifiedTime; + } + + public void setCreationTime(final long creationTime) { + this.creationTime = creationTime; + } + + public void setOwner(final String owner) { + this.owner = owner; + } + + public void setGroup(final String group) { + this.group = group; + } + + public void setPermission(final String permission) { + this.permission = permission; + } + + public void setAcl(final String acl) { + this.acl = acl; + } + + public void setContentLength(final Long contentLength) { + this.contentLength = contentLength; + } + + public void setCopyId(final String copyId) { + this.copyId = copyId; + } + + public void setCopyStatus(final String copyStatus) { + this.copyStatus = copyStatus; + } + + public void setCopySourceUrl(final String copySourceUrl) { + this.copySourceUrl = copySourceUrl; + } + + public void setCopyProgress(final String copyProgress) { + this.copyProgress = copyProgress; + } + + public void setCopyStatusDescription(final String copyStatusDescription) { + this.copyStatusDescription = copyStatusDescription; + } + + public void setCopyCompletionTime(final long copyCompletionTime) { + this.copyCompletionTime = copyCompletionTime; + } + + public void addMetadata(String key, String value) { + metadata.put(key, value); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlockStatus.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlockStatus.java new file mode 100644 index 00000000000000..387f09f4ecc1ac --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlockStatus.java @@ -0,0 +1,25 @@ +/** + * 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; + +public enum BlockStatus { + NEW, + SUCCESS, + FAILED +} 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 89d99471a82145..dee1d374d4a046 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 @@ -24,6 +24,8 @@ import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_CONTINUE; + /** * Retry policy used by AbfsClient. * */ @@ -56,6 +58,13 @@ public class ExponentialRetryPolicy { */ private static final double MAX_RANDOM_RATIO = 1.2; + /** + * Qualifies for retry based on + * https://learn.microsoft.com/en-us/azure/active-directory/ + * managed-identities-azure-resources/how-to-use-vm-token#error-handling + */ + private static final int HTTP_TOO_MANY_REQUESTS = 429; + /** * Holds the random number generator used to calculate randomized backoff intervals */ @@ -118,7 +127,12 @@ public ExponentialRetryPolicy(final int retryCount, final int minBackoff, final /** * Returns if a request should be retried based on the retry count, current response, - * and the current strategy. + * and the current strategy. The valid http status code lies in the range of 1xx-5xx. + * But an invalid status code might be set due to network or timeout kind of issues. + * Such invalid status code also qualify for retry. + * HTTP status code 410 qualifies for retry based on + * https://docs.microsoft.com/en-in/azure/virtual-machines/linux/ + * instance-metadata-service?tabs=windows#errors-and-debugging * * @param retryCount The current retry attempt count. * @param statusCode The status code of the response, or -1 for socket error. @@ -126,8 +140,10 @@ public ExponentialRetryPolicy(final int retryCount, final int minBackoff, final */ public boolean shouldRetry(final int retryCount, final int statusCode) { return retryCount < this.retryCount - && (statusCode == -1 + && (statusCode < HTTP_CONTINUE || statusCode == HttpURLConnection.HTTP_CLIENT_TIMEOUT + || statusCode == HttpURLConnection.HTTP_GONE + || statusCode == HTTP_TOO_MANY_REQUESTS || (statusCode >= HttpURLConnection.HTTP_INTERNAL_ERROR && statusCode != HttpURLConnection.HTTP_NOT_IMPLEMENTED && statusCode != HttpURLConnection.HTTP_VERSION)); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobConsumer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobConsumer.java new file mode 100644 index 00000000000000..bf6981505e4ead --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobConsumer.java @@ -0,0 +1,51 @@ +/** + * 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.util.List; + +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; + +public class ListBlobConsumer { + + private final ListBlobQueue listBlobQueue; + + public ListBlobConsumer(final ListBlobQueue listBlobQueue) { + this.listBlobQueue = listBlobQueue; + } + + public List consume() throws AzureBlobFileSystemException { + if (listBlobQueue.getException() != null) { + throw listBlobQueue.getException(); + } + return listBlobQueue.dequeue(); + } + + public Boolean isCompleted() { + return listBlobQueue.getIsCompleted() + && listBlobQueue.size() == 0; + } + + /** + * Register consumer failure. + */ + public void fail() { + listBlobQueue.consumptionFailed(); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobProducer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobProducer.java new file mode 100644 index 00000000000000..a198dc706c84a0 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobProducer.java @@ -0,0 +1,111 @@ +/** + * 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.classification.VisibleForTesting; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +/** + * ListBlob API can give maximum of 5000 blobs. If there are (~n*5000) blobs, the + * client would need to call the listBlob API n times. This would have two consequences: + *

    + *
  1. + * The consumer of the result of lists of blob would have to wait until all + * the blobs are received. The consumer could have used the time to start + * processing the blobs already in memory. The wait for receiving all the blobs + * would lead the processing more time. Lets say consumer need m time-units to process + * one blob. Lets assume that each set of blobs have x blobs. In total there + * have to be n sets Lets say that client needs t time to get all the blobs. If consumer + * wait for all the blobs to be received, the total time taken would be: + *
    t + (n * x * m)
    + * Now, lets assume that consumer in parallel work on the available the blobs, + * time taken would be: + *
    t + (((n * x) - t/m) * m)
    + *
  2. + *
  3. + * Since the information of the blobs have to be maintained in memory until the + * computation on the list is done. On low configuration machine, it may lead + * to OOM. + *
  4. + *
+ * In this design, the producer on a parallel thread to the main thread, will call + * ListBlob API and will populate {@link ListBlobQueue}, which would be dequeued by + * the main thread which will run the computation on the available blobs.
+ * + * How its different from {@link AbfsListStatusRemoteIterator}?
+ * It provides an iterator which on {@link AbfsListStatusRemoteIterator#hasNext()} checks + * if there are blobs available in memory. If not it will call Listing API on server for + * the next set of blobs. But here, it make the process sequential. As in, when the processing + * on whole set of blobs available in memory are done, after that only next set of blobs are + * fetched. + */ +public class ListBlobProducer { + + private final AbfsClient client; + + private final ListBlobQueue listBlobQueue; + + private final String src; + + private final TracingContext tracingContext; + + private String nextMarker; + private final Thread thread; + + public ListBlobProducer(final String src, + final AbfsClient abfsClient, + final ListBlobQueue listBlobQueue, + final String initNextMarker, + TracingContext tracingContext) { + this.src = src; + this.client = abfsClient; + this.tracingContext = tracingContext; + this.listBlobQueue = listBlobQueue; + listBlobQueue.setProducer(this); + this.nextMarker = initNextMarker; + thread = new Thread(() -> { + do { + int maxResult = listBlobQueue.availableSize(); + if (maxResult == 0) { + continue; + } + AbfsRestOperation op = null; + try { + op = client.getListBlobs(nextMarker, src, null, maxResult, tracingContext); + } catch (AzureBlobFileSystemException ex) { + listBlobQueue.setFailed(ex); + return; + } + BlobList blobList = op.getResult().getBlobList(); + nextMarker = blobList.getNextMarker(); + listBlobQueue.enqueue(blobList.getBlobPropertyList()); + if (nextMarker == null) { + listBlobQueue.complete(); + } + } while(nextMarker != null && !listBlobQueue.getConsumptionFailed()); + }); + thread.start(); + } + + @VisibleForTesting + public void waitForProcessCompletion() throws InterruptedException { + thread.join(); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobQueue.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobQueue.java new file mode 100644 index 00000000000000..3de0c85ec50006 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobQueue.java @@ -0,0 +1,126 @@ +/** + * 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.util.ArrayDeque; +import java.util.ArrayList; +import java.util.List; +import java.util.Queue; + +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; + +public class ListBlobQueue { + + private final Queue blobLists; + + private int totalProduced = 0; + + private int totalConsumed = 0; + + private Boolean isCompleted = false; + private Boolean isConsumptionFailed = false; + + private AzureBlobFileSystemException failureFromProducer; + + /** + * Since, Producer just spawns a thread and there are no public method for the + * class. Keeping its address in this object will prevent accidental GC close + * on the producer object. + */ + private ListBlobProducer producer; + + private final int maxSize; + private final int maxConsumedBlobCount; + + /** + * @param maxSize maxSize of the queue. + * @param maxConsumedBlobCount maximum number of blobs that would be returned + * by {@link #dequeue()} method. + */ + public ListBlobQueue(int maxSize, int maxConsumedBlobCount) { + blobLists = new ArrayDeque<>(maxSize); + this.maxSize = maxSize; + this.maxConsumedBlobCount = maxConsumedBlobCount; + } + + /** + * @param initBlobList list of blobProperties to be enqueued in th queue + * @param maxSize maxSize of the queue. + * @param maxConsumedBlobCount maximum number of blobs that would be returned + * by {@link #dequeue()} method. + */ + public ListBlobQueue(List initBlobList, int maxSize, int maxConsumedBlobCount) { + this(maxSize, maxConsumedBlobCount); + if (initBlobList != null) { + enqueue(initBlobList); + } + } + + void setProducer(ListBlobProducer producer) { + if (this.producer == null) { + this.producer = producer; + } + } + + void setFailed(AzureBlobFileSystemException failure) { + failureFromProducer = failure; + } + + public void complete() { + isCompleted = true; + } + + void consumptionFailed() { + isConsumptionFailed = true; + } + + Boolean getConsumptionFailed() { + return isConsumptionFailed; + } + + public Boolean getIsCompleted() { + return isCompleted; + } + + AzureBlobFileSystemException getException() { + return failureFromProducer; + } + + public void enqueue(List blobProperties) { + blobLists.addAll(blobProperties); + } + + public List dequeue() { + List blobProperties = new ArrayList<>(); + int counter = 0; + while (counter < maxConsumedBlobCount && blobLists.size() > 0) { + blobProperties.add(blobLists.poll()); + counter++; + } + return blobProperties; + } + + public int size() { + return blobLists.size(); + } + + public int availableSize() { + return maxSize - blobLists.size(); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/OperativeEndpoint.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/OperativeEndpoint.java new file mode 100644 index 00000000000000..93667eb77df84b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/OperativeEndpoint.java @@ -0,0 +1,48 @@ +/** + * 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.AbfsConfiguration; + +/** + * This class is mainly to unify the fallback for all API's to DFS endpoint at a single spot. + */ +public class OperativeEndpoint { + public static boolean isMkdirEnabledOnDFS(AbfsConfiguration abfsConfiguration) { + if (abfsConfiguration.getPrefixMode() == PrefixMode.BLOB) { + return abfsConfiguration.shouldMkdirFallbackToDfs(); + } else { + return true; + } + } + + public static boolean isIngressEnabledOnDFS(PrefixMode prefixMode, AbfsConfiguration abfsConfiguration) { + if (prefixMode == PrefixMode.BLOB) { + return abfsConfiguration.shouldIngressFallbackToDfs(); + } else { + return true; + } + } + + public static boolean isReadEnabledOnDFS(AbfsConfiguration abfsConfiguration) { + if (abfsConfiguration.getPrefixMode() == PrefixMode.BLOB) { + return abfsConfiguration.shouldReadFallbackToDfs(); + } + return true; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/PathInformation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/PathInformation.java new file mode 100644 index 00000000000000..e4f2790565fbff --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/PathInformation.java @@ -0,0 +1,37 @@ +/** + * 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; + +public class PathInformation { + private Boolean pathExists; + private Boolean isDirectory; + + public PathInformation(Boolean pathExists, Boolean isDirectory) { + this.pathExists = pathExists; + this.isDirectory = isDirectory; + } + + public Boolean getPathExists() { + return pathExists; + } + + public Boolean getIsDirectory() { + return isDirectory; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/PrefixMode.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/PrefixMode.java new file mode 100644 index 00000000000000..853a22f4f21c9e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/PrefixMode.java @@ -0,0 +1,25 @@ +/** + * 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; + +public enum PrefixMode { + DFS, + BLOB +} + diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java index 4eefb9fdf2c7e3..0f91afe0982dbf 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java @@ -101,6 +101,7 @@ private void init() { // hide instance constructor private ReadBufferManager() { + LOGGER.trace("Creating readbuffer manager with HADOOP-18546 patch"); } @@ -544,7 +545,6 @@ public synchronized void purgeBuffersForStream(AbfsInputStream stream) { LOGGER.debug("Purging stale buffers for AbfsInputStream {} ", stream); readAheadQueue.removeIf(readBuffer -> readBuffer.getStream() == stream); purgeList(stream, completedReadList); - purgeList(stream, inProgressList); } /** @@ -642,4 +642,9 @@ void testMimicFullUseAndAddFailedBuffer(ReadBuffer buf) { freeList.clear(); completedReadList.add(buf); } + + @VisibleForTesting + int getNumBuffers() { + return NUM_BUFFERS; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicityUtils.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicityUtils.java new file mode 100644 index 00000000000000..7490d3c090a4b6 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicityUtils.java @@ -0,0 +1,399 @@ +/** + * 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.FileNotFoundException; +import java.io.IOException; +import java.io.OutputStream; +import java.net.HttpURLConnection; +import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.TimeZone; + +import com.fasterxml.jackson.core.JsonParseException; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectReader; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +/** + * For a directory enabled for atomic-rename, before rename starts, a + * file with -RenamePending.json suffix is created. In this file, the states required + * for the rename are given. This file is created by {@link #preRename(Boolean, String)} ()} method. + * This is important in case the JVM process crashes during rename, the atomicity + * will be maintained, when the job calls {@link AzureBlobFileSystem#listStatus(Path)} + * or {@link AzureBlobFileSystem#getFileStatus(Path)}. On these API calls to filesystem, + * it will be checked if there is any RenamePending JSON file. If yes, the rename + * would be resumed as per the file. + */ +public class RenameAtomicityUtils { + + private static final Logger LOG = LoggerFactory.getLogger( + RenameAtomicityUtils.class); + + private final AzureBlobFileSystem azureBlobFileSystem; + private Path srcPath; + private Path dstPath; + private TracingContext tracingContext; + private Boolean isReDone; + + private static final int MAX_RENAME_PENDING_FILE_SIZE = 10000000; + private static final int FORMATTING_BUFFER = 10000; + + public static final String SUFFIX = "-RenamePending.json"; + + private static final ObjectReader READER = new ObjectMapper() + .configure(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES, true) + .readerFor(JsonNode.class); + + public RenameAtomicityUtils(final AzureBlobFileSystem azureBlobFileSystem, + final Path srcPath, + final Path dstPath, + final TracingContext tracingContext) throws IOException { + this.azureBlobFileSystem = azureBlobFileSystem; + this.srcPath = srcPath; + this.dstPath = dstPath; + this.tracingContext = tracingContext; + } + + public RenameAtomicityUtils(final AzureBlobFileSystem azureBlobFileSystem, + final Path renamePendingJsonPath, + final RedoRenameInvocation redoRenameInvocation, + final String srcEtag, + final AbfsInputStream renamePendingJsonInputStream) + throws IOException { + this.azureBlobFileSystem = azureBlobFileSystem; + final RenamePendingFileInfo renamePendingFileInfo = readFile( + renamePendingJsonPath, renamePendingJsonInputStream); + if (renamePendingFileInfo != null + && renamePendingFileInfo.eTag.equalsIgnoreCase(srcEtag)) { + redoRenameInvocation.redo(renamePendingFileInfo.destination, + renamePendingFileInfo.src); + isReDone = true; + } else { + isReDone = false; + } + } + + private RenamePendingFileInfo readFile(final Path redoFile, + final AbfsInputStream redoFileInputStream) + throws IOException { + Path f = redoFile; + byte[] bytes = new byte[MAX_RENAME_PENDING_FILE_SIZE]; + int l = redoFileInputStream.read(bytes); + if (l <= 0) { + // Jira HADOOP-12678 -Handle empty rename pending metadata file during + // atomic rename in redo path. If during renamepending file is created + // but not written yet, then this means that rename operation + // has not started yet. So we should delete rename pending metadata file. + LOG.error("Deleting empty rename pending file " + + redoFile + " -- no data available"); + deleteRenamePendingFile(azureBlobFileSystem, redoFile); + return null; + } + if (l == MAX_RENAME_PENDING_FILE_SIZE) { + throw new IOException( + "Error reading pending rename file contents -- " + + "maximum file size exceeded"); + } + String contents = new String(bytes, 0, l, StandardCharsets.UTF_8); + + // parse the JSON + JsonNode json = null; + boolean committed; + try { + json = READER.readValue(contents); + committed = true; + } catch (JsonMappingException e) { + + // The -RedoPending.json file is corrupted, so we assume it was + // not completely written + // and the redo operation did not commit. + committed = false; + } catch (JsonParseException e) { + committed = false; + } + + if (!committed) { + LOG.error("Deleting corruped rename pending file {} \n {}", + redoFile, contents); + + // delete the -RenamePending.json file + deleteRenamePendingFile(azureBlobFileSystem, redoFile); + return null; + } + + // initialize this object's fields + JsonNode oldFolderName = json.get("OldFolderName"); + JsonNode newFolderName = json.get("NewFolderName"); + JsonNode eTag = json.get("ETag"); + + if (oldFolderName != null && StringUtils.isNotEmpty( + oldFolderName.textValue()) + && newFolderName != null && StringUtils.isNotEmpty( + newFolderName.textValue()) && eTag != null && StringUtils.isNotEmpty( + eTag.textValue())) { + RenamePendingFileInfo renamePendingFileInfo = new RenamePendingFileInfo(); + renamePendingFileInfo.destination = new Path(newFolderName.textValue()); + renamePendingFileInfo.src = new Path(oldFolderName.textValue()); + renamePendingFileInfo.eTag = eTag.textValue(); + return renamePendingFileInfo; + } + return null; + } + + private void deleteRenamePendingFile(FileSystem fs, Path redoFile) + throws IOException { + try { + fs.delete(redoFile, false); + } catch (IOException e) { + // If the rename metadata was not found then somebody probably + // raced with us and finished the delete first + if (e instanceof FileNotFoundException) { + LOG.warn("rename pending file " + redoFile + " is already deleted"); + } else { + throw e; + } + } + } + + /** + * Write to disk the information needed to redo folder rename, + * in JSON format. The file name will be + * {@code abfs:///folderName-RenamePending.json} + * The file format will be: + *

{@code
+   * {
+   *   FormatVersion: "1.0",
+   *   OperationTime: "",
+   *   OldFolderName: "",
+   *   NewFolderName: ""
+   *   ETag: ""
+   * }
+   *
+   * Here's a sample:
+   * {
+   *  FormatVersion: "1.0",
+   *  OperationUTCTime: "2014-07-01 23:50:35.572",
+   *  OldFolderName: "user/ehans/folderToRename",
+   *  NewFolderName: "user/ehans/renamedFolder"
+   *  ETag: "ETag"
+   * } }
+ * @throws IOException Thrown when fail to write file. + */ + public void preRename(final Boolean isCreateOperationOnBlobEndpoint, + final String eTag) throws IOException { + Path path = getRenamePendingFilePath(); + LOG.debug("Preparing to write atomic rename state to {}", path.toString()); + OutputStream output = null; + + String contents = makeRenamePendingFileContents(eTag); + + // Write file. + try { + output = azureBlobFileSystem.create(path, false); + output.write(contents.getBytes(Charset.forName("UTF-8"))); + output.flush(); + output.close(); + } catch (IOException e) { + /* + * Scenario: file has been deleted by parallel thread before the RenameJSON + * could be written and flushed. + * ref: https://issues.apache.org/jira/browse/HADOOP-12678 + * On DFS endpoint, flush API is called. If file is not there, server returns + * 404. + * On blob endpoint, flush API is not there. PutBlockList is called with + * if-match header. If file is not there, the conditional header will fail, + * the server will return 412. + */ + if ((!isCreateOperationOnBlobEndpoint + && e instanceof FileNotFoundException) || ( + isCreateOperationOnBlobEndpoint && getWrappedException( + e) instanceof AbfsRestOperationException && + ((AbfsRestOperationException) getWrappedException( + e)).getStatusCode() + == HttpURLConnection.HTTP_PRECON_FAILED)) { + /* + * In case listStatus done on directory before any content could be written, + * that particular thread running on some worker-node of the cluster would + * delete the RenamePending JSON file. + * ref: https://issues.apache.org/jira/browse/HADOOP-12678. + * To recover from parallel delete, we will give it a second try. + */ + output = azureBlobFileSystem.create(path, false); + output.write(contents.getBytes(Charset.forName("UTF-8"))); + output.flush(); + output.close(); + return; + } + throw new IOException( + "Unable to write RenamePending file for folder rename from " + + srcPath.toUri().getPath() + " to " + dstPath.toUri().getPath(), + e); + } + } + + private Throwable getWrappedException(final IOException e) { + if (e.getCause() != null) { + return e.getCause().getCause(); + } + return e; + } + + /** + * Return the contents of the JSON file to represent the operations + * to be performed for a folder rename. + * + * @return JSON string which represents the operation. + */ + private String makeRenamePendingFileContents(String eTag) { + SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); + sdf.setTimeZone(TimeZone.getTimeZone("UTC")); + String time = sdf.format(new Date()); + if(!eTag.startsWith("\"") && !eTag.endsWith("\"")) { + eTag = quote(eTag); + } + + // Make file contents as a string. Again, quote file names, escaping + // characters as appropriate. + String contents = "{\n" + + " FormatVersion: \"1.0\",\n" + + " OperationUTCTime: \"" + time + "\",\n" + + " OldFolderName: " + quote(srcPath.toUri().getPath()) + ",\n" + + " NewFolderName: " + quote(dstPath.toUri().getPath()) + ",\n" + + " ETag: " + eTag + "\n" + + "}\n"; + + return contents; + } + + /** + * This is an exact copy of org.codehaus.jettison.json.JSONObject.quote + * method. + * + * Produce a string in double quotes with backslash sequences in all the + * right places. A backslash will be inserted within + * 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.util.LinkedList; +import java.util.List; + +import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.ClientErrorRetryReason; +import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.ConnectionResetRetryReason; +import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.ConnectionTimeoutRetryReason; +import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.ReadTimeoutRetryReason; +import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.RetryReasonCategory; +import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.ServerErrorRetryReason; +import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.UnknownHostRetryReason; +import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.UnknownIOExceptionRetryReason; +import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.UnknownSocketExceptionRetryReason; + + +/** + * This utility class exposes methods to convert a server response-error to a + * category of error. + */ +final class RetryReason { + + /** + * Linked-list of the implementations of RetryReasonCategory. The objects in the + * list are arranged by the rank of their significance. + *

    + *
  • ServerError (statusCode==5XX), ClientError (statusCode==4XX) are + * independent of other retryReason categories.
  • + *
  • Since {@link java.net.SocketException} is subclass of + * {@link java.io.IOException}, + * hence, {@link UnknownIOExceptionRetryReason} is placed before + * {@link UnknownSocketExceptionRetryReason}
  • + *
  • Since, connectionTimeout, readTimeout, and connectionReset are + * {@link java.net.SocketTimeoutException} exceptions with different messages, + * hence, {@link ConnectionTimeoutRetryReason}, {@link ReadTimeoutRetryReason}, + * {@link ConnectionResetRetryReason} are above {@link UnknownIOExceptionRetryReason}. + * There is no order between the three reasons as they are differentiated + * by exception-message.
  • + *
  • Since, {@link java.net.UnknownHostException} is subclass of + * {@link java.io.IOException}, {@link UnknownHostRetryReason} is placed + * over {@link UnknownIOExceptionRetryReason}
  • + *
+ */ + private static List rankedReasonCategories + = new LinkedList() {{ + add(new ServerErrorRetryReason()); + add(new ClientErrorRetryReason()); + add(new UnknownIOExceptionRetryReason()); + add(new UnknownSocketExceptionRetryReason()); + add(new ConnectionTimeoutRetryReason()); + add(new ReadTimeoutRetryReason()); + add(new UnknownHostRetryReason()); + add(new ConnectionResetRetryReason()); + }}; + + private RetryReason() { + + } + + /** + * Method to get correct abbreviation for a given set of exception, statusCode, + * storageStatusCode. + * + * @param ex exception caught during server communication. + * @param statusCode statusCode in the server response. + * @param storageErrorMessage storageErrorMessage in the server response. + * + * @return abbreviation for the the given set of exception, statusCode, storageStatusCode. + */ + static String getAbbreviation(Exception ex, + Integer statusCode, + String storageErrorMessage) { + String result = null; + for (RetryReasonCategory retryReasonCategory : rankedReasonCategories) { + final String abbreviation + = retryReasonCategory.captureAndGetAbbreviation(ex, + statusCode, storageErrorMessage); + if (abbreviation != null) { + result = abbreviation; + } + } + return result; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RetryReasonConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RetryReasonConstants.java new file mode 100644 index 00000000000000..8a0af183e30aea --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RetryReasonConstants.java @@ -0,0 +1,39 @@ +/** + * 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; + +public final class RetryReasonConstants { + + private RetryReasonConstants() { + + } + public static final String CONNECTION_TIMEOUT_JDK_MESSAGE = "connect timed out"; + public static final String READ_TIMEOUT_JDK_MESSAGE = "Read timed out"; + public static final String CONNECTION_RESET_MESSAGE = "Connection reset"; + public static final String OPERATION_BREACH_MESSAGE = "Operations per second is over the account limit."; + public static final String CONNECTION_RESET_ABBREVIATION = "CR"; + public static final String CONNECTION_TIMEOUT_ABBREVIATION = "CT"; + public static final String READ_TIMEOUT_ABBREVIATION = "RT"; + public static final String INGRESS_LIMIT_BREACH_ABBREVIATION = "ING"; + public static final String EGRESS_LIMIT_BREACH_ABBREVIATION = "EGR"; + public static final String OPERATION_LIMIT_BREACH_ABBREVIATION = "OPR"; + public static final String UNKNOWN_HOST_EXCEPTION_ABBREVIATION = "UH"; + public static final String IO_EXCEPTION_ABBREVIATION = "IOE"; + public static final String SOCKET_EXCEPTION_ABBREVIATION = "SE"; +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TimerFunctionality.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TimerFunctionality.java new file mode 100644 index 00000000000000..52428fdd54a19f --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TimerFunctionality.java @@ -0,0 +1,25 @@ +/** + * 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; + +public enum TimerFunctionality { + RESUME, + + SUSPEND +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ClientErrorRetryReason.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ClientErrorRetryReason.java new file mode 100644 index 00000000000000..cf1c47e3eb0dc5 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ClientErrorRetryReason.java @@ -0,0 +1,43 @@ +/** + * 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.retryReasonCategories; + +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_STATUS_CATEGORY_QUOTIENT; + +/** + * Category that can capture server-response errors for 4XX status-code. + */ +public class ClientErrorRetryReason extends RetryReasonCategory { + + @Override + Boolean canCapture(final Exception ex, + final Integer statusCode, + final String serverErrorMessage) { + if (statusCode == null || statusCode / HTTP_STATUS_CATEGORY_QUOTIENT != 4) { + return false; + } + return true; + } + + @Override + String getAbbreviation(final Integer statusCode, + final String serverErrorMessage) { + return statusCode + ""; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ConnectionResetRetryReason.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ConnectionResetRetryReason.java new file mode 100644 index 00000000000000..702f8875646324 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ConnectionResetRetryReason.java @@ -0,0 +1,42 @@ +/** + * 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.retryReasonCategories; + +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_RESET_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_RESET_MESSAGE; + +/** + * Category that can capture server-response errors for connection-reset exception. + */ +public class ConnectionResetRetryReason extends + RetryReasonCategory { + + @Override + Boolean canCapture(final Exception ex, + final Integer statusCode, + final String serverErrorMessage) { + return checkExceptionMessage(ex, CONNECTION_RESET_MESSAGE); + } + + @Override + String getAbbreviation(final Integer statusCode, + final String serverErrorMessage) { + return CONNECTION_RESET_ABBREVIATION; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ConnectionTimeoutRetryReason.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ConnectionTimeoutRetryReason.java new file mode 100644 index 00000000000000..28f35dcc805468 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ConnectionTimeoutRetryReason.java @@ -0,0 +1,43 @@ +/** + * 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.retryReasonCategories; + + +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_JDK_MESSAGE; + +/** + * Category that can capture server-response errors for connection-timeout. + */ +public class ConnectionTimeoutRetryReason extends + RetryReasonCategory { + + @Override + String getAbbreviation(final Integer statusCode, + final String serverErrorMessage) { + return CONNECTION_TIMEOUT_ABBREVIATION; + } + + @Override + Boolean canCapture(final Exception ex, + final Integer statusCode, + final String serverErrorMessage) { + return checkExceptionMessage(ex, CONNECTION_TIMEOUT_JDK_MESSAGE); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ReadTimeoutRetryReason.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ReadTimeoutRetryReason.java new file mode 100644 index 00000000000000..4663d9a52bbd6c --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ReadTimeoutRetryReason.java @@ -0,0 +1,41 @@ +/** + * 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.retryReasonCategories; + +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.READ_TIMEOUT_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.READ_TIMEOUT_JDK_MESSAGE; + +/** + * Category that can capture server-response errors for read-timeout. + */ +public class ReadTimeoutRetryReason extends RetryReasonCategory { + + @Override + Boolean canCapture(final Exception ex, + final Integer statusCode, + final String serverErrorMessage) { + return checkExceptionMessage(ex, READ_TIMEOUT_JDK_MESSAGE); + } + + @Override + String getAbbreviation(final Integer statusCode, + final String serverErrorMessage) { + return READ_TIMEOUT_ABBREVIATION; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/RetryReasonCategory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/RetryReasonCategory.java new file mode 100644 index 00000000000000..893451b496f457 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/RetryReasonCategory.java @@ -0,0 +1,90 @@ +/** + * 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.retryReasonCategories; + +import java.util.Locale; + +/** + * Provides methods to define if given exception can be categorised to certain category. + * Each category has a different implementation of the abstract class. + */ +public abstract class RetryReasonCategory { + + /** + * Returns if given server response error can be categorised by the implementation. + * + * @param ex exception captured in the server response. + * @param statusCode statusCode on the server response + * @param serverErrorMessage serverErrorMessage on the server response. + * + * @return

  1. true if server response error can be categorised by the implementation
  2. + *
  3. false if response error can not be categorised by the implementation
+ */ + abstract Boolean canCapture(Exception ex, + Integer statusCode, + String serverErrorMessage); + + /** + * Returns the abbreviation corresponding to the server response error. + * + * @param statusCode statusCode on the server response + * @param serverErrorMessage serverErrorMessage on the server response. + * + * @return abbreviation on the basis of the statusCode and the serverErrorMessage + */ + abstract String getAbbreviation(Integer statusCode, String serverErrorMessage); + + /** + * Converts the server-error response to an abbreviation if the response can be + * categorised by the implementation. + * + * @param ex exception received while making API request + * @param statusCode statusCode received in the server-response + * @param serverErrorMessage error-message received in the server-response + * + * @return abbreviation if the server-response can be categorised by the implementation. + * null if the server-response can not be categorised by the implementation. + */ + public String captureAndGetAbbreviation(Exception ex, + Integer statusCode, + String serverErrorMessage) { + if (canCapture(ex, statusCode, serverErrorMessage)) { + return getAbbreviation(statusCode, serverErrorMessage); + } + return null; + } + + /** + * Checks if a required search-string is in the exception's message. + */ + Boolean checkExceptionMessage(final Exception exceptionCaptured, + final String search) { + if (search == null) { + return false; + } + if (exceptionCaptured != null + && exceptionCaptured.getMessage() != null + && exceptionCaptured.getMessage() + .toLowerCase(Locale.US) + .contains(search.toLowerCase(Locale.US))) { + return true; + } + return false; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ServerErrorRetryReason.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ServerErrorRetryReason.java new file mode 100644 index 00000000000000..dd67a0cb8cbba2 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ServerErrorRetryReason.java @@ -0,0 +1,67 @@ +/** + * 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.retryReasonCategories; + +import static java.net.HttpURLConnection.HTTP_UNAVAILABLE; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_STATUS_CATEGORY_QUOTIENT; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.EGRESS_OVER_ACCOUNT_LIMIT; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.INGRESS_OVER_ACCOUNT_LIMIT; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.EGRESS_LIMIT_BREACH_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.INGRESS_LIMIT_BREACH_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.OPERATION_BREACH_MESSAGE; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.OPERATION_LIMIT_BREACH_ABBREVIATION; + +/** + * Category that can capture server-response errors for 5XX status-code. + */ +public class ServerErrorRetryReason extends RetryReasonCategory { + + @Override + Boolean canCapture(final Exception ex, + final Integer statusCode, + final String serverErrorMessage) { + if (statusCode == null || statusCode / HTTP_STATUS_CATEGORY_QUOTIENT != 5) { + return false; + } + return true; + } + + @Override + String getAbbreviation(final Integer statusCode, + final String serverErrorMessage) { + if (statusCode == HTTP_UNAVAILABLE && serverErrorMessage != null) { + String splitedServerErrorMessage = serverErrorMessage.split(System.lineSeparator(), + 2)[0]; + if (INGRESS_OVER_ACCOUNT_LIMIT.getErrorMessage().equalsIgnoreCase( + splitedServerErrorMessage)) { + return INGRESS_LIMIT_BREACH_ABBREVIATION; + } + if (EGRESS_OVER_ACCOUNT_LIMIT.getErrorMessage().equalsIgnoreCase( + splitedServerErrorMessage)) { + return EGRESS_LIMIT_BREACH_ABBREVIATION; + } + if (OPERATION_BREACH_MESSAGE.equalsIgnoreCase( + splitedServerErrorMessage)) { + return OPERATION_LIMIT_BREACH_ABBREVIATION; + } + return HTTP_UNAVAILABLE + ""; + } + return statusCode + ""; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownHostRetryReason.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownHostRetryReason.java new file mode 100644 index 00000000000000..c329348d81f8d6 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownHostRetryReason.java @@ -0,0 +1,45 @@ +/** + * 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.retryReasonCategories; + +import java.net.UnknownHostException; + +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.UNKNOWN_HOST_EXCEPTION_ABBREVIATION; + +/** + * Category that can capture server-response errors for {@link UnknownHostException}. + */ +public class UnknownHostRetryReason extends RetryReasonCategory { + + @Override + Boolean canCapture(final Exception ex, + final Integer statusCode, + final String serverErrorMessage) { + if (ex instanceof UnknownHostException) { + return true; + } + return false; + } + + @Override + String getAbbreviation(final Integer statusCode, + final String serverErrorMessage) { + return UNKNOWN_HOST_EXCEPTION_ABBREVIATION; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownIOExceptionRetryReason.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownIOExceptionRetryReason.java new file mode 100644 index 00000000000000..8a69ebb928d684 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownIOExceptionRetryReason.java @@ -0,0 +1,47 @@ +/** + * 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.retryReasonCategories; + +import java.io.IOException; + +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.IO_EXCEPTION_ABBREVIATION; + + +/** + * Category that can capture server-response errors for {@link IOException}. + */ +public class UnknownIOExceptionRetryReason extends + RetryReasonCategory { + + @Override + Boolean canCapture(final Exception ex, + final Integer statusCode, + final String serverErrorMessage) { + if (ex instanceof IOException) { + return true; + } + return false; + } + + @Override + String getAbbreviation(final Integer statusCode, + final String serverErrorMessage) { + return IO_EXCEPTION_ABBREVIATION; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownSocketExceptionRetryReason.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownSocketExceptionRetryReason.java new file mode 100644 index 00000000000000..18e9f115feaf69 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownSocketExceptionRetryReason.java @@ -0,0 +1,46 @@ +/** + * 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.retryReasonCategories; + +import java.net.SocketException; + +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.SOCKET_EXCEPTION_ABBREVIATION; + +/** + * Category that can capture server-response errors for {@link SocketException}. + */ +public class UnknownSocketExceptionRetryReason extends + RetryReasonCategory { + + @Override + Boolean canCapture(final Exception ex, + final Integer statusCode, + final String serverErrorMessage) { + if (ex instanceof SocketException) { + return true; + } + return false; + } + + @Override + String getAbbreviation(final Integer statusCode, + final String serverErrorMessage) { + return SOCKET_EXCEPTION_ABBREVIATION; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/package-info.java new file mode 100644 index 00000000000000..7d8078620af777 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/package-info.java @@ -0,0 +1,27 @@ +/* + * 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. + */ + +/** + * A retryReasonCategory defines methods applicable on server-response errors. + */ +@Private +@Evolving +package org.apache.hadoop.fs.azurebfs.services.retryReasonCategories; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Evolving; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index 5a115451df159b..e2ab95aec91028 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -61,12 +61,25 @@ public class TracingContext { private final TracingHeaderFormat format; // header ID display options private Listener listener = null; // null except when testing //final concatenated ID list set into x-ms-client-request-id header + private String fallbackDFSAppend = "B"; private String header = EMPTY_STRING; + /** + * If {@link #primaryRequestId} is null, this field shall be set equal + * to the last part of the {@link #clientRequestId}'s UUID + * in {@link #constructHeader(AbfsHttpOperation, String)} only on the + * first API call for an operation. Subsequent retries for that operation + * will not change this field. In case {@link #primaryRequestId} is non-null, + * this field shall not be set. + */ + private String primaryRequestIdForRetry; + private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*"; + private Integer operatedBlobCount = null; + /** * Initialize TracingContext * @param clientCorrelationID Provided over config by client @@ -139,6 +152,10 @@ public void setOperation(FSOperationType operation) { this.opType = operation; } + public int getRetryCount() { + return retryCount; + } + public void setRetryCount(int retryCount) { this.retryCount = retryCount; } @@ -147,20 +164,34 @@ public void setListener(Listener listener) { this.listener = listener; } + public void setFallbackDFSAppend(String fallbackDFSAppend) { + this.fallbackDFSAppend = fallbackDFSAppend; + } + + public String getFallbackDFSAppend() { + return fallbackDFSAppend; + } + /** * Concatenate all identifiers separated by (:) into a string and set into * X_MS_CLIENT_REQUEST_ID header of the http operation * @param httpOperation AbfsHttpOperation instance to set header into * connection + * @param previousFailure Failure seen before this API trigger on same operation + * from AbfsClient. */ - public void constructHeader(AbfsHttpOperation httpOperation) { + public void constructHeader(AbfsHttpOperation httpOperation, String previousFailure) { clientRequestId = UUID.randomUUID().toString(); switch (format) { case ALL_ID_FORMAT: // Optional IDs (e.g. streamId) may be empty header = clientCorrelationID + ":" + clientRequestId + ":" + fileSystemID + ":" - + primaryRequestId + ":" + streamID + ":" + opType + ":" - + retryCount; + + getPrimaryRequestIdForHeader(retryCount > 0) + ":" + streamID + + ":" + opType + ":" + retryCount; + header = addFailureReasons(header, previousFailure) + ":" + fallbackDFSAppend; + if (operatedBlobCount != null) { + header += (":" + operatedBlobCount); + } break; case TWO_ID_FORMAT: header = clientCorrelationID + ":" + clientRequestId; @@ -172,6 +203,39 @@ public void constructHeader(AbfsHttpOperation httpOperation) { listener.callTracingHeaderValidator(header, format); } httpOperation.setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, header); + /* + * In case the primaryRequestId is an empty-string and if it is the first try to + * API call (previousFailure shall be null), maintain the last part of clientRequestId's + * UUID in primaryRequestIdForRetry. This field shall be used as primaryRequestId part + * of the x-ms-client-request-id header in case of retry of the same API-request. + */ + if (primaryRequestId.isEmpty() && previousFailure == null) { + String[] clientRequestIdParts = clientRequestId.split("-"); + primaryRequestIdForRetry = clientRequestIdParts[ + clientRequestIdParts.length - 1]; + } + } + + /** + * Provide value to be used as primaryRequestId part of x-ms-client-request-id header. + * @param isRetry define if it's for a retry case. + * @return {@link #primaryRequestIdForRetry}:If the {@link #primaryRequestId} + * is an empty-string, and it's a retry iteration. + * {@link #primaryRequestId} for other cases. + */ + private String getPrimaryRequestIdForHeader(final Boolean isRetry) { + if (!primaryRequestId.isEmpty() || !isRetry) { + return primaryRequestId; + } + return primaryRequestIdForRetry; + } + + private String addFailureReasons(final String header, + final String previousFailure) { + if (previousFailure == null) { + return header; + } + return String.format("%s_%s", header, previousFailure); } /** @@ -182,4 +246,19 @@ public String getHeader() { return header; } + public String getPrimaryRequestId() { + return primaryRequestId; + } + + public void setOperatedBlobCount(Integer count) { + operatedBlobCount = count; + } + + public Integer getOperatedBlobCount() { + return operatedBlobCount; + } + + public FSOperationType getOpType() { + return opType; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/UriUtils.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/UriUtils.java index e27d54b443ca20..857b20b2fd80bd 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/UriUtils.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/UriUtils.java @@ -169,6 +169,20 @@ public static String getMaskedUrl(URL url) { return url.toString().replace(queryString, maskedQueryString); } + public static String encodeMetadataAttribute(String value) throws UnsupportedEncodingException { + // We have to URL encode the attribute as it could + // have URI special characters which unless encoded will result + // in 403 errors from the server. This is due to metadata properties + // being sent in the HTTP header of the request which is in turn used + // on the server side to authorize the request. + return value == null ? null : URLEncoder.encode(value, StandardCharsets.UTF_8.name()); + } + + public static String decodeMetadataAttribute(String encoded) throws UnsupportedEncodingException { + return encoded == null ? null : + java.net.URLDecoder.decode(encoded, StandardCharsets.UTF_8.name()); + } + private UriUtils() { } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/WeakReferenceMap.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/WeakReferenceMap.java new file mode 100644 index 00000000000000..cd47809689c416 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/WeakReferenceMap.java @@ -0,0 +1,333 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs.utils; + +import java.lang.ref.WeakReference; +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import java.util.function.Function; + +import javax.annotation.Nullable; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.store.LogExactlyOnce; + +import static java.util.Objects.requireNonNull; + +/** + * A map of keys type K to objects of type V which uses weak references, + * so does lot leak memory through long-lived references + * at the expense of losing references when GC takes place.. + * + * This class is intended be used instead of ThreadLocal storage when + * references are to be cleaned up when the instance holding. + * In this use case, the key is the Long key. + * + * Concurrency. + * The class assumes that map entries are rarely contended for when writing, + * and that not blocking other threads is more important than atomicity. + * - a ConcurrentHashMap is used to map keys to weak references, with + * all its guarantees. + * - there is no automatic pruning. + * - see {@link #create(Object)} for the concurrency semantics on entry creation. + */ +@InterfaceAudience.Private +public class WeakReferenceMap { + + private static final Logger LOG = + LoggerFactory.getLogger(WeakReferenceMap.class); + + /** + * The reference map. + */ + private final Map> map = new ConcurrentHashMap<>(); + + /** + * Supplier of new instances. + */ + private final Function factory; + + /** + * Nullable callback when a get on a key got a weak reference back. + * The assumption is that this is for logging/stats, which is why + * no attempt is made to use the call as a supplier of a new value. + */ + private final Consumer referenceLost; + + /** + * Counter of references lost. + */ + private final AtomicLong referenceLostCount = new AtomicLong(); + + /** + * Counter of entries created. + */ + private final AtomicLong entriesCreatedCount = new AtomicLong(); + + /** + * Log to report loss of a reference during the create phase, which + * is believed to be a cause of HADOOP-18456. + */ + private final LogExactlyOnce referenceLostDuringCreation = new LogExactlyOnce(LOG); + + /** + * instantiate. + * @param factory supplier of new instances + * @param referenceLost optional callback on lost references. + */ + public WeakReferenceMap( + Function factory, + @Nullable final Consumer referenceLost) { + + this.factory = requireNonNull(factory); + this.referenceLost = referenceLost; + } + + @Override + public String toString() { + return "WeakReferenceMap{" + + "size=" + size() + + ", referenceLostCount=" + referenceLostCount + + ", entriesCreatedCount=" + entriesCreatedCount + + '}'; + } + + /** + * Map size. + * @return the current map size. + */ + public int size() { + return map.size(); + } + + /** + * Clear all entries. + */ + public void clear() { + map.clear(); + } + + /** + * look up the value, returning the possibly empty weak reference + * to a value, or null if no value was found. + * @param key key to look up + * @return null if there is no entry, a weak reference if found + */ + public WeakReference lookup(K key) { + return map.get(key); + } + + /** + * Get the value, creating if needed. + * @param key key. + * @return an instance. + */ + public V get(K key) { + final WeakReference currentWeakRef = lookup(key); + // resolve it, after which if not null, we have a strong reference + V strongVal = resolve(currentWeakRef); + if (strongVal != null) { + // all good. + return strongVal; + } + + // here, either currentWeakRef was null, or its reference was GC'd. + if (currentWeakRef != null) { + // garbage collection removed the reference. + + // explicitly remove the weak ref from the map if it has not + // been updated by this point + // this is here just for completeness. + map.remove(key, currentWeakRef); + + // log/report the loss. + noteLost(key); + } + + // create a new value and add it to the map + return create(key); + } + + /** + * Create a new instance under a key. + *

+ * The instance is created, added to the map and then the + * map value retrieved. + * This ensures that the reference returned is that in the map, + * even if there is more than one entry being created at the same time. + * If that race does occur, it will be logged the first time it happens + * for this specific map instance. + *

+ * HADOOP-18456 highlighted the risk of a concurrent GC resulting a null + * value being retrieved and so returned. + * To prevent this: + *

    + *
  1. A strong reference is retained to the newly created instance + * in a local variable.
  2. + *
  3. That variable is used after the resolution process, to ensure + * the JVM doesn't consider it "unreachable" and so eligible for GC.
  4. + *
  5. A check is made for the resolved reference being null, and if so, + * the put() is repeated
  6. + *
+ * @param key key + * @return the created value + */ + public V create(K key) { + entriesCreatedCount.incrementAndGet(); + /* + Get a strong ref so even if a GC happens in this method the reference is not lost. + It is NOT enough to have a reference in a field, it MUST be used + so as to ensure the reference isn't optimized away prematurely. + "A reachable object is any object that can be accessed in any potential continuing + computation from any live thread." + */ + + final V strongRef = requireNonNull(factory.apply(key), + "factory returned a null instance"); + V resolvedStrongRef; + do { + WeakReference newWeakRef = new WeakReference<>(strongRef); + + // put it in the map + map.put(key, newWeakRef); + + // get it back from the map + WeakReference retrievedWeakRef = map.get(key); + // resolve that reference, handling the situation where somehow it was removed from the map + // between the put() and the get() + resolvedStrongRef = resolve(retrievedWeakRef); + if (resolvedStrongRef == null) { + referenceLostDuringCreation.warn("reference to %s lost during creation", key); + noteLost(key); + } + } while (resolvedStrongRef == null); + + // note if there was any change in the reference. + // as this forces strongRef to be kept in scope + if (strongRef != resolvedStrongRef) { + LOG.debug("Created instance for key {}: {} overwritten by {}", + key, strongRef, resolvedStrongRef); + } + + return resolvedStrongRef; + } + + /** + * Put a value under the key. + * A null value can be put, though on a get() call + * a new entry is generated + * + * @param key key + * @param value value + * @return any old non-null reference. + */ + public V put(K key, V value) { + return resolve(map.put(key, new WeakReference<>(value))); + } + + /** + * Remove any value under the key. + * @param key key + * @return any old non-null reference. + */ + public V remove(K key) { + return resolve(map.remove(key)); + } + + /** + * Does the map have a valid reference for this object? + * no-side effects: there's no attempt to notify or cleanup + * if the reference is null. + * @param key key to look up + * @return true if there is a valid reference. + */ + public boolean containsKey(K key) { + final WeakReference current = lookup(key); + return resolve(current) != null; + } + + /** + * Given a possibly null weak reference, resolve + * its value. + * @param r reference to resolve + * @return the value or null + */ + protected V resolve(WeakReference r) { + return r == null ? null : r.get(); + } + + /** + * Prune all null weak references, calling the referenceLost + * callback for each one. + * + * non-atomic and non-blocking. + * @return the number of entries pruned. + */ + public int prune() { + int count = 0; + final Iterator>> it = map.entrySet().iterator(); + while (it.hasNext()) { + final Map.Entry> next = it.next(); + if (next.getValue().get() == null) { + it.remove(); + count++; + noteLost(next.getKey()); + } + } + return count; + } + + /** + * Notify the reference lost callback. + * @param key key of lost reference + */ + private void noteLost(final K key) { + // increment local counter + referenceLostCount.incrementAndGet(); + + // and call any notification function supplied in the constructor + if (referenceLost != null) { + referenceLost.accept(key); + } + } + + /** + * Get count of references lost as detected + * during prune() or get() calls. + * @return count of references lost + */ + public final long getReferenceLostCount() { + return referenceLostCount.get(); + } + + /** + * Get count of entries created on demand. + * @return count of entries created + */ + public final long getEntriesCreatedCount() { + return entriesCreatedCount.get(); + } +} + diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md index dfb7f3f42a5cf9..ac77765f9e0b52 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md @@ -769,6 +769,26 @@ Hflush() being the only documented API that can provide persistent data transfer, Flush() also attempting to persist buffered data will lead to performance issues. +<<<<<<< HEAD +======= +### Hundred Continue Options + +`fs.azure.account.expect.header.enabled`: This configuration parameter is used +to specify whether you wish to send a expect 100 continue header with each +append request or not. It is configured to true by default. This flag configures +the client to check with the Azure store before uploading a block of data from +an output stream. This allows the client to throttle back gracefully -before +actually attempting to upload the block. In experiments this provides +significant throughput improvements under heavy load. For more information : +- https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Expect + + +### Account level throttling Options + +`fs.azure.account.operation.idle.timeout`: This value specifies the time after which the timer for the analyzer (read or +write) should be paused until no new request is made again. The default value for the same is 60 seconds. + +>>>>>>> c88011c6046... HADOOP-18146: ABFS: Added changes for expect hundred continue header (#4039) ### HNS Check Options Config `fs.azure.account.hns.enabled` provides an option to specify whether the storage account is HNS enabled or not. In case the config is not provided, @@ -874,6 +894,9 @@ when there are too many writes from the same process. time. Effectively this will be the threadpool size within the AbfsOutputStream instance. Set the value in between 1 to 8 both inclusive. +`fs.azure.analysis.period`: The time after which sleep duration is recomputed after analyzing metrics. The default value +for the same is 10 seconds. + `fs.azure.write.max.requests.to.queue`: To set the maximum write requests that can be queued. Memory consumption of AbfsOutputStream instance can be tuned with this config considering each queued request holds a buffer. Set diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md b/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md index 933f86be3e8961..e256fbef546cb2 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md @@ -602,26 +602,76 @@ various test combinations, it will: 2. Run tests for all combinations 3. Summarize results across all the test combination runs. -As a pre-requisite step, fill config values for test accounts and credentials -needed for authentication in `src/test/resources/azure-auth-keys.xml.template` -and rename as `src/test/resources/azure-auth-keys.xml`. +Below are the pre-requiste steps to follow: +1. Copy -**To add a new test combination:** Templates for mandatory test combinations -for PR validation are present in `dev-support/testrun-scripts/runtests.sh`. -If a new one needs to be added, add a combination set within -`dev-support/testrun-scripts/runtests.sh` similar to the ones already defined -and -1. Provide a new combination name -2. Update properties and values array which need to be effective for the test -combination -3. Call generateconfigs + ./src/test/resources/azure-auth-keys.xml.template + TO + ./src/test/resources/azure-auth-keys.xml + Update account names that should be used in the test run for HNS and non-HNS + combinations in the 2 properties present in the xml (account name should be + without domain part), namely + + fs.azure.hnsTestAccountName + fs.azure.nonHnsTestAccountName + azure-auth-keys.xml is listed in .gitignore, so any accidental account name leak is prevented. + +``` +XInclude is supported, so for extra security secrets may be +kept out of the source tree then referenced through an an XInclude element: + + +``` + +2. Create account config files (one config file per account) in folder: + + ./src/test/resources/accountSettings/ + Follow the instruction in the start of the template file + + accountName_settings.xml.template + within accountSettings folder while creating account config file. + New files created in folder accountSettings is listed in .gitignore to + prevent accidental cred leaks. **To run PR validation:** Running command -* `dev-support/testrun-scripts/runtests.sh` will generate configurations for -each of the combinations defined and run tests for all the combinations. -* `dev-support/testrun-scripts/runtests.sh -c {combinationname}` Specific -combinations can be provided with -c option. If combinations are provided -with -c option, tests for only those combinations will be run. +* `dev-support/testrun-scripts/runtests.sh` will prompt as below: +```bash +Choose action: +[Note - SET_ACTIVE_TEST_CONFIG will help activate the config for IDE/single test class runs] +1) SET_ACTIVE_TEST_CONFIG 4) SET_OR_CHANGE_TEST_ACCOUNT +2) RUN_TEST 5) PRINT_LOG4J_LOG_PATHS_FROM_LAST_RUN +3) CLEAN_UP_OLD_TEST_CONTAINERS +#? 2 +``` +Enter 1: for setting active combination for IDE test run/single mvn test class runs. + +Enter 2: for choosing the combination to choose for mvn full test suite. + +Enter 3: For clean-up of any abruptly ending test leaving auto generated test +container on the account. + +Enter 4: To create/modify the config file that decides the account to use for specific test combination. + +Enter 5: To print the log4j paths the last test runs. + +On next prompt, current list of combinations to choose are provided. +Sample for Run_TEST action: +```bash +Enter parallel test run process count [default - 8]: 4 +Set the active test combination to run the action: +1) HNS-OAuth 3) nonHNS-SharedKey 5) AllCombinationsTestRun +2) HNS-SharedKey 4) AppendBlob-HNS-OAuth 6) Quit +#? 1 + +Combination specific property setting: [ key=fs.azure.account.auth.type , value=OAuth ] + +Activated [src/test/resources/abfs-combination-test-configs.xml] - for account: snvijayacontracttest for combination HNS-OAuth +Running test for combination HNS-OAuth on account snvijayacontracttest [ProcessCount=4] +Test run report can be seen in dev-support/testlogs/2022-10-07_05-23-22/Test-Logs-HNS-OAuth.txt +```` + +Provide the option for the action chosen first. **Test logs:** Test runs will create a folder within dev-support/testlogs to save the test logs. Folder name will be the test start timestamp. The mvn verify @@ -632,25 +682,18 @@ consolidated results of all the combination runs will be saved into a file as Test-Results.log in the same folder. When run for PR validation, the consolidated test results needs to be pasted into the PR comment section. -**To generate config for use in IDE:** Running command with -a (activate) option -`dev-support/testrun-scripts/runtests.sh -a {combination name}` will update -the effective config relevant for the specific test combination. Hence the same -config files used by the mvn test runs can be used for IDE without any manual -updates needed within config file. - -**Other command line options:** -* -a Specify the combination name which needs to be -activated. This is to be used to generate config for use in IDE. -* -c Specify the combination name for test runs. If this -config is specified, tests for only the specified combinations will run. All -combinations of tests will be running if this config is not specified. -* -t ABFS mvn tests are run in parallel mode. Tests by default -are run with 8 thread count. It can be changed by providing -t - -In order to test ABFS, please add the following configuration to your -`src/test/resources/azure-auth-keys.xml` file. Note that the ABFS tests include -compatibility tests which require WASB credentials, in addition to the ABFS -credentials. +**To add a new test combination:** Templates for mandatory test combinations +for PR validation are present in `dev-support/testrun-scripts/runtests.sh`. +If a new one needs to be added, add a combination to +`dev-support/testrun-scripts/runtests.sh`. +(Refer to current active combinations within +`SECTION: COMBINATION DEFINITIONS AND TRIGGER` and +`SECTION: TEST COMBINATION METHODS` in the script). + +**Test Configuration Details:** + + Note that the ABFS tests include compatibility tests which require WASB + credentials, in addition to the ABFS credentials. ```xml diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AzureBlobStorageTestAccount.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AzureBlobStorageTestAccount.java index 5d2d5d4afdc3f4..72793eaecac04d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AzureBlobStorageTestAccount.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AzureBlobStorageTestAccount.java @@ -21,6 +21,7 @@ import com.microsoft.azure.storage.*; import com.microsoft.azure.storage.blob.*; import com.microsoft.azure.storage.core.Base64; +import jdk.nashorn.internal.parser.Token; import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,11 +34,14 @@ import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation; import org.apache.hadoop.fs.azure.metrics.AzureFileSystemMetricsSystem; import org.apache.hadoop.fs.azure.integration.AzureTestUtils; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException; +import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.hadoop.metrics2.AbstractMetric; import org.apache.hadoop.metrics2.MetricsRecord; import org.apache.hadoop.metrics2.MetricsSink; import org.apache.hadoop.metrics2.MetricsTag; import org.apache.hadoop.metrics2.impl.TestMetricsConfig; +import org.apache.kerby.kerberos.kerb.provider.TokenProvider; import java.io.File; import java.io.IOException; @@ -71,6 +75,7 @@ public final class AzureBlobStorageTestAccount implements AutoCloseable, public static final String MOCK_CONTAINER_NAME = "mockContainer"; public static final String WASB_AUTHORITY_DELIMITER = "@"; public static final String WASB_SCHEME = "wasb"; + public static final String WASBS_SCHEME = "wasbs"; public static final String PATH_DELIMITER = "/"; public static final String AZURE_ROOT_CONTAINER = "$root"; public static final String MOCK_WASB_URI = "wasb://" + MOCK_CONTAINER_NAME @@ -456,13 +461,13 @@ public static void setMockAccountKey(Configuration conf, String accountName) { private static URI createAccountUri(String accountName) throws URISyntaxException { - return new URI(WASB_SCHEME + ":" + PATH_DELIMITER + PATH_DELIMITER + return new URI(WASBS_SCHEME + ":" + PATH_DELIMITER + PATH_DELIMITER + accountName); } private static URI createAccountUri(String accountName, String containerName) throws URISyntaxException { - return new URI(WASB_SCHEME + ":" + PATH_DELIMITER + PATH_DELIMITER + return new URI(WASBS_SCHEME + ":" + PATH_DELIMITER + PATH_DELIMITER + containerName + WASB_AUTHORITY_DELIMITER + accountName); } @@ -505,6 +510,37 @@ static CloudStorageAccount createStorageAccount(String accountName, credentials = new StorageCredentialsAccountAndKey( accountName.split("\\.")[0], accountKey); } + return new CloudStorageAccount(credentials); + } + + static CloudStorageAccount createStorageAccount(String accountName, + Configuration conf, boolean allowAnonymous, NativeAzureFileSystem fs) + throws URISyntaxException, + KeyProviderException, IOException { + String accountKey = AzureNativeFileSystemStore + .getAccountKeyFromConfiguration(accountName, conf); + final StorageCredentials credentials; + if (fs.getStore().getTokenProviderStore() != null){ + AccessTokenProvider tokenProvider = fs.getStore().getTokenProviderStore(); + int iend = accountName.indexOf("."); //this finds the first occurrence of "." + if (iend != -1) { + accountName = accountName.substring(0 , iend); //this will give abc + } + credentials = new StorageCredentialsToken(accountName, tokenProvider.getToken().getAccessToken()); + return new CloudStorageAccount(credentials, true); + } + if (accountKey == null) { + if (allowAnonymous) { + credentials = StorageCredentialsAnonymous.ANONYMOUS; + } else { + LOG.warn("Skipping live Azure test because of missing key for" + + " account '" + accountName + "'."); + return null; + } + } else { + credentials = new StorageCredentialsAccountAndKey( + accountName.split("\\.")[0], accountKey); + } return new CloudStorageAccount(credentials); } @@ -540,6 +576,18 @@ static CloudStorageAccount createTestAccount(Configuration conf) return createStorageAccount(testAccountName, conf, false); } + static CloudStorageAccount createTestAccount(Configuration conf, NativeAzureFileSystem fs) + throws URISyntaxException, KeyProviderException, IOException { + AzureTestUtils.assumeNamespaceDisabled(conf); + + String testAccountName = verifyWasbAccountNameInConfig(conf); + if (testAccountName == null) { + LOG.warn("Skipping live Azure test because of missing test account"); + return null; + } + return createStorageAccount(testAccountName, conf, false, fs); + } + public static enum CreateOptions { UseSas, Readonly, CreateContainer, useThrottling } @@ -565,15 +613,12 @@ public static AzureBlobStorageTestAccount create( throws Exception { saveMetricsConfigFile(); NativeAzureFileSystem fs = null; + fs = new NativeAzureFileSystem(); CloudBlobContainer container = null; Configuration conf = createTestConfiguration(initialConfiguration); configurePageBlobDir(conf); configureAtomicRenameDir(conf); - CloudStorageAccount account = createTestAccount(conf); - if (account == null) { - return null; - } - fs = new NativeAzureFileSystem(); + CloudStorageAccount account = null; String containerName = useContainerSuffixAsContainerName ? containerNameSuffix : String.format( @@ -581,44 +626,45 @@ public static AzureBlobStorageTestAccount create( System.getProperty("user.name"), UUID.randomUUID().toString(), containerNameSuffix); + String accountName = verifyWasbAccountNameInConfig(conf); + // Check if throttling is turned on and set throttling parameters + // appropriately. + conf.setBoolean(KEY_DISABLE_THROTTLING, + !createOptions.contains(CreateOptions.useThrottling)); + +// // Set account URI and initialize Azure file system. + URI accountUri = createAccountUri(accountName, containerName); + fs.initialize(accountUri, conf); + if (fs.getStore().getTokenProviderStore() != null){ + account = createTestAccount(conf, fs); + } else { + account = createTestAccount(conf); + } + if (account == null) { + return null; + } container = account.createCloudBlobClient().getContainerReference( containerName); if (createOptions.contains(CreateOptions.CreateContainer)) { container.createIfNotExists(); } - String accountName = verifyWasbAccountNameInConfig(conf); if (createOptions.contains(CreateOptions.UseSas)) { String sas = generateSAS(container, createOptions.contains(CreateOptions.Readonly)); - if (!createOptions.contains(CreateOptions.CreateContainer)) { - // The caller doesn't want the container to be pre-created, - // so delete it now that we have generated the SAS. - container.delete(); - } - // Remove the account key from the configuration to make sure we don't - // cheat and use that. - // but only if not in secure mode, which requires that login if (!conf.getBoolean(AzureNativeFileSystemStore.KEY_USE_SECURE_MODE, false)) { conf.set(ACCOUNT_KEY_PROPERTY_NAME + accountName, ""); } // Set the SAS key. conf.set(SAS_PROPERTY_NAME + containerName + "." + accountName, sas); + fs.setConf(conf); + if (!createOptions.contains(CreateOptions.CreateContainer)) { + // The caller doesn't want the container to be pre-created, + // so delete it now that we have generated the SAS. + container.delete(); + } } - - // Check if throttling is turned on and set throttling parameters - // appropriately. - if (createOptions.contains(CreateOptions.useThrottling)) { - conf.setBoolean(KEY_DISABLE_THROTTLING, false); - } else { - conf.setBoolean(KEY_DISABLE_THROTTLING, true); - } - configureSecureModeTestSettings(conf); - // Set account URI and initialize Azure file system. - URI accountUri = createAccountUri(accountName, containerName); - fs.initialize(accountUri, conf); - // Create test account initializing the appropriate member variables. // AzureBlobStorageTestAccount testAcct = diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java index f86af954933626..ca3deefff695ac 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java @@ -189,7 +189,7 @@ public void run() { LOG.info("Freeing lease"); lease.free(); } - } catch (StorageException se) { + } catch (StorageException | IOException se) { LOG.warn("Unable to free lease.", se); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java index 6d11207c479b51..4f26777f512489 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java @@ -38,6 +38,7 @@ import org.apache.commons.codec.net.URLCodec; import org.apache.commons.lang3.NotImplementedException; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.http.client.utils.URIBuilder; import com.microsoft.azure.storage.AccessCondition; @@ -178,6 +179,13 @@ public CloudBlobContainerWrapper getContainerReference(String name) return container; } + @Override + public CloudBlobContainerWrapper getContainerReference(final String name, + final AccessTokenProvider tokenProvider) + throws URISyntaxException, StorageException { + return getContainerReference(name); + } + class MockCloudBlobContainerWrapper extends CloudBlobContainerWrapper { private boolean created = false; private HashMap metadata; @@ -234,6 +242,13 @@ public CloudBlobDirectoryWrapper getDirectoryReference(String relativePath) relativePath, true))); } + @Override + public CloudBlobDirectoryWrapper getDirectoryReference(String relativePath, AccessTokenProvider tokenProvider) + throws URISyntaxException, StorageException { + return new MockCloudBlobDirectoryWrapper(new URI(fullUriString( + relativePath, true))); + } + @Override public CloudBlockBlobWrapper getBlockBlobReference(String relativePath) throws URISyntaxException, StorageException { @@ -241,12 +256,27 @@ public CloudBlockBlobWrapper getBlockBlobReference(String relativePath) false)), null, 0); } + @Override + public CloudBlobWrapper getBlockBlobReference(final String relativePath, + final AccessTokenProvider tokenProvider) + throws URISyntaxException, StorageException, IOException { + return new MockCloudBlockBlobWrapper(new URI(fullUriString(relativePath, + false)), null, 0); + } + @Override public CloudPageBlobWrapper getPageBlobReference(String blobAddressUri) throws URISyntaxException, StorageException { return new MockCloudPageBlobWrapper(new URI(blobAddressUri), null, 0); } + @Override + public CloudBlobWrapper getPageBlobReference(String blobAddressUri, + final AccessTokenProvider tokenProvider) + throws URISyntaxException, StorageException, IOException { + return new MockCloudPageBlobWrapper(new URI(blobAddressUri), null, 0); + } + // helper to create full URIs for directory and blob. // use withTrailingSlash=true to get a good path for a directory. private String fullUriString(String relativePath, boolean withTrailingSlash) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java index 8ac36c299b65b5..a4dce44bde4055 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java @@ -1689,7 +1689,7 @@ public void run() { firstEndTime = System.currentTimeMillis(); lease.free(); LOG.info(name + " freed lease " + lease.getLeaseID()); - } catch (StorageException e) { + } catch (StorageException | IOException e) { fail("Unanticipated exception"); } } else if (name.equals("second-thread")) { @@ -1712,7 +1712,7 @@ public void run() { try { lease.free(); LOG.info(name + " freed lease " + lease.getLeaseID()); - } catch (StorageException e) { + } catch (StorageException | IOException e) { assertTrue("Unanticipated exception", false); } } else { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index fd2f2690daea21..b79e49390ac46f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -25,6 +25,9 @@ import java.util.UUID; import java.util.concurrent.Callable; +import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -38,8 +41,10 @@ import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.services.AuthType; +import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient; import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore; import org.apache.hadoop.fs.azure.NativeAzureFileSystem; import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation; @@ -55,6 +60,8 @@ import static org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.WASB_ACCOUNT_NAME_DOMAIN_SUFFIX; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.*; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DNS_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX; import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -69,7 +76,7 @@ public abstract class AbstractAbfsIntegrationTest extends AbstractAbfsTestWithTimeout { private static final Logger LOG = - LoggerFactory.getLogger(AbstractAbfsIntegrationTest.class); + LoggerFactory.getLogger(AbstractAbfsIntegrationTest.class); private boolean isIPAddress; private NativeAzureFileSystem wasb; @@ -106,7 +113,7 @@ protected AbstractAbfsIntegrationTest() throws Exception { if (authType == AuthType.SharedKey) { assumeTrue("Not set: " + FS_AZURE_ACCOUNT_KEY, - abfsConfig.get(FS_AZURE_ACCOUNT_KEY) != null); + abfsConfig.get(FS_AZURE_ACCOUNT_KEY) != null); // Update credentials } else { assumeTrue("Not set: " + FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME, @@ -239,6 +246,9 @@ public Hashtable call() throws Exception { } } + public AccessTokenProvider getAccessTokenProvider(final AzureBlobFileSystem fs) { + return ITestAbfsClient.getAccessTokenProvider(fs.getAbfsStore().getClient()); + } public void loadConfiguredFileSystem() throws Exception { // disable auto-creation of filesystem @@ -412,8 +422,7 @@ private static String convertTestUrls( } if (data != null) { - data = data.replace("." + fromDnsPrefix + ".", - "." + toDnsPrefix + "."); + data = data.replace(fromDnsPrefix, toDnsPrefix); } return data; } @@ -427,6 +436,24 @@ public AzureBlobFileSystemStore getAbfsStore(final AzureBlobFileSystem fs) { return fs.getAbfsStore(); } + public PrefixMode getPrefixMode(final AzureBlobFileSystem fs) { + return fs.getAbfsStore().getAbfsConfiguration().getPrefixMode(); + } + + public AbfsClient getClient(final AzureBlobFileSystem fs) { + return fs.getAbfsStore().getClient(); + } + + public boolean isNamespaceEnabled(final AzureBlobFileSystem fs) throws AzureBlobFileSystemException { + return fs.getAbfsStore() + .getIsNamespaceEnabled(getTestTracingContext(fs, true)); + } + + public void setAbfsClient(AzureBlobFileSystemStore abfsStore, + AbfsClient client) { + abfsStore.setClient(client); + } + public Path makeQualified(Path path) throws java.io.IOException { return getFileSystem().makeQualified(path); } @@ -469,7 +496,7 @@ protected AbfsOutputStream createAbfsOutputStreamWithFlushEnabled( return (AbfsOutputStream) abfss.createFile(path, fs.getFsStatistics(), true, FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()), - getTestTracingContext(fs, false)); + getTestTracingContext(fs, false), null); } /** @@ -486,4 +513,30 @@ protected long assertAbfsStatistics(AbfsStatistic statistic, (long) metricMap.get(statistic.getStatName())); return expectedValue; } + + /** + * For creating directory with implicit parents. Doesn't change already explicit + * parents. + */ + void createAzCopyDirectory(Path path) throws Exception { + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), getFileSystemName(), getFileSystem().getAbfsStore() + .getAbfsConfiguration() + .getRawConfiguration(), getFileSystem().getAbfsStore().getPrefixMode()); + azcopyHelper.createFolderUsingAzcopy( + getFileSystem().makeQualified(path).toUri().getPath().substring(1)); + } + + /** + * For creating files with implicit parents. Doesn't change already explicit + * parents. + */ + void createAzCopyFile(Path path) throws Exception { + AzcopyHelper azcopyHelper = new AzcopyHelper(getAccountName(), + getFileSystemName(), getFileSystem().getAbfsStore() + .getAbfsConfiguration() + .getRawConfiguration(), getFileSystem().getAbfsStore().getPrefixMode()); + azcopyHelper.createFileUsingAzcopy( + getFileSystem().makeQualified(path).toUri().getPath().substring(1)); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AzcopyHelper.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AzcopyHelper.java new file mode 100644 index 00000000000000..c15d094c83c907 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AzcopyHelper.java @@ -0,0 +1,217 @@ +/** + * 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 java.io.File; +import java.io.FileNotFoundException; +import java.io.FileWriter; +import java.io.IOException; + +import org.junit.Assume; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; + +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FORWARD_SLASH; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SAS_FIXED_TOKEN; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DNS_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX; + +public class AzcopyHelper { + + File hadoopAzureDir; + String azcopyDirPath; + private String accountName; + private String fileSystemName; + private Configuration configuration; + private PrefixMode mode; + + public AzcopyHelper(String accountName, String fileSystemName, Configuration configuration ,PrefixMode mode) throws Exception { + this.accountName = accountName.replace(ABFS_DNS_PREFIX, WASB_DNS_PREFIX); + this.fileSystemName = fileSystemName; + this.configuration = configuration; + this.mode = mode; + } + + public void downloadAzcopyExecutableIfNotPresent() throws IOException, InterruptedException { + // Skip execution if Prefix Mode is DFS. + Assume.assumeTrue(mode == PrefixMode.BLOB); + + // Find the hadoop-azure directory from the current working directory + File currentDir = new File(System.getProperty("user.dir")); + if (!currentDir.isDirectory() && !currentDir.getName().equals("hadoop-azure")) { + hadoopAzureDir = findHadoopAzureDir(currentDir); + if (hadoopAzureDir == null) { + throw new FileNotFoundException("hadoop-azure directory not found"); + } + } else { + hadoopAzureDir = currentDir; + } + + // Check if azcopy directory is present in the hadoop-azure directory, create it if it doesn't exist + azcopyDirPath = hadoopAzureDir.getAbsolutePath() + "/azcopy"; + File azcopyDir = new File(azcopyDirPath); + if (!azcopyDir.exists()) { + boolean created = azcopyDir.mkdir(); + // Check if azcopy is present in the azcopy directory + String azcopyPath = azcopyDirPath + "/azcopy"; + File azcopyFile = new File(azcopyPath); + if (!azcopyFile.exists()) { + // If azcopy is not present, download and extract it + String downloadUrl = "https://aka.ms/downloadazcopy-v10-linux"; + String downloadCmd = "wget " + downloadUrl + " -O azcopy.tar.gz" + " --no-check-certificate"; + String[] downloadCmdArr = {"bash", "-c", downloadCmd}; + Process downloadProcess = Runtime.getRuntime().exec(downloadCmdArr); + downloadProcess.waitFor(); + + // Extract the azcopy executable from the tarball + String extractCmd = "tar -xf azcopy.tar.gz -C " + hadoopAzureDir.getAbsolutePath(); + String[] extractCmdArr = {"bash", "-c", extractCmd}; + Process extractProcess = Runtime.getRuntime().exec(extractCmdArr); + extractProcess.waitFor(); + + // Rename the azcopy_linux_amd64_* directory to 'azcopy' and move it to the hadoop-azure directory + String renameCmd = "mv " + hadoopAzureDir.getAbsolutePath() + "/azcopy_linux_amd64_*/* " + azcopyDirPath; + String[] renameCmdArr = {"bash", "-c", renameCmd}; + Process renameProcess = Runtime.getRuntime().exec(renameCmdArr); + renameProcess.waitFor(); + + // Remove the downloaded tarball and azcopy folder + String cleanupCmd = "rm -rf " + hadoopAzureDir.getAbsolutePath() + "/azcopy_linux_amd64_* azcopy.tar.gz"; + String[] cleanupCmdArr = {"bash", "-c", cleanupCmd}; + Process cleanupProcess = Runtime.getRuntime().exec(cleanupCmdArr); + cleanupProcess.waitFor(); + + // Set the execute permission on the azcopy executable + String chmodCmd = "chmod +x " + azcopyDirPath; + String[] chmodCmdArr = {"bash", "-c", chmodCmd}; + Process chmodProcess = Runtime.getRuntime().exec(chmodCmdArr); + chmodProcess.waitFor(); + } + } + // Change working directory to the hadoop-azure directory + System.setProperty("user.dir", hadoopAzureDir.getAbsolutePath()); + } + + private File findHadoopAzureDir(File dir) { + if (dir == null) { + return null; + } + File[] files = dir.listFiles(); + if (files == null) { + return null; + } + for (File file : files) { + if (file.isDirectory() && file.getName().equals("hadoop-azure")) { + return file; + } else { + File hadoopAzureDir = findHadoopAzureDir(file); + if (hadoopAzureDir != null) { + return hadoopAzureDir; + } + } + } + return null; + } + + public void createFileOrFolder(String pathFromContainerRoot, boolean isFile) throws Exception { + downloadAzcopyExecutableIfNotPresent(); + String url = "https://" + accountName + FORWARD_SLASH + fileSystemName + FORWARD_SLASH + + pathFromContainerRoot; + // Add the SAS token in config file (should be Account SAS or Container SAS"). + String configuredFixedToken = configuration.get(FS_AZURE_SAS_FIXED_TOKEN, null); + if (configuredFixedToken != null) { + if (isFile) { + createFileCreationScript(azcopyDirPath, "createFile" + Thread.currentThread().getName() + ".sh", azcopyDirPath, configuredFixedToken, url); + } else { + createFolderCreationScript(azcopyDirPath, "createFolder" + Thread.currentThread().getName() + ".sh", azcopyDirPath, configuredFixedToken, url); + } + } else { + throw new Exception("The SAS token provided is null"); + } + String path; + if (isFile) { + path = azcopyDirPath + "/createFile" + Thread.currentThread().getName() + ".sh"; + } else { + path = azcopyDirPath + "/createFolder" + Thread.currentThread().getName() + ".sh"; + } + try { + ProcessBuilder pb = new ProcessBuilder(path); + Process p = pb.start(); + // wait for the process to finish + int exitCode = p.waitFor(); + } catch (IOException e) { + throw new IOException(e.getMessage()); + } catch (InterruptedException e) { + throw new InterruptedException(e.getMessage()); + } + String cleanupCmd = "rm -rf " + path; + String[] cleanupCmdArr = {"bash", "-c", cleanupCmd}; + Process cleanupProcess = Runtime.getRuntime().exec(cleanupCmdArr); + cleanupProcess.waitFor(); + } + + public void createFileUsingAzcopy(String pathFromContainerRoot) throws Exception { + // Add the path you want to copy to as config. + if (pathFromContainerRoot != null) { + createFileOrFolder(pathFromContainerRoot, true); + } + } + + public void createFolderUsingAzcopy(String pathFromContainerRoot) throws Exception { + // Add the path you want to copy to as config. + if (pathFromContainerRoot != null) { + createFileOrFolder(pathFromContainerRoot, false); + } + } + + public static void createFileCreationScript(String folderPath, String scriptName, String azcopyPath, String sasToken, String containerName) { + String blobPath = containerName + "?" + sasToken; // construct the blob path + String scriptContent = "blobPath=\"" + blobPath + "\"\n" + + "echo $blobPath\n" + + azcopyPath + "/azcopy copy \"" + azcopyPath + "/NOTICE.txt\" $blobPath\n"; // construct the script content + File scriptFile = new File(folderPath, scriptName); + try { + FileWriter writer = new FileWriter(scriptFile); + writer.write(scriptContent); + writer.close(); + boolean written = scriptFile.setExecutable(true); // make the script executable + System.out.println("Script created at " + scriptFile.getAbsolutePath()); + } catch (IOException e) { + e.printStackTrace(); + } + } + + public static void createFolderCreationScript(String folderPath, String scriptName, String azcopyPath, String sasToken, String containerName) { + String blobPath = containerName + "?" + sasToken; // construct the blob path + String scriptContent = "blobPath=\"" + blobPath + "\"\n" + + "echo $blobPath\n" + + azcopyPath + "/azcopy copy \"" + azcopyPath + "\" $blobPath --recursive\n"; // construct the script content + File scriptFile = new File(folderPath, scriptName); + try { + FileWriter writer = new FileWriter(scriptFile); + writer.write(scriptContent); + writer.close(); + boolean written = scriptFile.setExecutable(true); // make the script executable + System.out.println("Script created at " + scriptFile.getAbsolutePath()); + } catch (IOException e) { + e.printStackTrace(); + } + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/BlobDirectoryStateHelper.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/BlobDirectoryStateHelper.java new file mode 100644 index 00000000000000..6e174b36046cde --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/BlobDirectoryStateHelper.java @@ -0,0 +1,130 @@ +/** + * 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 java.io.IOException; +import java.net.HttpURLConnection; +import java.util.List; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.services.BlobProperty; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +public class BlobDirectoryStateHelper { + + /** + * To assert that a path exists as implicit directory we need two things to assert. + * 1. List blobs on the path should return some entries. + * 2. GetBlobProperties on path should fail. + * @param path to be checked + * @param fs AzureBlobFileSystem for API calls + * @return boolean whether the path exists as Implicit directory or not + */ + public static boolean isImplicitDirectory(Path path, AzureBlobFileSystem fs, TracingContext testTracingContext) throws Exception { + path = new Path(fs.makeQualified(path).toUri().getPath()); + if (fs.getAbfsStore().getPrefixMode() == PrefixMode.BLOB) { + List blobProperties = fs.getAbfsStore() + .getListBlobs(path,null, null, testTracingContext, 2, true); + if (blobProperties.size() == 0) { + return false; + } + try { + fs.getAbfsStore().getBlobProperty( + path, testTracingContext); + } + catch (AbfsRestOperationException ex) { + if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { + return true; + } + } + return false; + } + else { + FileStatus[] statuses = fs.getAbfsStore() + .listStatus(path, testTracingContext); + if (statuses.length == 0) { + return false; + } + try { + FileStatus status = fs.getAbfsStore().getFileStatus( + path, testTracingContext, true); + return !status.isDirectory(); + } + catch (AbfsRestOperationException ex) { + if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { + return true; + } + } + } + return false; + } + + /** + * To assert that a path exists as explicit directory + * For PrefixMode Blob: GetBlobProperties on path should succeed and marker should be present + * For PrefixMode DFS: GetFileStatus on path should succeed and marker should be present + * For Root on Blob: GetContainerProperty + * @param path to be checked + * @param fs AzureBlobFileSystem for API calls + * @return boolean whether the path exists as Implicit directory or not + */ + public static boolean isExplicitDirectory(Path path, AzureBlobFileSystem fs, TracingContext testTracingContext) { + path = new Path(fs.makeQualified(path).toUri().getPath()); + + if (fs.getAbfsStore().getPrefixMode() == PrefixMode.DFS) { + FileStatus status; + try { + status = fs.getAbfsStore() + .getFileStatus(path, testTracingContext, false); + } + catch (IOException ex) { + return false; + } + return status.isDirectory(); + } + + else if (path.isRoot()) { + BlobProperty prop; + try { + prop = fs.getAbfsStore().getContainerProperty(testTracingContext); + } + catch(AzureBlobFileSystemException ex) { + return false; + } + return prop.getIsDirectory(); + } + + else { + BlobProperty prop; + try { + prop = fs.getAbfsStore().getBlobProperty( + path, testTracingContext); + } + catch(AzureBlobFileSystemException ex) { + return false; + } + return prop.getIsDirectory(); + } + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsDurationTrackers.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsDurationTrackers.java index 0997b3dbd44d4b..afd91824e38e9e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsDurationTrackers.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsDurationTrackers.java @@ -101,11 +101,16 @@ public void testAbfsHttpCallsDurations() throws IOException { */ private void assertDurationTracker(IOStatistics ioStatistics) { for (AbfsStatistic abfsStatistic : HTTP_DURATION_TRACKER_LIST) { - Assertions.assertThat(lookupMeanStatistic(ioStatistics, - abfsStatistic.getStatName() + StoreStatisticNames.SUFFIX_MEAN).mean()) - .describedAs("The DurationTracker Named " + abfsStatistic.getStatName() - + " Doesn't match the expected value.") - .isGreaterThan(0.0); + // Since delete calls go to wasb endpoint and it doesn't have implementation for abfsStatistics + // we are avoiding incrementing stats for it. Can be reverted when we have blob implementation for + // delete. + if (!HTTP_DELETE_REQUEST.getStatName().equals(abfsStatistic.getStatName())) { + Assertions.assertThat(lookupMeanStatistic(ioStatistics, + abfsStatistic.getStatName() + StoreStatisticNames.SUFFIX_MEAN).mean()) + .describedAs("The DurationTracker Named " + abfsStatistic.getStatName() + + " Doesn't match the expected value.") + .isGreaterThan(0.0); + } } } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java index e2973912e23234..76e12b37f27455 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java @@ -33,13 +33,18 @@ import org.junit.Test; import org.mockito.Mockito; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator; import org.apache.hadoop.fs.azurebfs.services.ListingSupport; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DNS_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyList; @@ -344,4 +349,66 @@ private List createFilesUnderDirectory(int numFiles, Path rootPath, return fileNames; } + + @Test + public void testListStatusIteratorReturnStatusWithPathWithSameUriGivenInConfig() + throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + String accountName = getAccountName(); + Boolean isAccountNameInDfs = accountName.contains(ABFS_DNS_PREFIX); + String dnsAssertion; + if (isAccountNameInDfs) { + dnsAssertion = ABFS_DNS_PREFIX; + } else { + dnsAssertion = WASB_DNS_PREFIX; + } + + final Path path = new Path("/testDir/file"); + fs.create(path); + assertListStatusIteratorPath(fs, accountName, dnsAssertion, path); + + final Configuration configuration; + if (isAccountNameInDfs) { + configuration = new Configuration(getRawConfiguration()); + configuration.set(FS_DEFAULT_NAME_KEY, + configuration.get(FS_DEFAULT_NAME_KEY) + .replace(ABFS_DNS_PREFIX, WASB_DNS_PREFIX)); + dnsAssertion = WASB_DNS_PREFIX; + + } else { + configuration = new Configuration(getRawConfiguration()); + configuration.set(FS_DEFAULT_NAME_KEY, + configuration.get(FS_DEFAULT_NAME_KEY) + .replace(WASB_DNS_PREFIX, ABFS_DNS_PREFIX)); + dnsAssertion = ABFS_DNS_PREFIX; + } + fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + assertListStatusIteratorPath(fs, accountName, dnsAssertion, path); + } + + private void assertListStatusIteratorPath(final AzureBlobFileSystem fs, + final String accountName, + final String dnsAssertion, + final Path path) throws IOException { + RemoteIterator fileStatusRemoteIterator = fs.listStatusIterator( + new Path( + "abfs://" + fs.getAbfsClient().getFileSystem() + "@" + accountName + + path.getParent().toUri().getPath())); + + while (fileStatusRemoteIterator.hasNext()) { + FileStatus fileStatus = fileStatusRemoteIterator.next(); + Assertions.assertThat(fileStatus.getPath().toString()) + .contains(dnsAssertion); + } + + fileStatusRemoteIterator = fs.listStatusIterator(new Path( + "abfs://" + fs.getAbfsClient().getFileSystem() + "@" + accountName + + path.toUri().getPath())); + + while (fileStatusRemoteIterator.hasNext()) { + FileStatus fileStatus = fileStatusRemoteIterator.next(); + Assertions.assertThat(fileStatus.getPath().toString()) + .contains(dnsAssertion); + } + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsMsiTokenProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsMsiTokenProvider.java index d871befa430058..c93561005db7e5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsMsiTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsMsiTokenProvider.java @@ -19,15 +19,22 @@ package org.apache.hadoop.fs.azurebfs; import java.io.IOException; +import java.lang.reflect.Field; +import java.net.HttpURLConnection; import java.util.Date; - import org.junit.Test; +import org.mockito.MockedStatic; +import org.mockito.Mockito; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; +import org.apache.hadoop.fs.azurebfs.oauth2.AzureADAuthenticator; import org.apache.hadoop.fs.azurebfs.oauth2.AzureADToken; import org.apache.hadoop.fs.azurebfs.oauth2.MsiTokenProvider; +import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_OAUTH_TOKEN_FETCH_RETRY_MAX_ATTEMPTS; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.junit.Assume.assumeThat; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.not; @@ -40,6 +47,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT; +import static org.mockito.Mockito.times; /** * Test MsiTokenProvider. @@ -47,6 +55,8 @@ public final class ITestAbfsMsiTokenProvider extends AbstractAbfsIntegrationTest { + private static final int HTTP_TOO_MANY_REQUESTS = 429; + public ITestAbfsMsiTokenProvider() throws Exception { super(); } @@ -90,4 +100,109 @@ private String getTrimmedPasswordString(AbfsConfiguration conf, String key, return value.trim(); } + /** + * Test to verify that token fetch is retried for throttling errors (too many requests 429). + * @throws Exception + */ + @Test + public void testRetryForThrottling() throws Exception { + AbfsConfiguration conf = getConfiguration(); + + // Exception to be thrown with throttling error code 429. + AzureADAuthenticator.HttpException httpException + = new AzureADAuthenticator.HttpException(HTTP_TOO_MANY_REQUESTS, + "abc", "abc", "abc", "abc", "abc"); + + String tenantGuid = "abcd"; + String clientId = "abcd"; + String authEndpoint = getTrimmedPasswordString(conf, + FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT, + DEFAULT_FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT); + String authority = getTrimmedPasswordString(conf, + FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY, + DEFAULT_FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY); + + // Mock the getTokenSingleCall to throw exception so the retry logic comes into place. + try (MockedStatic adAuthenticator = Mockito.mockStatic( + AzureADAuthenticator.class, Mockito.CALLS_REAL_METHODS)) { + adAuthenticator.when( + () -> AzureADAuthenticator.getTokenSingleCall(Mockito.anyString(), + Mockito.anyString(), Mockito.any(), Mockito.anyString(), + Mockito.anyBoolean())).thenThrow(httpException); + + // Mock the tokenFetchRetryPolicy to verify retries. + ExponentialRetryPolicy exponentialRetryPolicy = Mockito.spy( + conf.getOauthTokenFetchRetryPolicy()); + Field tokenFetchRetryPolicy = AzureADAuthenticator.class.getDeclaredField( + "tokenFetchRetryPolicy"); + tokenFetchRetryPolicy.setAccessible(true); + tokenFetchRetryPolicy.set(ExponentialRetryPolicy.class, + exponentialRetryPolicy); + + AccessTokenProvider tokenProvider = new MsiTokenProvider(authEndpoint, + tenantGuid, clientId, authority); + AzureADToken token = null; + intercept(AzureADAuthenticator.HttpException.class, + tokenProvider::getToken); + + // If the status code doesn't qualify for retry shouldRetry returns false and the loop ends. + // It being called multiple times verifies that the retry was done for the throttling status code 429. + Mockito.verify(exponentialRetryPolicy, + times(DEFAULT_AZURE_OAUTH_TOKEN_FETCH_RETRY_MAX_ATTEMPTS + 1)) + .shouldRetry(Mockito.anyInt(), Mockito.anyInt()); + } + } + + /** + * Test to verify that token fetch is not retried for resource not found errors. + * @throws Exception + */ + @Test + public void testNoRetryForResourceNotFound() throws Exception { + AbfsConfiguration conf = getConfiguration(); + + // Exception to be thrown 404 error code. + AzureADAuthenticator.HttpException httpException + = new AzureADAuthenticator.HttpException(HttpURLConnection.HTTP_NOT_FOUND, + "abc", "abc", "abc", "abc", "abc"); + + String tenantGuid = "abcd"; + String clientId = "abcd"; + String authEndpoint = getTrimmedPasswordString(conf, + FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT, + DEFAULT_FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT); + String authority = getTrimmedPasswordString(conf, + FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY, + DEFAULT_FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY); + + // Mock the getTokenSingleCall to throw exception. + try (MockedStatic adAuthenticator = Mockito.mockStatic( + AzureADAuthenticator.class, Mockito.CALLS_REAL_METHODS)) { + adAuthenticator.when( + () -> AzureADAuthenticator.getTokenSingleCall(Mockito.anyString(), + Mockito.anyString(), Mockito.any(), Mockito.anyString(), + Mockito.anyBoolean())).thenThrow(httpException); + + // Mock the tokenFetchRetryPolicy to verify no retries. + ExponentialRetryPolicy exponentialRetryPolicy = Mockito.spy( + conf.getOauthTokenFetchRetryPolicy()); + Field tokenFetchRetryPolicy = AzureADAuthenticator.class.getDeclaredField( + "tokenFetchRetryPolicy"); + tokenFetchRetryPolicy.setAccessible(true); + tokenFetchRetryPolicy.set(ExponentialRetryPolicy.class, + exponentialRetryPolicy); + + AccessTokenProvider tokenProvider = new MsiTokenProvider(authEndpoint, + tenantGuid, clientId, authority); + AzureADToken token = null; + intercept(AzureADAuthenticator.HttpException.class, + tokenProvider::getToken); + + // If the status code doesn't qualify for retry shouldRetry returns false and the loop ends. + // It being called only once verifies that retry doesn't come into place.. + Mockito.verify(exponentialRetryPolicy, + times(1)) + .shouldRetry(Mockito.anyInt(), Mockito.anyInt()); + } + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java index 66b8da89572a1e..e7bed75f93143f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java @@ -21,6 +21,8 @@ import java.io.IOException; import java.util.Map; +import org.apache.hadoop.fs.azurebfs.services.OperativeEndpoint; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.junit.Test; @@ -127,8 +129,11 @@ public void testAbfsHttpSendStatistics() throws IOException { // Operation: AbfsOutputStream close. // Network Stats calculation: 1 flush (with close) is send. // 1 flush request = 1 connection and 1 send request - expectedConnectionsMade++; - expectedRequestsSent++; + if (OperativeEndpoint.isIngressEnabledOnDFS(getPrefixMode(fs), fs.getAbfsStore().getAbfsConfiguration())) { + // No network call made for flush without append in case of blob endpoint. + expectedConnectionsMade++; + expectedRequestsSent++; + } // -------------------------------------------------------------------- // Operation: Re-create the file / create overwrite scenario @@ -181,7 +186,6 @@ public void testAbfsHttpSendStatistics() throws IOException { assertAbfsStatistics(SEND_REQUESTS, expectedRequestsSent, metricMap); assertAbfsStatistics(AbfsStatistic.BYTES_SENT, expectedBytesSent, metricMap); } - } /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index f4f0f231037e18..beada775ae87b3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -32,11 +32,14 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_INFO; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.APPENDBLOB_MAX_WRITE_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtLevel; /** * Test read, write and seek. @@ -45,20 +48,29 @@ */ @RunWith(Parameterized.class) public class ITestAbfsReadWriteAndSeek extends AbstractAbfsScaleTest { - private static final Path TEST_PATH = new Path("/testfile"); - - @Parameterized.Parameters(name = "Size={0}") + private static final String TEST_PATH = "/testfile"; + + /** + * Parameterize on read buffer size and readahead. + * For test performance, a full x*y test matrix is not used. + * @return the test parameters + */ + @Parameterized.Parameters(name = "Size={0}-readahead={1}") public static Iterable sizes() { - return Arrays.asList(new Object[][]{{MIN_BUFFER_SIZE}, - {DEFAULT_READ_BUFFER_SIZE}, - {APPENDBLOB_MAX_WRITE_BUFFER_SIZE}, - {MAX_BUFFER_SIZE}}); + return Arrays.asList(new Object[][]{{MIN_BUFFER_SIZE, true}, + {DEFAULT_READ_BUFFER_SIZE, false}, + {DEFAULT_READ_BUFFER_SIZE, true}, + {APPENDBLOB_MAX_WRITE_BUFFER_SIZE, false}, + {MAX_BUFFER_SIZE, true}}); } private final int size; + private final boolean readaheadEnabled; - public ITestAbfsReadWriteAndSeek(final int size) throws Exception { + public ITestAbfsReadWriteAndSeek(final int size, + final boolean readaheadEnabled) throws Exception { this.size = size; + this.readaheadEnabled = readaheadEnabled; } @Test @@ -71,17 +83,25 @@ private void testReadWriteAndSeek(int bufferSize) throws Exception { final AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration(); abfsConfiguration.setWriteBufferSize(bufferSize); abfsConfiguration.setReadBufferSize(bufferSize); + abfsConfiguration.setReadAheadEnabled(readaheadEnabled); final byte[] b = new byte[2 * bufferSize]; new Random().nextBytes(b); - try (FSDataOutputStream stream = fs.create(TEST_PATH)) { + Path testPath = path(TEST_PATH); + FSDataOutputStream stream = fs.create(testPath); + try { stream.write(b); + } finally{ + stream.close(); } + logIOStatisticsAtLevel(LOG, IOSTATISTICS_LOGGING_LEVEL_INFO, stream); final byte[] readBuffer = new byte[2 * bufferSize]; int result; - try (FSDataInputStream inputStream = fs.open(TEST_PATH)) { + IOStatisticsSource statisticsSource = null; + try (FSDataInputStream inputStream = fs.open(testPath)) { + statisticsSource = inputStream; ((AbfsInputStream) inputStream.getWrappedStream()).registerListener( new TracingHeaderValidator(abfsConfiguration.getClientCorrelationId(), fs.getFileSystemId(), FSOperationType.READ, true, 0, @@ -99,6 +119,8 @@ private void testReadWriteAndSeek(int bufferSize) throws Exception { inputStream.seek(0); result = inputStream.read(readBuffer, 0, bufferSize); } + logIOStatisticsAtLevel(LOG, IOSTATISTICS_LOGGING_LEVEL_INFO, statisticsSource); + assertNotEquals("data read in final read()", -1, result); assertArrayEquals(readBuffer, b); } @@ -109,30 +131,35 @@ public void testReadAheadRequestID() throws java.io.IOException { final AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration(); int bufferSize = MIN_BUFFER_SIZE; abfsConfiguration.setReadBufferSize(bufferSize); + abfsConfiguration.setReadAheadEnabled(readaheadEnabled); final byte[] b = new byte[bufferSize * 10]; new Random().nextBytes(b); - try (FSDataOutputStream stream = fs.create(TEST_PATH)) { + Path testPath = path(TEST_PATH); + try (FSDataOutputStream stream = fs.create(testPath)) { ((AbfsOutputStream) stream.getWrappedStream()).registerListener( new TracingHeaderValidator(abfsConfiguration.getClientCorrelationId(), fs.getFileSystemId(), FSOperationType.WRITE, false, 0, ((AbfsOutputStream) stream.getWrappedStream()) .getStreamID())); stream.write(b); + logIOStatisticsAtLevel(LOG, IOSTATISTICS_LOGGING_LEVEL_INFO, stream); } + final byte[] readBuffer = new byte[4 * bufferSize]; int result; fs.registerListener( new TracingHeaderValidator(abfsConfiguration.getClientCorrelationId(), fs.getFileSystemId(), FSOperationType.OPEN, false, 0)); - try (FSDataInputStream inputStream = fs.open(TEST_PATH)) { + try (FSDataInputStream inputStream = fs.open(testPath)) { ((AbfsInputStream) inputStream.getWrappedStream()).registerListener( new TracingHeaderValidator(abfsConfiguration.getClientCorrelationId(), fs.getFileSystemId(), FSOperationType.READ, false, 0, ((AbfsInputStream) inputStream.getWrappedStream()) .getStreamID())); result = inputStream.read(readBuffer, 0, bufferSize*4); + logIOStatisticsAtLevel(LOG, IOSTATISTICS_LOGGING_LEVEL_INFO, inputStream); } fs.registerListener(null); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java index a71e7bc815f759..43702812fa2b38 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java @@ -18,8 +18,11 @@ package org.apache.hadoop.fs.azurebfs; +import java.io.FileNotFoundException; import java.io.IOException; +import java.util.Hashtable; +import org.apache.hadoop.fs.azurebfs.services.OperativeEndpoint; import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -31,6 +34,10 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +import org.mockito.Mockito; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.DOT; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION; @@ -45,8 +52,15 @@ public class ITestAbfsRestOperationException extends AbstractAbfsIntegrationTest{ private static final String RETRY_TEST_TOKEN_PROVIDER = "org.apache.hadoop.fs.azurebfs.oauth2.RetryTestTokenProvider"; + boolean useBlobEndpoint; public ITestAbfsRestOperationException() throws Exception { - super(); + super.setup(); + AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem()); + PrefixMode prefixMode = abfsStore.getPrefixMode(); + AbfsConfiguration abfsConfiguration = abfsStore.getAbfsConfiguration(); + useBlobEndpoint = !(OperativeEndpoint.isIngressEnabledOnDFS(prefixMode, abfsConfiguration) || + OperativeEndpoint.isMkdirEnabledOnDFS(abfsConfiguration) || + OperativeEndpoint.isReadEnabledOnDFS(abfsConfiguration)); } @Test @@ -60,12 +74,19 @@ public void testAbfsRestOperationExceptionFormat() throws IOException { String errorMessage = ex.getLocalizedMessage(); String[] errorFields = errorMessage.split(","); - Assert.assertEquals(4, errorFields.length); + // Expected Fields are: Message, StatusCode, Method, URL, ActivityId(rId) + Assert.assertEquals(5, errorFields.length); // Check status message, status code, HTTP Request Type and URL. - Assert.assertEquals("Operation failed: \"The specified path does not exist.\"", errorFields[0].trim()); + if (useBlobEndpoint) { + Assert.assertEquals("Operation failed: \"The specified blob does not exist.\"", errorFields[0].trim()); + } + else { + Assert.assertEquals("Operation failed: \"The specified path does not exist.\"", errorFields[0].trim()); + } Assert.assertEquals("404", errorFields[1].trim()); Assert.assertEquals("HEAD", errorFields[2].trim()); Assert.assertTrue(errorFields[3].trim().startsWith("http")); + Assert.assertTrue(errorFields[4].trim().startsWith("rId:")); } try { @@ -74,17 +95,63 @@ public void testAbfsRestOperationExceptionFormat() throws IOException { // verify its format String errorMessage = ex.getLocalizedMessage(); String[] errorFields = errorMessage.split(","); + // Flow is different for listStatusIterator being enabled or not. + if (!getAbfsStore(fs).getAbfsConfiguration().enableAbfsListIterator()) { + Assert.assertEquals(7, errorFields.length); + // Check status message, status code, HTTP Request Type and URL. + if (useBlobEndpoint) { + Assert.assertEquals("Operation failed: \"The specified blob does not exist.\"", errorFields[0].trim()); + } + else { + Assert.assertEquals("Operation failed: \"The specified path does not exist.\"", errorFields[0].trim()); + } + Assert.assertEquals("404", errorFields[1].trim()); + Assert.assertEquals("GET", errorFields[2].trim()); + Assert.assertTrue(errorFields[3].trim().startsWith("http")); + Assert.assertTrue(errorFields[4].trim().startsWith("rId:")); + // Check storage error code and storage error message. + Assert.assertEquals("PathNotFound", errorFields[4].trim()); + Assert.assertTrue(errorFields[5].contains("RequestId") + && errorFields[5].contains("Time")); + } else { + Assert.assertEquals(5, errorFields.length); + // Check status message, status code, HTTP Request Type and URL. + if (useBlobEndpoint) { + Assert.assertEquals("Operation failed: \"The specified blob does not exist.\"", errorFields[0].trim()); + } + else { + Assert.assertEquals("Operation failed: \"The specified path does not exist.\"", errorFields[0].trim()); + } + Assert.assertEquals("404", errorFields[1].trim()); + Assert.assertEquals("HEAD", errorFields[2].trim()); + Assert.assertTrue(errorFields[3].trim().startsWith("http")); + Assert.assertTrue(errorFields[4].trim().startsWith("rId:")); + } + } + // Check Exception Format For Put Method + try { + if (useBlobEndpoint) { + Hashtable metadata = new Hashtable<>(); + metadata.put("hi", "hello"); + fs.getAbfsStore().setBlobMetadata(fs.makeQualified(nonExistedFilePath1), + metadata, getTestTracingContext(fs, true)); + } + } catch (AbfsRestOperationException ex) { + String errorMessage = ex.getLocalizedMessage(); + String[] errorFields = errorMessage.split(","); - Assert.assertEquals(6, errorFields.length); + Assert.assertEquals(7, errorFields.length); // Check status message, status code, HTTP Request Type and URL. - Assert.assertEquals("Operation failed: \"The specified path does not exist.\"", errorFields[0].trim()); + if (useBlobEndpoint) { + Assert.assertEquals("Operation failed: \"The specified blob does not exist.\"", errorFields[0].trim()); + } + else { + Assert.assertEquals("Operation failed: \"The specified path does not exist.\"", errorFields[0].trim()); + } Assert.assertEquals("404", errorFields[1].trim()); - Assert.assertEquals("GET", errorFields[2].trim()); + Assert.assertEquals("PUT", errorFields[2].trim()); Assert.assertTrue(errorFields[3].trim().startsWith("http")); - // Check storage error code and storage error message. - Assert.assertEquals("PathNotFound", errorFields[4].trim()); - Assert.assertTrue(errorFields[5].contains("RequestId") - && errorFields[5].contains("Time")); + Assert.assertTrue(errorFields[4].trim().startsWith("rId:")); } } @@ -98,39 +165,43 @@ public void testCustomTokenFetchRetryCount() throws Exception { public void testWithDifferentCustomTokenFetchRetry(int numOfRetries) throws Exception { AzureBlobFileSystem fs = this.getFileSystem(); - Configuration config = new Configuration(this.getRawConfiguration()); + Configuration config = Mockito.spy(new Configuration(this.getRawConfiguration())); String accountName = config.get("fs.azure.abfs.account.name"); // Setup to configure custom token provider + config.set("fs.azure.account.auth.type", "Custom"); config.set("fs.azure.account.auth.type." + accountName, "Custom"); + config.set("fs.azure.account.oauth.provider.type", "org.apache.hadoop.fs" + + ".azurebfs.oauth2.RetryTestTokenProvider"); config.set("fs.azure.account.oauth.provider.type." + accountName, "org.apache.hadoop.fs" + ".azurebfs.oauth2.RetryTestTokenProvider"); config.set("fs.azure.custom.token.fetch.retry.count", Integer.toString(numOfRetries)); // Stop filesystem creation as it will lead to calls to store. config.set("fs.azure.createRemoteFileSystemDuringInitialization", "false"); - final AzureBlobFileSystem fs1 = - (AzureBlobFileSystem) FileSystem.newInstance(fs.getUri(), - config); - RetryTestTokenProvider.ResetStatusToFirstTokenFetch(); - intercept(Exception.class, - ()-> { - fs1.getFileStatus(new Path("/")); - }); + ()-> { + final AzureBlobFileSystem fs1 = + (AzureBlobFileSystem) FileSystem.newInstance(fs.getUri(), + config); + }); // Number of retries done should be as configured - Assert.assertTrue( - "Number of token fetch retries (" + RetryTestTokenProvider.reTryCount + Assert.assertEquals("Number of token fetch retries (" + RetryTestTokenProvider.reTryCount + ") done, does not match with fs.azure.custom.token.fetch.retry.count configured (" + numOfRetries - + ")", RetryTestTokenProvider.reTryCount == numOfRetries); + + ")", RetryTestTokenProvider.reTryCount, numOfRetries); + + RetryTestTokenProvider.ResetStatusToFirstTokenFetch(); } @Test public void testAuthFailException() throws Exception { - Configuration config = new Configuration(getRawConfiguration()); + Configuration config = Mockito.spy(new Configuration(getRawConfiguration())); String accountName = config .get(FS_AZURE_ABFS_ACCOUNT_NAME); // Setup to configure custom token provider + config.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, "Custom"); + config.set(FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME, + RETRY_TEST_TOKEN_PROVIDER); config.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME + DOT + accountName, "Custom"); config.set( @@ -139,9 +210,8 @@ public void testAuthFailException() throws Exception { // Stop filesystem creation as it will lead to calls to store. config.set(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, "false"); - final AzureBlobFileSystem fs = getFileSystem(config); try { - fs.getFileStatus(new Path("/")); + final AzureBlobFileSystem fs = getFileSystem(config); fail("Should fail at auth token fetch call"); } catch (AbfsRestOperationException e) { String errorDesc = "Should throw RestOp exception on AAD failure"; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java index e6b572de9717c4..ea0d074b6a1fd7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStatistics.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Map; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; import org.junit.Before; import org.junit.Test; @@ -102,9 +103,10 @@ public void testCreateStatistics() throws IOException { assertAbfsStatistics(AbfsStatistic.CALL_CREATE, 1, metricMap); assertAbfsStatistics(AbfsStatistic.CALL_CREATE_NON_RECURSIVE, 1, metricMap); assertAbfsStatistics(AbfsStatistic.FILES_CREATED, 1, metricMap); + // Child calls mkdirs for parent in case of blob. assertAbfsStatistics(AbfsStatistic.DIRECTORIES_CREATED, 1, metricMap); assertAbfsStatistics(AbfsStatistic.CALL_MKDIRS, 1, metricMap); - assertAbfsStatistics(AbfsStatistic.CALL_GET_FILE_STATUS, 2, metricMap); + assertAbfsStatistics(AbfsStatistic.CALL_GET_FILE_STATUS, 3, metricMap); //re-initialising Abfs to reset statistic values. fs.initialize(fs.getUri(), fs.getConf()); @@ -130,11 +132,12 @@ public void testCreateStatistics() throws IOException { assertAbfsStatistics(AbfsStatistic.CALL_CREATE_NON_RECURSIVE, NUMBER_OF_OPS, metricMap); assertAbfsStatistics(AbfsStatistic.FILES_CREATED, NUMBER_OF_OPS, metricMap); + // Child calls mkdirs for parent in case of blob. assertAbfsStatistics(AbfsStatistic.DIRECTORIES_CREATED, NUMBER_OF_OPS, - metricMap); + metricMap); assertAbfsStatistics(AbfsStatistic.CALL_MKDIRS, NUMBER_OF_OPS, metricMap); assertAbfsStatistics(AbfsStatistic.CALL_GET_FILE_STATUS, - 1 + NUMBER_OF_OPS, metricMap); + 1 + 2 * NUMBER_OF_OPS, metricMap); } /** @@ -170,9 +173,16 @@ public void testDeleteStatistics() throws IOException { since directory is delete recursively op_delete is called 2 times. 1 file is deleted, 1 listStatus() call is made. */ - assertAbfsStatistics(AbfsStatistic.CALL_DELETE, 2, metricMap); - assertAbfsStatistics(AbfsStatistic.FILES_DELETED, 1, metricMap); - assertAbfsStatistics(AbfsStatistic.CALL_LIST_STATUS, 1, metricMap); + // Calls go to wasb for now, will need to update this once goes to blob endpoint. + if (getPrefixMode(fs) == PrefixMode.BLOB) { + assertAbfsStatistics(AbfsStatistic.CALL_DELETE, 1, metricMap); + assertAbfsStatistics(AbfsStatistic.FILES_DELETED, 0, metricMap); + assertAbfsStatistics(AbfsStatistic.CALL_LIST_STATUS, 0, metricMap); + } else { + assertAbfsStatistics(AbfsStatistic.CALL_DELETE, 2, metricMap); + assertAbfsStatistics(AbfsStatistic.FILES_DELETED, 1, metricMap); + assertAbfsStatistics(AbfsStatistic.CALL_LIST_STATUS, 1, metricMap); + } /* creating a root directory and deleting it recursively to see if @@ -184,7 +194,12 @@ public void testDeleteStatistics() throws IOException { metricMap = fs.getInstrumentationMap(); //Test for directories_deleted. - assertAbfsStatistics(AbfsStatistic.DIRECTORIES_DELETED, 1, metricMap); + // Calls go to wasb for now, will need to update this once goes to blob endpoint. + if (getPrefixMode(fs) == PrefixMode.BLOB) { + assertAbfsStatistics(AbfsStatistic.DIRECTORIES_DELETED, 0, metricMap); + } else { + assertAbfsStatistics(AbfsStatistic.DIRECTORIES_DELETED, 1, metricMap); + } } /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java index 4b1e497f64f84b..fee6a3c2d68d17 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java @@ -20,8 +20,30 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; import java.util.Random; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +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.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; +import org.apache.hadoop.fs.azurebfs.services.OperativeEndpoint; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; +import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.junit.Assume; import org.junit.Test; import org.apache.hadoop.fs.FSDataOutputStream; @@ -29,6 +51,20 @@ import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.test.LambdaTestUtils; + +import org.mockito.Mockito; + +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_ENABLE_SMALL_WRITE_OPTIMIZATION; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_CONDITIONAL_CREATE_OVERWRITE; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_INFINITE_LEASE_KEY; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_THREADS; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.isNull; /** * Test append operations. @@ -37,6 +73,7 @@ public class ITestAzureBlobFileSystemAppend extends AbstractAbfsIntegrationTest { private static final Path TEST_FILE_PATH = new Path("testfile"); private static final Path TEST_FOLDER_PATH = new Path("testFolder"); + private static final String TEST_FILE = "testfile"; public ITestAzureBlobFileSystemAppend() throws Exception { super(); @@ -68,7 +105,6 @@ public void testAppendFileAfterDelete() throws Exception { final Path filePath = TEST_FILE_PATH; ContractTestUtils.touch(fs, filePath); fs.delete(filePath, false); - fs.append(filePath); } @@ -80,6 +116,108 @@ public void testAppendDirectory() throws Exception { fs.append(folderPath); } + @Test(expected = FileNotFoundException.class) + public void testAppendImplicitDirectory() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Path folderPath = TEST_FOLDER_PATH; + fs.mkdirs(folderPath); + fs.append(folderPath.getParent()); + } + + @Test(expected = FileNotFoundException.class) + public void testAppendFileNotExists() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.append(TEST_FOLDER_PATH); + } + + @Test(expected = IOException.class) + public void testIsAppendBlob() throws Exception { + final AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(true).when(store).isAppendBlobKey(anyString()); + fs.getAbfsStore().getAbfsConfiguration().setPrefixMode(PrefixMode.BLOB); + + // Set abfsStore as our mocked value. + Field privateField = AzureBlobFileSystem.class.getDeclaredField("abfsStore"); + privateField.setAccessible(true); + privateField.set(fs, store); + + fs.create(TEST_FILE_PATH); + } + + @Test(expected = IOException.class) + public void testSmallWriteBlob() throws Exception { + Configuration configuration = getRawConfiguration(); + configuration.set(AZURE_ENABLE_SMALL_WRITE_OPTIMIZATION, "true"); + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + fs.getAbfsStore().getAbfsConfiguration().setPrefixMode(PrefixMode.BLOB); + + fs.create(TEST_FILE_PATH); + } + + /** Create file over dfs endpoint and append over blob endpoint **/ + @Test + public void testCreateOverDfsAppendOverBlob() throws IOException { + final AzureBlobFileSystem fs = getFileSystem(); + Assume.assumeTrue(fs.getAbfsStore().getAbfsConfiguration().getPrefixMode() == PrefixMode.BLOB); + fs.getAbfsClient().createPath(makeQualified(TEST_FILE_PATH).toUri().getPath(), true, false, + null, null, false, + null, getTestTracingContext(fs, true)); + FSDataOutputStream outputStream = fs.append(TEST_FILE_PATH); + outputStream.write(10); + outputStream.hsync(); + outputStream.write(20); + outputStream.hsync(); + outputStream.write(30); + outputStream.hsync(); + } + + /** + * Create directory over dfs endpoint and append over blob endpoint. + * Should return error as append is not supported for directory. + * **/ + @Test(expected = IOException.class) + public void testCreateExplicitDirectoryOverDfsAppendOverBlob() throws IOException { + final AzureBlobFileSystem fs = getFileSystem(); + Assume.assumeTrue(fs.getAbfsStore().getAbfsConfiguration().getPrefixMode() == PrefixMode.BLOB); + fs.getAbfsClient().createPath(makeQualified(TEST_FOLDER_PATH).toUri().getPath(), false, false, + null, null, false, + null, getTestTracingContext(fs, true)); + FSDataOutputStream outputStream = fs.append(TEST_FOLDER_PATH); + outputStream.write(10); + outputStream.hsync(); + } + + /** + * Recreate file between append and flush. Etag mismatch happens. + **/ + @Test(expected = IOException.class) + public void testRecreateAppendAndFlush() throws IOException { + final AzureBlobFileSystem fs = getFileSystem(); + Assume.assumeTrue(!OperativeEndpoint.isIngressEnabledOnDFS(getPrefixMode(fs), getAbfsStore(fs).getAbfsConfiguration())); + fs.create(TEST_FILE_PATH); + FSDataOutputStream outputStream = fs.append(TEST_FILE_PATH); + outputStream.write(10); + final AzureBlobFileSystem fs1 = (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()); + FSDataOutputStream outputStream1 = fs1.create(TEST_FILE_PATH); + outputStream.hsync(); + } + + /** + * Recreate file between append and flush using dfs. Etag mismatch happens. + **/ + @Test(expected = IOException.class) + public void testRecreateDirectoryAppendAndFlush() throws IOException { + final AzureBlobFileSystem fs = getFileSystem(); + Assume.assumeTrue(fs.getAbfsStore().getAbfsConfiguration().getPrefixMode() == PrefixMode.BLOB); + fs.create(TEST_FILE_PATH); + FSDataOutputStream outputStream = fs.append(TEST_FILE_PATH); + outputStream.write(10); + final AzureBlobFileSystem fs1 = (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration()); + fs1.mkdirs(TEST_FILE_PATH); + outputStream.hsync(); + } + @Test public void testTracingForAppend() throws IOException { AzureBlobFileSystem fs = getFileSystem(); @@ -89,4 +227,507 @@ public void testTracingForAppend() throws IOException { fs.getFileSystemId(), FSOperationType.APPEND, false, 0)); fs.append(TEST_FILE_PATH, 10); } + + /** + * Verify that no calls to getBlockList were made. + */ + @Test + public void testCreateEmptyBlob() throws IOException { + Assume.assumeTrue(getFileSystem().getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + AbfsClient client = store.getClient(); + AbfsClient spiedClient = Mockito.spy(client); + store.setClient(spiedClient); + + fs.create(TEST_FILE_PATH); + Mockito.verify(spiedClient, Mockito.times(0)) + .getBlockList(Mockito.any(String.class), + Mockito.any(TracingContext.class)); + } + + @Test + public void testAppendImplicitDirectoryAzcopy() throws Exception { + Assume.assumeTrue(getFileSystem().getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + AzureBlobFileSystem fs = getFileSystem(); + createAzCopyDirectory(new Path("/src")); + createAzCopyFile(new Path("/src/file")); + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore().getBlobProperty(new Path("/src"), getTestTracingContext(fs, true)); + }); + intercept(FileNotFoundException.class, () -> fs.append(new Path("/src"))); + } + + /** + * Verify that no calls to getBlockList were made. + */ + @Test + public void testCreateNonEmptyBlob() throws IOException { + Assume.assumeTrue(getFileSystem().getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + AbfsClient client = store.getClient(); + AbfsClient testClient = Mockito.spy(ITestAbfsClient.createTestClientFromCurrentContext( + client, + fs.getAbfsStore().getAbfsConfiguration())); + store.setClient(testClient); + + FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH); + outputStream.write(10); + outputStream.hsync(); + outputStream.close(); + fs.append(TEST_FILE_PATH); + Mockito.verify(testClient, Mockito.times(1)) + .getBlockList(Mockito.any(String.class), + Mockito.any(TracingContext.class)); + } + + /** + * Verify that if getBlockList throws exception append should fail. + */ + @Test + public void testValidateGetBlockList() throws Exception { + Assume.assumeTrue(getFileSystem().getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + AbfsClient client = store.getClient(); + AbfsClient testClient = Mockito.spy(ITestAbfsClient.createTestClientFromCurrentContext( + client, + fs.getAbfsStore().getAbfsConfiguration())); + store.setClient(testClient); + + AzureBlobFileSystemException exception = Mockito.mock(AzureBlobFileSystemException.class); + // Throw exception when getBlockList is called + Mockito.doThrow(exception).when(testClient) + .getBlockList(Mockito.any(), Mockito.any(TracingContext.class)); + + // Create a non-empty file + FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH); + outputStream.write(10); + outputStream.hsync(); + outputStream.close(); + + intercept(AzureBlobFileSystemException.class, () -> fs.getAbfsStore() + .openFileForWrite(TEST_FILE_PATH, null, false, getTestTracingContext(fs, true))); + } + + /** + * Verify that parallel write with same offset from different output streams will not throw exception. + **/ + @Test + public void testParallelWriteSameOffsetDifferentOutputStreams() throws Exception { + Configuration configuration = getRawConfiguration(); + configuration.set(FS_AZURE_ENABLE_CONDITIONAL_CREATE_OVERWRITE, "false"); + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + ExecutorService executorService = Executors.newFixedThreadPool(5); + List> futures = new ArrayList<>(); + + final byte[] b = new byte[8 * ONE_MB]; + new Random().nextBytes(b); + + // Create three output streams + FSDataOutputStream out1 = fs.create(TEST_FILE_PATH); + FSDataOutputStream out2 = fs.append(TEST_FILE_PATH); + FSDataOutputStream out3 = fs.append(TEST_FILE_PATH); + + // Submit tasks to write to each output stream with the same offset + futures.add(executorService.submit(() -> { + try { + out1.write(b, 10, 200); + } catch (IOException e) { + throw new RuntimeException(e); + } + })); + + futures.add(executorService.submit(() -> { + try { + out2.write(b, 10, 200); + } catch (IOException e) { + throw new RuntimeException(e); + } + })); + + futures.add(executorService.submit(() -> { + try { + out3.write(b, 10, 200); + } catch (IOException e) { + throw new RuntimeException(e); + } + })); + + int exceptionCaught = 0; + for (Future future : futures) { + try { + future.get(); // wait for the task to complete and handle any exceptions thrown by the lambda expression + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + if (cause instanceof RuntimeException) { + exceptionCaught++; + } else { + System.err.println("Unexpected exception caught: " + cause); + } + } catch (InterruptedException e) { + // handle interruption + } + } + assertEquals(exceptionCaught, 0); + } + + + /** + * Verify that parallel write for different content length will not throw exception. + **/ + @Test + public void testParallelWriteDifferentContentLength() throws Exception { + Configuration configuration = getRawConfiguration(); + configuration.set(FS_AZURE_ENABLE_CONDITIONAL_CREATE_OVERWRITE, "false"); + FileSystem fs = FileSystem.newInstance(configuration); + ExecutorService executorService = Executors.newFixedThreadPool(5); + List> futures = new ArrayList<>(); + + // Create three output streams with different content length + FSDataOutputStream out1 = fs.create(TEST_FILE_PATH); + final byte[] b1 = new byte[8 * ONE_MB]; + new Random().nextBytes(b1); + + FSDataOutputStream out2 = fs.append(TEST_FILE_PATH); + FSDataOutputStream out3 = fs.append(TEST_FILE_PATH); + + // Submit tasks to write to each output stream + futures.add(executorService.submit(() -> { + try { + out1.write(b1, 10, 200); + } catch (IOException e) { + throw new RuntimeException(e); + } + })); + + futures.add(executorService.submit(() -> { + try { + out2.write(b1, 20, 300); + } catch (IOException e) { + throw new RuntimeException(e); + } + })); + + futures.add(executorService.submit(() -> { + try { + out3.write(b1, 30, 400); + } catch (IOException e) { + throw new RuntimeException(e); + } + })); + + int exceptionCaught = 0; + for (Future future : futures) { + try { + future.get(); // wait for the task to complete and handle any exceptions thrown by the lambda expression + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + if (cause instanceof RuntimeException) { + exceptionCaught++; + } else { + System.err.println("Unexpected exception caught: " + cause); + } + } catch (InterruptedException e) { + // handle interruption + } + } + assertEquals(exceptionCaught, 0); + } + + /** + * Verify that parallel write for different content length will not throw exception. + **/ + @Test + public void testParallelWriteOutputStreamClose() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + Assume.assumeTrue(!OperativeEndpoint.isIngressEnabledOnDFS(getPrefixMode(fs), getAbfsStore(fs).getAbfsConfiguration())); + final Path SECONDARY_FILE_PATH = new Path("secondarytestfile"); + ExecutorService executorService = Executors.newFixedThreadPool(2); + List> futures = new ArrayList<>(); + + FSDataOutputStream out1 = fs.create(SECONDARY_FILE_PATH); + AbfsOutputStream outputStream1 = (AbfsOutputStream) out1.getWrappedStream(); + String fileETag = outputStream1.getETag(); + final byte[] b1 = new byte[8 * ONE_MB]; + new Random().nextBytes(b1); + final byte[] b2 = new byte[8 * ONE_MB]; + new Random().nextBytes(b2); + + FSDataOutputStream out2 = fs.append(SECONDARY_FILE_PATH); + + // Submit tasks to write to each output stream + futures.add(executorService.submit(() -> { + try { + out1.write(b1, 0, 200); + out1.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + })); + + futures.add(executorService.submit(() -> { + try { + out2.write(b2, 0, 400); + out2.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + })); + + int exceptionCaught = 0; + + for (Future future : futures) { + try { + future.get(); // wait for the task to complete and handle any exceptions thrown by the lambda expression + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + if (cause instanceof RuntimeException) { + exceptionCaught++; + } else { + System.err.println("Unexpected exception caught: " + cause); + } + } catch (InterruptedException e) { + // handle interruption + } + } + + assertEquals(exceptionCaught, 1); + // Validate that the data written in the buffer is the same as what was read + final byte[] readBuffer = new byte[8 * ONE_MB]; + int result; + FSDataInputStream inputStream = fs.open(SECONDARY_FILE_PATH); + inputStream.seek(0); + + AbfsOutputStream outputStream2 = (AbfsOutputStream) out1.getWrappedStream(); + String out1Etag = outputStream2.getETag(); + + AbfsOutputStream outputStream3 = (AbfsOutputStream) out2.getWrappedStream(); + String out2Etag = outputStream3.getETag(); + + if (!fileETag.equals(out1Etag)) { + result = inputStream.read(readBuffer, 0, 4 * ONE_MB); + assertEquals(result, 200); // Verify that the number of bytes read matches the number of bytes written + assertArrayEquals(Arrays.copyOfRange(readBuffer, 0, result), Arrays.copyOfRange(b1, 0, result)); // Verify that the data read matches the original data written + } else if (!fileETag.equals(out2Etag)) { + result = inputStream.read(readBuffer, 0, 4 * ONE_MB); + assertEquals(result, 400); // Verify that the number of bytes read matches the number of bytes written + assertArrayEquals(Arrays.copyOfRange(readBuffer, 0, result), Arrays.copyOfRange(b2, 0, result)); // Verify that the data read matches the original data written + } else { + fail("Neither out1 nor out2 was flushed successfully."); + } + } + + /** + * Verify that once flushed etag changes. + **/ + @Test + public void testEtagMismatch() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + Assume.assumeTrue(!OperativeEndpoint.isIngressEnabledOnDFS(getPrefixMode(fs), getAbfsStore(fs).getAbfsConfiguration())); + FSDataOutputStream out1 = fs.create(TEST_FILE_PATH); + FSDataOutputStream out2 = fs.create(TEST_FILE_PATH); + + out2.write(10); + out2.hsync(); + out1.write(10); + intercept(IOException.class, () -> out1.hsync()); + } + + /** + * Verify that for flush without append no network calls are made. + **/ + @Test + public void testNoNetworkCallsForFlush() throws Exception { + Assume.assumeTrue(getFileSystem().getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + AbfsClient client = store.getClient(); + AbfsClient spiedClient = Mockito.spy(client); + store.setClient(spiedClient); + + FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH); + outputStream.hsync(); + Mockito.verify(spiedClient, Mockito.times(0)) + .append(Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any(TracingContext.class), Mockito.any()); + Mockito.verify(spiedClient, Mockito.times(0)). + flush(any(byte[].class), anyString(), anyBoolean(), isNull(), isNull(), any(), + any(TracingContext.class)); + } + + /** + * Verify that for flush without append no network calls are made. + **/ + @Test + public void testNoNetworkCallsForSecondFlush() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + Assume.assumeTrue(!OperativeEndpoint.isIngressEnabledOnDFS(getPrefixMode(fs), getAbfsStore(fs).getAbfsConfiguration())); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + AbfsClient client = store.getClient(); + AbfsClient spiedClient = Mockito.spy(client); + store.setClient(spiedClient); + + FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH); + outputStream.write(10); + outputStream.hsync(); + outputStream.close(); + Mockito.verify(spiedClient, Mockito.times(1)) + .append(Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any(TracingContext.class), Mockito.any()); + Mockito.verify(spiedClient, Mockito.times(1)). + flush(any(byte[].class), anyString(), anyBoolean(), isNull(), isNull(), any(), + any(TracingContext.class)); + } + + private AzureBlobFileSystem getCustomFileSystem(Path infiniteLeaseDirs, int numLeaseThreads) throws Exception { + Configuration conf = getRawConfiguration(); + conf.setBoolean(String.format("fs.%s.impl.disable.cache", getAbfsScheme()), true); + conf.set(FS_AZURE_INFINITE_LEASE_KEY, infiniteLeaseDirs.toUri().getPath()); + conf.setInt(FS_AZURE_LEASE_THREADS, numLeaseThreads); + return getFileSystem(conf); + } + + @Test + public void testAppendWithLease() throws Exception { + Assume.assumeTrue(getFileSystem().getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); + final AzureBlobFileSystem fs = Mockito.spy(getCustomFileSystem(testFilePath.getParent(), 1)); + AbfsOutputStream outputStream = (AbfsOutputStream) fs.getAbfsStore().createFile(testFilePath, null, true, + null, null, getTestTracingContext(fs, true), null); + outputStream.write(10); + outputStream.close(); + assertNotNull(outputStream.getLeaseId()); + } + + + /** + * If a write operation fails asynchronously, when the next write comes once failure is + * registered, that operation would fail with the exception caught on previous + * write operation. + * The next close, hsync, hflush would also fail for the last caught exception. + */ + @Test + public void testIntermittentAppendFailureToBeReported() throws Exception { + AzureBlobFileSystem fs = Mockito.spy( + (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration())); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient spiedClient = Mockito.spy(store.getClient()); + store.setClient(spiedClient); + Mockito.doReturn(store).when(fs).getAbfsStore(); + Mockito.doThrow( + new AbfsRestOperationException(503, "", "", new Exception())) + .when(spiedClient) + .append(Mockito.anyString(), Mockito.anyString(), + Mockito.any(byte[].class), Mockito.any( + AppendRequestParameters.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class), Mockito.nullable(String.class)); + + Mockito.doThrow( + new AbfsRestOperationException(503, "", "", new Exception())) + .when(spiedClient) + .append(Mockito.anyString(), Mockito.any(byte[].class), Mockito.any( + AppendRequestParameters.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + + byte[] bytes = new byte[1024 * 1024 * 8]; + new Random().nextBytes(bytes); + LambdaTestUtils.intercept(IOException.class, () -> { + try (FSDataOutputStream os = fs.create(new Path("/test/file"))) { + os.write(bytes); + } + }); + + LambdaTestUtils.intercept(IOException.class, () -> { + FSDataOutputStream os = fs.create(new Path("/test/file")); + os.write(bytes); + os.close(); + }); + + LambdaTestUtils.intercept(IOException.class, () -> { + FSDataOutputStream os = fs.create(new Path("/test/file")); + os.write(bytes); + os.hsync(); + }); + + LambdaTestUtils.intercept(IOException.class, () -> { + FSDataOutputStream os = fs.create(new Path("/test/file")); + os.write(bytes); + os.hflush(); + }); + + LambdaTestUtils.intercept(IOException.class, () -> { + AbfsOutputStream os = (AbfsOutputStream) fs.create(new Path("/test/file")) + .getWrappedStream(); + os.write(bytes); + while (!os.areWriteOperationsTasksDone()) ; + os.write(bytes); + }); + } + + /** + * Test to check when async write takes time, the close, hsync, hflush method + * wait to get async ops completed and then flush. If async ops fail, the methods + * will throw exception. + */ + @Test + public void testWriteAsyncOpFailedAfterCloseCalled() throws Exception { + AzureBlobFileSystem fs = Mockito.spy( + (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration())); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient spiedClient = Mockito.spy(store.getClient()); + store.setClient(spiedClient); + Mockito.doReturn(store).when(fs).getAbfsStore(); + + byte[] bytes = new byte[1024 * 1024 * 8]; + new Random().nextBytes(bytes); + + AtomicInteger count = new AtomicInteger(0); + + Mockito.doAnswer(answer -> { + count.incrementAndGet(); + while (count.get() < 2) ; + Thread.sleep(1000); + throw new AbfsRestOperationException(503, "", "", new Exception()); + }) + .when(spiedClient) + .append(Mockito.anyString(), Mockito.anyString(), + Mockito.any(byte[].class), Mockito.any( + AppendRequestParameters.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class), Mockito.nullable(String.class)); + + Mockito.doAnswer(answer -> { + count.incrementAndGet(); + while (count.get() < 2) ; + Thread.sleep(1000); + throw new AbfsRestOperationException(503, "", "", new Exception()); + }) + .when(spiedClient) + .append(Mockito.anyString(), Mockito.any(byte[].class), Mockito.any( + AppendRequestParameters.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + + FSDataOutputStream os = fs.create(new Path("/test/file")); + os.write(bytes); + os.write(bytes); + LambdaTestUtils.intercept(IOException.class, os::close); + + count.set(0); + FSDataOutputStream os1 = fs.create(new Path("/test/file1")); + os1.write(bytes); + os1.write(bytes); + LambdaTestUtils.intercept(IOException.class, os1::hsync); + + count.set(0); + FSDataOutputStream os2 = fs.create(new Path("/test/file2")); + os2.write(bytes); + os2.write(bytes); + LambdaTestUtils.intercept(IOException.class, os2::hflush); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java index beb7d0ebaaa8ea..07a8799f1c19f5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAttributes.java @@ -19,6 +19,8 @@ package org.apache.hadoop.fs.azurebfs; import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.EnumSet; import org.junit.Assume; @@ -26,8 +28,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.XAttrSetFlag; -import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; -import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -42,70 +43,210 @@ public ITestAzureBlobFileSystemAttributes() throws Exception { super(); } + /** + * Test GetXAttr() and SetXAttr() with Unicode Attribute Values. + * DFS does not support Unicode characters in user-defined metadata properties. + * Blob Endpoint supports Unicode encoded in UTF_8 character encoding. + * @throws Exception + */ @Test - public void testSetGetXAttr() throws Exception { + public void testGetSetXAttr() throws Exception { AzureBlobFileSystem fs = getFileSystem(); - AbfsConfiguration conf = fs.getAbfsStore().getAbfsConfiguration(); - Assume.assumeTrue(getIsNamespaceEnabled(fs)); - - byte[] attributeValue1 = fs.getAbfsStore().encodeAttribute("hi"); - byte[] attributeValue2 = fs.getAbfsStore().encodeAttribute("你好"); - String attributeName1 = "user.asciiAttribute"; - String attributeName2 = "user.unicodeAttribute"; - Path testFile = path("setGetXAttr"); - - // after creating a file, the xAttr should not be present - touch(testFile); - assertNull(fs.getXAttr(testFile, attributeName1)); - - // after setting the xAttr on the file, the value should be retrievable - fs.registerListener( - new TracingHeaderValidator(conf.getClientCorrelationId(), - fs.getFileSystemId(), FSOperationType.SET_ATTR, true, 0)); - fs.setXAttr(testFile, attributeName1, attributeValue1); - fs.setListenerOperation(FSOperationType.GET_ATTR); - assertArrayEquals(attributeValue1, fs.getXAttr(testFile, attributeName1)); - fs.registerListener(null); - - // after setting a second xAttr on the file, the first xAttr values should not be overwritten - fs.setXAttr(testFile, attributeName2, attributeValue2); - assertArrayEquals(attributeValue1, fs.getXAttr(testFile, attributeName1)); - assertArrayEquals(attributeValue2, fs.getXAttr(testFile, attributeName2)); + final Path testPath = new Path("a/b"); + fs.create(testPath); + testGetSetXAttrHelper(fs, testPath, testPath); + } + + @Test + public void testGetSetXAttrOnRoot() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + // TODO: Support SetXAttr() on root on DFS endpoint + Assume.assumeTrue(fs.getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + final Path filePath = new Path("a/b"); + final Path testPath = new Path("/"); + fs.create(filePath); + testGetSetXAttrHelper(fs, filePath, testPath); + } + + @Test + public void testGetSetXAttrOnImplicitDir() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + final Path testPath = new Path("a/b"); + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getRawConfiguration(), + fs.getAbfsStore().getPrefixMode() + ); + + azcopyHelper.createFolderUsingAzcopy(fs.makeQualified(testPath).toUri().getPath().substring(1)); + // Assert that the folder is implicit + BlobDirectoryStateHelper.isExplicitDirectory(testPath, fs, getTestTracingContext(fs, true)); + testGetSetXAttrHelper(fs, testPath, testPath); + + // Assert that the folder is now explicit + BlobDirectoryStateHelper.isExplicitDirectory(testPath, fs, getTestTracingContext(fs, true)); + } + + /** + * Test that setting metadata over marker blob do not override + * x-ms-meta-hdi_IsFolder + * @throws Exception + */ + @Test + public void testSetXAttrOverMarkerBlob() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + final Path testPath = new Path("ab"); + fs.mkdirs(testPath); + testGetSetXAttrHelper(fs, testPath, testPath); + + // Assert that the folder is now explicit + BlobDirectoryStateHelper.isExplicitDirectory(testPath, fs, getTestTracingContext(fs, true)); + } + + private void testGetSetXAttrHelper(final AzureBlobFileSystem fs, + final Path filePath, final Path testPath) throws Exception { + + String attributeName1 = "user.attribute1"; + String attributeName2 = "user.attribute2"; + String decodedAttributeValue1; + String decodedAttributeValue2; + byte[] attributeValue1; + byte[] attributeValue2; + + if(fs.getAbfsStore().getPrefixMode() == PrefixMode.BLOB) { + Assume.assumeTrue(!getIsNamespaceEnabled(fs)); // Blob endpoint Currently Supports FNS only + decodedAttributeValue1 = "hi"; + decodedAttributeValue2 = "hello"; //Блюз //你好 + // TODO: Modify them to unicode characters when support is added + attributeValue1 = fs.getAbfsStore().encodeAttribute(decodedAttributeValue1); + attributeValue2 = fs.getAbfsStore().encodeAttribute(decodedAttributeValue2); + } + else { + decodedAttributeValue1 = "hi"; + decodedAttributeValue2 = "hello"; // DFS Endpoint only Supports ASCII + attributeValue1 = fs.getAbfsStore().encodeAttribute(decodedAttributeValue1); + attributeValue2 = fs.getAbfsStore().encodeAttribute(decodedAttributeValue2); + } + + // Attribute not present initially + assertNull(fs.getXAttr(testPath, attributeName1)); + assertNull(fs.getXAttr(testPath, attributeName2)); + + // Set the Attributes + fs.setXAttr(testPath, attributeName1, attributeValue1); + + // Check if the attribute is retrievable + byte[] rv = fs.getXAttr(testPath, attributeName1); + assertTrue(Arrays.equals(rv, attributeValue1)); + assertEquals(new String(rv, StandardCharsets.UTF_8), decodedAttributeValue1); + + // Set the second Attribute + fs.setXAttr(testPath, attributeName2, attributeValue2); + + // Check all the attributes present and previous Attribute not overridden + rv = fs.getXAttr(testPath, attributeName1); + assertTrue(Arrays.equals(rv, attributeValue1)); + assertEquals(new String(rv, StandardCharsets.UTF_8), decodedAttributeValue1); + rv = fs.getXAttr(testPath, attributeName2); + assertTrue(Arrays.equals(rv, attributeValue2)); + assertEquals(new String(rv, StandardCharsets.UTF_8), decodedAttributeValue2); + } + + /** + * Trying to set same attribute multiple times should result in no failure + * @throws Exception + */ + @Test + public void testSetXAttrMultipleOperations() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + final Path path = new Path("a/b"); + fs.create(path); + + String attributeName1 = "user.attribute1"; + byte[] attributeValue1; + String decodedAttributeValue1 = "hi"; + + if(fs.getAbfsStore().getPrefixMode() == PrefixMode.BLOB) { + Assume.assumeTrue(!getIsNamespaceEnabled(fs)); + // TODO: Modify them to unicode characters when support is added + attributeValue1 = fs.getAbfsStore().encodeAttribute("hi"); + } + else { + attributeValue1 = fs.getAbfsStore().encodeAttribute("hi"); + } + + // Attribute not present initially + assertNull(fs.getXAttr(path, attributeName1)); + + // Set the Attributes Multiple times + // Filesystem internally adds create and replace flags + fs.setXAttr(path, attributeName1, attributeValue1); + fs.setXAttr(path, attributeName1, attributeValue1); + + // Check if the attribute is retrievable + byte[] rv = fs.getXAttr(path, attributeName1); + assertTrue(Arrays.equals(rv, attributeValue1)); + assertEquals(new String(rv, StandardCharsets.UTF_8), decodedAttributeValue1); } @Test public void testSetGetXAttrCreateReplace() throws Exception { AzureBlobFileSystem fs = getFileSystem(); - Assume.assumeTrue(getIsNamespaceEnabled(fs)); - byte[] attributeValue = fs.getAbfsStore().encodeAttribute("one"); - String attributeName = "user.someAttribute"; - Path testFile = path("createReplaceXAttr"); + final Path testFile = new Path("a/b"); + + String attributeName = "user.attribute1"; + String decodedAttributeValue1; + byte[] attributeValue; + + if(fs.getAbfsStore().getPrefixMode() == PrefixMode.BLOB) { + Assume.assumeTrue(!getIsNamespaceEnabled(fs)); // Blob endpoint Currently Supports FNS only + decodedAttributeValue1 = "hi"; + attributeValue = decodedAttributeValue1.getBytes(StandardCharsets.UTF_8); + } + else { + decodedAttributeValue1 = "hi"; + attributeValue = fs.getAbfsStore().encodeAttribute(decodedAttributeValue1); + } // after creating a file, it must be possible to create a new xAttr - touch(testFile); + fs.create(testFile); fs.setXAttr(testFile, attributeName, attributeValue, CREATE_FLAG); assertArrayEquals(attributeValue, fs.getXAttr(testFile, attributeName)); - // however after the xAttr is created, creating it again must fail + // however, after the xAttr is created, creating it again must fail intercept(IOException.class, () -> fs.setXAttr(testFile, attributeName, attributeValue, CREATE_FLAG)); } @Test public void testSetGetXAttrReplace() throws Exception { AzureBlobFileSystem fs = getFileSystem(); - Assume.assumeTrue(getIsNamespaceEnabled(fs)); - byte[] attributeValue1 = fs.getAbfsStore().encodeAttribute("one"); - byte[] attributeValue2 = fs.getAbfsStore().encodeAttribute("two"); - String attributeName = "user.someAttribute"; - Path testFile = path("replaceXAttr"); + final Path testFile = new Path("a/b"); + + String attributeName = "user.attribute1"; + String decodedAttributeValue1 = "one"; + String decodedAttributeValue2 = "two"; + + byte[] attributeValue1; + byte[] attributeValue2; + + if(fs.getAbfsStore().getPrefixMode() == PrefixMode.BLOB) { + Assume.assumeTrue(!getIsNamespaceEnabled(fs)); // Blob endpoint Currently Supports FNS only + attributeValue1 = decodedAttributeValue1.getBytes(StandardCharsets.UTF_8); + attributeValue2 = decodedAttributeValue2.getBytes(StandardCharsets.UTF_8); + } + else { + attributeValue1 = fs.getAbfsStore().encodeAttribute(decodedAttributeValue1); + attributeValue2 = fs.getAbfsStore().encodeAttribute(decodedAttributeValue2); + } // after creating a file, it must not be possible to replace an xAttr intercept(IOException.class, () -> { - touch(testFile); + fs.create(testFile); fs.setXAttr(testFile, attributeName, attributeValue1, REPLACE_FLAG); }); - // however after the xAttr is created, replacing it must succeed + // however, after the xAttr is created, replacing it must succeed fs.setXAttr(testFile, attributeName, attributeValue1, CREATE_FLAG); fs.setXAttr(testFile, attributeName, attributeValue2, REPLACE_FLAG); assertArrayEquals(attributeValue2, fs.getXAttr(testFile, attributeName)); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAuthorization.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAuthorization.java index 589c3a285f48f4..449574189ad538 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAuthorization.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAuthorization.java @@ -95,12 +95,10 @@ public void testSASTokenProviderEmptySASToken() throws Exception { testConfig.set(ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, TEST_ERR_AUTHZ_CLASS); testConfig.set(MOCK_SASTOKENPROVIDER_RETURN_EMPTY_SAS_TOKEN, "true"); - testFs.initialize(fs.getUri(), - this.getConfiguration().getRawConfiguration()); - intercept(SASTokenProviderException.class, - () -> { - testFs.create(new org.apache.hadoop.fs.Path("/testFile")); - }); + intercept(SASTokenProviderException.class, () -> { + testFs.initialize(fs.getUri(), + this.getConfiguration().getRawConfiguration()); + }); } @Test @@ -111,11 +109,9 @@ public void testSASTokenProviderNullSASToken() throws Exception { Configuration testConfig = this.getConfiguration().getRawConfiguration(); testConfig.set(ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE, TEST_ERR_AUTHZ_CLASS); - testFs.initialize(fs.getUri(), this.getConfiguration().getRawConfiguration()); - intercept(SASTokenProviderException.class, - ()-> { - testFs.create(new org.apache.hadoop.fs.Path("/testFile")); - }); + intercept(SASTokenProviderException.class, () -> { + testFs.initialize(fs.getUri(), this.getConfiguration().getRawConfiguration()); + }); } @Test diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBlobConfig.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBlobConfig.java new file mode 100644 index 00000000000000..e88aa1c511e554 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBlobConfig.java @@ -0,0 +1,731 @@ +/** + * 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 java.io.IOException; +import java.util.HashMap; +import java.util.Random; + +import org.apache.hadoop.fs.azurebfs.services.OperativeEndpoint; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Test; +import org.mockito.Mockito; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azure.NativeAzureFileSystem; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_BLOB_ENDPOINT; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_INGRESS_FALLBACK_TO_DFS; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_MKDIRS_FALLBACK_TO_DFS; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_REDIRECT_DELETE; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_REDIRECT_RENAME; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_FS_AZURE_ENABLE_BLOBENDPOINT; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DNS_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX; + +public class ITestAzureBlobFileSystemBlobConfig + extends AbstractAbfsIntegrationTest { + + public ITestAzureBlobFileSystemBlobConfig() throws Exception { + super(); + } + + @Override + public void setup() throws Exception { + super.setup(); + Assume.assumeFalse(getFileSystem().getIsNamespaceEnabled( + Mockito.mock(TracingContext.class))); + } + + @Test + public void testDfsEndpointWhenBlobEndpointConfigIsDisabled() + throws Exception { + AzureBlobFileSystem fs = createFileSystemForEndpointConfigPair( + FS_AZURE_ENABLE_BLOB_ENDPOINT, false, true); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + fs.getAbfsStore().setClient(client); + fs.create(new Path("/tmp")); + verifyCreatePathExecution(client); + } + + private void verifyCreatePathExecution(final AbfsClient client) + throws AzureBlobFileSystemException { + Mockito.verify(client, Mockito.times(1)) + .createPath(Mockito.anyString(), Mockito.anyBoolean(), + Mockito.anyBoolean(), Mockito.nullable(String.class), + Mockito.nullable(String.class), + Mockito.anyBoolean(), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + } + + @Test + public void testDfsEndpointWhenBlobEndpointConfigIsEnabled() + throws Exception { + AzureBlobFileSystem fs = createFileSystemForEndpointConfigPair( + FS_AZURE_ENABLE_BLOB_ENDPOINT, true, true); + Assume.assumeTrue(!OperativeEndpoint.isIngressEnabledOnDFS(getPrefixMode(fs), fs.getAbfsStore().getAbfsConfiguration())); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + fs.getAbfsStore().setClient(client); + fs.create(new Path("/tmp")); + verifyCreatePathBlobExecution(client); + } + + @Test + public void testBlobEndpointWhenBlobEndpointConfigIsEnabled() + throws Exception { + AzureBlobFileSystem fs = createFileSystemForEndpointConfigPair( + FS_AZURE_ENABLE_BLOB_ENDPOINT, true, false); + Assume.assumeTrue(!OperativeEndpoint.isIngressEnabledOnDFS(getPrefixMode(fs), fs.getAbfsStore().getAbfsConfiguration())); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + fs.getAbfsStore().setClient(client); + fs.create(new Path("/tmp")); + verifyCreatePathBlobExecution(client); + } + + @Test + public void testBlobEndpointWhenBlobEndpointConfigIsDisabled() + throws Exception { + AzureBlobFileSystem fs = createFileSystemForEndpointConfigPair( + FS_AZURE_ENABLE_BLOB_ENDPOINT, false, false); + Assume.assumeTrue(!OperativeEndpoint.isIngressEnabledOnDFS(getPrefixMode(fs), fs.getAbfsStore().getAbfsConfiguration())); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + fs.getAbfsStore().setClient(client); + fs.create(new Path("/tmp")); + verifyCreatePathBlobExecution(client); + } + + @Test + public void testBlobEndpointWhenBlobEndpointConfigIsNull() + throws Exception { + AzureBlobFileSystem fs = createFileSystemForEndpointConfigPair( + FS_AZURE_ENABLE_BLOB_ENDPOINT, null, false); + Assume.assumeTrue(!OperativeEndpoint.isIngressEnabledOnDFS(getPrefixMode(fs), fs.getAbfsStore().getAbfsConfiguration())); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + fs.getAbfsStore().setClient(client); + fs.create(new Path("/tmp")); + verifyCreatePathBlobExecution(client); + } + + @Test + public void testBlobEndpointWithMkdirsOnDFS() throws Exception { + AzureBlobFileSystem fs = createFileSystemForEndpointConfigPair( + FS_AZURE_MKDIRS_FALLBACK_TO_DFS, true, false); + Assume.assumeTrue(!OperativeEndpoint.isIngressEnabledOnDFS(getPrefixMode(fs), fs.getAbfsStore().getAbfsConfiguration())); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + fs.getAbfsStore().setClient(client); + int[] dirCreatedOverDFSExecCount = new int[1]; + dirCreatedOverDFSExecCount[0] = 0; + int[] fileCreatedOverDFSExecCount = new int[1]; + fileCreatedOverDFSExecCount[0] = 0; + checkDirAndFileCreationOnDFS(client, dirCreatedOverDFSExecCount, + fileCreatedOverDFSExecCount); + fs.mkdirs(new Path("/tmp")); + fs.create(new Path("/file")); + verifyCreatePathBlobExecution(client); + Assert.assertTrue(dirCreatedOverDFSExecCount[0] == 1); + } + + @Test + public void testBlobEndpointWithMkdirsOnDfsNoOverride() throws Exception { + AzureBlobFileSystem fs = createFileSystemForEndpointConfigPair( + FS_AZURE_MKDIRS_FALLBACK_TO_DFS, false, false); + Assume.assumeTrue(!OperativeEndpoint.isIngressEnabledOnDFS(getPrefixMode(fs), fs.getAbfsStore().getAbfsConfiguration())); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + fs.getAbfsStore().setClient(client); + int[] dirCreatedOverBlobExecCount = new int[1]; + dirCreatedOverBlobExecCount[0] = 0; + int[] fileCreatedOverBlobExecCount = new int[1]; + fileCreatedOverBlobExecCount[0] = 0; + + checkDirAndFileCreationOnBlob(client, dirCreatedOverBlobExecCount, + fileCreatedOverBlobExecCount); + + fs.mkdirs(new Path("/tmp")); + fs.create(new Path("/file")); + Assert.assertTrue(dirCreatedOverBlobExecCount[0] == 1); + Assert.assertTrue(fileCreatedOverBlobExecCount[0] == 1); + } + + private void checkDirAndFileCreationOnBlob(final AbfsClient client, + final int[] dirCreatedOverBlobExecCount, + final int[] fileCreatedOverBlobExecCount) + throws AzureBlobFileSystemException { + Mockito.doAnswer(answer -> { + if (!(Boolean) answer.getArgument(1)) { + dirCreatedOverBlobExecCount[0]++; + } else { + fileCreatedOverBlobExecCount[0]++; + } + return answer.callRealMethod(); + }).when(client) + .createPathBlob(Mockito.anyString(), Mockito.anyBoolean(), + Mockito.anyBoolean(), Mockito.nullable( + HashMap.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + } + + @Test + public void testDFSEndpointWithMkdirsOnDFS() throws Exception { + AzureBlobFileSystem fs = createFileSystemForEndpointConfigPair( + FS_AZURE_MKDIRS_FALLBACK_TO_DFS, true, true); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + fs.getAbfsStore().setClient(client); + int[] dirCreatedOverDFSExecCount = new int[1]; + dirCreatedOverDFSExecCount[0] = 0; + int[] fileCreatedOverDFSExecCount = new int[1]; + fileCreatedOverDFSExecCount[0] = 0; + checkDirAndFileCreationOnDFS(client, dirCreatedOverDFSExecCount, + fileCreatedOverDFSExecCount); + fs.mkdirs(new Path("/tmp")); + fs.create(new Path("/file")); + Assert.assertTrue(dirCreatedOverDFSExecCount[0] == 1); + Assert.assertTrue(fileCreatedOverDFSExecCount[0] == 1); + } + + @Test + public void testDFSEndpointWithMkdirsOnDFSNoOverride() throws Exception { + AzureBlobFileSystem fs = createFileSystemForEndpointConfigPair( + FS_AZURE_MKDIRS_FALLBACK_TO_DFS, false, true); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + fs.getAbfsStore().setClient(client); + int[] dirCreatedOverDFSExecCount = new int[1]; + dirCreatedOverDFSExecCount[0] = 0; + int[] fileCreatedOverDFSExecCount = new int[1]; + fileCreatedOverDFSExecCount[0] = 0; + checkDirAndFileCreationOnDFS(client, dirCreatedOverDFSExecCount, fileCreatedOverDFSExecCount); + fs.mkdirs(new Path("/tmp")); + fs.create(new Path("/file")); + Assert.assertTrue(dirCreatedOverDFSExecCount[0] == 1); + Assert.assertTrue(fileCreatedOverDFSExecCount[0] == 1); + } + + @Test + public void testBlobEndpointNoDeleteWasbRedirect() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(createFileSystemForEndpointConfigPair( + FS_AZURE_REDIRECT_DELETE, false, false)); + Assert.assertNull(fs.getNativeFs()); + NativeAzureFileSystem nativeAzureFileSystem = null; + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + assignStoreMocksToFs(fs, nativeAzureFileSystem, store, client); + + Boolean[] isDeleteOverNativeFS = new Boolean[1]; + isDeleteOverNativeFS[0] = false; + Boolean[] isDeleteOverDFSEndpoint = new Boolean[1]; + isDeleteOverDFSEndpoint[0] = false; + Boolean[] isDeleteOverAbfsBlobEndpoint = new Boolean[1]; + isDeleteOverAbfsBlobEndpoint[0] = false; + + countDeleteOverAbfsAndWasb(nativeAzureFileSystem, client, isDeleteOverNativeFS, + isDeleteOverDFSEndpoint, isDeleteOverAbfsBlobEndpoint); + + fs.create(new Path("/file")); + fs.delete(new Path("/file"), true); + + Assert.assertTrue(isDeleteOverDFSEndpoint[0] || isDeleteOverAbfsBlobEndpoint[0]); + Assert.assertFalse(isDeleteOverNativeFS[0]); + } + + @Test + public void testBlobEndpointWithDeleteWasbRedirect() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(createFileSystemForEndpointConfigPair( + FS_AZURE_REDIRECT_DELETE, true, false)); + Assert.assertNotNull(fs.getNativeFs()); + NativeAzureFileSystem nativeAzureFileSystem = Mockito.spy(fs.getNativeFs()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + assignStoreMocksToFs(fs, nativeAzureFileSystem, store, client); + + Boolean[] isDeleteOverNativeFS = new Boolean[1]; + isDeleteOverNativeFS[0] = false; + Boolean[] isDeleteOverDFSEndpoint = new Boolean[1]; + isDeleteOverDFSEndpoint[0] = false; + Boolean[] isDeleteOverAbfsBlobEndpoint = new Boolean[1]; + isDeleteOverAbfsBlobEndpoint[0] = false; + + countDeleteOverAbfsAndWasb(nativeAzureFileSystem, client, isDeleteOverNativeFS, + isDeleteOverDFSEndpoint, isDeleteOverAbfsBlobEndpoint); + + fs.create(new Path("/file")); + fs.delete(new Path("/file"), true); + + Assert.assertFalse(isDeleteOverDFSEndpoint[0]); + Assert.assertTrue(isDeleteOverNativeFS[0]); + } + + @Test + public void testDfsEndpointWithDeleteWasbRedirect() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(createFileSystemForEndpointConfigPair( + FS_AZURE_REDIRECT_DELETE, true, true)); + Assert.assertNotNull(fs.getNativeFs()); + NativeAzureFileSystem nativeAzureFileSystem = Mockito.spy(fs.getNativeFs()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + assignStoreMocksToFs(fs, nativeAzureFileSystem, store, client); + + Boolean[] isDeleteOverNativeFS = new Boolean[1]; + isDeleteOverNativeFS[0] = false; + Boolean[] isDeleteOverDFSEndpoint = new Boolean[1]; + isDeleteOverDFSEndpoint[0] = false; + Boolean[] isDeleteOverAbfsBlobEndpoint = new Boolean[1]; + isDeleteOverAbfsBlobEndpoint[0] = false; + + countDeleteOverAbfsAndWasb(nativeAzureFileSystem, client, isDeleteOverNativeFS, + isDeleteOverDFSEndpoint, isDeleteOverAbfsBlobEndpoint); + + fs.create(new Path("/file")); + fs.delete(new Path("/file"), true); + + Assert.assertFalse(isDeleteOverDFSEndpoint[0]); + Assert.assertTrue(isDeleteOverNativeFS[0]); + } + + @Test + public void testDfsEndpointNoDeleteWasbRedirect() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(createFileSystemForEndpointConfigPair( + FS_AZURE_REDIRECT_DELETE, false, true)); + Assert.assertNull(fs.getNativeFs()); + NativeAzureFileSystem nativeAzureFileSystem = null; + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + assignStoreMocksToFs(fs, nativeAzureFileSystem, store, client); + + Boolean[] isDeleteOverNativeFS = new Boolean[1]; + isDeleteOverNativeFS[0] = false; + Boolean[] isDeleteOverDFSEndpoint = new Boolean[1]; + isDeleteOverDFSEndpoint[0] = false; + Boolean[] isDeleteOverAbfsBlobEndpoint = new Boolean[1]; + isDeleteOverAbfsBlobEndpoint[0] = false; + + countDeleteOverAbfsAndWasb(nativeAzureFileSystem, client, isDeleteOverNativeFS, + isDeleteOverDFSEndpoint, isDeleteOverAbfsBlobEndpoint); + + fs.create(new Path("/file")); + fs.delete(new Path("/file"), true); + + Assert.assertTrue(isDeleteOverDFSEndpoint[0]); + Assert.assertFalse(isDeleteOverNativeFS[0]); + } + + @Test + public void testBlobEndpointForNoRenameRedirectToWasb() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(createFileSystemForEndpointConfigPair( + FS_AZURE_REDIRECT_RENAME, false, false)); + + NativeAzureFileSystem nativeAzureFileSystem = null; + if (fs.getNativeFs() != null) { + nativeAzureFileSystem = Mockito.spy(fs.getNativeFs()); + } + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + assignStoreMocksToFs(fs, nativeAzureFileSystem, store, client); + + Boolean[] isRenameOverNativeFS = new Boolean[1]; + isRenameOverNativeFS[0] = false; + Boolean[] isRenameOverDFSEndpoint = new Boolean[1]; + isRenameOverDFSEndpoint[0] = false; + Boolean[] isRenameOverBlobEndpoint = new Boolean[1]; + isRenameOverBlobEndpoint[0] = false; + + countRenameOverDfsBlobAndWasb(nativeAzureFileSystem, client, isRenameOverNativeFS, + isRenameOverDFSEndpoint, isRenameOverBlobEndpoint); + + fs.create(new Path("/file1")); + fs.rename(new Path("/file1"), new Path("/file2")); + Assert.assertTrue(isRenameOverBlobEndpoint[0]); + Assert.assertFalse(isRenameOverDFSEndpoint[0]); + Assert.assertFalse(isRenameOverNativeFS[0]); + } + + @Test + public void testBlobEndpointForRenameRedirectToWasb() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(createFileSystemForEndpointConfigPair( + FS_AZURE_REDIRECT_RENAME, true, false)); + + NativeAzureFileSystem nativeAzureFileSystem = Mockito.spy(fs.getNativeFs()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + assignStoreMocksToFs(fs, nativeAzureFileSystem, store, client); + + Boolean[] isRenameOverNativeFS = new Boolean[1]; + isRenameOverNativeFS[0] = false; + Boolean[] isRenameOverDFSEndpoint = new Boolean[1]; + isRenameOverDFSEndpoint[0] = false; + Boolean[] isRenameOverBlobEndpoint = new Boolean[1]; + isRenameOverBlobEndpoint[0] = false; + + countRenameOverDfsBlobAndWasb(nativeAzureFileSystem, client, isRenameOverNativeFS, + isRenameOverDFSEndpoint, isRenameOverBlobEndpoint); + + fs.create(new Path("/file1")); + fs.rename(new Path("/file1"), new Path("/file2")); + Assert.assertFalse(isRenameOverBlobEndpoint[0]); + Assert.assertFalse(isRenameOverDFSEndpoint[0]); + Assert.assertTrue(isRenameOverNativeFS[0]); + } + + @Test + public void testDfsEndpointForRenameRedirectToWasb() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(createFileSystemForEndpointConfigPair( + FS_AZURE_REDIRECT_RENAME, true, true)); + + NativeAzureFileSystem nativeAzureFileSystem = Mockito.spy(fs.getNativeFs()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + assignStoreMocksToFs(fs, nativeAzureFileSystem, store, client); + + Boolean[] isRenameOverNativeFS = new Boolean[1]; + isRenameOverNativeFS[0] = false; + Boolean[] isRenameOverDFSEndpoint = new Boolean[1]; + isRenameOverDFSEndpoint[0] = false; + Boolean[] isRenameOverBlobEndpoint = new Boolean[1]; + isRenameOverBlobEndpoint[0] = false; + + countRenameOverDfsBlobAndWasb(nativeAzureFileSystem, client, isRenameOverNativeFS, + isRenameOverDFSEndpoint, isRenameOverBlobEndpoint); + + fs.create(new Path("/file1")); + fs.rename(new Path("/file1"), new Path("/file2")); + Assert.assertFalse(isRenameOverBlobEndpoint[0]); + Assert.assertFalse(isRenameOverDFSEndpoint[0]); + Assert.assertTrue(isRenameOverNativeFS[0]); + } + + @Test + public void testDfsEndpointForNoRenameRedirectToWasb() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(createFileSystemForEndpointConfigPair( + FS_AZURE_REDIRECT_RENAME, false, true)); + + NativeAzureFileSystem nativeAzureFileSystem = null; + if (fs.getNativeFs() != null) { + nativeAzureFileSystem = Mockito.spy(fs.getNativeFs()); + } + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + assignStoreMocksToFs(fs, nativeAzureFileSystem, store, client); + + Boolean[] isRenameOverNativeFS = new Boolean[1]; + isRenameOverNativeFS[0] = false; + Boolean[] isRenameOverDFSEndpoint = new Boolean[1]; + isRenameOverDFSEndpoint[0] = false; + Boolean[] isRenameOverBlobEndpoint = new Boolean[1]; + isRenameOverBlobEndpoint[0] = false; + + countRenameOverDfsBlobAndWasb(nativeAzureFileSystem, client, isRenameOverNativeFS, + isRenameOverDFSEndpoint, isRenameOverBlobEndpoint); + + fs.create(new Path("/file1")); + fs.rename(new Path("/file1"), new Path("/file2")); + Assert.assertFalse(isRenameOverBlobEndpoint[0]); + Assert.assertTrue(isRenameOverDFSEndpoint[0]); + Assert.assertFalse(isRenameOverNativeFS[0]); + } + + @Test + public void testBlobEndpointIngressRedirectToDfs() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(createFileSystemForEndpointConfigPair( + FS_AZURE_INGRESS_FALLBACK_TO_DFS, true, false)); + + NativeAzureFileSystem nativeAzureFileSystem = null; + if (fs.getNativeFs() != null) { + nativeAzureFileSystem = Mockito.spy(fs.getNativeFs()); + } + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + assignStoreMocksToFs(fs, nativeAzureFileSystem, store, client); + + int[] createOverBlob = new int[1]; + createOverBlob[0] = 0; + int[] createOverDfs = new int[1]; + createOverDfs[0] = 0; + int[] appendOverBlob = new int[1]; + appendOverBlob[0] = 0; + int[] appendOverDfs = new int[1]; + appendOverDfs[0] = 0; + int[] flushOverBlob = new int[1]; + flushOverBlob[0] = 0; + int[] flushOverDfs = new int[1]; + flushOverDfs[0] = 0; + + countIngressOverDfsAndBlob(client, createOverBlob, createOverDfs, appendOverBlob, + appendOverDfs, + flushOverBlob, flushOverDfs); + + FSDataOutputStream outputStream = fs.create(new Path("/file")); + byte[] bytes = new byte[ONE_MB * 4]; + new Random().nextBytes(bytes); + outputStream.write(bytes); + outputStream.close(); + + Assert.assertTrue(createOverDfs[0] == 1); + Assert.assertTrue(appendOverDfs[0] == 1); + Assert.assertTrue(flushOverDfs[0] == 1); + } + + @Test + public void testBlobEndpointNoIngressRedirectToDfs() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(createFileSystemForEndpointConfigPair( + FS_AZURE_INGRESS_FALLBACK_TO_DFS, false, false)); + + NativeAzureFileSystem nativeAzureFileSystem = null; + if (fs.getNativeFs() != null) { + nativeAzureFileSystem = Mockito.spy(fs.getNativeFs()); + } + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + assignStoreMocksToFs(fs, nativeAzureFileSystem, store, client); + + int[] createOverBlob = new int[1]; + createOverBlob[0] = 0; + int[] createOverDfs = new int[1]; + createOverDfs[0] = 0; + int[] appendOverBlob = new int[1]; + appendOverBlob[0] = 0; + int[] appendOverDfs = new int[1]; + appendOverDfs[0] = 0; + int[] flushOverBlob = new int[1]; + flushOverBlob[0] = 0; + int[] flushOverDfs = new int[1]; + flushOverDfs[0] = 0; + + countIngressOverDfsAndBlob(client, createOverBlob, createOverDfs, appendOverBlob, + appendOverDfs, + flushOverBlob, flushOverDfs); + + FSDataOutputStream outputStream = fs.create(new Path("/file")); + byte[] bytes = new byte[ONE_MB * 4]; + new Random().nextBytes(bytes); + outputStream.write(bytes); + outputStream.close(); + + Assert.assertTrue(createOverBlob[0] == 1); + Assert.assertTrue(appendOverBlob[0] == 1); + Assert.assertTrue(flushOverBlob[0] == 1); + } + + @Test + public void testDfsEndpointNoIngressRedirectToDfs() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(createFileSystemForEndpointConfigPair( + FS_AZURE_INGRESS_FALLBACK_TO_DFS, true, true)); + + NativeAzureFileSystem nativeAzureFileSystem = null; + if (fs.getNativeFs() != null) { + nativeAzureFileSystem = Mockito.spy(fs.getNativeFs()); + } + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + assignStoreMocksToFs(fs, nativeAzureFileSystem, store, client); + + int[] createOverBlob = new int[1]; + createOverBlob[0] = 0; + int[] createOverDfs = new int[1]; + createOverDfs[0] = 0; + int[] appendOverBlob = new int[1]; + appendOverBlob[0] = 0; + int[] appendOverDfs = new int[1]; + appendOverDfs[0] = 0; + int[] flushOverBlob = new int[1]; + flushOverBlob[0] = 0; + int[] flushOverDfs = new int[1]; + flushOverDfs[0] = 0; + + countIngressOverDfsAndBlob(client, createOverBlob, createOverDfs, appendOverBlob, + appendOverDfs, + flushOverBlob, flushOverDfs); + + FSDataOutputStream outputStream = fs.create(new Path("/file")); + byte[] bytes = new byte[ONE_MB * 4]; + new Random().nextBytes(bytes); + outputStream.write(bytes); + outputStream.close(); + + Assert.assertTrue(createOverDfs[0] == 1); + Assert.assertTrue(appendOverDfs[0] == 1); + Assert.assertTrue(flushOverDfs[0] == 1); + } + + private void countIngressOverDfsAndBlob(final AbfsClient client, + final int[] createOverBlob, + final int[] createOverDfs, + final int[] appendOverBlob, + final int[] appendOverDfs, + final int[] flushOverBlob, + final int[] flushOverDfs) throws IOException { + Mockito.doAnswer(answer -> { + createOverDfs[0]++; + return answer.callRealMethod(); + }).when(client).createPath(Mockito.anyString(), Mockito.anyBoolean(), + Mockito.anyBoolean(), Mockito.nullable(String.class), Mockito.nullable(String.class), + Mockito.anyBoolean(), Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + Mockito.doAnswer(answer -> { + createOverBlob[0]++; + return answer.callRealMethod(); + }).when(client).createPathBlob(Mockito.anyString(), Mockito.anyBoolean(), + Mockito.anyBoolean(), Mockito.nullable(HashMap.class), Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + + Mockito.doAnswer(answer -> { + appendOverDfs[0]++; + return answer.callRealMethod(); + }).when(client).append(Mockito.anyString(), Mockito.any(byte[].class), Mockito.any( + AppendRequestParameters.class), Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + Mockito.doAnswer(answer -> { + appendOverBlob[0]++; + return answer.callRealMethod(); + }).when(client).append(Mockito.anyString(), Mockito.anyString(), Mockito.any(byte[].class), Mockito.any( + AppendRequestParameters.class), Mockito.nullable(String.class), Mockito.any(TracingContext.class), Mockito.nullable(String.class)); + + Mockito.doAnswer(answer -> { + flushOverDfs[0]++; + return answer.callRealMethod(); + }).when(client).flush(Mockito.anyString(), Mockito.anyLong(), Mockito.anyBoolean(), + Mockito.anyBoolean(), Mockito.nullable(String.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + Mockito.doAnswer(answer -> { + flushOverBlob[0]++; + return answer.callRealMethod(); + }).when(client).flush(Mockito.any(byte[].class), Mockito.anyString(), + Mockito.anyBoolean(), Mockito.nullable(String.class),Mockito.nullable(String.class),Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + } + + private void countRenameOverDfsBlobAndWasb(final NativeAzureFileSystem nativeAzureFileSystem, + final AbfsClient client, + final Boolean[] isRenameOverNativeFS, + final Boolean[] isRenameOverDFSEndpoint, + final Boolean[] isRenameOverBlobEndpoint) throws IOException { + if(nativeAzureFileSystem != null) { + Mockito.doAnswer(answer -> { + isRenameOverNativeFS[0] = true; + return answer.callRealMethod(); + }).when(nativeAzureFileSystem).rename(Mockito.any(Path.class), Mockito.any(Path.class)); + } + Mockito.doAnswer(answer -> { + isRenameOverDFSEndpoint[0] = true; + return answer.callRealMethod(); + }).when(client).renamePath(Mockito.anyString(), Mockito.anyString(), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + Mockito.doAnswer(answer -> { + isRenameOverBlobEndpoint[0] = true; + return answer.callRealMethod(); + }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + } + + private void countDeleteOverAbfsAndWasb(final NativeAzureFileSystem nativeAzureFileSystem, + final AbfsClient client, + final Boolean[] isDeleteOverNativeFS, + final Boolean[] isDeleteOverDFSEndpoint, + final Boolean[] isDeleteOverAbfsBlobEndpoint) throws IOException { + Mockito.doAnswer(answer -> { + isDeleteOverDFSEndpoint[0] = true; + return answer.callRealMethod(); + }).when(client).deletePath(Mockito.any(String.class), Mockito.anyBoolean(), Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + + Mockito.doAnswer(answer -> { + isDeleteOverAbfsBlobEndpoint[0] = true; + return answer.callRealMethod(); + }).when(client).deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + + if(nativeAzureFileSystem != null) { + Mockito.doAnswer(answer -> { + isDeleteOverNativeFS[0] = true; + return answer.callRealMethod(); + }).when(nativeAzureFileSystem).delete(Mockito.any(Path.class), Mockito.anyBoolean()); + } + } + + private void assignStoreMocksToFs(final AzureBlobFileSystem fs, + final NativeAzureFileSystem nativeAzureFileSystem, + final AzureBlobFileSystemStore store, + final AbfsClient client) { + Mockito.doReturn(store).when(fs).getAbfsStore(); + store.setClient(client); + Mockito.doReturn(nativeAzureFileSystem).when(fs).getNativeFs(); + } + + private void checkDirAndFileCreationOnDFS(final AbfsClient client, + final int[] dirCreatedOverDFSExecCount, + final int[] fileCreatedOverDFSExecCount) + throws AzureBlobFileSystemException { + Mockito.doAnswer(answer -> { + if (!(Boolean) answer.getArgument(1)) { + dirCreatedOverDFSExecCount[0]++; + } else { + fileCreatedOverDFSExecCount[0]++; + } + return answer.callRealMethod(); + }) + .when(client) + .createPath(Mockito.anyString(), Mockito.anyBoolean(), + Mockito.anyBoolean(), Mockito.nullable(String.class), + Mockito.nullable(String.class), Mockito.anyBoolean(), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + } + + + private void verifyCreatePathBlobExecution(final AbfsClient client) + throws AzureBlobFileSystemException { + Mockito.verify(client, Mockito.times(1)) + .createPathBlob(Mockito.anyString(), Mockito.anyBoolean(), + Mockito.anyBoolean(), Mockito.nullable( + HashMap.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + } + + private AzureBlobFileSystem createFileSystemForEndpointConfigPair(String configName, + Boolean configVal, + Boolean dfsEndpoint) throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + Assume.assumeFalse( + fs.getIsNamespaceEnabled(getTestTracingContext(fs, true))); + Configuration configuration = Mockito.spy(getRawConfiguration()); + if(!FS_AZURE_ENABLE_BLOB_ENDPOINT.equalsIgnoreCase(configName)) { + configuration.set(FS_AZURE_ENABLE_BLOB_ENDPOINT, Boolean.toString(DEFAULT_FS_AZURE_ENABLE_BLOBENDPOINT)); + } + fixEndpointAsPerTest(configuration, dfsEndpoint); + if (configVal != null) { + configuration.set(configName, configVal.toString()); + } + return (AzureBlobFileSystem) FileSystem.newInstance(configuration); + } + + private void fixEndpointAsPerTest(Configuration configuration, + final Boolean dfsEndpoint) { + if (dfsEndpoint) { + String url = getTestUrl(); + url = url.replace(WASB_DNS_PREFIX, ABFS_DNS_PREFIX); + configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, + url); + } else { + String url = getTestUrl(); + url = url.replace(ABFS_DNS_PREFIX, WASB_DNS_PREFIX); + configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, + url); + } + } +} 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 7ec03552833f09..80f26c8d46291f 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 @@ -20,8 +20,10 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.lang.reflect.Field; +import java.nio.file.AccessDeniedException; import java.util.List; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; import org.apache.hadoop.thirdparty.com.google.common.collect.Lists; import org.junit.Assume; import org.junit.Test; @@ -41,9 +43,12 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_IS_HNS_ENABLED; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_CHECK_ACCESS; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DNS_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID; @@ -84,29 +89,64 @@ private void setTestUserFs() throws Exception { if (this.testUserFs != null) { return; } + AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem()); + String accountName = this.getAccountName(); + if (abfsStore.getPrefixMode() == PrefixMode.BLOB) { + if (abfsStore.getAbfsConfiguration().shouldEnableBlobEndPoint()) { + accountName = getAccountName().replace(ABFS_DNS_PREFIX, WASB_DNS_PREFIX); + } + } checkIfConfigIsSet(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT - + "." + getAccountName()); + + "." + accountName); Configuration conf = getRawConfiguration(); - setTestFsConf(FS_AZURE_BLOB_FS_CLIENT_ID, + setTestFsConf(FS_AZURE_BLOB_FS_CLIENT_ID, accountName, FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID); - setTestFsConf(FS_AZURE_BLOB_FS_CLIENT_SECRET, + setTestFsConf(FS_AZURE_BLOB_FS_CLIENT_SECRET, accountName, FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET); conf.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.OAuth.name()); conf.set(FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME + "." - + getAccountName(), ClientCredsTokenProvider.class.getName()); + + accountName, ClientCredsTokenProvider.class.getName()); conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, false); this.testUserFs = FileSystem.newInstance(getRawConfiguration()); } - private void setTestFsConf(final String fsConfKey, + private void setTestUserFsNonHNS() throws Exception { + AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem()); + String accountName = this.getAccountName(); + checkIfConfigIsSet(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT + + "." + accountName); + Configuration conf = Mockito.spy(getRawConfiguration()); + setTestFsConf1(FS_AZURE_BLOB_FS_CLIENT_ID, accountName, + FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID, conf); + setTestFsConf1(FS_AZURE_BLOB_FS_CLIENT_SECRET, accountName, + FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET, conf); + conf.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.OAuth.name()); + conf.set(FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME + "." + + accountName, ClientCredsTokenProvider.class.getName()); + conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, + false); + conf.set(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, ""); + FileSystem testUserFsNonHns; + testUserFsNonHns = FileSystem.newInstance(conf); + } + + private void setTestFsConf(final String fsConfKey, String accountName, final String testFsConfKey) { - final String confKeyWithAccountName = fsConfKey + "." + getAccountName(); + final String confKeyWithAccountName = fsConfKey + "." + accountName; final String confValue = getConfiguration() .getString(testFsConfKey, ""); getRawConfiguration().set(confKeyWithAccountName, confValue); } + private void setTestFsConf1(final String fsConfKey, String accountName, + final String testFsConfKey, Configuration conf) { + final String confKeyWithAccountName = fsConfKey + "." + accountName; + final String confValue = getConfiguration() + .getString(testFsConfKey, ""); + conf.set(confKeyWithAccountName, confValue); + } + @Test(expected = IllegalArgumentException.class) public void testCheckAccessWithNullPath() throws IOException { superUserFs.access(null, FsAction.READ); @@ -174,14 +214,16 @@ public void testCheckAccessForAccountWithoutNS() throws Exception { checkIfConfigIsSet(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_SECRET); checkIfConfigIsSet(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_USER_GUID); - setTestUserFs(); + Configuration configuration = Mockito.spy(getFileSystem().getConf()); + FileSystem testUserFSWithoutNS; + testUserFSWithoutNS = FileSystem.newInstance(configuration); // When the driver does not know if the account is HNS enabled or not it // makes a server call and fails - intercept(AccessControlException.class, + + intercept(Exception.class, "\"This request is not authorized to perform this operation using " - + "this permission.\", 403", - () -> testUserFs.access(new Path("/"), FsAction.READ)); + + "this permission.\", 403", this::setTestUserFsNonHNS); // When the driver has already determined if the account is HNS enabled // or not, and as the account is non HNS the AzureBlobFileSystem#access @@ -194,8 +236,8 @@ public void testCheckAccessForAccountWithoutNS() throws Exception { Field abfsStoreField = AzureBlobFileSystem.class.getDeclaredField( "abfsStore"); abfsStoreField.setAccessible(true); - abfsStoreField.set(testUserFs, mockAbfsStore); - testUserFs.access(new Path("/"), FsAction.READ); + abfsStoreField.set(testUserFSWithoutNS, mockAbfsStore); + testUserFSWithoutNS.access(new Path("/"), FsAction.READ); superUserFs.access(new Path("/"), FsAction.READ); } @@ -304,9 +346,9 @@ public void testFsActionALL() throws Exception { } private void checkPrerequisites() throws Exception { - setTestUserFs(); Assume.assumeTrue(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false", - isHNSEnabled); + isHNSEnabled); + setTestUserFs(); Assume.assumeTrue(FS_AZURE_ENABLE_CHECK_ACCESS + " is false", isCheckAccessEnabled); checkIfConfigIsSet(FS_AZURE_BLOB_FS_CHECKACCESS_TEST_CLIENT_ID); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 86689a1e61940d..42c54979499dec 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -22,17 +22,36 @@ import java.io.FilterOutputStream; import java.io.IOException; import java.lang.reflect.Field; +import java.util.ArrayList; import java.util.EnumSet; +import java.util.List; import java.util.UUID; -import org.junit.Test; - -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; + +import java.util.HashMap; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; +import org.apache.hadoop.fs.azurebfs.services.OperativeEndpoint; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; +import org.apache.hadoop.test.LambdaTestUtils; + +import org.junit.Assert; +import org.junit.Assume; + +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.test.GenericTestUtils; @@ -41,11 +60,12 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConcurrentWriteOperationDetectedException; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; -import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; -import org.apache.hadoop.fs.azurebfs.services.TestAbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; +import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; +import org.mockito.Mockito; import static java.net.HttpURLConnection.HTTP_CONFLICT; import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR; @@ -53,7 +73,14 @@ import static java.net.HttpURLConnection.HTTP_OK; import static java.net.HttpURLConnection.HTTP_PRECON_FAILED; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.TRUE; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_BLOB_MKDIR_OVERWRITE; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_CREATE_NON_RECURSIVE; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_MKDIRS_FALLBACK_TO_DFS; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_META_HDI_ISFOLDER; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -71,9 +98,16 @@ public class ITestAzureBlobFileSystemCreate extends private static final Path TEST_FILE_PATH = new Path("testfile"); private static final Path TEST_FOLDER_PATH = new Path("testFolder"); private static final String TEST_CHILD_FILE = "childFile"; + private boolean useBlobEndpoint; public ITestAzureBlobFileSystemCreate() throws Exception { - super(); + super.setup(); + AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem()); + PrefixMode prefixMode = abfsStore.getPrefixMode(); + AbfsConfiguration abfsConfiguration = abfsStore.getAbfsConfiguration(); + useBlobEndpoint = !(OperativeEndpoint.isIngressEnabledOnDFS(prefixMode, abfsConfiguration) || + OperativeEndpoint.isMkdirEnabledOnDFS(abfsConfiguration) || + OperativeEndpoint.isReadEnabledOnDFS(abfsConfiguration)); } @Test @@ -88,6 +122,799 @@ public void testEnsureFileCreatedImmediately() throws Exception { assertIsFile(fs, TEST_FILE_PATH); } + /** + * Creating subdirectory on existing file path should fail. + * @throws Exception + */ + @Test + public void testMkdirsFailsForSubdirectoryOfExistingFile() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.create(new Path("a/b/c")); + fs.mkdirs(new Path("a/b/d")); + intercept(IOException.class, () -> fs.mkdirs(new Path("a/b/c/d/e"))); + + assertTrue(fs.exists(new Path("a/b/c"))); + assertTrue(fs.exists(new Path("a/b/d"))); + // Asserting directory created still exists as explicit. + assertTrue( + BlobDirectoryStateHelper.isExplicitDirectory(new Path("a/b/d"), + fs, getTestTracingContext(fs, true))); + } + + /** + * Try creating file same as an existing directory. + * @throws Exception + */ + @Test + public void testCreateDirectoryAndFile() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.mkdirs(new Path("a/b/c")); + assertTrue(fs.exists(new Path("a/b/c"))); + intercept(IOException.class, () -> fs.create(new Path("a/b/c"))); + + // Asserting that directory still exists as explicit + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(new Path("a/b/c"), + fs, getTestTracingContext(fs, true))); + } + + /** + * Creating same file without specifying overwrite. + * @throws Exception + */ + @Test + public void testCreateSameFile() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.create(new Path("a/b/c")); + fs.create(new Path("a/b/c")); + assertTrue(fs.exists(new Path("a/b/c"))); + } + + /** + * Creating same file with overwrite flag set to false. + * @throws Exception + */ + @Test + public void testCreateSameFileWithOverwriteFalse() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.create(new Path("a/b/c")); + assertTrue(fs.exists(new Path("a/b/c"))); + intercept(IOException.class, () -> fs.create(new Path("a/b/c"), false)); + } + + /** + * Creation of already existing subpath should fail. + * @throws Exception + */ + @Test + public void testCreateSubPath() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.create(new Path("a/b/c")); + assertTrue(fs.exists(new Path("a/b/c"))); + intercept(IOException.class, () -> fs.create(new Path("a/b"))); + } + + /** + * Creating path with parent explicit. + */ + @Test + public void testCreatePathParentExplicit() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.mkdirs(new Path("a/b/c")); + assertTrue(fs.exists(new Path("a/b/c"))); + fs.create(new Path("a/b/c/d")); + assertTrue(fs.exists(new Path("a/b/c/d"))); + + // asserting that parent stays explicit + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(new Path("a/b/c"), + fs, getTestTracingContext(fs, true))); + } + + + /** + * Test create on implicit directory with explicit parent. + * @throws Exception + */ + @Test + public void testParentExplicitPathImplicit() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final AzureBlobFileSystemStore store = fs.getAbfsStore(); + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getRawConfiguration(), + store.getPrefixMode() + ); + fs.mkdirs(new Path("/explicitParent")); + String sourcePathName = "/explicitParent/implicitDir"; + Path sourcePath = new Path(sourcePathName); + azcopyHelper.createFolderUsingAzcopy(fs.makeQualified(sourcePath).toUri().getPath().substring(1)); + + intercept(IOException.class, () -> + fs.create(sourcePath, true)); + intercept(IOException.class, () -> + fs.create(sourcePath, false)); + + assertTrue("Parent directory should be explicit.", + BlobDirectoryStateHelper.isExplicitDirectory(sourcePath.getParent(), + fs, getTestTracingContext(fs, true))); + assertTrue("Path should be implicit.", + BlobDirectoryStateHelper.isImplicitDirectory(sourcePath, + fs, getTestTracingContext(fs, true))); + } + + /** + * Test create on implicit directory with implicit parent + * @throws Exception + */ + @Test + public void testParentImplicitPathImplicit() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final AzureBlobFileSystemStore store = fs.getAbfsStore(); + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getRawConfiguration(), + store.getPrefixMode() + ); + String parentPathName = "/implicitParent"; + Path parentPath = new Path(parentPathName); + String sourcePathName = "/implicitParent/implicitDir"; + Path sourcePath = new Path(sourcePathName); + + azcopyHelper.createFolderUsingAzcopy(fs.makeQualified(parentPath).toUri().getPath().substring(1)); + azcopyHelper.createFolderUsingAzcopy(fs.makeQualified(sourcePath).toUri().getPath().substring(1)); + + intercept(IOException.class, () -> + fs.create(sourcePath, true)); + intercept(IOException.class, () -> + fs.create(sourcePath, false)); + + + assertTrue("Parent directory is implicit.", + BlobDirectoryStateHelper.isImplicitDirectory(parentPath, + fs, getTestTracingContext(fs, true))); + + assertTrue("Path should also be implicit.", + BlobDirectoryStateHelper.isImplicitDirectory(sourcePath, + fs, getTestTracingContext(fs, true))); + + } + + /** + * Tests create file when file exists already + * Verifies using eTag for overwrite = true/false + */ + @Test + public void testCreateFileExistsImplicitParent() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final AzureBlobFileSystemStore store = fs.getAbfsStore(); + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getRawConfiguration(), + store.getPrefixMode() + ); + String parentPathName = "/implicitParent"; + Path parentPath = new Path(parentPathName); + azcopyHelper.createFolderUsingAzcopy(parentPathName); + + String fileName = "/implicitParent/testFile"; + Path filePath = new Path(fileName); + fs.create(filePath); + String eTag = extractFileEtag(fileName); + + // testing createFile on already existing file path + fs.create(filePath, true); + + String eTagAfterCreateOverwrite = extractFileEtag(fileName); + + assertTrue("New file eTag after create overwrite should be different from old", + !eTag.equals(eTagAfterCreateOverwrite)); + + intercept(IOException.class, () -> + fs.create(filePath, false)); + + String eTagAfterCreate = extractFileEtag(fileName); + + assertTrue("File eTag should not change as creation fails", + eTagAfterCreateOverwrite.equals(eTagAfterCreate)); + + assertTrue("Parent path should also change to explicit.", + BlobDirectoryStateHelper.isExplicitDirectory(parentPath, + fs, getTestTracingContext(fs, true))); + } + + /** + * Tests create file when the parent is an existing file + * should fail but parent directory should change to explicit + * @throws Exception FileAlreadyExists for blob and IOException for dfs. + */ + @Test + public void testCreateFileParentFile() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final AzureBlobFileSystemStore store = fs.getAbfsStore(); + + String parentName = "/testParentFile"; + Path parent = new Path(parentName); + fs.create(parent); + + String childName = "/testParentFile/testChildFile"; + Path child = new Path(childName); + IOException e = intercept(IOException.class, () -> + fs.create(child, false)); + + assertFalse("Parent Path should be a file.", + fs.getAbfsStore().getBlobProperty(parent, getTestTracingContext(fs, false)) + .getIsDirectory()); + + } + + + /** + * Creating directory on existing file path should fail. + * @throws Exception + */ + @Test + public void testCreateMkdirs() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.create(new Path("a/b/c")); + intercept(IOException.class, () -> fs.mkdirs(new Path("a/b/c/d"))); + } + + /** + * Test mkdirs. + * @throws Exception + */ + @Test + public void testMkdirs() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.mkdirs(new Path("a/b")); + fs.mkdirs(new Path("a/b/c/d")); + fs.mkdirs(new Path("a/b/c/e")); + + assertTrue(fs.exists(new Path("a/b"))); + assertTrue(fs.exists(new Path("a/b/c/d"))); + assertTrue(fs.exists(new Path("a/b/c/e"))); + + //Asserting that directories created as explicit + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(new Path("a/b"), + fs, getTestTracingContext(fs, true))); + assertTrue( + BlobDirectoryStateHelper.isExplicitDirectory(new Path("a/b/c/d"), + fs, getTestTracingContext(fs, true))); + assertTrue( + BlobDirectoryStateHelper.isExplicitDirectory(new Path("a/b/c/e"), + fs, getTestTracingContext(fs, true))); + } + + /** + * Creating subpath of directory path should fail. + * @throws Exception + */ + @Test + public void testMkdirsCreateSubPath() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.mkdirs(new Path("a/b/c")); + assertTrue(fs.exists(new Path("a/b/c"))); + intercept(IOException.class, () -> fs.create(new Path("a/b"))); + + //Asserting that directories created as explicit + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(new Path("a/b/c"), + fs, getTestTracingContext(fs, true))); + } + + /** + * Test creation of directory by level. + * @throws Exception + */ + @Test + public void testMkdirsByLevel() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.mkdirs(new Path("a")); + fs.mkdirs(new Path("a/b/c")); + fs.mkdirs(new Path("a/b/c/d/e")); + + assertTrue(fs.exists(new Path("a"))); + assertTrue(fs.exists(new Path("a/b/c"))); + assertTrue(fs.exists(new Path("a/b/c/d/e"))); + + //Asserting that directories created as explicit + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(new Path("a/"), + fs, getTestTracingContext(fs, true))); + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(new Path("a/b/c"), + fs, getTestTracingContext(fs, true))); + assertTrue( + BlobDirectoryStateHelper.isExplicitDirectory(new Path("a/b/c/d/e"), + fs, getTestTracingContext(fs, true))); + } + + /* + Delete part of a path and validate sub path exists. + */ + @Test + public void testMkdirsWithDelete() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.mkdirs(new Path("a/b")); + fs.mkdirs(new Path("a/b/c/d")); + fs.delete(new Path("a/b/c/d")); + fs.getFileStatus(new Path("a/b/c")); + assertTrue(fs.exists(new Path("a/b/c"))); + } + + /** + * Verify mkdir and rename of parent. + */ + @Test + public void testMkdirsWithRename() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.mkdirs(new Path("a/b/c/d")); + fs.create(new Path("e/file")); + fs.delete(new Path("a/b/c/d")); + assertTrue(fs.rename(new Path("e"), new Path("a/b/c/d"))); + assertTrue(fs.exists(new Path("a/b/c/d/file"))); + } + + /** + * Create a file with name /dir1 and then mkdirs for /dir1/dir2 should fail. + * @throws Exception + */ + @Test + public void testFileCreateMkdirsRoot() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.setWorkingDirectory(new Path("/")); + final Path p1 = new Path("dir1"); + fs.create(p1); + intercept(IOException.class, () -> fs.mkdirs(new Path("dir1/dir2"))); + } + + /** + * Create a file with name /dir1 and then mkdirs for /dir1/dir2 should fail. + * @throws Exception + */ + @Test + public void testFileCreateMkdirsNonRoot() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Path p1 = new Path("dir1"); + fs.create(p1); + intercept(IOException.class, () -> fs.mkdirs(new Path("dir1/dir2"))); + } + + /** + * Creation of same directory without overwrite flag should pass. + * @throws Exception + */ + @Test + public void testCreateSameDirectory() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.mkdirs(new Path("a/b/c")); + fs.mkdirs(new Path("a/b/c")); + + assertTrue(fs.exists(new Path("a/b/c"))); + //Asserting that directories created as explicit + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(new Path("a/b/c"), + fs, getTestTracingContext(fs, true))); + } + + /** + * Creation of same directory without overwrite flag should pass. + * @throws Exception + */ + @Test + public void testCreateSamePathDirectory() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.create(new Path("a")); + intercept(IOException.class, () -> fs.mkdirs(new Path("a"))); + } + + /** + * Creation of directory with root as parent + */ + @Test + public void testMkdirOnRootAsParent() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Path path = new Path("a"); + fs.setWorkingDirectory(new Path("/")); + fs.mkdirs(path); + + // Asserting that the directory created by mkdir exists as explicit. + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(path, + fs, getTestTracingContext(fs, true))); + } + + /** + * Creation of directory on root + */ + @Test + public void testMkdirOnRoot() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Path path = new Path("/"); + fs.setWorkingDirectory(new Path("/")); + fs.mkdirs(path); + + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(path, + fs, getTestTracingContext(fs, true))); + } + + /** + * Creation of directory on path with unicode chars + */ + @Test + public void testMkdirUnicode() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Path path = new Path("/dir\u0031"); + fs.mkdirs(path); + + // Asserting that the directory created by mkdir exists as explicit. + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(path, + fs, getTestTracingContext(fs, true))); + } + + /** + * Creation of directory on same path with parallel threads + */ + @Test + public void testMkdirParallelRequests() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Path path = new Path("/dir1"); + + ExecutorService es = Executors.newFixedThreadPool(3); + + List> tasks = new ArrayList<>(); + + for (int i = 0; i < 3; i++) { + CompletableFuture future = CompletableFuture.runAsync(() -> { + try { + fs.mkdirs(path); + } catch (IOException e) { + throw new CompletionException(e); + } + }, es); + tasks.add(future); + } + + // Wait for all the tasks to complete + CompletableFuture.allOf(tasks.toArray(new CompletableFuture[0])).join(); + + // Assert that the directory created by mkdir exists as explicit + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(path, + fs, getTestTracingContext(fs, true))); + + } + + + /** + * Creation of directory with overwrite set to false should not fail according to DFS code. + * @throws Exception + */ + @Test + public void testCreateSameDirectoryOverwriteFalse() throws Exception { + Configuration configuration = getRawConfiguration(); + configuration.setBoolean(FS_AZURE_ENABLE_BLOB_MKDIR_OVERWRITE, false); + AzureBlobFileSystem fs1 = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + fs1.mkdirs(new Path("a/b/c")); + fs1.mkdirs(new Path("a/b/c")); + + //Asserting that directories created as explicit + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(new Path("a/b/c"), + fs1, getTestTracingContext(fs1, true))); + } + + /** + * Try creating directory same as an existing file. + */ + @Test + public void testCreateDirectoryAndFileRecreation() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.mkdirs(new Path("a/b/c")); + fs.create(new Path("a/b/c/d")); + assertTrue(fs.exists(new Path("a/b/c"))); + assertTrue(fs.exists(new Path("a/b/c/d"))); + intercept(IOException.class, () -> fs.mkdirs(new Path("a/b/c/d"))); + } + + /** + * Creation of directory with parent directory existing as implicit. + * And the directory to be created does not exist + * @throws Exception + */ + @Test + public void testMkdirOnNonExistingPathWithImplicitParentDir() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Path implicitPath = new Path("dir1"); + final Path path = new Path("dir1/dir2"); + + // Creating implicit directory to be used as parent + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getFileSystem().getAbfsStore().getAbfsConfiguration().getRawConfiguration(), + fs.getAbfsStore().getPrefixMode()); + azcopyHelper.createFolderUsingAzcopy( + getFileSystem().makeQualified(implicitPath).toUri().getPath().substring(1)); + + // Creating a directory on non-existing path inside an implicit directory + fs.mkdirs(path); + + // Asserting that path created by azcopy becomes explicit. + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(implicitPath, + fs, getTestTracingContext(fs, true))); + + // Asserting that the directory created by mkdir exists as explicit. + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(path, + fs, getTestTracingContext(fs, true))); + } + + /** + * Creation of directory with parent directory existing as implicit. + * And the directory to be created existing as explicit directory + * @throws Exception + */ + @Test + public void testMkdirOnExistingExplicitDirWithImplicitParentDir() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Path implicitPath = new Path("dir1"); + final Path path = new Path("dir1/dir2"); + + // Creating implicit directory to be used as parent + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getFileSystem().getAbfsStore().getAbfsConfiguration().getRawConfiguration(), + fs.getAbfsStore().getPrefixMode()); + azcopyHelper.createFolderUsingAzcopy( + getFileSystem().makeQualified(implicitPath).toUri().getPath().substring(1)); + + // Creating an explicit directory on the path first + fs.mkdirs(path); + + // Creating a directory on existing explicit directory inside an implicit directory + fs.mkdirs(path); + + // Asserting that path created by azcopy becomes explicit. + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(implicitPath, + fs, getTestTracingContext(fs, true))); + + // Asserting that the directory created by mkdir exists as explicit. + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(path, + fs, getTestTracingContext(fs, true))); + } + + /** + * Creation of directory with parent directory existing as explicit. + * And the directory to be created existing as implicit directory + * @throws Exception + */ + @Test + public void testMkdirOnExistingImplicitDirWithExplicitParentDir() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Path explicitPath = new Path("dir1"); + final Path path = new Path("dir1/dir2"); + + // Creating an explicit directory to be used a parent + fs.mkdirs(explicitPath); + + // Creating implicit directory on the path + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getFileSystem().getAbfsStore().getAbfsConfiguration().getRawConfiguration(), + fs.getAbfsStore().getPrefixMode()); + azcopyHelper.createFolderUsingAzcopy( + getFileSystem().makeQualified(path).toUri().getPath().substring(1)); + + // Creating a directory on existing implicit directory inside an explicit directory + fs.mkdirs(path); + + // Asserting that the directory created by mkdir exists as explicit. + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(explicitPath, + fs, getTestTracingContext(fs, true))); + + // Asserting that the directory created by mkdir exists as explicit. + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(path, + fs, getTestTracingContext(fs, true))); + } + + /** + * Creation of directory with parent directory existing as implicit. + * And the directory to be created existing as implicit directory + * @throws Exception + */ + @Test + public void testMkdirOnExistingImplicitDirWithImplicitParentDir() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Path implicitPath = new Path("dir1"); + final Path path = new Path("dir1/dir2"); + + // Creating implicit directory to be used as parent + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getFileSystem().getAbfsStore().getAbfsConfiguration().getRawConfiguration(), + fs.getAbfsStore().getPrefixMode()); + azcopyHelper.createFolderUsingAzcopy( + getFileSystem().makeQualified(implicitPath).toUri().getPath().substring(1)); + + // Creating an implicit directory on path + azcopyHelper.createFolderUsingAzcopy( + getFileSystem().makeQualified(path).toUri().getPath().substring(1)); + + // Creating a directory on existing implicit directory inside an implicit directory + fs.mkdirs(path); + + // Asserting that path created by azcopy becomes explicit. + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(implicitPath, + fs, getTestTracingContext(fs, true))); + + // Asserting that the directory created by mkdir exists as explicit. + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(path, + fs, getTestTracingContext(fs, true))); + } + + /** + * Creation of directory with parent directory existing as implicit. + * And the directory to be created existing as file + * @throws Exception + */ + @Test + public void testMkdirOnExistingFileWithImplicitParentDir() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Path implicitPath = new Path("dir1"); + final Path path = new Path("dir1/dir2"); + + // Creating implicit directory to be used as parent + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getFileSystem().getAbfsStore().getAbfsConfiguration().getRawConfiguration(), + fs.getAbfsStore().getPrefixMode()); + azcopyHelper.createFolderUsingAzcopy( + getFileSystem().makeQualified(implicitPath).toUri().getPath().substring(1)); + + // Creating a file on path + fs.create(path); + + // Creating a directory on existing file inside an implicit directory + // Asserting that the mkdir fails + LambdaTestUtils.intercept(FileAlreadyExistsException.class, () -> { + fs.mkdirs(path); + }); + + // Asserting that path created by azcopy becomes explicit. + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(implicitPath, + fs, getTestTracingContext(fs, true))); + + // Asserting that the file still exists at path. + assertFalse( + fs.getAbfsStore().getBlobProperty( + new Path(getFileSystem().makeQualified(path).toUri().getPath()), + getTestTracingContext(fs, true) + ).getIsDirectory() + ); + } + + /** + * Test to validate that if prefix mode is BLOB and client encryption key is not null, exception is thrown. + */ + @Test + public void testCPKOverBlob() throws Exception { + AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem()); + Assume.assumeTrue(abfsStore.getPrefixMode() == PrefixMode.BLOB); + Configuration configuration = Mockito.spy(getRawConfiguration()); + String accountName = getAccountName(); + + configuration.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, "abcd"); + intercept(InvalidConfigurationValueException.class, () -> (AzureBlobFileSystem) FileSystem.newInstance(configuration)); + } + + /** + * Test to validate that if prefix mode is BLOB and even if client encryption key empty, exception is thrown. + */ + @Test + public void testCPKOverBlobEmptyKey() throws Exception { + AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem()); + Assume.assumeTrue(abfsStore.getPrefixMode() == PrefixMode.BLOB); + Configuration configuration = Mockito.spy(getRawConfiguration()); + String accountName = getAccountName(); + + configuration.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, ""); + intercept(InvalidConfigurationValueException.class, () -> (AzureBlobFileSystem) FileSystem.newInstance(configuration)); + } + + /** + * Test to validate that if prefix mode is BLOB and if client encryption key is not set, no exception is thrown. + */ + @Test + public void testCPKOverBlobNullKey() throws Exception { + Assume.assumeTrue(getFileSystem().getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + Configuration configuration = Mockito.spy(getRawConfiguration()); + AzureBlobFileSystem fs1 = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + } + + /** + * 1. a/b/c as implicit. + * 2. Create marker for b. + * 3. Do mkdir on a/b/c/d. + * 4. Verify all b,c,d have marker but a is implicit. + */ + @Test + public void testImplicitExplicitFolder() throws Exception { + Configuration configuration = Mockito.spy(getRawConfiguration()); + configuration.setBoolean(FS_AZURE_MKDIRS_FALLBACK_TO_DFS, false); + final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + final Path implicitPath = new Path("a/b/c"); + + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getFileSystem().getAbfsStore().getAbfsConfiguration().getRawConfiguration(), + fs.getAbfsStore().getPrefixMode()); + azcopyHelper.createFolderUsingAzcopy( + getFileSystem().makeQualified(implicitPath).toUri().getPath().substring(1)); + + Path path = makeQualified(new Path("a/b")); + HashMap metadata = new HashMap<>(); + metadata.put(X_MS_META_HDI_ISFOLDER, TRUE); + fs.getAbfsStore().createFile(path, null, true, + null, null, getTestTracingContext(fs, true), metadata); + + fs.mkdirs(new Path("a/b/c/d")); + + assertTrue(BlobDirectoryStateHelper.isImplicitDirectory(new Path("a"), + fs, getTestTracingContext(fs, true))); + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(new Path("a/b"), + fs, getTestTracingContext(fs, true))); + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(new Path("a/b/c"), + fs, getTestTracingContext(fs, true))); + assertTrue( + BlobDirectoryStateHelper.isExplicitDirectory(new Path("a/b/c/d"), + fs, getTestTracingContext(fs, true))); + } + + /** + * 1. a/b/c implicit. + * 2. Marker for a and c. + * 3. mkdir on a/b/c/d. + * 4. Verify a,c,d are explicit but b is implicit. + */ + @Test + public void testImplicitExplicitFolder1() throws Exception { + Configuration configuration = Mockito.spy(getRawConfiguration()); + configuration.setBoolean(FS_AZURE_MKDIRS_FALLBACK_TO_DFS, false); + final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + final Path implicitPath = new Path("a/b/c"); + + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getFileSystem().getAbfsStore().getAbfsConfiguration().getRawConfiguration(), + fs.getAbfsStore().getPrefixMode()); + azcopyHelper.createFolderUsingAzcopy( + getFileSystem().makeQualified(implicitPath).toUri().getPath().substring(1)); + + Path path = makeQualified(new Path("a")); + HashMap metadata = new HashMap<>(); + metadata.put(X_MS_META_HDI_ISFOLDER, TRUE); + fs.getAbfsStore().createFile(path, null, true, + null, null, getTestTracingContext(fs, true), metadata); + + fs.getAbfsStore().createFile(makeQualified(new Path("a/b/c")), null, true, + null, null, getTestTracingContext(fs, true), metadata); + + fs.mkdirs(new Path("a/b/c/d")); + + assertTrue(BlobDirectoryStateHelper.isImplicitDirectory(new Path("a/b"), + fs, getTestTracingContext(fs, true))); + + // Asserting that the directory created by mkdir exists as explicit. + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(new Path("a"), + fs, getTestTracingContext(fs, true))); + assertTrue(BlobDirectoryStateHelper.isExplicitDirectory(new Path("a/b/c"), + fs, getTestTracingContext(fs, true))); + assertTrue( + BlobDirectoryStateHelper.isExplicitDirectory(new Path("a/b/c/d"), + fs, getTestTracingContext(fs, true))); + } + @Test @SuppressWarnings("deprecation") public void testCreateNonRecursive() throws Exception { @@ -143,6 +970,151 @@ public void testCreateNonRecursive2() throws Exception { assertIsFile(fs, testFile); } + @Test + @SuppressWarnings("deprecation") + public void testCreateNonRecursiveOnRoot() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + + Path testFile = new Path("/"); + try { + fs.createNonRecursive(testFile, FsPermission.getDefault(), false, 1024, (short) 1, 1024, null); + fail("Should've thrown AbfsRestOperationException with HTTP_CONFLICT"); + } catch (AbfsRestOperationException e) { + assertEquals(e.getStatusCode(), HTTP_CONFLICT); + } + } + + @Test + public void testCreateNonRecursiveForAtomicDirectoryFile() throws Exception { + AzureBlobFileSystem fileSystem = getFileSystem(); + Assume.assumeTrue( + fileSystem.getAbfsStore().getAbfsConfiguration().getPrefixMode() + == PrefixMode.BLOB); + fileSystem.setWorkingDirectory(new Path("/")); + fileSystem.mkdirs(new Path("/hbase/dir")); + fileSystem.createFile(new Path("/hbase/dir/file")) + .overwrite(false) + .replication((short) 1) + .bufferSize(1024) + .blockSize(1024) + .build(); + Assert.assertTrue(fileSystem.exists(new Path("/hbase/dir/file"))); + } + + @Test + public void testActiveCreateNonRecursiveDenyParallelReadOnAtomicDir() throws Exception { + Assume.assumeTrue(getPrefixMode(getFileSystem()) == PrefixMode.BLOB); + Configuration configuration = Mockito.spy(getRawConfiguration()); + configuration.set(FS_AZURE_LEASE_CREATE_NON_RECURSIVE, "true"); + AzureBlobFileSystem fileSystem = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + AbfsClient client = Mockito.spy(fileSystem.getAbfsClient()); + fileSystem.getAbfsStore().setClient(client); + fileSystem.setWorkingDirectory(new Path("/")); + fileSystem.mkdirs(new Path("/hbase/dir")); + fileSystem.create(new Path("/hbase/dir/file")); + AtomicBoolean createCalled = new AtomicBoolean(false); + AtomicBoolean parallelRenameDone = new AtomicBoolean(false); + AtomicBoolean exceptionCaught = new AtomicBoolean(false); + + Mockito.doAnswer(answer -> { + AbfsRestOperation op = (AbfsRestOperation) answer.callRealMethod(); + createCalled.set(true); + while(!parallelRenameDone.get()); + return op; + }).when(client).createPathBlob(Mockito.anyString(), Mockito.anyBoolean(), + Mockito.anyBoolean(), Mockito.nullable(HashMap.class), Mockito.nullable(String.class), Mockito.nullable(TracingContext.class)); + + Mockito.doAnswer(answer -> { + AbfsRestOperation op = (AbfsRestOperation) answer.callRealMethod(); + createCalled.set(true); + while (!parallelRenameDone.get()); + return op; + }) + .when(client) + .createPath(Mockito.anyString(), Mockito.anyBoolean(), + Mockito.anyBoolean(), Mockito.nullable(String.class), + Mockito.nullable(String.class), Mockito.anyBoolean(), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + + new Thread(() -> { + try { + while(!createCalled.get()); + getFileSystem().rename(new Path("/hbase/dir/"), new Path("/hbase/dir2")); + } catch (Exception e) { + exceptionCaught.set(true); + } finally { + parallelRenameDone.set(true); + } + }).start(); + + fileSystem.createFile(new Path("/hbase/dir/file1")) + .overwrite(false) + .replication((short) 1) + .bufferSize(1024) + .blockSize(1024) + .build(); + + Assert.assertTrue(exceptionCaught.get()); + Assert.assertTrue(fileSystem.exists(new Path("/hbase/dir/file"))); + } + + @Test + public void testActiveCreateNonRecursiveNotDenyParallelRenameOnAtomicDirIfLeaseConfigDisabled() throws Exception { + Assume.assumeTrue(getPrefixMode(getFileSystem()) == PrefixMode.BLOB); + Configuration configuration = Mockito.spy(getRawConfiguration()); + AzureBlobFileSystem fileSystem = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + AbfsClient client = Mockito.spy(fileSystem.getAbfsClient()); + fileSystem.getAbfsStore().setClient(client); + fileSystem.setWorkingDirectory(new Path("/")); + fileSystem.mkdirs(new Path("/hbase/dir")); + fileSystem.create(new Path("/hbase/dir/file")); + AtomicBoolean createCalled = new AtomicBoolean(false); + AtomicBoolean parallelRenameDone = new AtomicBoolean(false); + AtomicBoolean exceptionCaught = new AtomicBoolean(false); + + Mockito.doAnswer(answer -> { + AbfsRestOperation op = (AbfsRestOperation) answer.callRealMethod(); + createCalled.set(true); + while(!parallelRenameDone.get()); + return op; + }).when(client).createPathBlob(Mockito.anyString(), Mockito.anyBoolean(), + Mockito.anyBoolean(), Mockito.nullable(HashMap.class), Mockito.nullable(String.class), Mockito.nullable(TracingContext.class)); + + Mockito.doAnswer(answer -> { + AbfsRestOperation op = (AbfsRestOperation) answer.callRealMethod(); + createCalled.set(true); + while (!parallelRenameDone.get()); + return op; + }) + .when(client) + .createPath(Mockito.anyString(), Mockito.anyBoolean(), + Mockito.anyBoolean(), Mockito.nullable(String.class), + Mockito.nullable(String.class), Mockito.anyBoolean(), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + + new Thread(() -> { + try { + while(!createCalled.get()); + getFileSystem().rename(new Path("/hbase/dir/"), new Path("/hbase/dir2")); + } catch (Exception e) { + exceptionCaught.set(true); + } finally { + parallelRenameDone.set(true); + } + }).start(); + + fileSystem.createFile(new Path("/hbase/dir/file1")) + .overwrite(false) + .replication((short) 1) + .bufferSize(1024) + .blockSize(1024) + .build(); + + Assert.assertFalse(exceptionCaught.get()); + Assert.assertFalse(fileSystem.exists(new Path("/hbase/dir/file"))); + Assert.assertTrue(fileSystem.exists(new Path("/hbase/dir2/file"))); + } + /** * Attempts to use to the ABFS stream after it is closed. */ @@ -178,7 +1150,7 @@ public void testTryWithResources() throws Throwable { out.write('2'); out.hsync(); fail("Expected a failure"); - } catch (FileNotFoundException fnfe) { + } catch (IOException fnfe) { //appendblob outputStream does not generate suppressed exception on close as it is //single threaded code if (!fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(testPath).toString())) { @@ -204,7 +1176,7 @@ public void testFilterFSWriteAfterClose() throws Throwable { final AzureBlobFileSystem fs = getFileSystem(); Path testPath = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE); FSDataOutputStream out = fs.create(testPath); - intercept(FileNotFoundException.class, + intercept(IOException.class, () -> { try (FilterOutputStream fos = new FilterOutputStream(out)) { fos.write('a'); @@ -212,12 +1184,12 @@ public void testFilterFSWriteAfterClose() throws Throwable { out.hsync(); fs.delete(testPath, false); // trigger the first failure - throw intercept(FileNotFoundException.class, + throw intercept(IOException.class, () -> { - fos.write('b'); - out.hsync(); - return "hsync didn't raise an IOE"; - }); + fos.write('b'); + out.hsync(); + return "hsync didn't raise an IOE"; + }); } }); } @@ -262,13 +1234,18 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) fs.create(nonOverwriteFile, false); // One request to server to create path should be issued - createRequestCount++; + // two calls added for - + // 1. getFileStatus on DFS endpoint : 1 + // getFileStatus on Blob endpoint: 2 (Additional List blob call) + // 2. actual create call: 1 + createRequestCount += (useBlobEndpoint ? 3: 2); assertAbfsStatistics( CONNECTIONS_MADE, totalConnectionMadeBeforeTest + createRequestCount, fs.getInstrumentationMap()); + // Case 2: Not Overwrite - File pre-exists fs.registerListener(new TracingHeaderValidator( fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), @@ -278,7 +1255,11 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) fs.registerListener(null); // One request to server to create path should be issued - createRequestCount++; + // Only single tryGetFileStatus should happen + // 1. getFileStatus on DFS endpoint : 1 + // getFileStatus on Blob endpoint: 1 (No Additional List blob call as file exists) + + createRequestCount ++; assertAbfsStatistics( CONNECTIONS_MADE, @@ -292,8 +1273,12 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) // create should be successful fs.create(overwriteFilePath, true); - // One request to server to create path should be issued - createRequestCount++; + /// One request to server to create path should be issued + // two calls added for - + // 1. getFileStatus on DFS endpoint : 1 + // getFileStatus on Blob endpoint: 2 (Additional List blob call for non-existing path) + // 2. actual create call: 1 + createRequestCount += (useBlobEndpoint ? 3: 2); assertAbfsStatistics( CONNECTIONS_MADE, @@ -307,6 +1292,14 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) fs.create(overwriteFilePath, true); fs.registerListener(null); + // One request to server to create path should be issued + // First tryGetFileStatus should happen + // 1. getFileStatus on DFS endpoint : 1 + // getFileStatus on Blob endpoint: 1 (No Additional List blob call as file exists) + + createRequestCount ++; + + // Second actual create call will hap if (enableConditionalCreateOverwrite) { // Three requests will be sent to server to create path, // 1. create without overwrite @@ -350,17 +1343,18 @@ public void testNegativeScenariosForCreateOverwriteDisabled() Boolean.toString(true)); final AzureBlobFileSystem fs = - (AzureBlobFileSystem) FileSystem.newInstance(currentFs.getUri(), - config); + Mockito.spy((AzureBlobFileSystem) FileSystem.newInstance(currentFs.getUri(), + config)); // Get mock AbfsClient with current config AbfsClient mockClient - = TestAbfsClient.getMockAbfsClient( + = ITestAbfsClient.getMockAbfsClient( fs.getAbfsStore().getClient(), fs.getAbfsStore().getAbfsConfiguration()); - AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); + AzureBlobFileSystemStore abfsStore = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(abfsStore).when(fs).getAbfsStore(); abfsStore = setAzureBlobSystemStoreField(abfsStore, "client", mockClient); boolean isNamespaceEnabled = abfsStore .getIsNamespaceEnabled(getTestTracingContext(fs, false)); @@ -369,6 +1363,7 @@ public void testNegativeScenariosForCreateOverwriteDisabled() AbfsRestOperation.class); AbfsHttpOperation http200Op = mock( AbfsHttpOperation.class); + AzureBlobFileSystemStore.VersionedFileStatus fileStatus = mock(AzureBlobFileSystemStore.VersionedFileStatus.class); when(http200Op.getStatusCode()).thenReturn(HTTP_OK); when(successOp.getResult()).thenReturn(http200Op); @@ -396,12 +1391,34 @@ public void testNegativeScenariosForCreateOverwriteDisabled() isNamespaceEnabled ? any(String.class) : eq(null), any(boolean.class), eq(null), any(TracingContext.class)); + // mock for overwrite=false + doThrow(conflictResponseEx) // Scn1: GFS fails with Http404 + .doThrow(conflictResponseEx) // Scn2: GFS fails with Http500 + .doThrow( + conflictResponseEx) // Scn3: create overwrite=true fails with Http412 + .doThrow( + conflictResponseEx) // Scn4: create overwrite=true fails with Http500 + .doThrow( + serverErrorResponseEx) // Scn5: create overwrite=false fails with Http500 + .when(mockClient) + .createPathBlob(any(String.class), eq(true), eq(false), + any(), eq(null), any(TracingContext.class)); + doThrow(fileNotFoundResponseEx) // Scn1: GFS fails with Http404 - .doThrow(serverErrorResponseEx) // Scn2: GFS fails with Http500 - .doReturn(successOp) // Scn3: create overwrite=true fails with Http412 - .doReturn(successOp) // Scn4: create overwrite=true fails with Http500 + .doThrow(serverErrorResponseEx) // Scn2: GFS fails with Http500 + .doReturn(fileStatus) + .doReturn(fileStatus) + .when(abfsStore) + .getFileStatus(any(Path.class), any(TracingContext.class), anyBoolean()); + + // mock for overwrite=true + doThrow( + preConditionResponseEx) // Scn3: create overwrite=true fails with Http412 + .doThrow( + serverErrorResponseEx) // Scn4: create overwrite=true fails with Http500 .when(mockClient) - .getPathStatus(any(String.class), eq(false), any(TracingContext.class)); + .createPathBlob(any(String.class), eq(true), eq(true), + any(), eq(null), any(TracingContext.class)); // mock for overwrite=true doThrow( @@ -479,10 +1496,74 @@ private void validateCreateFileException(final Class ex intercept( exceptionClass, () -> abfsStore.createFile(testPath, null, true, permission, umask, - getTestTracingContext(getFileSystem(), true))); + getTestTracingContext(getFileSystem(), true), null)); } private AbfsRestOperationException getMockAbfsRestOperationException(int status) { return new AbfsRestOperationException(status, "", "", new Exception()); } + + /** + * Attempts to test multiple flush calls. + */ + @Test + public void testMultipleFlush() throws Throwable { + final AzureBlobFileSystem fs = getFileSystem(); + Path testPath = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE); + try (FSDataOutputStream out = fs.create(testPath)) { + out.write('1'); + out.hsync(); + out.write('2'); + out.hsync(); + } + } + + /** + * Delete the blob before flush and verify that an exception should be thrown. + */ + @Test + public void testDeleteBeforeFlush() throws Throwable { + final AzureBlobFileSystem fs = getFileSystem(); + Path testPath = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE); + try (FSDataOutputStream out = fs.create(testPath)) { + out.write('1'); + fs.delete(testPath, false); + out.hsync(); + // this will cause the next write to failAll + } catch (IOException fnfe) { + //appendblob outputStream does not generate suppressed exception on close as it is + //single threaded code + if (!fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(testPath).toString())) { + // the exception raised in close() must be in the caught exception's + // suppressed list + Throwable[] suppressed = fnfe.getSuppressed(); + assertEquals("suppressed count", 1, suppressed.length); + Throwable inner = suppressed[0]; + if (!(inner instanceof IOException)) { + throw inner; + } + GenericTestUtils.assertExceptionContains(fnfe.getMessage(), inner.getCause(), inner.getCause().getMessage()); + } + } + } + + /** + * Extracts the eTag for an existing file + * @param fileName file Path in String from container root + * @return String etag for the file + * @throws IOException + */ + private String extractFileEtag(String fileName) throws IOException { + final AzureBlobFileSystem fs = getFileSystem(); + final AbfsClient client = fs.getAbfsClient(); + final TracingContext testTracingContext = getTestTracingContext(fs, false); + AbfsRestOperation op; + if (fs.getAbfsStore().getPrefixMode() == PrefixMode.BLOB) { + op = client.getBlobProperty(new Path(fileName), testTracingContext); + } else { + op = client.getPathStatus(fileName, true, testTracingContext); + } + return AzureBlobFileSystemStore.extractEtagHeader(op.getResult()); + } } + diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSASForBlobEndpoint.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSASForBlobEndpoint.java new file mode 100644 index 00000000000000..32fdc78394779d --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSASForBlobEndpoint.java @@ -0,0 +1,177 @@ +/** + * 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 java.util.List; + +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.extensions.MockDelegationSASTokenProvider; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; +import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; +import org.apache.hadoop.fs.azurebfs.services.AuthType; +import org.apache.hadoop.fs.azurebfs.services.BlobProperty; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_PENDING; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE; + +public class ITestAzureBlobFileSystemDelegationSASForBlobEndpoint + extends AbstractAbfsIntegrationTest { + + public ITestAzureBlobFileSystemDelegationSASForBlobEndpoint() + throws Exception { + // These tests rely on specific settings in azure-auth-keys.xml: + String sasProvider = getRawConfiguration().get( + FS_AZURE_SAS_TOKEN_PROVIDER_TYPE); + Assume.assumeTrue(MockDelegationSASTokenProvider.class.getCanonicalName() + .equals(sasProvider)); + Assume.assumeNotNull( + getRawConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_APP_ID)); + Assume.assumeNotNull(getRawConfiguration().get( + TestConfigurationKeys.FS_AZURE_TEST_APP_SECRET)); + Assume.assumeNotNull(getRawConfiguration().get( + TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_TENANT_ID)); + Assume.assumeNotNull(getRawConfiguration().get( + TestConfigurationKeys.FS_AZURE_TEST_APP_SERVICE_PRINCIPAL_OBJECT_ID)); + // The test uses shared key to create a random filesystem and then creates another + // instance of this filesystem using SAS authorization. + Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); + } + + @Override + public void setup() throws Exception { + createFilesystemForSASTests(); + super.setup(); + Assume.assumeTrue( + getFileSystem().getAbfsStore().getAbfsConfiguration().getPrefixMode() + == PrefixMode.BLOB); + } + + @Before + public void before() throws Exception { + Assume.assumeTrue( + getFileSystem().getAbfsStore().getAbfsConfiguration().getPrefixMode() + == PrefixMode.BLOB); + } + + @Test + public void testPosixRenameDirectoryWhereDirectoryAlreadyThereOnDestination() + throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.mkdirs(new Path("testDir2/test1/test2/test3")); + fs.create(new Path("testDir2/test1/test2/test3/file")); + fs.mkdirs(new Path("testDir2/test4/test3")); + assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file"))); + Assert.assertFalse(fs.rename(new Path("testDir2/test1/test2/test3"), + new Path("testDir2/test4"))); + assertTrue(fs.exists(new Path("testDir2"))); + assertTrue(fs.exists(new Path("testDir2/test1/test2"))); + assertTrue(fs.exists(new Path("testDir2/test4"))); + assertTrue(fs.exists(new Path("testDir2/test1/test2/test3"))); + if (getIsNamespaceEnabled(fs) || fs.getAbfsStore() + .getAbfsConfiguration() + .get("fs.azure.abfs.account.name") + .contains(".dfs.core") || fs.getAbfsStore() + .getAbfsConfiguration() + .get("fs.azure.abfs.account.name") + .contains(".blob.core")) { + assertFalse(fs.exists(new Path("testDir2/test4/test3/file"))); + assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file"))); + } else { + assertTrue(fs.exists(new Path("testDir2/test4/test3/file"))); + assertFalse(fs.exists(new Path("testDir2/test1/test2/test3/file"))); + } + } + + @Test + public void testPosixRenameDirectory() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.mkdirs(new Path("testDir2/test1/test2/test3")); + fs.mkdirs(new Path("testDir2/test4")); + Assert.assertTrue(fs.rename(new Path("testDir2/test1/test2/test3"), + new Path("testDir2/test4"))); + assertTrue(fs.exists(new Path("testDir2"))); + assertTrue(fs.exists(new Path("testDir2/test1/test2"))); + assertTrue(fs.exists(new Path("testDir2/test4"))); + assertTrue(fs.exists(new Path("testDir2/test4/test3"))); + assertFalse(fs.exists(new Path("testDir2/test1/test2/test3"))); + } + + @Test + public void testCopyBlobTakeTime() throws Exception { + AzureBlobFileSystem fileSystem = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fileSystem.getAbfsStore()); + Mockito.doReturn(store).when(fileSystem).getAbfsStore(); + AbfsClient client = store.getClient(); + AbfsClient spiedClient = Mockito.spy(client); + store.setClient(spiedClient); + + Mockito.doAnswer(answer -> { + AbfsRestOperation op = Mockito.spy((AbfsRestOperation) answer.callRealMethod()); + AbfsHttpOperation httpOp = Mockito.spy(op.getResult()); + Mockito.doReturn(COPY_STATUS_PENDING).when(httpOp).getResponseHeader( + HttpHeaderConfigurations.X_MS_COPY_STATUS); + Mockito.doReturn(httpOp).when(op).getResult(); + return op; + }).when(spiedClient).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + fileSystem.create(new Path("/test1/file")); + fileSystem.rename(new Path("/test1/file"), new Path("/test1/file2")); + Assert.assertTrue(fileSystem.exists(new Path("/test1/file2"))); + Mockito.verify(store, Mockito.times(1)) + .handleCopyInProgress(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.any(String.class)); + } + + @Test + public void testListBlob() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + int i = 0; + while (i < 10) { + fs.create(new Path("/dir/" + i)); + i++; + } + List blobProperties = fs.getAbfsStore() + .getListBlobs(new Path("dir"), null, null, + getTestTracingContext(fs, true), null, false); + Assertions.assertThat(blobProperties) + .describedAs( + "BlobList should match the number of files created in tests + the directory itself") + .hasSize(11); + + blobProperties = fs.getAbfsStore() + .getListBlobs(new Path("dir"), null, null, + getTestTracingContext(fs, true), null, true); + Assertions.assertThat(blobProperties) + .describedAs( + "BlobList should match the number of files created in tests") + .hasSize(10); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index 8afe3d4bca1b34..3bdbddd8dedcbb 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -19,23 +19,35 @@ package org.apache.hadoop.fs.azurebfs; import java.io.FileNotFoundException; +import java.io.IOException; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.assertj.core.api.Assertions; +import org.junit.Assert; import org.junit.Assume; +import org.junit.Ignore; import org.junit.Test; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil; import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; -import org.apache.hadoop.fs.azurebfs.services.TestAbfsClient; +import org.apache.hadoop.fs.azurebfs.services.ListBlobProducer; +import org.apache.hadoop.fs.azurebfs.services.ListBlobQueue; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; +import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient; import org.apache.hadoop.fs.azurebfs.services.TestAbfsPerfTracker; import org.apache.hadoop.fs.azurebfs.utils.TestMockHelpers; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; @@ -43,11 +55,19 @@ import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.test.LambdaTestUtils; + +import org.mockito.Mockito; import static java.net.HttpURLConnection.HTTP_BAD_REQUEST; +import static java.net.HttpURLConnection.HTTP_FORBIDDEN; import static java.net.HttpURLConnection.HTTP_NOT_FOUND; import static java.net.HttpURLConnection.HTTP_OK; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_MKDIRS_FALLBACK_TO_DFS; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_PRODUCER_QUEUE_MAX_SIZE; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_REDIRECT_DELETE; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_REDIRECT_RENAME; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doCallRealMethod; import static org.mockito.Mockito.doReturn; @@ -115,6 +135,20 @@ public void testEnsureFileIsDeleted() throws Exception { assertPathDoesNotExist(fs, "deleted", testfile); } + @Test + public void testEnsureFileIsDeletedWithRedirection() throws Exception { + + // Set redirect to wasb delete as true and assert deletion. + Configuration configuration = Mockito.spy(getRawConfiguration()); + configuration.setBoolean(FS_AZURE_REDIRECT_DELETE, true); + AzureBlobFileSystem fs1 = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + + Path testfile = makeQualified(new Path("testfile")); + touch(testfile); + assertDeleted(fs1, testfile, false); + assertPathDoesNotExist(fs1, "deleted", testfile); + } + @Test public void testDeleteDirectory() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); @@ -127,6 +161,7 @@ public void testDeleteDirectory() throws Exception { assertPathDoesNotExist(fs, "deleted", dir); } + @Ignore @Test public void testDeleteFirstLevelDirectory() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); @@ -156,7 +191,7 @@ public Void call() throws Exception { fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), fs.getFileSystemId(), FSOperationType.DELETE, false, 0)); // first try a non-recursive delete, expect failure - intercept(FileAlreadyExistsException.class, + intercept(IOException.class, () -> fs.delete(dir, false)); fs.registerListener(null); assertDeleted(fs, dir, true); @@ -175,7 +210,7 @@ public void testDeleteIdempotency() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); AbfsClient abfsClient = fs.getAbfsStore().getClient(); - AbfsClient testClient = TestAbfsClient.createTestClientFromCurrentContext( + AbfsClient testClient = ITestAbfsClient.createTestClientFromCurrentContext( abfsClient, abfsConfig); @@ -222,7 +257,7 @@ public void testDeleteIdempotency() throws Exception { public void testDeleteIdempotencyTriggerHttp404() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); - AbfsClient client = TestAbfsClient.createTestClientFromCurrentContext( + AbfsClient client = ITestAbfsClient.createTestClientFromCurrentContext( fs.getAbfsStore().getClient(), this.getConfiguration()); @@ -241,7 +276,7 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { getTestTracingContext(fs, true))); // mock idempotency check to mimic retried case - AbfsClient mockClient = TestAbfsClient.getMockAbfsClient( + AbfsClient mockClient = ITestAbfsClient.getMockAbfsClient( fs.getAbfsStore().getClient(), this.getConfiguration()); AzureBlobFileSystemStore mockStore = mock(AzureBlobFileSystemStore.class); @@ -256,10 +291,10 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { // Case 2: Mimic retried case // Idempotency check on Delete always returns success - AbfsRestOperation idempotencyRetOp = TestAbfsClient.getRestOp( + AbfsRestOperation idempotencyRetOp = ITestAbfsClient.getRestOp( DeletePath, mockClient, HTTP_METHOD_DELETE, - TestAbfsClient.getTestUrl(mockClient, "/NonExistingPath"), - TestAbfsClient.getTestRequestHeaders(mockClient)); + ITestAbfsClient.getTestUrl(mockClient, "/NonExistingPath"), + ITestAbfsClient.getTestRequestHeaders(mockClient)); idempotencyRetOp.hardSetResult(HTTP_OK); doReturn(idempotencyRetOp).when(mockClient).deleteIdempotencyCheckOp(any()); @@ -282,4 +317,291 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { mockStore.delete(new Path("/NonExistingPath"), false, getTestTracingContext(fs, false)); } + @Test + public void testDeleteImplicitDir() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + Assume.assumeTrue(fs.getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + fs.mkdirs(new Path("/testDir/dir1")); + fs.create(new Path("/testDir/dir1/file1")); + fs.getAbfsClient().deleteBlobPath(new Path("/testDir/dir1"), + null, getTestTracingContext(fs, true)); + + fs.delete(new Path("/testDir/dir1"), true); + + Assert.assertTrue(!fs.exists(new Path("/testDir/dir1"))); + Assert.assertTrue(!fs.exists(new Path("/testDir/dir1/file1"))); + } + + @Test + public void testDeleteImplicitDirWithSingleListResults() throws Exception { + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance( + getRawConfiguration()); + Assume.assumeTrue(fs.getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + AbfsClient client = fs.getAbfsClient(); + AbfsClient spiedClient = Mockito.spy(client); + fs.getAbfsStore().setClient(spiedClient); + fs.mkdirs(new Path("/testDir/dir1")); + for (int i = 0; i < 10; i++) { + fs.create(new Path("/testDir/dir1/file" + i)); + } + Mockito.doAnswer(answer -> { + String marker = answer.getArgument(0); + String prefix = answer.getArgument(1); + TracingContext context = answer.getArgument(4); + return client.getListBlobs(marker, prefix, null, 1, context); + }) + .when(spiedClient) + .getListBlobs(Mockito.nullable(String.class), Mockito.anyString(), + Mockito.nullable(String.class), Mockito.nullable(Integer.class), + Mockito.any(TracingContext.class)); + fs.getAbfsClient().deleteBlobPath(new Path("/testDir/dir1"), + null, getTestTracingContext(fs, true)); + + fs.delete(new Path("/testDir/dir1"), true); + + Assert.assertTrue(!fs.exists(new Path("/testDir/dir1"))); + } + + @Test + public void testDeleteExplicitDirInImplicitParentDir() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + Assume.assumeTrue(fs.getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + fs.mkdirs(new Path("/testDir/dir1")); + fs.create(new Path("/testDir/dir1/file1")); + fs.getAbfsClient().deleteBlobPath(new Path("/testDir/"), + null, getTestTracingContext(fs, true)); + + fs.delete(new Path("/testDir/dir1"), true); + + Assert.assertTrue(!fs.exists(new Path("/testDir/dir1"))); + Assert.assertTrue(!fs.exists(new Path("/testDir/dir1/file1"))); + Assert.assertTrue(fs.exists(new Path("/testDir/"))); + } + + @Test + public void testDeleteParallelBlobFailure() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + Assume.assumeTrue(fs.getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + store.setClient(client); + Mockito.doReturn(store).when(fs).getAbfsStore(); + + fs.mkdirs(new Path("/testDir")); + fs.create(new Path("/testDir/file1")); + fs.create(new Path("/testDir/file2")); + fs.create(new Path("/testDir/file3")); + + Mockito.doThrow( + new AbfsRestOperationException(HTTP_FORBIDDEN, "", "", new Exception())) + .when(client) + .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + + LambdaTestUtils.intercept(RuntimeException.class, () -> { + fs.delete(new Path("/testDir"), true); + }); + } + + @Test + public void testDeleteRootWithNonRecursion() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + fs.mkdirs(new Path("/testDir")); + Assertions.assertThat(fs.delete(new Path("/"), false)).isFalse(); + } + + @Test + public void testDeleteCheckIfParentLMTChange() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + AbfsConfiguration conf = getConfiguration(); + /* + * LMT of parent directory doesn't change when delete directory triggered with + * DFS endpoint (both hns and non-hns account). In Blob endpoint, if there + * is no redirect for ingress / mkdirs, the LMT doesn't change. But in case + * of ingress redirection, the directory creation of parent overrides the + * path which changes the LMT. Hence, for tests running with redirect + * configuration, this test is ignored. + */ + Assume.assumeFalse( + getPrefixMode(fs) == PrefixMode.BLOB && (conf.shouldMkdirFallbackToDfs() + || conf.shouldIngressFallbackToDfs())); + fs.mkdirs(new Path("/dir1/dir2")); + fs.create(new Path("/dir1/dir2/file")); + FileStatus status = fs.getFileStatus(new Path("/dir1")); + Long lmt = status.getModificationTime(); + + fs.delete(new Path("/dir1/dir2"), true); + Long newLmt = fs.getFileStatus(new Path("/dir1")).getModificationTime(); + Assertions.assertThat(lmt).isEqualTo(newLmt); + } + + /** + * Test to assert that the CID in src marker delete contains the + * total number of blobs operated in the delete directory. + * Also, to assert that all operations in the delete-directory flow have same + * primaryId and opType. + */ + @Test + public void testDeleteEmitDeletionCountInClientRequestId() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + Assume.assumeTrue(getPrefixMode(fs) == PrefixMode.BLOB); + String dirPathStr = "/testDir/dir1"; + fs.mkdirs(new Path(dirPathStr)); + ExecutorService executorService = Executors.newFixedThreadPool(5); + List futures = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + final int iter = i; + Future future = executorService.submit(() -> { + return fs.create(new Path("/testDir/dir1/file" + iter)); + }); + futures.add(future); + } + + for (Future future : futures) { + future.get(); + } + executorService.shutdown(); + + + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + AbfsClient client = Mockito.spy(store.getClient()); + store.setClient(client); + + final TracingHeaderValidator tracingHeaderValidator + = new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.DELETE, true, 0); + fs.registerListener(tracingHeaderValidator); + + Mockito.doAnswer(answer -> { + Mockito.doAnswer(deleteAnswer -> { + if (dirPathStr.equalsIgnoreCase( + ((Path) deleteAnswer.getArgument(0)).toUri().getPath())) { + tracingHeaderValidator.setOperatedBlobCount(11); + Object result = deleteAnswer.callRealMethod(); + tracingHeaderValidator.setOperatedBlobCount(null); + return result; + } + return deleteAnswer.callRealMethod(); + }) + .when(client) + .deleteBlobPath(Mockito.any(Path.class), + Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + + return answer.callRealMethod(); + }) + .when(store) + .delete(Mockito.any(Path.class), Mockito.anyBoolean(), + Mockito.any(TracingContext.class)); + + fs.delete(new Path(dirPathStr), true); + } + + @Test + public void testProducerStopOnDeleteFailure() throws Exception { + Assume.assumeTrue(getPrefixMode(getFileSystem()) == PrefixMode.BLOB); + Configuration configuration = Mockito.spy(getRawConfiguration()); + AzureBlobFileSystem fs = Mockito.spy( + (AzureBlobFileSystem) FileSystem.get(configuration)); + + fs.mkdirs(new Path("/src")); + ExecutorService executorService = Executors.newFixedThreadPool(10); + List futureList = new ArrayList<>(); + for (int i = 0; i < 20; i++) { + int iter = i; + Future future = executorService.submit(() -> { + try { + fs.create(new Path("/src/file" + iter)); + } catch (IOException ex) {} + }); + futureList.add(future); + } + + for (Future future : futureList) { + future.get(); + } + + AbfsClient client = fs.getAbfsClient(); + AbfsClient spiedClient = Mockito.spy(client); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + store.setClient(spiedClient); + Mockito.doReturn(store).when(fs).getAbfsStore(); + + ListBlobProducer[] producers = new ListBlobProducer[1]; + Mockito.doAnswer(answer -> { + producers[0] = (ListBlobProducer) answer.callRealMethod(); + return producers[0]; + }).when(store).getListBlobProducer(Mockito.anyString(), Mockito.any( + ListBlobQueue.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + + AtomicInteger listCounter = new AtomicInteger(0); + AtomicBoolean hasConsumerStarted = new AtomicBoolean(false); + + Mockito.doAnswer(answer -> { + String marker = answer.getArgument(0); + String prefix = answer.getArgument(1); + String delimiter = answer.getArgument(2); + TracingContext tracingContext = answer.getArgument(4); + int counter = listCounter.incrementAndGet(); + if (counter > 1) { + while (!hasConsumerStarted.get()) { + Thread.sleep(1_000L); + } + } + Object result = client.getListBlobs(marker, prefix, delimiter, 1, + tracingContext); + return result; + }) + .when(spiedClient) + .getListBlobs(Mockito.nullable(String.class), + Mockito.nullable(String.class), Mockito.nullable(String.class), + Mockito.nullable(Integer.class), Mockito.any(TracingContext.class)); + + Mockito.doAnswer(answer -> { + spiedClient.acquireBlobLease( + ((Path) answer.getArgument(0)).toUri().getPath(), -1, + answer.getArgument(2)); + hasConsumerStarted.set(true); + return answer.callRealMethod(); + }) + .when(spiedClient) + .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + + intercept(Exception.class, () -> { + fs.delete(new Path("/src"), true); + }); + + producers[0].waitForProcessCompletion(); + + Mockito.verify(spiedClient, Mockito.atMost(3)) + .getListBlobs(Mockito.nullable(String.class), + Mockito.nullable(String.class), Mockito.nullable(String.class), + Mockito.nullable(Integer.class), Mockito.any(TracingContext.class)); + } + + @Test + public void deleteBlobDirParallelThreadToDeleteOnDifferentTracingContext() + throws Exception { + Assume.assumeTrue(getPrefixMode(getFileSystem()) == PrefixMode.BLOB); + Configuration configuration = getRawConfiguration(); + AzureBlobFileSystem fs = Mockito.spy( + (AzureBlobFileSystem) FileSystem.newInstance(configuration)); + AzureBlobFileSystemStore spiedStore = Mockito.spy(fs.getAbfsStore()); + AbfsClient spiedClient = Mockito.spy(fs.getAbfsClient()); + + Mockito.doReturn(spiedStore).when(fs).getAbfsStore(); + spiedStore.setClient(spiedClient); + + fs.mkdirs(new Path("/testDir")); + fs.create(new Path("/testDir/file1")); + fs.create(new Path("/testDir/file2")); + + AbfsClientTestUtil.hookOnRestOpsForTracingContextSingularity(spiedClient); + + fs.delete(new Path("/testDir"), true); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java index 05c3855f5c89de..e525e709affbdd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java @@ -18,11 +18,14 @@ package org.apache.hadoop.fs.azurebfs; +import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; import java.util.Arrays; import java.util.Random; +import org.apache.hadoop.fs.azurebfs.services.OperativeEndpoint; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; import org.junit.Test; import org.apache.hadoop.conf.Configuration; @@ -195,7 +198,7 @@ public void testWriteWithFileNotFoundException() throws Exception { assertFalse(fs.exists(testFilePath)); // trigger append call - intercept(FileNotFoundException.class, + intercept(IOException.class, () -> stream.close()); } @@ -213,8 +216,13 @@ public void testFlushWithFileNotFoundException() throws Exception { fs.delete(testFilePath, true); assertFalse(fs.exists(testFilePath)); - intercept(FileNotFoundException.class, - () -> stream.close()); + // No call goes to backend for empty map in case of blob endpoint. + if (OperativeEndpoint.isIngressEnabledOnDFS(getPrefixMode(fs), fs.getAbfsStore().getAbfsConfiguration())) { + intercept(FileNotFoundException.class, + () -> stream.close()); + } else { + stream.close(); + } } private void testWriteOneByteToFile(Path testFilePath) throws Exception { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemExplictImplicitRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemExplictImplicitRename.java new file mode 100644 index 00000000000000..f597e3f3419215 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemExplictImplicitRename.java @@ -0,0 +1,964 @@ +/** + * 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 java.io.IOException; +import java.net.HttpURLConnection; + +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Test; +import org.mockito.Mockito; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +public class ITestAzureBlobFileSystemExplictImplicitRename + extends AbstractAbfsIntegrationTest { + + public ITestAzureBlobFileSystemExplictImplicitRename() throws Exception { + super(); + } + + @Override + public void setup() throws Exception { + super.setup(); + Assume.assumeTrue( + getFileSystem().getAbfsStore().getAbfsConfiguration().getPrefixMode() + == PrefixMode.BLOB); + } + + @Test + public void testRenameSrcFileInImplicitParentDirectory() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + createAzCopyDirectory(new Path("/src")); + createAzCopyFile(new Path("/src/file")); + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore().getBlobProperty(new Path("/src"), getTestTracingContext(fs, true)); + }); + Assert.assertNotNull(fs.getAbfsStore() + .getBlobProperty(new Path("/src/file"), + getTestTracingContext(fs, true))); + Assert.assertTrue(fs.rename(new Path("/src/file"), new Path("/dstFile"))); + Assert.assertNotNull(fs.getAbfsStore() + .getBlobProperty(new Path("/dstFile"), + getTestTracingContext(fs, true))); + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore() + .getBlobProperty(new Path("/src/file"), + getTestTracingContext(fs, true)); + }); + + Assert.assertFalse(fs.rename(new Path("/src/file"), new Path("/dstFile2"))); + } + + @Test + public void testRenameNonExistentFileInImplicitParent() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + createAzCopyDirectory(new Path("/src")); + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore().getBlobProperty(new Path("/src"), getTestTracingContext(fs, true)); + }); + + Assert.assertFalse(fs.rename(new Path("/src/file"), new Path("/dstFile2"))); + } + + @Test + public void testRenameFileToNonExistingDstInImplicitParent() + throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + createAzCopyFile(new Path("/file")); + createAzCopyDirectory(new Path("/dstDir")); + createAzCopyFile(new Path("/dstDir/file2")); + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore() + .getBlobProperty(new Path("/dstDir"), + getTestTracingContext(fs, true)); + }); + Assert.assertTrue(fs.rename(new Path("/file"), new Path("/dstDir"))); + Assert.assertTrue(fs.exists(new Path("/dstDir/file"))); + } + + @Test + public void testRenameFileAsExistingExplicitDirectoryInImplicitDirectory() + throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + createAzCopyFile(new Path("/file")); + createAzCopyDirectory(new Path("/dst")); + fs.mkdirs(new Path("/dst/dir")); + deleteBlobPath(fs, new Path("/dst")); + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore() + .getBlobProperty(new Path("/dst"), + getTestTracingContext(fs, true)); + }); + Assert.assertTrue(fs.rename(new Path("/file"), new Path("/dst/dir"))); + Assert.assertTrue(fs.exists(new Path("/dst/dir/file"))); + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore() + .getBlobProperty(new Path("/file"), + getTestTracingContext(fs, true)); + }); + } + + @Test + public void testRenameFileAsExistingImplicitDirectoryInExplicitDirectory() + throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + createAzCopyFile(new Path("/file")); + fs.mkdirs(new Path("/dst")); + createAzCopyDirectory(new Path("/dst/dir")); + createAzCopyFile(new Path("/dst/dir/file2")); + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore() + .getBlobProperty(new Path("/dst/dir"), + getTestTracingContext(fs, true)); + }); + Assert.assertTrue(fs.rename(new Path("/file"), new Path("/dst/dir"))); + Assert.assertTrue(fs.exists(new Path("/dst/dir/file"))); + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore() + .getBlobProperty(new Path("/file"), + getTestTracingContext(fs, true)); + }); + } + + @Test + public void testRenameFileAsExistingImplicitDirectoryInImplicitDirectory() + throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + createAzCopyFile(new Path("/file")); + createAzCopyDirectory(new Path("/dst")); + createAzCopyDirectory(new Path("/dst/dir")); + createAzCopyFile(new Path("/dst/dir/file2")); + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore() + .getBlobProperty(new Path("/dst"), + getTestTracingContext(fs, true)); + }); + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore() + .getBlobProperty(new Path("/dst/dir"), + getTestTracingContext(fs, true)); + }); + Assert.assertTrue(fs.rename(new Path("/file"), new Path("/dst/dir"))); + Assert.assertTrue(fs.exists(new Path("/dst/dir/file"))); + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore() + .getBlobProperty(new Path("/file"), + getTestTracingContext(fs, true)); + }); + } + + @Test + public void testRenameDirectoryContainingImplicitDirectory() + throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + fs.mkdirs(new Path("/src")); + fs.mkdirs(new Path("/dst")); + createAzCopyDirectory(new Path("/src/subDir")); + createAzCopyFile(new Path("/src/subDir/subFile")); + createAzCopyFile(new Path("/src/subFile")); + Assert.assertTrue(fs.rename(new Path("/src"), new Path("/dst/dir"))); + Assert.assertTrue(fs.exists(new Path("/dst/dir/subFile"))); + Assert.assertTrue(fs.exists(new Path("/dst/dir/subDir/subFile"))); + } + + @Test + public void testRenameImplicitDirectoryContainingExplicitDirectory() + throws Exception { + explicitImplicitDirectoryRenameTest( + /*srcParentExplicit*/true, + /*srcExplicit*/false, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/false, + /*isDstFile*/false, + /*shouldRenamePass*/true + ); + } + + @Test + public void testRenameImplicitDirectoryContainingImplicitDirectory() + throws Exception { + explicitImplicitDirectoryRenameTest( + /*srcParentExplicit*/true, + /*srcExplicit*/false, + /*srcSubDirExplicit*/false, + /*dstParentExplicit*/true, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/false, + /*isDstFile*/false, + /*shouldRenamePass*/true + ); + } + + @Test + public void testRenameExplicitDirectoryContainingExplicitDirectoryInImplicitSrcParent() + throws Exception { + explicitImplicitDirectoryRenameTest( + /*srcParentExplicit*/false, + /*srcExplicit*/true, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/false, + /*isDstFile*/false, + /*shouldRenamePass*/true + ); + } + + @Test + public void testRenameExplicitDirectoryContainingImplicitDirectoryInImplicitSrcParent() + throws Exception { + explicitImplicitDirectoryRenameTest( + /*srcParentExplicit*/false, + /*srcExplicit*/true, + /*srcSubDirExplicit*/false, + /*dstParentExplicit*/true, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/false, + /*isDstFile*/false, + /*shouldRenamePass*/true + ); + } + + @Test + public void testRenameImplicitDirectoryContainingExplicitDirectoryInImplicitSrcParent() + throws Exception { + explicitImplicitDirectoryRenameTest( + /*srcParentExplicit*/false, + /*srcExplicit*/false, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/false, + /*isDstFile*/false, + /*shouldRenamePass*/true + ); + } + + @Test + public void testRenameImplicitDirectoryContainingImplicitDirectoryInImplicitSrcParent() + throws Exception { + explicitImplicitDirectoryRenameTest( + /*srcParentExplicit*/false, + /*srcExplicit*/false, + /*srcSubDirExplicit*/false, + /*dstParentExplicit*/true, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/false, + /*isDstFile*/false, + /*shouldRenamePass*/true + ); + } + + @Test + public void testRenameDirectoryWhereDstParentDoesntExist() throws Exception { + explicitImplicitDirectoryRenameTest( + /*srcParentExplicit*/true, + /*srcExplicit*/true, + /*srcSubDirExplicit*/false, + /*dstParentExplicit*/false, + /*dstExplicit*/false, + /*dstParentExists*/false, + /*isDstParentFile*/false, + /*dstExist*/false, + /*isDstFile*/false, + /*shouldRenamePass*/false + ); + } + + @Test + public void testRenameImplicitDirectoryWhereDstParentDoesntExist() + throws Exception { + explicitImplicitDirectoryRenameTest( + /*srcParentExplicit*/true, + /*srcExplicit*/false, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/false, + /*dstExplicit*/false, + /*dstParentExists*/false, + /*isDstParentFile*/false, + /*dstExist*/false, + /*isDstFile*/false, + /*shouldRenamePass*/false + ); + } + + @Test + public void testRenameImplicitDirectoryToNonExistentDstWithImplicitParent() + throws Exception { + explicitImplicitDirectoryRenameTest( + /*srcParentExplicit*/true, + /*srcExplicit*/false, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/false, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/false, + /*isDstFile*/false, + /*shouldRenamePass*/true + ); + } + + @Test + public void testRenameImplicitDirectoryToNonExistentDstWithParentIsFile() + throws Exception { + explicitImplicitDirectoryRenameTest( + /*srcParentExplicit*/true, + /*srcExplicit*/false, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/false, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/true, + /*dstExist*/false, + /*isDstFile*/false, + /*shouldRenamePass*/false + ); + } + + @Test + public void testRenameExplicitDirectoryToFileDst() throws Exception { + explicitImplicitDirectoryRenameTest( + /*srcParentExplicit*/true, + /*srcExplicit*/true, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/true, + /*shouldRenamePass*/false + ); + } + + @Test + public void testRenameimplicitDirectoryToFileDst() throws Exception { + explicitImplicitDirectoryRenameTest( + /*srcParentExplicit*/true, + /*srcExplicit*/false, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/true, + /*shouldRenamePass*/false + ); + } + + @Test + public void testDirectoryIntoSameNameDestination() throws Exception { + explicitImplicitDirectoryRenameTestWithDestPathNames( + /*srcParentExplicit*/true, + /*srcExplicit*/true, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/true, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/false, + /*srcName*/"src", + /*dstName*/"src", + /*dstSubFileName*/null, + /*dstSubDirName*/null, + /*isSubDirExplicit*/true, /*shouldRenamePass*/true + ); + } + + @Test + public void testRenameDirectoryToSameNameImplicitDirectoryDestination() + throws Exception { + explicitImplicitDirectoryRenameTestWithDestPathNames( + /*srcParentExplicit*/true, + /*srcExplicit*/true, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/false, + /*srcName*/"src", + /*dstName*/"src", + /*dstSubFileName*/null, + /*dstSubDirName*/null, + /*isSubDirExplicit*/true, /*shouldRenamePass*/true + ); + } + + @Test + public void testRenameDirectoryToImplicitDirectoryDestinationHavingSameNameSubDir() + throws Exception { + explicitImplicitDirectoryRenameTestWithDestPathNames( + /*srcParentExplicit*/true, + /*srcExplicit*/true, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/false, + /*srcName*/"src", + /*dstName*/null, + /*dstSubFileName*/null, + /*dstSubDirName*/"src", + /*isSubDirExplicit*/true, /*shouldRenamePass*/false + ); + } + + @Test + public void testRenameDirectoryToImplicitDirectoryDestinationHavingSameNameSubFile() + throws Exception { + explicitImplicitDirectoryRenameTestWithDestPathNames( + /*srcParentExplicit*/true, + /*srcExplicit*/true, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/false, + /*srcName*/"src", + /*dstName*/null, + /*dstSubFileName*/"src", + /*dstSubDirName*/null, + /*isSubDirExplicit*/true, /*shouldRenamePass*/false + ); + } + + @Test + public void testRenameDirectoryToImplicitDirectoryDestinationHavingSameNameImplicitSubDir() + throws Exception { + explicitImplicitDirectoryRenameTestWithDestPathNames( + /*srcParentExplicit*/true, + /*srcExplicit*/true, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/false, + /*srcName*/"src", + /*dstName*/null, + /*dstSubFileName*/null, + /*dstSubDirName*/"src", + /*isSubDirExplicit*/false, /*shouldRenamePass*/false + ); + } + + @Test + public void testImplicitDirectoryIntoSameNameDestination() throws Exception { + explicitImplicitDirectoryRenameTestWithDestPathNames( + /*srcParentExplicit*/true, + /*srcExplicit*/false, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/true, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/false, + /*srcName*/"src", + /*dstName*/"src", + /*dstSubFileName*/null, + /*dstSubDirName*/null, + /*isSubDirExplicit*/true, /*shouldRenamePass*/true + ); + } + + @Test + public void testImplicitDirectoryIntoExplicitDestination() throws Exception { + explicitImplicitDirectoryRenameTestWithDestPathNames( + /*srcParentExplicit*/true, + /*srcExplicit*/false, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/true, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/false, + /*srcName*/"src", + /*dstName*/null, + /*dstSubFileName*/null, + /*dstSubDirName*/null, + /*isSubDirExplicit*/true, /*shouldRenamePass*/true + ); + } + + @Test + public void testRenameImplicitDirectoryToExplicitDirectoryDestinationHavingSameNameSubDir() + throws Exception { + explicitImplicitDirectoryRenameTestWithDestPathNames( + /*srcParentExplicit*/true, + /*srcExplicit*/false, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/true, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/false, + /*srcName*/"src", + /*dstName*/null, + /*dstSubFileName*/null, + /*dstSubDirName*/"src", + /*isSubDirExplicit*/true, /*shouldRenamePass*/false + ); + } + + @Test + public void testRenameImplicitDirectoryToExplicitDirectoryDestinationHavingSameNameSubFile() + throws Exception { + explicitImplicitDirectoryRenameTestWithDestPathNames( + /*srcParentExplicit*/true, + /*srcExplicit*/false, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/true, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/false, + /*srcName*/"src", + /*dstName*/null, + /*dstSubFileName*/"src", + /*dstSubDirName*/null, + /*isSubDirExplicit*/true, /*shouldRenamePass*/false + ); + } + + @Test + public void testRenameImplicitDirectoryToExplicitDirectoryDestinationHavingSameNameImplicitSubDir() + throws Exception { + explicitImplicitDirectoryRenameTestWithDestPathNames( + /*srcParentExplicit*/true, + /*srcExplicit*/false, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/true, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/false, + /*srcName*/"src", + /*dstName*/null, + /*dstSubFileName*/null, + /*dstSubDirName*/"src", + /*isSubDirExplicit*/false, /*shouldRenamePass*/false + ); + } + + @Test + public void testImplicitDirectoryIntoSameNameImplicitDestination() + throws Exception { + explicitImplicitDirectoryRenameTestWithDestPathNames( + /*srcParentExplicit*/true, + /*srcExplicit*/false, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/false, + /*srcName*/"src", + /*dstName*/"src", + /*dstSubFileName*/null, + /*dstSubDirName*/null, + /*isSubDirExplicit*/true, /*shouldRenamePass*/true + ); + } + + @Test + public void testImplicitDirectoryIntoImplicitDestination() throws Exception { + explicitImplicitDirectoryRenameTestWithDestPathNames( + /*srcParentExplicit*/true, + /*srcExplicit*/false, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/false, + /*srcName*/"src", + /*dstName*/null, + /*dstSubFileName*/null, + /*dstSubDirName*/null, + /*isSubDirExplicit*/true, /*shouldRenamePass*/true + ); + } + + @Test + public void testRenameImplicitDirectoryToImplicitDirectoryDestinationHavingSameNameSubDir() + throws Exception { + explicitImplicitDirectoryRenameTestWithDestPathNames( + /*srcParentExplicit*/true, + /*srcExplicit*/false, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/false, + /*srcName*/"src", + /*dstName*/null, + /*dstSubFileName*/null, + /*dstSubDirName*/"src", + /*isSubDirExplicit*/true, /*shouldRenamePass*/false + ); + } + + @Test + public void testRenameImplicitDirectoryToImplicitDirectoryDestinationHavingSameNameSubFile() + throws Exception { + explicitImplicitDirectoryRenameTestWithDestPathNames( + /*srcParentExplicit*/true, + /*srcExplicit*/false, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/false, + /*srcName*/"src", + /*dstName*/null, + /*dstSubFileName*/"src", + /*dstSubDirName*/null, + /*isSubDirExplicit*/true, /*shouldRenamePass*/false + ); + } + + @Test + public void testRenameImplicitDirectoryToImplicitDirectoryDestinationHavingSameNameImplicitSubDir() + throws Exception { + explicitImplicitDirectoryRenameTestWithDestPathNames( + /*srcParentExplicit*/true, + /*srcExplicit*/false, + /*srcSubDirExplicit*/true, + /*dstParentExplicit*/true, + /*dstExplicit*/false, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/false, + /*srcName*/"src", + /*dstName*/null, + /*dstSubFileName*/null, + /*dstSubDirName*/"src", + /*isSubDirExplicit*/false, /*shouldRenamePass*/false + ); + } + + @Test + public void testRenameExplicitSrcWithImplicitSubDirToImplicitDstWithExplicitSubDir() + throws Exception { + explicitImplicitDirectoryRenameTestWithDestPathNames( + /*srcParentExplicit*/true, + /*srcExplicit*/true, + /*srcSubDirExplicit*/false, + /*dstParentExplicit*/false, + /*dstExplicit*/true, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/false, + /*srcName*/"src", + /*dstName*/null, + /*dstSubFileName*/null, + /*dstSubDirName*/null, + /*isSubDirExplicit*/true, /*shouldRenamePass*/true + ); + } + + @Test + public void testRenameExplicitSrcWithImplicitSubDirToImplicitDstWithImplicitSubDir() + throws Exception { + explicitImplicitDirectoryRenameTestWithDestPathNames( + /*srcParentExplicit*/true, + /*srcExplicit*/true, + /*srcSubDirExplicit*/false, + /*dstParentExplicit*/false, + /*dstExplicit*/true, + /*dstParentExists*/true, + /*isDstParentFile*/false, + /*dstExist*/true, + /*isDstFile*/false, + /*srcName*/"src", + /*dstName*/null, + /*dstSubFileName*/null, + /*dstSubDirName*/null, + /*isSubDirExplicit*/false, /*shouldRenamePass*/true + ); + } + + + private void explicitImplicitDirectoryRenameTest(Boolean srcParentExplicit, + Boolean srcExplicit, + Boolean srcSubDirExplicit, + Boolean dstParentExplicit, + Boolean dstExplicit, + Boolean dstParentExists, + Boolean isDstParentFile, + Boolean dstExist, + Boolean isDstFile, + Boolean shouldRenamePass) throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + Path srcParent = new Path("/srcParent"); + Path src = new Path(srcParent, "src"); + createSourcePaths(srcParentExplicit, srcExplicit, srcSubDirExplicit, fs, + srcParent, + src); + + Path dstParent = new Path("/dstParent"); + Path dst = new Path(dstParent, "dst"); + createDestinationPaths(dstParentExplicit, dstExplicit, dstParentExists, + isDstParentFile, + dstExist, isDstFile, fs, dstParent, dst, null, null, true); + + if (dstParentExists && !isDstParentFile && !dstParentExplicit) { + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore() + .getBlobProperty(dstParent, getTestTracingContext(fs, true)); + }); + } + + explicitImplicitCaseRenameAssert(dstExist, shouldRenamePass, fs, src, dst); + } + + private void explicitImplicitDirectoryRenameTestWithDestPathNames(Boolean srcParentExplicit, + Boolean srcExplicit, + Boolean srcSubDirExplicit, + Boolean dstParentExplicit, + Boolean dstExplicit, + Boolean dstParentExists, + Boolean isDstParentFile, + Boolean dstExist, + Boolean isDstFile, + String srcName, + String dstName, + String dstSubFileName, + String dstSubDirName, + final Boolean isSubDirExplicit, Boolean shouldRenamePass) + throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + Path srcParent = new Path("/srcParent"); + Path src = new Path(srcParent, srcName != null ? srcName : "src"); + createSourcePaths(srcParentExplicit, srcExplicit, srcSubDirExplicit, fs, + srcParent, + src); + + Path dstParent = new Path("/dstParent"); + Path dst = new Path(dstParent, dstName != null ? dstName : "dst"); + createDestinationPaths(dstParentExplicit, dstExplicit, dstParentExists, + isDstParentFile, + dstExist, isDstFile, fs, dstParent, dst, dstSubFileName, dstSubDirName, + isSubDirExplicit); + + if (dstParentExists && !isDstParentFile && !dstParentExplicit && !dstExplicit) { + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore() + .getBlobProperty(dstParent, getTestTracingContext(fs, true)); + }); + } + + explicitImplicitCaseRenameAssert(dstExist, shouldRenamePass, fs, src, dst); + } + + private void createSourcePaths(final Boolean srcParentExplicit, + final Boolean srcExplicit, + final Boolean srcSubDirExplicit, + final AzureBlobFileSystem fs, + final Path srcParent, + final Path src) throws Exception { + if (srcParentExplicit) { + fs.mkdirs(srcParent); + } else { + createAzCopyDirectory(srcParent); + } + + if (srcExplicit) { + fs.mkdirs(src); + if(!srcParentExplicit) { + deleteBlobPath(fs, srcParent); + } + } else { + createAzCopyDirectory(src); + } + createAzCopyFile(new Path(src, "subFile")); + if (srcSubDirExplicit) { + fs.mkdirs(new Path(src, "subDir")); + if(!srcParentExplicit) { + deleteBlobPath(fs, srcParent); + } + if(!srcExplicit) { + deleteBlobPath(fs, src); + } + } else { + Path srcSubDir = new Path(src, "subDir"); + createAzCopyDirectory(srcSubDir); + createAzCopyFile(new Path(srcSubDir, "subFile")); + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore() + .getBlobProperty(srcSubDir, getTestTracingContext(fs, true)); + }); + } + if (!srcParentExplicit) { + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore() + .getBlobProperty(srcParent, getTestTracingContext(fs, true)); + }); + } + if (!srcExplicit) { + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore() + .getBlobProperty(src, getTestTracingContext(fs, true)); + }); + } + } + + private void deleteBlobPath(final AzureBlobFileSystem fs, final Path srcParent) + throws AzureBlobFileSystemException { + try { + fs.getAbfsClient() + .deleteBlobPath(srcParent, null, getTestTracingContext(fs, true)); + } catch (AbfsRestOperationException ex) { + if(ex.getStatusCode() != HttpURLConnection.HTTP_NOT_FOUND) { + throw ex; + } + } + } + + private void createDestinationPaths(final Boolean dstParentExplicit, + final Boolean dstExplicit, + final Boolean dstParentExists, + final Boolean isDstParentFile, + final Boolean dstExist, + final Boolean isDstFile, + final AzureBlobFileSystem fs, + final Path dstParent, + final Path dst, final String subFileName, final String subDirName, + final Boolean isSubDirExplicit) throws Exception { + if (dstParentExists) { + if (!isDstParentFile) { + if (dstParentExplicit) { + fs.mkdirs(dstParent); + } else { + createAzCopyDirectory(dstParent); + } + } else { + createAzCopyFile(dstParent); + } + } + + if (dstExist) { + if (!isDstFile) { + if (dstExplicit) { + fs.mkdirs(dst); + if(!dstParentExplicit) { + deleteBlobPath(fs, dstParent); + } + } else { + createAzCopyDirectory(dst); + } + if (subFileName != null) { + createAzCopyFile(new Path(dst, subFileName)); + } + if (subDirName != null) { + if (isSubDirExplicit) { + fs.mkdirs(new Path(dst, subDirName)); + if(!dstParentExplicit) { + deleteBlobPath(fs, dstParent); + } + if(!dstExplicit) { + deleteBlobPath(fs, dst); + } + } else { + createAzCopyDirectory(new Path(dst, subDirName)); + } + } + } else { + createAzCopyFile(dst); + } + } + } + + private void explicitImplicitCaseRenameAssert(final Boolean dstExist, + final Boolean shouldRenamePass, + final AzureBlobFileSystem fs, + final Path src, + final Path dst) throws IOException { + if (shouldRenamePass) { + Assert.assertTrue(fs.rename(src, dst)); + if (dstExist) { + Assert.assertTrue(fs.getAbfsStore() + .getBlobProperty(new Path(dst, src.getName()), + getTestTracingContext(fs, true)) + .getIsDirectory()); + } else { + Assert.assertTrue(fs.getAbfsStore() + .getBlobProperty(dst, getTestTracingContext(fs, true)) + .getIsDirectory()); + } + } else { + Assert.assertFalse(fs.rename(src, dst)); + Assert.assertTrue(fs.getAbfsStore() + .getListBlobs(src, null, null, getTestTracingContext(fs, true), null, + false) + .size() > 0); + if (dstExist) { + Assert.assertTrue(fs.getAbfsStore() + .getListBlobs(dst, null, null, getTestTracingContext(fs, true), null, + false) + .size() > 0); + } + } + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java index b9498be89a1ffe..b4fb5d3f394ffc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java @@ -20,12 +20,27 @@ import java.io.IOException; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Assume; import org.junit.Test; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; +import org.mockito.Mockito; + +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DNS_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.mockito.Mockito.times; /** * Test FileStatus. @@ -140,4 +155,265 @@ public void testLastModifiedTime() throws IOException { assertTrue("lastModifiedTime should be before createEndTime", createEndTime > lastModifiedTime); } + + @Test + public void testFileStatusOnFileWithImplicitParent() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getRawConfiguration(), + fs.getAbfsStore().getPrefixMode() + ); + + Path testPath = new Path("a/b.txt"); + azcopyHelper.createFileUsingAzcopy(fs.makeQualified(testPath).toUri().getPath().substring(1)); + + assertTrue("Parent directory is implicit.", + BlobDirectoryStateHelper.isImplicitDirectory(testPath.getParent(), fs, getTestTracingContext(fs, true))); + + // Assert getFileStatus Succeed on path + FileStatus fileStatus = fs.getFileStatus(testPath); + assertNotNull(fileStatus.getPath()); + assertFalse(fileStatus.isDirectory()); + assertNotEquals(0L, fileStatus.getLen()); + } + + @Test + public void testFileStatusOnFileWithExplicitParent() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + Path testPath = new Path("a/b.txt"); + fs.create(testPath); + + assertTrue("Parent directory is explicit.", + BlobDirectoryStateHelper.isExplicitDirectory(testPath.getParent(), fs, getTestTracingContext(fs, true))); + + FileStatus fileStatus = fs.getFileStatus(testPath); + assertNotNull(fileStatus.getPath()); + assertFalse(fileStatus.isDirectory()); + } + + @Test + public void testFileStatusOnImplicitDirWithImplicitParent() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getRawConfiguration(), + fs.getAbfsStore().getPrefixMode() + ); + + Path testPath = new Path("a/b"); + azcopyHelper.createFolderUsingAzcopy(fs.makeQualified(testPath).toUri().getPath().substring(1)); + + assertTrue("Path is implicit.", + BlobDirectoryStateHelper.isImplicitDirectory(testPath, fs, getTestTracingContext(fs, true))); + assertTrue("Parent directory is implicit.", + BlobDirectoryStateHelper.isImplicitDirectory(testPath.getParent(), fs, getTestTracingContext(fs, true))); + + // Assert that getFileStatus succeeds + FileStatus fileStatus = fs.getFileStatus(testPath); + assertNotNull(fileStatus.getPath()); + assertTrue(fileStatus.isDirectory()); + assertEquals(0L, fileStatus.getLen()); + } + + @Test + public void testFileStatusOnImplicitDirWithExplicitParent() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getRawConfiguration(), + fs.getAbfsStore().getPrefixMode() + ); + + Path testPath = new Path("a/b"); + azcopyHelper.createFolderUsingAzcopy(fs.makeQualified(testPath).toUri().getPath().substring(1)); + fs.mkdirs(testPath.getParent()); + + assertTrue("Path is implicit.", + BlobDirectoryStateHelper.isImplicitDirectory(testPath, fs, getTestTracingContext(fs, true))); + assertTrue("Parent directory is explicit.", + BlobDirectoryStateHelper.isExplicitDirectory(testPath.getParent(), fs, getTestTracingContext(fs, true))); + + // Assert that getFileStatus succeeds + FileStatus fileStatus = fs.getFileStatus(testPath); + assertNotNull(fileStatus.getPath()); + assertTrue(fileStatus.isDirectory()); + assertEquals(0L, fileStatus.getLen()); + } + + @Test + public void testFileStatusOnExplicitDirWithExplicitParent() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + Path testPath = new Path("a/b"); + fs.mkdirs(testPath); + + assertTrue("Parent directory is explicit.", + BlobDirectoryStateHelper.isExplicitDirectory(testPath.getParent(), fs, getTestTracingContext(fs, true))); + assertTrue("Path is explicit.", + BlobDirectoryStateHelper.isExplicitDirectory(testPath, fs, getTestTracingContext(fs, true))); + + // Assert that getFileStatus Succeeds + FileStatus fileStatus = fs.getFileStatus(testPath); + assertNotNull(fileStatus.getPath()); + assertTrue(fileStatus.isDirectory()); + assertEquals(0L, fileStatus.getLen()); + } + + @Test + public void testFileStatusOnNonExistingPathWithExplicitParent() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + Path testPath = new Path("a/b.txt"); + fs.mkdirs(testPath.getParent()); + + assertTrue("Parent directory is explicit.", + BlobDirectoryStateHelper.isExplicitDirectory(testPath.getParent(), fs, getTestTracingContext(fs, true))); + + // assert that getFileStatus fails + intercept(IOException.class, + () -> fs.getFileStatus(testPath)); + } + + @Test + public void testFileStatusOnNonExistingPathWithImplicitParent() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getRawConfiguration(), + fs.getAbfsStore().getPrefixMode() + ); + + Path testPath = new Path("a/b.txt"); + azcopyHelper.createFolderUsingAzcopy(fs.makeQualified( + testPath.getParent()).toUri().getPath().substring(1)); + + assertTrue("Parent directory is implicit.", + BlobDirectoryStateHelper.isImplicitDirectory(testPath.getParent(), fs, getTestTracingContext(fs, true))); + + // assert that getFileStatus Fails with IOException + intercept(IOException.class, + () -> fs.getFileStatus(testPath)); + } + + @Test + public void testFileStatusOnRoot() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Path path = new Path("/"); + fs.setWorkingDirectory(new Path("/")); + + // Assert that getFileSus on root path succeed. + FileStatus fileStatus = fs.getFileStatus(path); + assertTrue(fileStatus.isDirectory()); + assertTrue(fileStatus.getLen() == 0L); + } + + @Test + public void testGetPathPropertyCalled() throws Exception { + Assume.assumeTrue(getFileSystem().getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + + fs.create(new Path("/testGetPathProperty")); + FileStatus fileStatus = fs.getFileStatus(new Path("/testGetPathProperty")); + + Assert.assertFalse(fileStatus.isDirectory()); + + Mockito.verify(store, times(2)).getPathProperty(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.any(Boolean.class)); + final AbfsConfiguration configuration= fs.getAbfsStore().getAbfsConfiguration(); + final int listBlobAssertionTimes; + if (!configuration.shouldMkdirFallbackToDfs() + && !configuration.shouldReadFallbackToDfs() + && !configuration.shouldIngressFallbackToDfs()) { + listBlobAssertionTimes = 1; + } else{ + listBlobAssertionTimes = 0; + } + + Mockito.verify(store, times(listBlobAssertionTimes)).getListBlobs(Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class), Mockito.any(Integer.class), + Mockito.any(Boolean.class)); + } + + @Test + public void testGetPathPropertyCalledImplicit() throws Exception { + Assume.assumeTrue(getFileSystem().getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + + createAzCopyDirectory(new Path("/testImplicitDirectory")); + FileStatus fileStatus = fs.getFileStatus(new Path("/testImplicitDirectory")); + + Assert.assertTrue(fileStatus.isDirectory()); + + final AbfsConfiguration configuration= fs.getAbfsStore().getAbfsConfiguration(); + final int listBlobAssertionTimes; + if (!configuration.shouldMkdirFallbackToDfs() + && !configuration.shouldReadFallbackToDfs() + && !configuration.shouldIngressFallbackToDfs()) { + listBlobAssertionTimes = 1; + } else{ + listBlobAssertionTimes = 0; + } + + Mockito.verify(store, times(1)).getPathProperty(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.any(Boolean.class)); + Mockito.verify(store, times(listBlobAssertionTimes)).getListBlobs(Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class), Mockito.any(Integer.class), + Mockito.any(Boolean.class)); + } + + @Test + public void testGetFileStatusReturnStatusWithPathWithSameUriGivenInConfig() + throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + String accountName = getAccountName(); + Boolean isAccountNameInDfs = accountName.contains(ABFS_DNS_PREFIX); + String dnsAssertion; + if (isAccountNameInDfs) { + dnsAssertion = ABFS_DNS_PREFIX; + } else { + dnsAssertion = WASB_DNS_PREFIX; + } + + final Path path = new Path("/testDir/file"); + fs.create(path); + assertGetFileStatusPath(fs, accountName, dnsAssertion, path); + + final Configuration configuration; + if (isAccountNameInDfs) { + configuration = new Configuration(getRawConfiguration()); + configuration.set(FS_DEFAULT_NAME_KEY, + configuration.get(FS_DEFAULT_NAME_KEY) + .replace(ABFS_DNS_PREFIX, WASB_DNS_PREFIX)); + dnsAssertion = WASB_DNS_PREFIX; + + } else { + configuration = new Configuration(getRawConfiguration()); + configuration.set(FS_DEFAULT_NAME_KEY, + configuration.get(FS_DEFAULT_NAME_KEY) + .replace(WASB_DNS_PREFIX, ABFS_DNS_PREFIX)); + dnsAssertion = ABFS_DNS_PREFIX; + } + fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + assertGetFileStatusPath(fs, accountName, dnsAssertion, path); + } + + private void assertGetFileStatusPath(final AzureBlobFileSystem fs, + final String accountName, + final String dnsAssertion, + final Path path) throws IOException { + FileStatus fileStatus = fs.getFileStatus(new Path( + "abfs://" + fs.getAbfsClient().getFileSystem() + "@" + accountName + + path.toUri().getPath())); + Assertions.assertThat(fileStatus.getPath().toString()) + .contains(dnsAssertion); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index 07b8a6f2bb203d..0e7457e852e5b8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -20,6 +20,10 @@ import java.io.IOException; import java.util.concurrent.RejectedExecutionException; +import org.apache.hadoop.fs.azurebfs.services.AbfsBlobLease; +import org.apache.hadoop.fs.azurebfs.services.AbfsDfsLease; +import org.apache.hadoop.fs.azurebfs.services.OperativeEndpoint; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; import org.junit.Assert; import org.junit.Test; @@ -37,6 +41,9 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.CONDITION_NOT_MET; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_LEASE_EXPIRED_DFS; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_NO_LEASE_ID_SPECIFIED_BLOB; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyString; @@ -123,7 +130,7 @@ public void testSubDir() throws Exception { getCustomFileSystem(testFilePath.getParent().getParent(), 1); fs.mkdirs(testFilePath.getParent().getParent()); - FSDataOutputStream out = fs.create(testFilePath); + FSDataOutputStream out = fs.create(testFilePath.getParent()); Assert.assertTrue("Output stream should have lease", ((AbfsOutputStream) out.getWrappedStream()).hasLease()); out.close(); @@ -137,10 +144,13 @@ public void testTwoCreate() throws Exception { final Path testFilePath = new Path(path(methodName.getMethodName()), TEST_FILE); final AzureBlobFileSystem fs = getCustomFileSystem(testFilePath.getParent(), 1); fs.mkdirs(testFilePath.getParent()); - + PrefixMode prefixMode = getPrefixMode(fs); try (FSDataOutputStream out = fs.create(testFilePath)) { + AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration(); LambdaTestUtils.intercept(IOException.class, isHNSEnabled ? ERR_PARALLEL_ACCESS_DETECTED - : ERR_NO_LEASE_ID_SPECIFIED, () -> { + : !OperativeEndpoint.isIngressEnabledOnDFS(prefixMode, abfsConfiguration) + ? ERR_NO_LEASE_ID_SPECIFIED_BLOB + : ERR_NO_LEASE_ID_SPECIFIED, () -> { try (FSDataOutputStream out2 = fs.create(testFilePath)) { } return "Expected second create on infinite lease dir to fail"; @@ -162,7 +172,23 @@ private void twoWriters(AzureBlobFileSystem fs, Path testFilePath, boolean expec } } out.writeInt(1); - out.hsync(); + try { + out.hsync(); + } catch (IOException e) { + // Etag mismatch leads to condition not met error for blob endpoint. + if (getPrefixMode(fs) == PrefixMode.BLOB) { + GenericTestUtils.assertExceptionContains(CONDITION_NOT_MET, e); + } else { + throw e; + } + } + } catch (IOException e) { + // Etag mismatch leads to condition not met error for blob endpoint. + if (getPrefixMode(fs) == PrefixMode.BLOB) { + GenericTestUtils.assertExceptionContains(CONDITION_NOT_MET, e); + } else { + throw e; + } } Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed()); @@ -219,14 +245,17 @@ public void testWriteAfterBreakLease() throws Exception { FSOperationType.BREAK_LEASE, false, 0)); fs.breakLease(testFilePath); fs.registerListener(null); - - LambdaTestUtils.intercept(IOException.class, ERR_LEASE_EXPIRED, () -> { + PrefixMode prefixMode = getPrefixMode(fs); + AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration(); + LambdaTestUtils.intercept(IOException.class, !OperativeEndpoint.isIngressEnabledOnDFS(prefixMode, abfsConfiguration) + ? ERR_LEASE_EXPIRED : ERR_LEASE_EXPIRED_DFS, () -> { out.write(1); out.hsync(); return "Expected exception on write after lease break but got " + out; }); - LambdaTestUtils.intercept(IOException.class, ERR_LEASE_EXPIRED, () -> { + LambdaTestUtils.intercept(IOException.class, !OperativeEndpoint.isIngressEnabledOnDFS(prefixMode, abfsConfiguration) + ? ERR_LEASE_EXPIRED : ERR_LEASE_EXPIRED_DFS, () -> { out.close(); return "Expected exception on close after lease break but got " + out; }); @@ -252,8 +281,10 @@ public void testLeaseFreedAfterBreak() throws Exception { out.write(0); fs.breakLease(testFilePath); - - LambdaTestUtils.intercept(IOException.class, ERR_LEASE_EXPIRED, () -> { + PrefixMode prefixMode = getPrefixMode(fs); + AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration(); + LambdaTestUtils.intercept(IOException.class, !OperativeEndpoint.isIngressEnabledOnDFS(prefixMode, abfsConfiguration) + ? ERR_LEASE_EXPIRED : ERR_LEASE_EXPIRED_DFS, () -> { out.close(); return "Expected exception on close after lease break but got " + out; }); @@ -318,8 +349,14 @@ public void testAcquireRetry() throws Exception { FSOperationType.TEST_OP, true, 0); tracingContext.setListener(listener); - AbfsLease lease = new AbfsLease(fs.getAbfsClient(), - testFilePath.toUri().getPath(), tracingContext); + AbfsLease lease; + if (getPrefixMode(fs) == PrefixMode.BLOB) { + lease = new AbfsBlobLease(fs.getAbfsClient(), + testFilePath.toUri().getPath(), null, tracingContext); + } else { + lease = new AbfsDfsLease(fs.getAbfsClient(), + testFilePath.toUri().getPath(), null, tracingContext); + } Assert.assertNotNull("Did not successfully lease file", lease.getLeaseID()); listener.setOperation(FSOperationType.RELEASE_LEASE); lease.free(); @@ -333,7 +370,20 @@ public void testAcquireRetry() throws Exception { .doCallRealMethod().when(mockClient) .acquireLease(anyString(), anyInt(), any(TracingContext.class)); - lease = new AbfsLease(mockClient, testFilePath.toUri().getPath(), 5, 1, tracingContext); + doThrow(new AbfsLease.LeaseException("failed to acquire 1")) + .doThrow(new AbfsLease.LeaseException("failed to acquire 2")) + .doCallRealMethod().when(mockClient) + .acquireBlobLease(anyString(), anyInt(), any(TracingContext.class)); + + if (getPrefixMode(fs) == PrefixMode.BLOB) { + lease = new AbfsBlobLease(mockClient, testFilePath.toUri().getPath(), 5, + 1, null, + tracingContext); + } else { + lease = new AbfsDfsLease(mockClient, testFilePath.toUri().getPath(), 5, 1, + null, + tracingContext); + } Assert.assertNotNull("Acquire lease should have retried", lease.getLeaseID()); lease.free(); Assert.assertEquals("Unexpected acquire retry count", 2, lease.getAcquireRetryCount()); @@ -341,9 +391,18 @@ public void testAcquireRetry() throws Exception { doThrow(new AbfsLease.LeaseException("failed to acquire")).when(mockClient) .acquireLease(anyString(), anyInt(), any(TracingContext.class)); + doThrow(new AbfsLease.LeaseException("failed to acquire")).when(mockClient) + .acquireBlobLease(anyString(), anyInt(), any(TracingContext.class)); + LambdaTestUtils.intercept(AzureBlobFileSystemException.class, () -> { - new AbfsLease(mockClient, testFilePath.toUri().getPath(), 5, 1, - tracingContext); + if (getPrefixMode(fs) == PrefixMode.BLOB) { + new AbfsBlobLease(mockClient, testFilePath.toUri().getPath(), 5, 1, + null, + tracingContext); + } else { + new AbfsDfsLease(mockClient, testFilePath.toUri().getPath(), 5, 1, null, + tracingContext); + } }); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java index dc9ef9bffb9452..4cae9a4ac5f565 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java @@ -20,6 +20,7 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.net.SocketTimeoutException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.Callable; @@ -27,6 +28,16 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil; +import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperationTestUtil; +import org.apache.hadoop.fs.azurebfs.services.BlobList; +import org.apache.hadoop.fs.azurebfs.services.BlobProperty; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.assertj.core.api.Assertions; +import org.junit.Assume; import org.junit.Test; import org.apache.hadoop.conf.Configuration; @@ -36,16 +47,29 @@ import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.mockito.Mockito; +import org.mockito.stubbing.Stubber; + +import static java.net.HttpURLConnection.HTTP_OK; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_LIST_MAX_RESULTS; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DNS_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_JDK_MESSAGE; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists; import static org.apache.hadoop.fs.contract.ContractTestUtils.rename; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.nullable; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.when; /** * Test listStatus operation. @@ -60,7 +84,7 @@ public ITestAzureBlobFileSystemListStatus() throws Exception { @Test public void testListPath() throws Exception { - Configuration config = new Configuration(this.getRawConfiguration()); + Configuration config = Mockito.spy(this.getRawConfiguration()); config.set(AZURE_LIST_MAX_RESULTS, "5000"); final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem .newInstance(getFileSystem().getUri(), config); @@ -92,6 +116,62 @@ public Void call() throws Exception { assertEquals(TEST_FILES_NUMBER, files.length /* user directory */); } + /** + * Test to verify that each paginated call to ListBlobs uses a new tracing context. + * @throws Exception + */ + @Test + public void testListPathTracingContext() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + Assume.assumeTrue("To work on only on non-HNS Blob endpoint", + fs.getAbfsStore().getAbfsConfiguration().getPrefixMode() + == PrefixMode.BLOB); + TracingHeaderValidator validator = new TracingHeaderValidator(getConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.LISTSTATUS, true, 0); + validator.setDisableValidation(true); + fs.registerListener(validator); + + final AzureBlobFileSystem spiedFs = Mockito.spy(fs); + final AzureBlobFileSystemStore spiedStore = Mockito.spy(fs.getAbfsStore()); + final AbfsClient spiedClient = Mockito.spy(fs.getAbfsClient()); + final TracingContext spiedTracingContext = Mockito.spy( + new TracingContext( + fs.getClientCorrelationId(), fs.getFileSystemId(), + FSOperationType.LISTSTATUS, true, TracingHeaderFormat.ALL_ID_FORMAT,validator)); + + Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore(); + spiedStore.setClient(spiedClient); + spiedFs.setWorkingDirectory(new Path("/")); + + AbfsClientTestUtil.setMockAbfsRestOperationForListBlobOperation(spiedClient, + (httpOperation) -> { + BlobProperty blob = new BlobProperty(); + blob.setPath(new Path("/abc.txt")); + blob.setName("abc.txt"); + BlobList blobListWithNextMarker = new BlobList(); + AbfsClientTestUtil.populateBlobListHelper(blobListWithNextMarker, blob, "nextMarker"); + BlobList blobListWithoutNextMarker = new BlobList(); + AbfsClientTestUtil.populateBlobListHelper(blobListWithNextMarker, blob, AbfsHttpConstants.EMPTY_STRING); + when(httpOperation.getBlobList()).thenReturn(blobListWithNextMarker) + .thenReturn(blobListWithoutNextMarker); + + Stubber stubber = Mockito.doThrow( + new SocketTimeoutException(CONNECTION_TIMEOUT_JDK_MESSAGE)); + stubber.doNothing().when(httpOperation).processResponse( + nullable(byte[].class), nullable(int.class), nullable(int.class)); + + when(httpOperation.getStatusCode()).thenReturn(-1).thenReturn(HTTP_OK); + return httpOperation; + }); + + List fileStatuses = new ArrayList<>(); + spiedStore.listStatus(new Path("/"), "", fileStatuses, true, null, spiedTracingContext); + + // Assert that there were 2 paginated ListBlob calls made and new tracing context were created. + Mockito.verify(spiedClient, times(2)).getListBlobs(any(), any(), any(), any(), any()); + Mockito.verify(spiedTracingContext, times(0)).constructHeader(any(), any()); + } + /** * Creates a file, verifies that listStatus returns it, * even while the file is still open for writing. @@ -247,4 +327,328 @@ public void testRenameTrailingPeriodFile() throws IOException { assertTrue("Attempt to create file that ended with a dot should" + " throw IllegalArgumentException", exceptionThrown); } + + @Test + public void testListStatusImplicitExplicitChildren() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + fs.setWorkingDirectory(new Path("/")); + Path root = new Path("/"); + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getRawConfiguration(), + fs.getAbfsStore().getPrefixMode() + ); + + // Create an implicit directory under root + Path dir1 = new Path("a"); + azcopyHelper.createFolderUsingAzcopy(fs.makeQualified(dir1).toUri().getPath().substring(1)); + assertTrue("Path is implicit.", + BlobDirectoryStateHelper.isImplicitDirectory(dir1, fs, getTestTracingContext(fs, true))); + + // Assert that implicit directory is returned + FileStatus[] fileStatuses = fs.listStatus(root); + Assertions.assertThat(fileStatuses.length).isEqualTo(1); + assertImplicitDirectoryFileStatus(fileStatuses[0], fs.makeQualified(dir1)); + + // Create a marker blob for the directory. + fs.mkdirs(dir1); + + // Assert that only one entry of explicit directory is returned + fileStatuses = fs.listStatus(root); + Assertions.assertThat(fileStatuses.length).isEqualTo(1); + assertExplicitDirectoryFileStatus(fileStatuses[0], fs.makeQualified(dir1)); + + // Create a file under root + Path file1 = new Path("b"); + fs.create(file1); + + // Assert that two entries are returned in alphabetic order. + fileStatuses = fs.listStatus(root); + Assertions.assertThat(fileStatuses.length).isEqualTo(2); + assertExplicitDirectoryFileStatus(fileStatuses[0], fs.makeQualified(dir1)); + assertFileFileStatus(fileStatuses[1], fs.makeQualified(file1)); + + // Create another implicit directory under root. + Path dir2 = new Path("c"); + azcopyHelper.createFolderUsingAzcopy(fs.makeQualified(dir2).toUri().getPath().substring(1)); + assertTrue("Path is implicit.", + BlobDirectoryStateHelper.isImplicitDirectory(dir2, fs, getTestTracingContext(fs, true))); + + // Assert that three entries are returned in alphabetic order. + fileStatuses = fs.listStatus(root); + Assertions.assertThat(fileStatuses.length).isEqualTo(3); + assertExplicitDirectoryFileStatus(fileStatuses[0], fs.makeQualified(dir1)); + assertFileFileStatus(fileStatuses[1], fs.makeQualified(file1)); + assertImplicitDirectoryFileStatus(fileStatuses[2], fs.makeQualified(dir2)); + } + + @Test + public void testListStatusOnNonExistingPath() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + Path testPath = new Path("a/b"); + + intercept(FileNotFoundException.class, + () -> fs.listFiles(testPath, false).next()); + } + + @Test + public void testListStatusOnImplicitDirectory() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getRawConfiguration(), + fs.getAbfsStore().getPrefixMode() + ); + + // Create an implicit directory with another implicit directory inside + Path testPath = new Path("testDir"); + Path childPath = new Path("testDir/azcopy"); + azcopyHelper.createFolderUsingAzcopy( + fs.makeQualified(testPath).toUri().getPath().substring(1)); + assertTrue("Path is implicit.", + BlobDirectoryStateHelper.isImplicitDirectory(testPath, fs, getTestTracingContext(fs, true))); + + // Assert that one entry is returned as implicit child. + FileStatus[] fileStatuses = fs.listStatus(testPath); + Assertions.assertThat(fileStatuses.length).isEqualTo(1); + assertImplicitDirectoryFileStatus(fileStatuses[0], fs.makeQualified(childPath)); + } + + @Test + public void testListStatusOnExplicitDirectory() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getRawConfiguration(), + fs.getAbfsStore().getPrefixMode() + ); + + // Create an explicit directory with all kind of children. + Path testPath = new Path("testDir"); + Path explicitChild = new Path ("testDir/a/subdir"); + Path fileChild = new Path ("testDir/b"); + Path implicitChild = new Path ("testDir/c"); + fs.mkdirs(explicitChild); + fs.create(fileChild); + azcopyHelper.createFolderUsingAzcopy( + fs.makeQualified(implicitChild).toUri().getPath().substring(1)); + + assertTrue("Test path is explicit", + BlobDirectoryStateHelper.isExplicitDirectory(testPath, fs, getTestTracingContext(fs, true))); + assertTrue("explicitChild Path is explicit", + BlobDirectoryStateHelper.isExplicitDirectory(explicitChild, fs, getTestTracingContext(fs, true))); + assertTrue("implicitChild Path is implicit.", + BlobDirectoryStateHelper.isImplicitDirectory(implicitChild, fs, getTestTracingContext(fs, true))); + + // Assert that three entry is returned. + FileStatus[] fileStatuses = fs.listStatus(testPath); + Assertions.assertThat(fileStatuses.length).isEqualTo(3); + assertExplicitDirectoryFileStatus(fileStatuses[0], fs.makeQualified(explicitChild.getParent())); + assertFileFileStatus(fileStatuses[1], fs.makeQualified(fileChild)); + assertImplicitDirectoryFileStatus(fileStatuses[2], fs.makeQualified(implicitChild)); + } + + @Test + public void testListStatusImplicitExplicitWithDotInFolderName() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + AzcopyHelper azcopyHelper = new AzcopyHelper( + getAccountName(), + getFileSystemName(), + getRawConfiguration(), + fs.getAbfsStore().getPrefixMode() + ); + + // Create two implicit folder with same prefix one having dot. + Path testPath = new Path("Try1/DirA"); + Path implicitChild1 = new Path ("Try1/DirA/DirB/file.txt"); + Path implicitChild2 = new Path ("Try1/DirA/DirB.bak/file.txt"); + Path explicitChild = new Path ("Try1/DirA/DirB"); + + azcopyHelper.createFolderUsingAzcopy( + fs.makeQualified(implicitChild1).toUri().getPath().substring(1)); + azcopyHelper.createFolderUsingAzcopy( + fs.makeQualified(implicitChild2).toUri().getPath().substring(1)); + fs.mkdirs(explicitChild); + + assertTrue("Test path is explicit", + BlobDirectoryStateHelper.isExplicitDirectory(testPath, fs, getTestTracingContext(fs, true))); + assertTrue("explicitChild Path is explicit", + BlobDirectoryStateHelper.isExplicitDirectory(explicitChild, fs, getTestTracingContext(fs, true))); + assertTrue("implicitChild2 Path is implicit.", + BlobDirectoryStateHelper.isImplicitDirectory(implicitChild2, fs, getTestTracingContext(fs, true))); + + // Assert that only 2 entry is returned. + FileStatus[] fileStatuses = fs.listStatus(testPath); + Assertions.assertThat(fileStatuses.length).isEqualTo(2); + assertExplicitDirectoryFileStatus(fileStatuses[0], fs.makeQualified(explicitChild)); + assertImplicitDirectoryFileStatus(fileStatuses[1], fs.makeQualified(implicitChild2.getParent())); + } + + @Test + public void testListStatusOnEmptyDirectory() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + Path testPath = new Path("testPath"); + fs.mkdirs(testPath); + FileStatus[] fileStatuses = fs.listStatus(testPath); + Assertions.assertThat(fileStatuses.length).isEqualTo(0); + } + + @Test + public void testListStatusUsesGfs() throws Exception { + Assume.assumeTrue(getFileSystem().getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(store.getClient()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + Mockito.doReturn(client).when(store).getClient(); + store.setClient(client); + + intercept(FileNotFoundException.class, () -> + fs.listStatus(new Path("a/b"))); + + Mockito.verify(store, times(1)).getPathProperty(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.any(boolean.class)); + + // getListBlobs from within store should not be called - as this is invoked from getFileStatus + Mockito.verify(store, times(0)).getListBlobs(Mockito.any(Path.class), + Mockito.any(String.class), Mockito.any(String.class), Mockito.any(TracingContext.class), + Mockito.any(Integer.class), Mockito.any(Boolean.class)); + + // getListBlobs from client should be called once - the call for listStatus that would fail + // as this blob is actually non-existent + Mockito.verify(client, times(1)).getListBlobs(Mockito.nullable(String.class), + Mockito.nullable(String.class), Mockito.nullable(String.class), + Mockito.any(Integer.class), Mockito.any(TracingContext.class)); + } + + private void assertFileFileStatus(final FileStatus fileStatus, + final Path qualifiedPath) { + Assertions.assertThat(fileStatus.getPath()).isEqualTo(qualifiedPath); + Assertions.assertThat(fileStatus.isDirectory()).isEqualTo(false); + Assertions.assertThat(fileStatus.isFile()).isEqualTo(true); + Assertions.assertThat(fileStatus.getModificationTime()).isNotEqualTo(0); + } + + private void assertImplicitDirectoryFileStatus(final FileStatus fileStatus, + final Path qualifiedPath) { + assertDirectoryFileStatus(fileStatus, qualifiedPath); + Assertions.assertThat(fileStatus.getModificationTime()).isEqualTo(0); + } + + private void assertExplicitDirectoryFileStatus(final FileStatus fileStatus, + final Path qualifiedPath) { + assertDirectoryFileStatus(fileStatus, qualifiedPath); + Assertions.assertThat(fileStatus.getModificationTime()).isNotEqualTo(0); + } + + private void assertDirectoryFileStatus(final FileStatus fileStatus, + final Path qualifiedPath) { + Assertions.assertThat(fileStatus.getPath()).isEqualTo(qualifiedPath); + Assertions.assertThat(fileStatus.isDirectory()).isEqualTo(true); + Assertions.assertThat(fileStatus.isFile()).isEqualTo(false); + Assertions.assertThat(fileStatus.getLen()).isEqualTo(0); + } + + @Test + public void testListStatusNotTriesToRenameResumeForNonAtomicDir() + throws Exception { + Assume.assumeTrue(getPrefixMode(getFileSystem()) == PrefixMode.BLOB); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + Mockito.doReturn(new FileStatus[1]) + .when(store) + .listStatus(Mockito.any(Path.class), Mockito.any( + TracingContext.class)); + fs.listStatus(new Path("/testDir/")); + Mockito.verify(store, Mockito.times(0)) + .getRenamePendingFileStatus(Mockito.any(FileStatus[].class)); + } + + @Test + public void testListStatusTriesToRenameResumeForAtomicDir() throws Exception { + Assume.assumeTrue(getPrefixMode(getFileSystem()) == PrefixMode.BLOB); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + Mockito.doReturn(new FileStatus[0]) + .when(store) + .listStatus(Mockito.any(Path.class), Mockito.any( + TracingContext.class)); + fs.listStatus(new Path("/hbase/")); + Mockito.verify(store, Mockito.times(1)) + .getRenamePendingFileStatus(Mockito.any(FileStatus[].class)); + } + + @Test + public void testListStatusTriesToRenameResumeForAbsoluteAtomicDir() + throws Exception { + Assume.assumeTrue(getPrefixMode(getFileSystem()) == PrefixMode.BLOB); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + Mockito.doReturn(new FileStatus[0]) + .when(store) + .listStatus(Mockito.any(Path.class), Mockito.any( + TracingContext.class)); + fs.listStatus(new Path("/hbase")); + Mockito.verify(store, Mockito.times(1)) + .getRenamePendingFileStatus(Mockito.any(FileStatus[].class)); + } + + @Test + public void testListStatusReturnStatusWithPathWithSameUriGivenInConfig() + throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + String accountName = getAccountName(); + Boolean isAccountNameInDfs = accountName.contains(ABFS_DNS_PREFIX); + String dnsAssertion; + if (isAccountNameInDfs) { + dnsAssertion = ABFS_DNS_PREFIX; + } else { + dnsAssertion = WASB_DNS_PREFIX; + } + + final Path path = new Path("/testDir/file"); + fs.create(path); + assertListStatusPath(fs, accountName, dnsAssertion, path); + + final Configuration configuration; + if (isAccountNameInDfs) { + configuration = new Configuration(getRawConfiguration()); + configuration.set(FS_DEFAULT_NAME_KEY, + configuration.get(FS_DEFAULT_NAME_KEY) + .replace(ABFS_DNS_PREFIX, WASB_DNS_PREFIX)); + dnsAssertion = WASB_DNS_PREFIX; + + } else { + configuration = new Configuration(getRawConfiguration()); + configuration.set(FS_DEFAULT_NAME_KEY, + configuration.get(FS_DEFAULT_NAME_KEY) + .replace(WASB_DNS_PREFIX, ABFS_DNS_PREFIX)); + dnsAssertion = ABFS_DNS_PREFIX; + } + fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + assertListStatusPath(fs, accountName, dnsAssertion, path); + } + + private void assertListStatusPath(final AzureBlobFileSystem fs, + final String accountName, + final String dnsAssertion, + final Path path) throws IOException { + FileStatus[] fileStatuses = fs.listStatus(new Path( + "abfs://" + fs.getAbfsClient().getFileSystem() + "@" + accountName + + path.toUri().getPath())); + Assertions.assertThat(fileStatuses[0].getPath().toString()) + .contains(dnsAssertion); + + fileStatuses = fs.listStatus(new Path( + "abfs://" + fs.getAbfsClient().getFileSystem() + "@" + accountName + + path.getParent().toUri().getPath())); + Assertions.assertThat(fileStatuses[0].getPath().toString()) + .contains(dnsAssertion); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java index a9a059e7cda560..688e9b06a82c34 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java @@ -20,6 +20,12 @@ import java.util.UUID; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.OperativeEndpoint; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; +import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Assume; import org.junit.Test; @@ -27,9 +33,11 @@ import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.mockito.Mockito; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CONNECTIONS_MADE; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_MKDIR_OVERWRITE; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_MKDIRS_FALLBACK_TO_DFS; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_FS_AZURE_ENABLE_MKDIR_OVERWRITE; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -122,8 +130,12 @@ public void testCreateDirOverwrite(boolean enableConditionalCreateOverwrite) // Case 1: Dir does not pre-exist fs.mkdirs(dirPath); - // One request to server - mkdirRequestCount++; + // One request to server for dfs and 2 for blob because child calls mkdir for parent. + if (!OperativeEndpoint.isMkdirEnabledOnDFS(getAbfsStore(fs).getAbfsConfiguration())) { + mkdirRequestCount += 2; + } else { + mkdirRequestCount++; + } assertAbfsStatistics( CONNECTIONS_MADE, @@ -134,12 +146,88 @@ public void testCreateDirOverwrite(boolean enableConditionalCreateOverwrite) // Mkdir on existing Dir path will not lead to failure fs.mkdirs(dirPath); - // One request to server - mkdirRequestCount++; + // One request to server for dfs and 3 for blob because child calls mkdir for parent. + if (!OperativeEndpoint.isMkdirEnabledOnDFS(getAbfsStore(fs).getAbfsConfiguration())) { + mkdirRequestCount += 3; + } else { + mkdirRequestCount++; + } assertAbfsStatistics( CONNECTIONS_MADE, totalConnectionMadeBeforeTest + mkdirRequestCount, fs.getInstrumentationMap()); } + + @Test + public void testVerifyGetBlobProperty() throws Exception { + Assume.assumeTrue(getFileSystem().getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + AbfsClient client = store.getClient(); + AbfsClient testClient = Mockito.spy(ITestAbfsClient.createTestClientFromCurrentContext( + client, + fs.getAbfsStore().getAbfsConfiguration())); + store.setClient(testClient); + + createAzCopyDirectory(new Path("/src")); + intercept(AbfsRestOperationException.class, () -> { + store.getBlobProperty(new Path("/src"), getTestTracingContext(fs, true)); + }); + fs.mkdirs(new Path("/src/dir")); + Mockito.verify(testClient, Mockito.times(0)).getPathStatus(Mockito.any(String.class), + Mockito.anyBoolean(), Mockito.any(TracingContext.class)); + Mockito.verify(testClient, Mockito.atLeast(1)).getBlobProperty(Mockito.any(Path.class), + Mockito.any(TracingContext.class)); + + } + + @Test + public void testGetPathPropertyCalledOnMkdirBlobEndpoint() throws Exception { + Assume.assumeTrue(getFileSystem().getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + AbfsConfiguration abfsConfig = new AbfsConfiguration(getRawConfiguration(), getAccountName()); + abfsConfig.setBoolean(FS_AZURE_MKDIRS_FALLBACK_TO_DFS, false); + AzureBlobFileSystem fs = Mockito.spy((AzureBlobFileSystem) FileSystem.newInstance(abfsConfig.getRawConfiguration())); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + + fs.mkdirs(new Path("/testPath")); + Mockito.verify(store, Mockito.times(1)).tryGetPathProperty(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.any(Boolean.class)); + Mockito.verify(store, Mockito.times(1)).getPathProperty(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.any(Boolean.class)); + + } + + @Test + public void testGetPathPropertyCalledOnMkdirFallbackDFS() throws Exception { + Assume.assumeTrue(getFileSystem().getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + AbfsConfiguration abfsConfig = new AbfsConfiguration(getRawConfiguration(), getAccountName()); + abfsConfig.setBoolean(FS_AZURE_MKDIRS_FALLBACK_TO_DFS, true); + AzureBlobFileSystem fs = Mockito.spy((AzureBlobFileSystem) FileSystem.newInstance(abfsConfig.getRawConfiguration())); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + + + fs.mkdirs(new Path("/testPath")); + Mockito.verify(store, Mockito.times(0)).tryGetPathProperty(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.any(Boolean.class)); + Mockito.verify(store, Mockito.times(0)).getPathProperty(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.any(Boolean.class)); + + } + + @Test + public void testMkdirWithExistingFilenameBlobEndpoint() throws Exception { + Assume.assumeTrue(getFileSystem().getAbfsStore().getPrefixMode() == PrefixMode.BLOB); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + + fs.create(new Path("/testFilePath")); + intercept(FileAlreadyExistsException.class, () -> fs.mkdirs(new Path("/testFilePath"))); + intercept(FileAlreadyExistsException.class, () -> fs.mkdirs(new Path("/testFilePath/newDir"))); + } } + diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java index 0a9713cffa94b5..6b6e5e7f23581a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java @@ -18,14 +18,21 @@ package org.apache.hadoop.fs.azurebfs; import java.io.EOFException; +import java.io.FileNotFoundException; import java.io.IOException; import java.util.Random; import java.util.concurrent.Callable; import java.util.UUID; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.OperativeEndpoint; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; +import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.junit.Assume; import org.junit.Ignore; import org.junit.Test; +import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -574,6 +581,49 @@ public void testAlwaysReadBufferSizeConfig(boolean alwaysReadBufferSizeConfigVal assertAbfsStatistics(BYTES_RECEIVED, dateSizeReadStatAtStart + newDataSizeRead, fs.getInstrumentationMap()); } + @Test + public void testReadBlob() throws IOException { + Assume.assumeTrue(PrefixMode.BLOB == getFileSystem().getAbfsStore().getPrefixMode()); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = store.getClient(); + AbfsClient mockClient = Mockito.spy(ITestAbfsClient.createTestClientFromCurrentContext( + client, + fs.getAbfsStore().getAbfsConfiguration() + )); + store.setClient(mockClient); + Mockito.doReturn(mockClient).when(store).getClient(); + Mockito.doReturn(store).when(fs).getAbfsStore(); + + Path testPath = new Path("/testReadFile"); + fs.create(testPath); + FSDataInputStream in = fs.open(testPath); + if (!OperativeEndpoint.isReadEnabledOnDFS(store.getAbfsConfiguration())) { + Mockito.verify(mockClient, Mockito.atLeast(1)).getBlobProperty( + Mockito.any(Path.class), Mockito.any(TracingContext.class)); + } else { + Mockito.verify(mockClient, Mockito.times(2)).getPathStatus( + Mockito.any(String.class), Mockito.anyBoolean(), Mockito.any(TracingContext.class)); + } + } + + @Test + public void testInvalidImplicitDirRead() throws Exception { + AzureBlobFileSystem fs = (AzureBlobFileSystem) getFileSystem(); + AzcopyHelper azcopyhelper = new AzcopyHelper(getAccountName(), + getFileSystemName(), + getRawConfiguration(), + fs.getAbfsStore().getPrefixMode()); + String fullPath = "/implicitDirPath/testFile"; + String path = "/implicitDirPath"; + azcopyhelper.createFolderUsingAzcopy( + fs.makeQualified(new Path(fullPath)).toUri().getPath().substring(1) + ); + + intercept(FileNotFoundException.class, () -> + fs.open(new Path(path))); + + } private long sequentialRead(String version, Path testPath, FileSystem fs, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java index 48c3a9d9665e33..fd415538f4a3f5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java @@ -18,23 +18,76 @@ package org.apache.hadoop.fs.azurebfs; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.SocketException; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.Random; import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; + +import org.assertj.core.api.Assertions; import org.junit.Assert; +import org.junit.Assume; import org.junit.Test; +import org.mockito.Mockito; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +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.AzureServiceErrorCode; +import org.apache.hadoop.fs.azurebfs.services.AbfsBlobLease; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil; +import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOpTestUtil; +import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; +import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; +import org.apache.hadoop.fs.azurebfs.services.AbfsLease; +import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; +import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperationTestUtil; +import org.apache.hadoop.fs.azurebfs.services.ListBlobProducer; +import org.apache.hadoop.fs.azurebfs.services.ListBlobQueue; +import org.apache.hadoop.fs.azurebfs.services.RenameAtomicityUtils; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; +import org.apache.hadoop.fs.impl.OpenFileParameters; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_CREATE_NON_RECURSIVE; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_MKDIRS_FALLBACK_TO_DFS; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_REDIRECT_RENAME; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.TRUE; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_INGRESS_FALLBACK_TO_DFS; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_ID; +import static org.apache.hadoop.fs.azurebfs.services.RenameAtomicityUtils.SUFFIX; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_ABORTED; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_FAILED; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_PENDING; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.COPY_BLOB_ABORTED; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.COPY_BLOB_FAILED; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test rename operation. @@ -87,6 +140,28 @@ public void testRenameFileUnderDir() throws Exception { filename, status.getPath().getName()); } + @Test + public void testRenameFileUnderDirRedirection() throws Exception { + Configuration configuration = Mockito.spy(getRawConfiguration()); + + // Set redirect to wasb rename true and assert rename. + configuration.setBoolean(FS_AZURE_REDIRECT_RENAME, true); + AzureBlobFileSystem fs1 = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + Path sourceDir = makeQualified(new Path("/testSrc")); + assertMkdirs(fs1, sourceDir); + String filename = "file1"; + Path file1 = new Path(sourceDir, filename); + touch(file1); + + Path destDir = makeQualified(new Path("/testDst")); + assertRenameOutcome(fs1, sourceDir, destDir, true); + FileStatus[] fileStatus = fs1.listStatus(destDir); + assertNotNull("Null file status", fileStatus); + FileStatus status = fileStatus[0]; + assertEquals("Wrong filename in " + status, + filename, status.getPath().getName()); + } + @Test public void testRenameDirectory() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); @@ -160,4 +235,2279 @@ public void testPosixRenameDirectory() throws Exception { assertFalse(fs.exists(new Path("testDir2/test1/test2/test3"))); } + @Test + public void testRenameToRoot() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + fs.mkdirs(new Path("/src1/src2")); + Assert.assertTrue(fs.rename(new Path("/src1/src2"), new Path("/"))); + Assert.assertTrue(fs.exists(new Path("/src2"))); + } + + @Test + public void testRenameNotFoundBlobToEmptyRoot() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + Assert.assertFalse(fs.rename(new Path("/file"), new Path("/"))); + } + + @Test(expected = IOException.class) + public void testRenameBlobToDstWithColonInPath() throws Exception{ + AzureBlobFileSystem fs = getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + fs.create(new Path("/src")); + fs.rename(new Path("/src"), new Path("/dst:file")); + } + + @Test + public void testRenameBlobInSameDirectoryWithNoMarker() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + fs.create(new Path("/srcDir/dir/file")); + fs.getAbfsStore().getClient().deleteBlobPath(new Path("/srcDir/dir"), null, + getTestTracingContext(fs, true)); + Assert.assertTrue(fs.rename(new Path("/srcDir/dir"), new Path("/srcDir"))); + } + + /** + *
+   * Test to check behaviour of rename API if the destination directory is already
+   * there. The HNS call and the one for Blob endpoint should have same behaviour.
+   *
+   * /testDir2/test1/test2/test3 contains (/file)
+   * There is another path that exists: /testDir2/test4/test3
+   * On rename(/testDir2/test1/test2/test3, /testDir2/test4).
+   * 
+ * + * Expectation for HNS / Blob endpoint:
    + *
  1. Rename should fail
  2. + *
  3. No file should be transferred to destination directory
  4. + *
+ */ + @Test + public void testPosixRenameDirectoryWhereDirectoryAlreadyThereOnDestination() + throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.mkdirs(new Path("testDir2/test1/test2/test3")); + fs.create(new Path("testDir2/test1/test2/test3/file")); + fs.mkdirs(new Path("testDir2/test4/test3")); + assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file"))); + Assert.assertFalse(fs.rename(new Path("testDir2/test1/test2/test3"), + new Path("testDir2/test4"))); + assertTrue(fs.exists(new Path("testDir2"))); + assertTrue(fs.exists(new Path("testDir2/test1/test2"))); + assertTrue(fs.exists(new Path("testDir2/test4"))); + assertTrue(fs.exists(new Path("testDir2/test1/test2/test3"))); + if (getIsNamespaceEnabled(fs) + || fs.getAbfsStore().getAbfsConfiguration().getPrefixMode() + == PrefixMode.BLOB) { + assertFalse(fs.exists(new Path("testDir2/test4/test3/file"))); + assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file"))); + } else { + assertTrue(fs.exists(new Path("testDir2/test4/test3/file"))); + assertFalse(fs.exists(new Path("testDir2/test1/test2/test3/file"))); + } + } + + @Test + public void testPosixRenameDirectoryWherePartAlreadyThereOnDestination() + throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.mkdirs(new Path("testDir2/test1/test2/test3")); + fs.create(new Path("testDir2/test1/test2/test3/file")); + fs.create(new Path("testDir2/test1/test2/test3/file1")); + fs.mkdirs(new Path("testDir2/test4/")); + fs.create(new Path("testDir2/test4/file1")); + byte[] etag = fs.getXAttr(new Path("testDir2/test4/file1"), "ETag"); + assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file"))); + assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file1"))); + Assert.assertTrue(fs.rename(new Path("testDir2/test1/test2/test3"), + new Path("testDir2/test4"))); + assertTrue(fs.exists(new Path("testDir2"))); + assertTrue(fs.exists(new Path("testDir2/test1/test2"))); + assertTrue(fs.exists(new Path("testDir2/test4"))); + assertFalse(fs.exists(new Path("testDir2/test1/test2/test3"))); + + + assertFalse(fs.exists(new Path("testDir2/test4/file"))); + assertTrue(fs.exists(new Path("testDir2/test4/file1"))); + assertTrue(fs.exists(new Path("testDir2/test4/test3/file"))); + assertTrue(fs.exists(new Path("testDir2/test4/test3/file1"))); + assertTrue(fs.exists(new Path("testDir2/test4/file1"))); + assertFalse(fs.exists(new Path("testDir2/test1/test2/test3/file"))); + assertFalse(fs.exists(new Path("testDir2/test1/test2/test3/file1"))); + } + + private void assumeNonHnsAccountBlobEndpoint(final AzureBlobFileSystem fs) { + Assume.assumeTrue("To work on only on non-HNS Blob endpoint", + fs.getAbfsStore().getAbfsConfiguration().getPrefixMode() + == PrefixMode.BLOB); + } + + /** + * Test that after completing rename for a directory which is enabled for + * AtomicRename, the RenamePending JSON file is deleted. + */ + @Test + public void testRenamePendingJsonIsRemovedPostSuccessfulRename() + throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + fs.setWorkingDirectory(new Path("/")); + fs.mkdirs(new Path("hbase/test1/test2/test3")); + fs.create(new Path("hbase/test1/test2/test3/file")); + fs.create(new Path("hbase/test1/test2/test3/file1")); + fs.mkdirs(new Path("hbase/test4/")); + fs.create(new Path("hbase/test4/file1")); + final AzureBlobFileSystem spiedFs = Mockito.spy(fs); + final Integer[] correctDeletePathCount = new Integer[1]; + correctDeletePathCount[0] = 0; + + Mockito.doAnswer(answer -> { + final String correctDeletePath = "/hbase/test1/test2/test3" + SUFFIX; + if (correctDeletePath.equals( + ((Path) answer.getArgument(0)).toUri().getPath())) { + correctDeletePathCount[0] = 1; + } + return null; + }).when(spiedFs).delete(Mockito.any(Path.class), Mockito.anyBoolean()); + Assert.assertTrue(spiedFs.rename(new Path("hbase/test1/test2/test3"), + new Path("hbase/test4"))); + Assert.assertTrue(correctDeletePathCount[0] == 1); + } + + /** + * Test for a directory in /hbase directory. To simulate the crash of process, + * test will throw an exception with 503 on a copy of one of the blob.
+ * ListStatus API will be called on the directory. Expectation is that the ListStatus + * API of {@link AzureBlobFileSystem} should recover the paused rename. + */ + @Test + public void testHBaseHandlingForFailedRename() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + final String failedCopyPath = "hbase/test1/test2/test3/file1"; + fs.setWorkingDirectory(new Path("/")); + fs.mkdirs(new Path("hbase/test1/test2/test3")); + fs.create(new Path("hbase/test1/test2/test3/file")); + fs.create(new Path(failedCopyPath)); + fs.mkdirs(new Path("hbase/test4/")); + fs.create(new Path("hbase/test4/file1")); + final AzureBlobFileSystem spiedFs = Mockito.spy(fs); + final AzureBlobFileSystemStore spiedAbfsStore = Mockito.spy( + spiedFs.getAbfsStore()); + Mockito.doReturn(spiedAbfsStore).when(spiedFs).getAbfsStore(); + AbfsClient spiedClient = Mockito.spy(spiedAbfsStore.getClient()); + spiedAbfsStore.setClient(spiedClient); + Map pathLeaseIdMap = new HashMap<>(); + Mockito.doAnswer(answer -> { + AbfsRestOperation op = (AbfsRestOperation) answer.callRealMethod(); + String leaseId = op.getResult().getResponseHeader(X_MS_LEASE_ID); + pathLeaseIdMap.put(answer.getArgument(0), leaseId); + return op; + }).when(spiedClient).acquireBlobLease(Mockito.anyString(), Mockito.anyInt(), Mockito.any(TracingContext.class)); + final Integer[] correctDeletePathCount = new Integer[1]; + correctDeletePathCount[0] = 0; + + //fail copy of /hbase/test1/test2/test3/file1. + Mockito.doAnswer(answer -> { + final Path srcPath = answer.getArgument(0); + final Path dstPath = answer.getArgument(1); + final String leaseId = answer.getArgument(2); + final TracingContext tracingContext = answer.getArgument(3); + if (("/" + failedCopyPath).equalsIgnoreCase(srcPath.toUri().getPath())) { + throw new AbfsRestOperationException(HttpURLConnection.HTTP_UNAVAILABLE, + AzureServiceErrorCode.INGRESS_OVER_ACCOUNT_LIMIT.getErrorCode(), + "Ingress is over the account limit.", new Exception()); + } + fs.getAbfsStore().copyBlob(srcPath, dstPath, leaseId, tracingContext); + return null; + }) + .when(spiedAbfsStore) + .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + try { + spiedFs.rename(new Path("hbase/test1/test2/test3"), + new Path("hbase/test4")); + } catch (Exception ex) { + + } + Assert.assertTrue(fs.exists(new Path(failedCopyPath))); + Assert.assertFalse(spiedFs.exists(new Path( + failedCopyPath.replace("test1/test2/test3/", "test4/test3/")))); + + //call listPath API, it will recover the rename atomicity. + final AzureBlobFileSystem spiedFsForListPath = Mockito.spy(fs); + final AzureBlobFileSystemStore spiedStoreForListPath = Mockito.spy( + spiedFsForListPath.getAbfsStore()); + Mockito.doReturn(spiedStoreForListPath) + .when(spiedFsForListPath) + .getAbfsStore(); + + /* + * Check if the fs.delete is on the renameJson file. + */ + AtomicInteger deletedCount = new AtomicInteger(0); + AtomicBoolean srcSuffixDeletion = new AtomicBoolean(false); + Mockito.doAnswer(answer -> { + Path path = answer.getArgument(0); + Boolean recursive = answer.getArgument(1); + Assert.assertTrue( + ("/" + "hbase/test1/test2/test3" + SUFFIX).equalsIgnoreCase( + path.toUri().getPath())); + srcSuffixDeletion.set(true); + deletedCount.incrementAndGet(); + return fs.delete(path, recursive); + }) + .when(spiedFsForListPath) + .delete(Mockito.any(Path.class), Mockito.anyBoolean()); + + /* + * Check if the blob which will be retried is deleted from the renameBlob + * method. + */ + AbfsClient client = spiedFsForListPath.getAbfsClient(); + final AbfsClient spiedClientForListPath = Mockito.spy(client); + spiedFsForListPath.getAbfsStore().setClient(spiedClientForListPath); + Mockito.doAnswer(answer -> { + Path path = answer.getArgument(0); + String leaseId = answer.getArgument(1); + TracingContext tracingContext = answer.getArgument(2); + if (srcSuffixDeletion.get()) { + Assert.assertTrue( + ("/" + "hbase/test1/test2/test3" + SUFFIX).equalsIgnoreCase( + path.toUri().getPath())); + } else { + Assert.assertTrue( + ("/" + failedCopyPath).equalsIgnoreCase(path.toUri().getPath()) + || "/hbase/test1/test2/test3".equalsIgnoreCase( + path.toUri().getPath())); + deletedCount.incrementAndGet(); + } + client.deleteBlobPath(path, leaseId, tracingContext); + return null; + }) + .when(spiedClientForListPath) + .deleteBlobPath(Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + + for(Map.Entry entry : pathLeaseIdMap.entrySet()) { + try { + fs.getAbfsClient() + .releaseBlobLease(entry.getKey(), entry.getValue(), + getTestTracingContext(fs, true)); + } catch (AbfsRestOperationException ex) { + + } + } + spiedFsForListPath.listStatus(new Path("hbase/test1/test2")); + /* + * The invocation of getPathProperty will happen on the reinvocation of listStatus + * of /hbase/test2/test3 as after the resume, there will be no listing for the path + * and hence, getFileStatus would be required. + */ + Mockito.verify(spiedStoreForListPath, Mockito.times(1)) + .getPathProperty(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.anyBoolean()); + Assert.assertTrue(deletedCount.get() == 3); + Assert.assertFalse(spiedFsForListPath.exists(new Path(failedCopyPath))); + Assert.assertTrue(spiedFsForListPath.exists(new Path( + failedCopyPath.replace("test1/test2/test3/", "test4/test3/")))); + } + + /** + * Test for a directory in /hbase directory. To simulate the crash of process, + * test will throw an exception with 503 on a copy of one of the blob. The + * source directory is a nested directory.
+ * ListStatus API will be called on the directory. Expectation is that the ListStatus + * API of {@link AzureBlobFileSystem} should recover the paused rename. + */ + @Test + public void testHBaseHandlingForFailedRenameForNestedSourceThroughListFile() + throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + final String failedCopyPath = "hbase/test1/test2/test3/file1"; + fs.setWorkingDirectory(new Path("/")); + fs.mkdirs(new Path("hbase/test1/test2/test3")); + fs.create(new Path("hbase/test1/test2/test3/file")); + fs.create(new Path(failedCopyPath)); + fs.mkdirs(new Path("hbase/test4/")); + fs.create(new Path("hbase/test4/file1")); + final AzureBlobFileSystem spiedFs = Mockito.spy(fs); + final AzureBlobFileSystemStore spiedAbfsStore = Mockito.spy( + spiedFs.getAbfsStore()); + Mockito.doReturn(spiedAbfsStore).when(spiedFs).getAbfsStore(); + AbfsClient spiedClient = Mockito.spy(spiedAbfsStore.getClient()); + spiedAbfsStore.setClient(spiedClient); + Map pathLeaseIdMap = new HashMap<>(); + Mockito.doAnswer(answer -> { + AbfsRestOperation op = (AbfsRestOperation) answer.callRealMethod(); + String leaseId = op.getResult().getResponseHeader(X_MS_LEASE_ID); + pathLeaseIdMap.put(answer.getArgument(0), leaseId); + return op; + }).when(spiedClient).acquireBlobLease(Mockito.anyString(), Mockito.anyInt(), Mockito.any(TracingContext.class)); + final Integer[] correctDeletePathCount = new Integer[1]; + correctDeletePathCount[0] = 0; + + //fail copy of /hbase/test1/test2/test3/file1. + Mockito.doAnswer(answer -> { + final Path srcPath = answer.getArgument(0); + final Path dstPath = answer.getArgument(1); + final String leaseId = answer.getArgument(2); + final TracingContext tracingContext = answer.getArgument(3); + if (("/" + failedCopyPath).equalsIgnoreCase(srcPath.toUri().getPath())) { + throw new AbfsRestOperationException(HttpURLConnection.HTTP_UNAVAILABLE, + AzureServiceErrorCode.INGRESS_OVER_ACCOUNT_LIMIT.getErrorCode(), + "Ingress is over the account limit.", new Exception()); + } + fs.getAbfsStore().copyBlob(srcPath, dstPath, leaseId, tracingContext); + return null; + }) + .when(spiedAbfsStore) + .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + try { + spiedFs.rename(new Path("hbase/test1/test2"), + new Path("hbase/test4")); + } catch (Exception ex) { + + } + Assert.assertTrue(fs.exists(new Path(failedCopyPath))); + Assert.assertFalse(spiedFs.exists(new Path( + failedCopyPath.replace("test1/test2/test3/", "test4/test3/")))); + + //call listPath API, it will recover the rename atomicity. + final AzureBlobFileSystem spiedFsForListPath = Mockito.spy(fs); + final AzureBlobFileSystemStore spiedStoreForListPath = Mockito.spy( + spiedFsForListPath.getAbfsStore()); + Mockito.doReturn(spiedStoreForListPath) + .when(spiedFsForListPath) + .getAbfsStore(); + + /* + * Check if the fs.delete is on the renameJson file. + */ + AtomicInteger deletedCount = new AtomicInteger(0); + AtomicBoolean srcDirSuffixDeletion = new AtomicBoolean(false); + Mockito.doAnswer(answer -> { + Path path = answer.getArgument(0); + Boolean recursive = answer.getArgument(1); + Assert.assertTrue(("/" + "hbase/test1/test2" + SUFFIX).equalsIgnoreCase( + path.toUri().getPath())); + srcDirSuffixDeletion.set(true); + deletedCount.incrementAndGet(); + return fs.delete(path, recursive); + }) + .when(spiedFsForListPath) + .delete(Mockito.any(Path.class), Mockito.anyBoolean()); + + /* + * Check if the blob which will be retried is deleted from the renameBlob + * method. + */ + AbfsClient client = spiedFsForListPath.getAbfsClient(); + final AbfsClient spiedClientForListPath = Mockito.spy(client); + spiedFsForListPath.getAbfsStore().setClient(spiedClientForListPath); + Mockito.doAnswer(answer -> { + Path path = answer.getArgument(0); + String leaseId = answer.getArgument(1); + TracingContext tracingContext = answer.getArgument(2); + if (srcDirSuffixDeletion.get()) { + Assert.assertTrue(("/" + "hbase/test1/test2" + SUFFIX).equalsIgnoreCase( + path.toUri().getPath())); + } else { + Assert.assertTrue( + ("/" + failedCopyPath).equalsIgnoreCase(path.toUri().getPath()) + || "/hbase/test1/test2".equalsIgnoreCase( + path.toUri().getPath())); + deletedCount.incrementAndGet(); + } + client.deleteBlobPath(path, leaseId, tracingContext); + return null; + }) + .when(spiedClientForListPath) + .deleteBlobPath(Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + + /* + * listFile on /hbase/test1 would give no result because + * /hbase/test1/test2 would be totally moved to /hbase/test4. + * + */ + for(Map.Entry entry : pathLeaseIdMap.entrySet()) { + try { + fs.getAbfsClient() + .releaseBlobLease(entry.getKey(), entry.getValue(), + getTestTracingContext(fs, true)); + } catch (AbfsRestOperationException ex) { + + } + } + final FileStatus[] listFileResult = spiedFsForListPath.listStatus( + new Path("hbase/test1")); + /* + * The invocation of getPathProperty will happen on the reinvocation of listStatus + * of /hbase/test2/test3 as after the resume, there will be no listing for the path + * and hence, getFileStatus would be required. + */ + Mockito.verify(spiedStoreForListPath, Mockito.times(1)) + .getPathProperty(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.anyBoolean()); + Assert.assertTrue(deletedCount.get() == 3); + Assert.assertFalse(spiedFsForListPath.exists(new Path(failedCopyPath))); + Assert.assertTrue(spiedFsForListPath.exists(new Path( + failedCopyPath.replace("test1/test2/test3/", "test4/test2/test3/")))); + Assert.assertTrue(listFileResult.length == 0); + } + + /** + * Test for a directory in /hbase directory. To simulate the crash of process, + * test will throw an exception with 503 on a copy of one of the blob. The + * source directory is a nested directory.
+ * GetFileStatus API will be called on the directory. Expectation is that the + * GetFileStatus API of {@link AzureBlobFileSystem} should recover the paused + * rename. + */ + @Test + public void testHBaseHandlingForFailedRenameForNestedSourceThroughGetPathStatus() + throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + final String failedCopyPath = "hbase/test1/test2/test3/file1"; + fs.setWorkingDirectory(new Path("/")); + fs.mkdirs(new Path("hbase/test1/test2/test3")); + fs.create(new Path("hbase/test1/test2/test3/file")); + fs.create(new Path(failedCopyPath)); + fs.mkdirs(new Path("hbase/test4/")); + fs.create(new Path("hbase/test4/file1")); + final AzureBlobFileSystem spiedFs = Mockito.spy(fs); + final AzureBlobFileSystemStore spiedAbfsStore = Mockito.spy( + spiedFs.getAbfsStore()); + Mockito.doReturn(spiedAbfsStore).when(spiedFs).getAbfsStore(); + AbfsClient spiedClient = Mockito.spy(spiedAbfsStore.getClient()); + spiedAbfsStore.setClient(spiedClient); + Map pathLeaseIdMap = new HashMap<>(); + Mockito.doAnswer(answer -> { + AbfsRestOperation op = (AbfsRestOperation) answer.callRealMethod(); + String leaseId = op.getResult().getResponseHeader(X_MS_LEASE_ID); + pathLeaseIdMap.put(answer.getArgument(0), leaseId); + return op; + }).when(spiedClient).acquireBlobLease(Mockito.anyString(), Mockito.anyInt(), Mockito.any(TracingContext.class)); + final Integer[] correctDeletePathCount = new Integer[1]; + correctDeletePathCount[0] = 0; + + //fail copy of /hbase/test1/test2/test3/file1. + Mockito.doAnswer(answer -> { + final Path srcPath = answer.getArgument(0); + final Path dstPath = answer.getArgument(1); + final String leaseId = answer.getArgument(2); + final TracingContext tracingContext = answer.getArgument(3); + if (("/" + failedCopyPath).equalsIgnoreCase(srcPath.toUri().getPath())) { + throw new AbfsRestOperationException(HttpURLConnection.HTTP_UNAVAILABLE, + AzureServiceErrorCode.INGRESS_OVER_ACCOUNT_LIMIT.getErrorCode(), + "Ingress is over the account limit.", new Exception()); + } + fs.getAbfsStore().copyBlob(srcPath, dstPath, leaseId, tracingContext); + return null; + }) + .when(spiedAbfsStore) + .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + try { + spiedFs.rename(new Path("hbase/test1/test2"), + new Path("hbase/test4")); + } catch (Exception ex) { + + } + Assert.assertTrue(fs.exists(new Path(failedCopyPath))); + Assert.assertFalse(spiedFs.exists(new Path( + failedCopyPath.replace("test1/test2/test3/", "test4/test3/")))); + + //call listPath API, it will recover the rename atomicity. + final AzureBlobFileSystem spiedFsForListPath = Mockito.spy(fs); + final AzureBlobFileSystemStore spiedStoreForListPath = Mockito.spy( + spiedFsForListPath.getAbfsStore()); + Mockito.doReturn(spiedStoreForListPath) + .when(spiedFsForListPath) + .getAbfsStore(); + + /* + * Check if the fs.delete is on the renameJson file. + */ + AtomicInteger deletedCount = new AtomicInteger(0); + AtomicBoolean srcSuffixDeletion = new AtomicBoolean(false); + Mockito.doAnswer(answer -> { + Path path = answer.getArgument(0); + Boolean recursive = answer.getArgument(1); + Assert.assertTrue(("/" + "hbase/test1/test2" + SUFFIX).equalsIgnoreCase( + path.toUri().getPath())); + srcSuffixDeletion.set(true); + deletedCount.incrementAndGet(); + return fs.delete(path, recursive); + }) + .when(spiedFsForListPath) + .delete(Mockito.any(Path.class), Mockito.anyBoolean()); + + /* + * Check if the blob which will be retried is deleted from the renameBlob + * method. + */ + AbfsClient client = spiedFsForListPath.getAbfsClient(); + final AbfsClient spiedClientForListPath = Mockito.spy(client); + spiedFsForListPath.getAbfsStore().setClient(spiedClientForListPath); + Mockito.doAnswer(answer -> { + Path path = answer.getArgument(0); + String leaseId = answer.getArgument(1); + TracingContext tracingContext = answer.getArgument(2); + if (srcSuffixDeletion.get()) { + Assert.assertTrue(("/" + "hbase/test1/test2" + SUFFIX).equalsIgnoreCase( + path.toUri().getPath())); + } else { + Assert.assertTrue( + ("/" + failedCopyPath).equalsIgnoreCase(path.toUri().getPath()) + || "/hbase/test1/test2".equalsIgnoreCase( + path.toUri().getPath())); + deletedCount.incrementAndGet(); + } + client.deleteBlobPath(path, leaseId, tracingContext); + return null; + }) + .when(spiedClientForListPath) + .deleteBlobPath(Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + + /* + * getFileStatus on /hbase/test2 should give NOT_FOUND exception, since, + * /hbase/test1/test2 was partially renamed. On the invocation of getFileStatus + * on the directory, the remaining rename will be made. And as the directory is renamed, + * the method should give NOT_FOUND exception. + */ + for(Map.Entry entry : pathLeaseIdMap.entrySet()) { + try { + fs.getAbfsClient().releaseBlobLease(entry.getKey(), entry.getValue(), getTestTracingContext(fs, true)); + } catch (AbfsRestOperationException ex) { + + } + } + FileStatus fileStatus = null; + Boolean notFoundExceptionReceived = false; + try { + fileStatus = spiedFsForListPath.getFileStatus( + new Path("hbase/test1/test2")); + } catch (FileNotFoundException ex) { + notFoundExceptionReceived = true; + + } + Assert.assertTrue(notFoundExceptionReceived); + Assert.assertNull(fileStatus); + /* + * GetPathProperty on store would be called to get FileStatus for srcDirectory + * and the corresponding renamePendingJson file. + */ + Mockito.verify(spiedStoreForListPath, Mockito.times(2)) + .getPathProperty(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.anyBoolean()); + Assert.assertTrue(deletedCount.get() == 3); + Assert.assertFalse(spiedFsForListPath.exists(new Path(failedCopyPath))); + Assert.assertTrue(spiedFsForListPath.exists(new Path( + failedCopyPath.replace("test1/test2/test3/", "test4/test2/test3/")))); + } + + /** + * Simulates a scenario where HMaster in Hbase starts up and executes listStatus + * API on the directory that has to be renamed by some other executor-machine. + * The scenario is that RenamePending JSON is created but before it could be + * appended, it has been opened by the HMaster. The HMaster will delete it. The + * machine doing rename would have to recreate the JSON file. + * ref: issue + */ + @Test + public void testHbaseListStatusBeforeRenamePendingFileAppendedWithIngressOnBlob() + throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + fs.setWorkingDirectory(new Path("/")); + testHbaseListStatusBeforeRenamePendingFileAppended(fs); + } + + @Test + public void testHbaseListStatusBeforeRenamePendingFileAppendedWithIngressOnDFS() + throws Exception { + AzureBlobFileSystem fs = this.getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + + + Configuration configuration = Mockito.spy(fs.getAbfsStore().getAbfsConfiguration().getRawConfiguration()); + configuration.set(FS_AZURE_INGRESS_FALLBACK_TO_DFS, TRUE); + fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + fs.setWorkingDirectory(new Path("/")); + testHbaseListStatusBeforeRenamePendingFileAppended(fs); + } + + private void testHbaseListStatusBeforeRenamePendingFileAppended(final AzureBlobFileSystem fs) throws IOException { + final String failedCopyPath = "hbase/test1/test2/test3/file1"; + fs.mkdirs(new Path("hbase/test1/test2/test3")); + fs.create(new Path("hbase/test1/test2/test3/file")); + fs.create(new Path(failedCopyPath)); + fs.mkdirs(new Path("hbase/test4/")); + fs.create(new Path("hbase/test4/file1")); + final AzureBlobFileSystem spiedFs = Mockito.spy(fs); + final AzureBlobFileSystemStore spiedAbfsStore = Mockito.spy( + spiedFs.getAbfsStore()); + Mockito.doReturn(spiedAbfsStore).when(spiedFs).getAbfsStore(); + final Integer[] correctDeletePathCount = new Integer[1]; + correctDeletePathCount[0] = 0; + + Boolean[] renamePendingJsonCreated = new Boolean[1]; + renamePendingJsonCreated[0] = false; + Boolean[] parallelListStatusCalledOnTheDirBeingRenamed = new Boolean[1]; + parallelListStatusCalledOnTheDirBeingRenamed[0] = false; + Mockito.doAnswer(answer -> { + Path path = answer.getArgument(0); + Boolean recursive = answer.getArgument(1); + FSDataOutputStream outputStream = fs.create(path, recursive); + renamePendingJsonCreated[0] = true; + while (!parallelListStatusCalledOnTheDirBeingRenamed[0]) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + return outputStream; + }).when(spiedFs).create(Mockito.any(Path.class), Mockito.anyBoolean()); + + try { + new Thread(() -> { + //wait for the renamePending created; + while (!renamePendingJsonCreated[0]) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + try { + spiedFs.listStatus(new Path("hbase/test1")); + parallelListStatusCalledOnTheDirBeingRenamed[0] = true; + } catch (IOException e) { + throw new RuntimeException(e); + } + }).start(); + spiedFs.rename(new Path("hbase/test1/test2"), + new Path("hbase/test4")); + } catch (Exception ex) { + + } + Assert.assertFalse(fs.exists(new Path(failedCopyPath))); + Assert.assertTrue( + spiedFs.exists(new Path(failedCopyPath.replace("test1/", "test4/")))); + } + + @Test + public void testHbaseEmptyRenamePendingJsonDeletedBeforeListStatusCanDelete() + throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + final String failedCopyPath = "hbase/test1/test2/test3/file1"; + fs.setWorkingDirectory(new Path("/")); + fs.mkdirs(new Path("hbase/test1/test2/test3")); + fs.create(new Path("hbase/test1/test2/test3/file")); + fs.create(new Path(failedCopyPath)); + fs.mkdirs(new Path("hbase/test4/")); + fs.create(new Path("hbase/test4/file1")); + final AzureBlobFileSystem spiedFs = Mockito.spy(fs); + final AzureBlobFileSystemStore spiedStore = Mockito.spy( + spiedFs.getAbfsStore()); + Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore(); + final Integer[] correctDeletePathCount = new Integer[1]; + correctDeletePathCount[0] = 0; + + AzureBlobFileSystem listFileFs = Mockito.spy(fs); + + + Boolean[] renamePendingJsonCreated = new Boolean[1]; + renamePendingJsonCreated[0] = false; + Boolean[] parallelListStatusCalledOnTheDirBeingRenamed = new Boolean[1]; + parallelListStatusCalledOnTheDirBeingRenamed[0] = false; + Boolean[] parallelDeleteOfRenamePendingFileFromRenameFlow = new Boolean[1]; + parallelDeleteOfRenamePendingFileFromRenameFlow[0] = false; + Mockito.doAnswer(answer -> { + Path path = answer.getArgument(0); + Boolean recursive = answer.getArgument(1); + FSDataOutputStream outputStream = fs.create(path, recursive); + renamePendingJsonCreated[0] = true; + while (!parallelListStatusCalledOnTheDirBeingRenamed[0]) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + return outputStream; + }).when(spiedFs).create(Mockito.any(Path.class), Mockito.anyBoolean()); + + Mockito.doAnswer(answer -> { + Path path = answer.getArgument(0); + Boolean recursive = answer.getArgument(1); + if (("/hbase/test1/test2" + SUFFIX).equalsIgnoreCase( + path.toUri().getPath())) { + while (!parallelListStatusCalledOnTheDirBeingRenamed[0]) { + Thread.sleep(100); + } + parallelDeleteOfRenamePendingFileFromRenameFlow[0] = true; + return fs.delete(path, recursive); + } + return fs.delete(path, recursive); + }).when(spiedFs).delete(Mockito.any(Path.class), Mockito.anyBoolean()); + + Mockito.doAnswer(answer -> { + Path path = answer.getArgument(0); + if (("/hbase/test1/test2" + SUFFIX).equalsIgnoreCase( + path.toUri().getPath())) { + FSDataInputStream inputStream = fs.open(path); + parallelListStatusCalledOnTheDirBeingRenamed[0] = true; + while (!parallelDeleteOfRenamePendingFileFromRenameFlow[0]) { + Thread.sleep(100); + } + return inputStream; + } + return fs.open(path); + }).when(listFileFs).open(Mockito.any(Path.class)); + + try { + new Thread(() -> { + //wait for the renamePending created; + while (!renamePendingJsonCreated[0]) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + try { + listFileFs.listStatus(new Path("hbase/test1")); + parallelListStatusCalledOnTheDirBeingRenamed[0] = true; + } catch (IOException e) { + throw new RuntimeException(e); + } + }).start(); + spiedFs.rename(new Path("hbase/test1/test2"), + new Path("hbase/test4")); + } catch (Exception ex) { + + } + Assert.assertFalse(fs.exists(new Path(failedCopyPath))); + Assert.assertTrue( + spiedFs.exists(new Path(failedCopyPath.replace("test1/", "test4/")))); + } + + @Test + public void testInvalidJsonForRenamePendingFile() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + fs.setWorkingDirectory(new Path("/")); + fs.mkdirs(new Path("hbase/test1/test2/test3")); + fs.create(new Path("hbase/test1/test2/test3/file")); + fs.create(new Path("hbase/test1/test2/test3/file1")); + FSDataOutputStream outputStream = fs.create( + new Path("hbase/test1/test2/test3" + SUFFIX)); + outputStream.writeChars("{ some wrong json"); + outputStream.flush(); + outputStream.close(); + + fs.listStatus(new Path("hbase/test1/test2")); + Assert.assertFalse(fs.exists(new Path("hbase/test1/test2/test3" + SUFFIX))); + } + + @Test + public void testEmptyDirRenameResolveFromGetFileStatus() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + String srcDir = "/hbase/test1/test2/test3"; + fs.setWorkingDirectory(new Path("/")); + fs.mkdirs(new Path(srcDir)); + fs.create(new Path(srcDir, "file1")); + fs.mkdirs(new Path("hbase/test4")); + + AzureBlobFileSystem spiedFs = Mockito.spy(fs); + + AzureBlobFileSystemStore spiedAbfsStore = Mockito.spy( + spiedFs.getAbfsStore()); + Mockito.doReturn(spiedAbfsStore).when(spiedFs).getAbfsStore(); + AbfsClient spiedClient = Mockito.spy(spiedAbfsStore.getClient()); + spiedAbfsStore.setClient(spiedClient); + Map pathLeaseIdMap = new HashMap<>(); + Mockito.doAnswer(answer -> { + AbfsRestOperation op = (AbfsRestOperation) answer.callRealMethod(); + String leaseId = op.getResult().getResponseHeader(X_MS_LEASE_ID); + pathLeaseIdMap.put(answer.getArgument(0), leaseId); + return op; + }).when(spiedClient).acquireBlobLease(Mockito.anyString(), Mockito.anyInt(), Mockito.any(TracingContext.class)); + Mockito.doAnswer(answer -> { + throw new AbfsRestOperationException(HttpURLConnection.HTTP_UNAVAILABLE, + AzureServiceErrorCode.INGRESS_OVER_ACCOUNT_LIMIT.getErrorCode(), + "Ingress is over the account limit.", new Exception()); + }) + .when(spiedAbfsStore) + .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + try { + spiedFs.rename(new Path(srcDir), + new Path("hbase/test4")); + } catch (Exception ex) { + } + + Assert.assertFalse(spiedFs.exists( + new Path(srcDir.replace("test1/test2/test3", "test4/test3/")))); + + //call listPath API, it will recover the rename atomicity. + for(Map.Entry entry : pathLeaseIdMap.entrySet()) { + try { + fs.getAbfsClient() + .releaseBlobLease(entry.getKey(), entry.getValue(), + getTestTracingContext(fs, true)); + } catch (Exception e) {} + } + + final AzureBlobFileSystem spiedFsForListPath = Mockito.spy(fs); + final AzureBlobFileSystemStore spiedStoreForListPath = Mockito.spy( + spiedFsForListPath.getAbfsStore()); + Mockito.doReturn(spiedStoreForListPath) + .when(spiedFsForListPath) + .getAbfsStore(); + + /* + * Check if the fs.delete is on the renameJson file. + */ + AtomicInteger deletedCount = new AtomicInteger(0); + AtomicBoolean deletedSrcDirSuffix = new AtomicBoolean(false); + Mockito.doAnswer(answer -> { + Path path = answer.getArgument(0); + Boolean recursive = answer.getArgument(1); + Assert.assertTrue( + (srcDir + SUFFIX).equalsIgnoreCase(path.toUri().getPath())); + deletedSrcDirSuffix.set(true); + deletedCount.incrementAndGet(); + return fs.delete(path, recursive); + }) + .when(spiedFsForListPath) + .delete(Mockito.any(Path.class), Mockito.anyBoolean()); + + /* + * Check if the blob which will be retried is deleted from the renameBlob + * method. + */ + AbfsClient client = spiedFsForListPath.getAbfsClient(); + final AbfsClient spiedClientForListPath = Mockito.spy(client); + spiedFsForListPath.getAbfsStore().setClient(spiedClientForListPath); + Mockito.doAnswer(answer -> { + Path path = answer.getArgument(0); + String leaseId = answer.getArgument(1); + TracingContext tracingContext = answer.getArgument(2); + if (deletedSrcDirSuffix.get()) { + Assert.assertTrue( + (srcDir + SUFFIX).equalsIgnoreCase(path.toUri().getPath())); + } else { + Assert.assertTrue( + ((srcDir).equalsIgnoreCase(path.toUri().getPath()) || (srcDir + + "/file1").equalsIgnoreCase(path.toUri().getPath()))); + deletedCount.incrementAndGet(); + } + client.deleteBlobPath(path, leaseId, tracingContext); + return null; + }) + .when(spiedClientForListPath) + .deleteBlobPath(Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + + /* + * getFileStatus on /hbase/test2 should give NOT_FOUND exception, since, + * /hbase/test1/test2 was partially renamed. On the invocation of getFileStatus + * on the directory, the remaining rename will be made. And as the directory is renamed, + * the method should give NOT_FOUND exception. + */ + FileStatus fileStatus = null; + Boolean notFoundExceptionReceived = false; + try { + fileStatus = spiedFsForListPath.getFileStatus(new Path(srcDir)); + } catch (FileNotFoundException ex) { + notFoundExceptionReceived = true; + + } + Assert.assertTrue(notFoundExceptionReceived); + Assert.assertNull(fileStatus); + /* + * GetFileStatus on store would be called to get FileStatus for srcDirectory + * and the corresponding renamePendingJson file. + */ + Mockito.verify(spiedStoreForListPath, Mockito.times(2)) + .getPathProperty(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.anyBoolean()); + Assert.assertTrue(deletedCount.get() == 3); + Assert.assertFalse(spiedFsForListPath.exists(new Path(srcDir))); + Assert.assertTrue(spiedFsForListPath.getFileStatus( + new Path(srcDir.replace("test1/test2/test3", "test4/test3/"))) + .isDirectory()); + } + + @Test + public void testRenameBlobIdempotency() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + String srcDir = "/test1/test2/test3"; + fs.mkdirs(new Path(srcDir)); + fs.create(new Path(srcDir, "file1")); + fs.create(new Path(srcDir, "file2")); + + fs.mkdirs(new Path("/test4")); + + final AzureBlobFileSystem spiedFs = Mockito.spy(fs); + final AzureBlobFileSystemStore spiedStore = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore(); + final AbfsClient spiedClient = Mockito.spy(fs.getAbfsClient()); + spiedStore.setClient(spiedClient); + + /* + * First call to copyBlob for file1 will fail with connection-reset, but the + * backend has got the call. Retry of that API would give 409 error. + */ + boolean[] hasBeenCalled = new boolean[1]; + hasBeenCalled[0] = false; + + boolean[] connectionResetThrown = new boolean[1]; + connectionResetThrown[0] = false; + + AbfsClientTestUtil.setMockAbfsRestOperationForCopyBlobOperation(spiedClient, + (spiedRestOp, actualCallMakerOp) -> { + + Mockito.doAnswer(answer -> { + if (spiedRestOp.getUrl().toString().contains("file1") + && !hasBeenCalled[0]) { + hasBeenCalled[0] = true; + actualCallMakerOp.execute(answer.getArgument(0)); + AbfsRestOperationTestUtil.addAbfsHttpOpProcessResponseMock( + spiedRestOp, (mockAbfsHttpOp, actualAbfsHttpOp) -> { + Mockito.doAnswer(sendRequestAnswer -> { + if (!connectionResetThrown[0]) { + connectionResetThrown[0] = true; + throw new SocketException("connection-reset"); + } + spiedRestOp.signRequest(actualAbfsHttpOp, + sendRequestAnswer.getArgument(2)); + actualAbfsHttpOp.sendRequest( + sendRequestAnswer.getArgument(0), + sendRequestAnswer.getArgument(1), + sendRequestAnswer.getArgument(2)); + AbfsHttpOpTestUtil.setConnection(mockAbfsHttpOp, + actualAbfsHttpOp); + return mockAbfsHttpOp; + }).when(mockAbfsHttpOp) + .sendRequest(Mockito.nullable(byte[].class), + Mockito.anyInt(), Mockito.anyInt()); + + return mockAbfsHttpOp; + }); + Mockito.doCallRealMethod() + .when(spiedRestOp) + .execute(Mockito.any(TracingContext.class)); + spiedRestOp.execute(answer.getArgument(0)); + return spiedRestOp; + } else { + actualCallMakerOp.execute(answer.getArgument(0)); + AbfsRestOperationTestUtil.setResult(spiedRestOp, + actualCallMakerOp.getResult()); + return actualCallMakerOp; + } + }).when(spiedRestOp).execute(Mockito.any(TracingContext.class)); + return spiedRestOp; + }); + + spiedFs.setWorkingDirectory(new Path("/")); + + Assert.assertTrue(spiedFs.rename(new Path(srcDir), new Path("/test4"))); + Assert.assertTrue(spiedFs.exists(new Path("test4/test3/file1"))); + Assert.assertTrue(spiedFs.exists(new Path("test4/test3/file2"))); + Assert.assertTrue(hasBeenCalled[0]); + Assert.assertTrue(connectionResetThrown[0]); + } + + @Test + public void testRenameBlobIdempotencyWhereDstIsCreatedFromSomeOtherProcess() + throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + String srcDir = "/test1/test2/test3"; + fs.mkdirs(new Path(srcDir)); + fs.create(new Path(srcDir, "file1")); + fs.create(new Path(srcDir, "file2")); + + fs.mkdirs(new Path("/test4")); + + final AzureBlobFileSystem spiedFs = Mockito.spy(fs); + final AzureBlobFileSystemStore spiedStore = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore(); + final AbfsClient spiedClient = Mockito.spy(fs.getAbfsClient()); + spiedStore.setClient(spiedClient); + + /* + * First call to copyBlob for file1 will fail with connection-reset, but the + * backend has got the call. Retry of that API would give 409 error. + */ + boolean[] hasBeenCalled = new boolean[1]; + hasBeenCalled[0] = false; + + boolean[] connectionResetThrown = new boolean[1]; + connectionResetThrown[0] = false; + + AbfsClientTestUtil.setMockAbfsRestOperationForCopyBlobOperation(spiedClient, + (spiedRestOp, actualCallMakerOp) -> { + + Mockito.doAnswer(answer -> { + if (spiedRestOp.getUrl().toString().contains("file1") + && !hasBeenCalled[0]) { + hasBeenCalled[0] = true; + fs.create(new Path("/test4/test3", "file1")); + AbfsRestOperationTestUtil.addAbfsHttpOpProcessResponseMock( + spiedRestOp, (mockAbfsHttpOp, actualAbfsHttpOp) -> { + Mockito.doAnswer(sendRequestAnswer -> { + if (!connectionResetThrown[0]) { + connectionResetThrown[0] = true; + throw new SocketException("connection-reset"); + } + spiedRestOp.signRequest(actualAbfsHttpOp, + sendRequestAnswer.getArgument(2)); + actualAbfsHttpOp.sendRequest( + sendRequestAnswer.getArgument(0), + sendRequestAnswer.getArgument(1), + sendRequestAnswer.getArgument(2)); + AbfsHttpOpTestUtil.setConnection(mockAbfsHttpOp, + actualAbfsHttpOp); + return mockAbfsHttpOp; + }).when(mockAbfsHttpOp) + .sendRequest(Mockito.nullable(byte[].class), + Mockito.anyInt(), Mockito.anyInt()); + + return mockAbfsHttpOp; + }); + Mockito.doCallRealMethod() + .when(spiedRestOp) + .execute(Mockito.any(TracingContext.class)); + spiedRestOp.execute(answer.getArgument(0)); + return spiedRestOp; + } else { + actualCallMakerOp.execute(answer.getArgument(0)); + AbfsRestOperationTestUtil.setResult(spiedRestOp, + actualCallMakerOp.getResult()); + return actualCallMakerOp; + } + }).when(spiedRestOp).execute(Mockito.any(TracingContext.class)); + return spiedRestOp; + }); + + spiedFs.setWorkingDirectory(new Path("/")); + + Boolean dstAlreadyThere = false; + try { + spiedFs.rename(new Path(srcDir), new Path("/test4")); + } catch (RuntimeException ex) { + if (ex.getMessage().contains(HttpURLConnection.HTTP_CONFLICT + "")) { + dstAlreadyThere = true; + } + } + Assert.assertTrue(dstAlreadyThere); + Assert.assertTrue(hasBeenCalled[0]); + Assert.assertTrue(connectionResetThrown[0]); + } + + @Test + public void testRenameBlobIdempotencyWhereDstIsCopiedFromSomeOtherProcess() + throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + String srcDir = "/test1/test2/test3"; + fs.mkdirs(new Path(srcDir)); + fs.create(new Path(srcDir, "file1")); + fs.create(new Path(srcDir, "file2")); + + fs.mkdirs(new Path("/test4")); + + final AzureBlobFileSystem spiedFs = Mockito.spy(fs); + final AzureBlobFileSystemStore spiedStore = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore(); + final AbfsClient spiedClient = Mockito.spy(fs.getAbfsClient()); + spiedStore.setClient(spiedClient); + + /* + * First call to copyBlob for file1 will fail with connection-reset, but the + * backend has got the call. Retry of that API would give 409 error. + */ + boolean[] hasBeenCalled = new boolean[1]; + hasBeenCalled[0] = false; + + boolean[] connectionResetThrown = new boolean[1]; + connectionResetThrown[0] = false; + + AbfsClientTestUtil.setMockAbfsRestOperationForCopyBlobOperation(spiedClient, + (spiedRestOp, actualCallMakerOp) -> { + + Mockito.doAnswer(answer -> { + if (spiedRestOp.getUrl().toString().contains("file1") + && !hasBeenCalled[0]) { + hasBeenCalled[0] = true; + fs.create(new Path("/randomDir/test3/file1")); + fs.rename(new Path("/randomDir/test3/file1"), + new Path("/test4/test3/file1")); + AbfsRestOperationTestUtil.addAbfsHttpOpProcessResponseMock( + spiedRestOp, (mockAbfsHttpOp, actualAbfsHttpOp) -> { + Mockito.doAnswer(sendRequestAnswer -> { + if (!connectionResetThrown[0]) { + connectionResetThrown[0] = true; + throw new SocketException("connection-reset"); + } + spiedRestOp.signRequest(actualAbfsHttpOp, + sendRequestAnswer.getArgument(2)); + actualAbfsHttpOp.sendRequest( + sendRequestAnswer.getArgument(0), + sendRequestAnswer.getArgument(1), + sendRequestAnswer.getArgument(2)); + AbfsHttpOpTestUtil.setConnection(mockAbfsHttpOp, + actualAbfsHttpOp); + return mockAbfsHttpOp; + }).when(mockAbfsHttpOp) + .sendRequest(Mockito.nullable(byte[].class), + Mockito.anyInt(), Mockito.anyInt()); + + return mockAbfsHttpOp; + }); + Mockito.doCallRealMethod() + .when(spiedRestOp) + .execute(Mockito.any(TracingContext.class)); + spiedRestOp.execute(answer.getArgument(0)); + return spiedRestOp; + } else { + actualCallMakerOp.execute(answer.getArgument(0)); + AbfsRestOperationTestUtil.setResult(spiedRestOp, + actualCallMakerOp.getResult()); + return actualCallMakerOp; + } + }).when(spiedRestOp).execute(Mockito.any(TracingContext.class)); + return spiedRestOp; + }); + + spiedFs.setWorkingDirectory(new Path("/")); + + Boolean dstAlreadyThere = false; + try { + spiedFs.rename(new Path(srcDir), new Path("/test4")); + } catch (RuntimeException ex) { + if (ex.getMessage().contains(HttpURLConnection.HTTP_CONFLICT + "")) { + dstAlreadyThere = true; + } + } + Assert.assertTrue(dstAlreadyThere); + Assert.assertTrue(hasBeenCalled[0]); + Assert.assertTrue(connectionResetThrown[0]); + } + + @Test + public void testRenameLargeNestedDir() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + String dir = "/"; + for (int i = 0; i < 100; i++) { + dir += ("dir" + i + "/"); + fs.mkdirs(new Path(dir)); + } + fs.mkdirs(new Path("/dst")); + fs.rename(new Path("/dir0"), new Path("/dst")); + dir = ""; + for (int i = 0; i < 100; i++) { + dir += ("dir" + i + "/"); + Assert.assertTrue("" + i, fs.exists(new Path("/dst/" + dir))); + } + } + + @Test + public void testRenameDirWhenMarkerBlobIsAbsent() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + fs.mkdirs(new Path("/test1")); + fs.mkdirs(new Path("/test1/test2")); + fs.mkdirs(new Path("/test1/test2/test3")); + fs.create(new Path("/test1/test2/test3/file")); + + fs.getAbfsClient() + .deleteBlobPath(new Path("/test1/test2"), + null, getTestTracingContext(fs, true)); + fs.mkdirs(new Path("/test4/test5")); + fs.rename(new Path("/test4"), new Path("/test1/test2")); + + Assert.assertTrue(fs.exists(new Path("/test1/test2/test4/test5"))); + + fs.mkdirs(new Path("/test6")); + fs.rename(new Path("/test6"), new Path("/test1/test2/test4/test5")); + Assert.assertTrue(fs.exists(new Path("/test1/test2/test4/test5/test6"))); + + fs.getAbfsClient() + .deleteBlobPath(new Path("/test1/test2/test4/test5/test6"), + null, getTestTracingContext(fs, true)); + fs.mkdirs(new Path("/test7")); + fs.create(new Path("/test7/file")); + fs.rename(new Path("/test7"), new Path("/test1/test2/test4/test5/test6")); + Assert.assertTrue( + fs.exists(new Path("/test1/test2/test4/test5/test6/file"))); + } + + @Test + public void testBlobRenameSrcDirHasNoMarker() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + fs.create(new Path("/test1/test2/file1")); + fs.getAbfsStore() + .getClient() + .deleteBlobPath(new Path("/test1"), null, getTestTracingContext(fs, true)); + intercept(AbfsRestOperationException.class, () -> { + fs.getAbfsStore().getBlobProperty(new Path("/test1"), + getTestTracingContext(fs, true)); + }); + fs.mkdirs(new Path("/test2")); + fs.rename(new Path("/test1"), new Path("/test2")); + Assert.assertTrue(fs.getAbfsStore() + .getBlobProperty(new Path("/test2/test1"), + getTestTracingContext(fs, true)).getIsDirectory()); + } + + @Test + public void testCopyBlobTakeTime() throws Exception { + AzureBlobFileSystem fileSystem = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fileSystem); + AzureBlobFileSystemStore store = Mockito.spy(fileSystem.getAbfsStore()); + Mockito.doReturn(store).when(fileSystem).getAbfsStore(); + AbfsClient client = store.getClient(); + AbfsClient spiedClient = Mockito.spy(client); + store.setClient(spiedClient); + + Mockito.doAnswer(answer -> { + AbfsRestOperation op = Mockito.spy((AbfsRestOperation) answer.callRealMethod()); + AbfsHttpOperation httpOp = Mockito.spy(op.getResult()); + Mockito.doReturn(COPY_STATUS_PENDING).when(httpOp).getResponseHeader( + HttpHeaderConfigurations.X_MS_COPY_STATUS); + Mockito.doReturn(httpOp).when(op).getResult(); + return op; + }).when(spiedClient).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + fileSystem.create(new Path("/test1/file")); + fileSystem.rename(new Path("/test1/file"), new Path("/test1/file2")); + Assert.assertTrue(fileSystem.exists(new Path("/test1/file2"))); + Mockito.verify(store, Mockito.times(1)) + .handleCopyInProgress(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.any(String.class)); + } + + @Test + public void testCopyBlobTakeTimeAndEventuallyFail() throws Exception { + AzureBlobFileSystem fileSystem = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fileSystem); + AzureBlobFileSystemStore store = Mockito.spy(fileSystem.getAbfsStore()); + Mockito.doReturn(store).when(fileSystem).getAbfsStore(); + AbfsClient client = store.getClient(); + AbfsClient spiedClient = Mockito.spy(client); + store.setClient(spiedClient); + + Mockito.doAnswer(answer -> { + AbfsRestOperation op = Mockito.spy((AbfsRestOperation) answer.callRealMethod()); + AbfsHttpOperation httpOp = Mockito.spy(op.getResult()); + Mockito.doReturn(COPY_STATUS_PENDING).when(httpOp).getResponseHeader( + HttpHeaderConfigurations.X_MS_COPY_STATUS); + Mockito.doReturn(httpOp).when(op).getResult(); + return op; + }).when(spiedClient).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + Mockito.doAnswer(answer -> { + AbfsRestOperation op = Mockito.spy((AbfsRestOperation) answer.callRealMethod()); + AbfsHttpOperation httpOp = Mockito.spy(op.getResult()); + Mockito.doReturn(COPY_STATUS_FAILED).when(httpOp).getResponseHeader( + HttpHeaderConfigurations.X_MS_COPY_STATUS); + Mockito.doReturn(httpOp).when(op).getResult(); + return op; + }).when(spiedClient).getBlobProperty(Mockito.any(Path.class), Mockito.any(TracingContext.class)); + + fileSystem.create(new Path("/test1/file")); + Boolean copyBlobFailureCaught = false; + try { + fileSystem.rename(new Path("/test1/file"), new Path("/test1/file2")); + } catch (AbfsRestOperationException e) { + if (COPY_BLOB_FAILED.equals(e.getErrorCode())) { + copyBlobFailureCaught = true; + } + } + Assert.assertTrue(copyBlobFailureCaught); + Assert.assertTrue(fileSystem.exists(new Path("/test1/file"))); + Mockito.verify(store, Mockito.times(1)) + .handleCopyInProgress(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.any(String.class)); + } + + @Test + public void testCopyBlobTakeTimeAndEventuallyAborted() throws Exception { + AzureBlobFileSystem fileSystem = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fileSystem); + AzureBlobFileSystemStore store = Mockito.spy(fileSystem.getAbfsStore()); + Mockito.doReturn(store).when(fileSystem).getAbfsStore(); + AbfsClient client = store.getClient(); + AbfsClient spiedClient = Mockito.spy(client); + store.setClient(spiedClient); + + Mockito.doAnswer(answer -> { + AbfsRestOperation op = Mockito.spy((AbfsRestOperation) answer.callRealMethod()); + AbfsHttpOperation httpOp = Mockito.spy(op.getResult()); + Mockito.doReturn(COPY_STATUS_PENDING).when(httpOp).getResponseHeader( + HttpHeaderConfigurations.X_MS_COPY_STATUS); + Mockito.doReturn(httpOp).when(op).getResult(); + return op; + }).when(spiedClient).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + Mockito.doAnswer(answer -> { + AbfsRestOperation op = Mockito.spy((AbfsRestOperation) answer.callRealMethod()); + AbfsHttpOperation httpOp = Mockito.spy(op.getResult()); + Mockito.doReturn(COPY_STATUS_ABORTED).when(httpOp).getResponseHeader( + HttpHeaderConfigurations.X_MS_COPY_STATUS); + Mockito.doReturn(httpOp).when(op).getResult(); + return op; + }).when(spiedClient).getBlobProperty(Mockito.any(Path.class), Mockito.any(TracingContext.class)); + + fileSystem.create(new Path("/test1/file")); + Boolean copyBlobFailureCaught = false; + try { + fileSystem.rename(new Path("/test1/file"), new Path("/test1/file2")); + } catch (AbfsRestOperationException e) { + if (COPY_BLOB_ABORTED.equals(e.getErrorCode())) { + copyBlobFailureCaught = true; + } + } + Assert.assertTrue(copyBlobFailureCaught); + Assert.assertTrue(fileSystem.exists(new Path("/test1/file"))); + Mockito.verify(store, Mockito.times(1)) + .handleCopyInProgress(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.any(String.class)); + } + + @Test + public void testCopyBlobTakeTimeAndBlobIsDeleted() throws Exception { + AzureBlobFileSystem fileSystem = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fileSystem); + AzureBlobFileSystemStore store = Mockito.spy(fileSystem.getAbfsStore()); + String srcFile = "/test1/file"; + String dstFile = "/test1/file2"; + Mockito.doReturn(store).when(fileSystem).getAbfsStore(); + AbfsClient client = store.getClient(); + AbfsClient spiedClient = Mockito.spy(client); + store.setClient(spiedClient); + + Mockito.doAnswer(answer -> { + AbfsRestOperation op = Mockito.spy((AbfsRestOperation) answer.callRealMethod()); + fileSystem.delete(new Path(dstFile), false); + AbfsHttpOperation httpOp = Mockito.spy(op.getResult()); + Mockito.doReturn(COPY_STATUS_PENDING).when(httpOp).getResponseHeader( + HttpHeaderConfigurations.X_MS_COPY_STATUS); + Mockito.doReturn(httpOp).when(op).getResult(); + return op; + }).when(spiedClient).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + + fileSystem.create(new Path(srcFile)); + + + Assert.assertFalse(fileSystem.rename(new Path(srcFile), new Path(dstFile))); + Assert.assertFalse(fileSystem.exists(new Path(dstFile))); + } + + @Test + public void testParallelCopy() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + fs.create(new Path("/src")); + boolean[] dstBlobAlreadyThereExceptionReceived = new boolean[1]; + dstBlobAlreadyThereExceptionReceived[0] = false; + AtomicInteger threadsCompleted = new AtomicInteger(0); + new Thread(() -> { + parallelCopyRunnable(fs, dstBlobAlreadyThereExceptionReceived, + threadsCompleted); + }).start(); + new Thread(() -> { + parallelCopyRunnable(fs, dstBlobAlreadyThereExceptionReceived, + threadsCompleted); + }).start(); + while (threadsCompleted.get() < 2) ; + Assert.assertTrue(dstBlobAlreadyThereExceptionReceived[0]); + } + + private void parallelCopyRunnable(final AzureBlobFileSystem fs, + final boolean[] dstBlobAlreadyThereExceptionReceived, + final AtomicInteger threadsCompleted) { + try { + fs.getAbfsClient().copyBlob(new Path("/src"), + new Path("/dst"), null, getTestTracingContext(fs, true)); + } catch (AbfsRestOperationException ex) { + if (ex.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { + dstBlobAlreadyThereExceptionReceived[0] = true; + } + } catch ( + AzureBlobFileSystemException e) { + } + threadsCompleted.incrementAndGet(); + } + + @Test + public void testCopyAfterSourceHasBeenDeleted() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + fs.create(new Path("/src")); + TracingContext tracingContext = new TracingContext("clientCorrelationId", + "fileSystemId", FSOperationType.TEST_OP, + getConfiguration().getTracingHeaderFormat(), + null); + fs.getAbfsStore() + .getClient() + .deleteBlobPath(new Path("/src"), null, getTestTracingContext(fs, true)); + Boolean srcBlobNotFoundExReceived = false; + + tracingContext = new TracingContext(tracingContext); + try { + fs.getAbfsStore() + .copyBlob(new Path("/src"), new Path("/dst"), + null, getTestTracingContext(fs, true)); + } catch (AbfsRestOperationException ex) { + if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { + srcBlobNotFoundExReceived = true; + } + } + Assert.assertTrue(srcBlobNotFoundExReceived); + } + + @Test + public void testParallelRenameForAtomicDirShouldFail() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + fs.setWorkingDirectory(new Path("/")); + fs.mkdirs(new Path("/hbase/dir1")); + fs.create(new Path("/hbase/dir1/file1")); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + store.setClient(client); + AtomicBoolean leaseAcquired = new AtomicBoolean(false); + AtomicBoolean exceptionOnParallelRename = new AtomicBoolean(false); + AtomicBoolean parallelThreadDone = new AtomicBoolean(false); + Mockito.doAnswer(answer -> { + AbfsRestOperation op = (AbfsRestOperation) answer.callRealMethod(); + leaseAcquired.set(true); + while (!parallelThreadDone.get()) ; + return op; + }) + .when(client) + .acquireBlobLease(Mockito.anyString(), Mockito.anyInt(), + Mockito.any(TracingContext.class)); + + new Thread(() -> { + while (!leaseAcquired.get()) ; + try { + fs.rename(new Path("/hbase/dir1/file1"), new Path("/hbase/dir2/")); + } catch (Exception e) { + if (e.getCause() instanceof AbfsRestOperationException + && ((AbfsRestOperationException) e.getCause()).getStatusCode() + == HttpURLConnection.HTTP_CONFLICT) { + exceptionOnParallelRename.set(true); + } + } finally { + parallelThreadDone.set(true); + } + }).start(); + fs.rename(new Path("/hbase/dir1/file1"), new Path("/hbase/dir2/")); + while (!parallelThreadDone.get()) ; + Assert.assertTrue(exceptionOnParallelRename.get()); + } + + @Test + public void testParallelAppendToFileBeingCopiedInAtomicDirectory() + throws Exception { + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + fs.setWorkingDirectory(new Path("/")); + fs.mkdirs(new Path("/hbase/dir1")); + fs.create(new Path("/hbase/dir1/file1")); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + store.setClient(client); + AtomicBoolean copyOfSrcFile = new AtomicBoolean(false); + AtomicBoolean parallelAppendDone = new AtomicBoolean(false); + AtomicBoolean exceptionCaught = new AtomicBoolean(false); + + Mockito.doAnswer(answer -> { + answer.callRealMethod(); + if ("/hbase/dir1/file1".equalsIgnoreCase( + ((Path) answer.getArgument(0)).toUri().getPath())) { + copyOfSrcFile.set(true); + while (!parallelAppendDone.get()) ; + } + return null; + }) + .when(store) + .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + + FSDataOutputStream outputStream = fs.append(new Path("/hbase/dir1/file1")); + + new Thread(() -> { + while (!copyOfSrcFile.get()) ; + try { + byte[] bytes = new byte[4 * ONE_MB]; + new Random().nextBytes(bytes); + outputStream.write(bytes); + outputStream.hsync(); + } catch (Exception e) { + if (e.getCause() instanceof AbfsRestOperationException + && ((AbfsRestOperationException) e.getCause()).getStatusCode() + == HttpURLConnection.HTTP_PRECON_FAILED) { + exceptionCaught.set(true); + } + } finally { + parallelAppendDone.set(true); + } + }).start(); + + fs.rename(new Path("/hbase/dir1"), new Path("/hbase/dir2")); + + while (!parallelAppendDone.get()) ; + Assert.assertTrue(exceptionCaught.get()); + } + + @Test + public void testParallelBlobLeaseOnChildBlobInRenameSrcDir() + throws Exception { + assumeNonHnsAccountBlobEndpoint(getFileSystem()); + AzureBlobFileSystem fs = Mockito.spy( + (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration())); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Path srcDirPath = new Path("/hbase/testDir"); + fs.mkdirs(srcDirPath); + fs.create(new Path(srcDirPath, "file1")); + fs.create(new Path(srcDirPath, "file2")); + Mockito.doReturn(store).when(fs).getAbfsStore(); + AbfsClient client = store.getClient(); + AbfsClient spiedClient = Mockito.spy(client); + store.setClient(spiedClient); + + fs.getAbfsClient() + .acquireBlobLease("/hbase/testDir/file2", -1, + getTestTracingContext(fs, true)); + + AbfsLease[] leases = new AbfsLease[1]; + Mockito.doAnswer(answer -> { + String path = answer.getArgument(0); + AbfsLease lease = (AbfsLease) answer.callRealMethod(); + if (srcDirPath.toUri().getPath().equalsIgnoreCase(path)) { + lease = Mockito.spy(lease); + leases[0] = lease; + } + return lease; + }) + .when(store) + .getBlobLease(Mockito.anyString(), Mockito.nullable(Integer.class), + Mockito.any(TracingContext.class)); + + Boolean renameFailed = false; + try { + fs.rename(srcDirPath, new Path("/hbase/newDir")); + } catch (Exception e) { + renameFailed = true; + } + + Assertions.assertThat(renameFailed).isTrue(); + Mockito.verify(leases[0], Mockito.times(1)).free(); + } + + @Test + public void testParallelCreateNonRecursiveToFilePartOfAtomicDirectoryInRename() + throws Exception { + Configuration configuration = Mockito.spy(getRawConfiguration()); + configuration.set(FS_AZURE_LEASE_CREATE_NON_RECURSIVE, "true"); + FileSystem fsCreate = FileSystem.newInstance(configuration); + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + assumeNonHnsAccountBlobEndpoint(fs); + fs.setWorkingDirectory(new Path("/")); + fs.mkdirs(new Path("/hbase/dir1")); + fs.create(new Path("/hbase/dir1/file1")); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + fs.getAbfsStore().setClient(client); + AtomicBoolean leaseAcquired = new AtomicBoolean(false); + AtomicBoolean parallelCreateDone = new AtomicBoolean(false); + AtomicBoolean exceptionCaught = new AtomicBoolean(false); + AtomicBoolean parallelRenameDone = new AtomicBoolean(false); + + Mockito.doAnswer(answer -> { + AbfsRestOperation op = (AbfsRestOperation) answer.callRealMethod(); + leaseAcquired.set(true); + while(!parallelCreateDone.get()); + return op; + }) + .when(client) + .acquireBlobLease(Mockito.anyString(), Mockito.anyInt(), + Mockito.any(TracingContext.class)); + + new Thread(() -> { + try { + fs.rename(new Path("/hbase/dir1"), new Path("/hbase/dir2")); + } catch (Exception e) {} finally { + parallelRenameDone.set(true); + } + }).start(); + + Path createNewFilePath = new Path("/hbase/dir1/file2"); + while (!leaseAcquired.get()) ; + try { + fsCreate.createFile(createNewFilePath) + .overwrite(false) + .replication((short) 1) + .bufferSize(1024) + .blockSize(1024) + .build(); + } catch (AbfsRestOperationException e) { + if (e.getStatusCode() + == HttpURLConnection.HTTP_CONFLICT) { + exceptionCaught.set(true); + } + } finally { + parallelCreateDone.set(true); + } + + + while (!parallelRenameDone.get()) ; + Assert.assertTrue(exceptionCaught.get()); + } + + @Test + public void testBlobRenameOfDirectoryHavingNeighborWithSamePrefix() + throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + fs.mkdirs(new Path("/testDir/dir")); + fs.mkdirs(new Path("/testDir/dirSamePrefix")); + fs.create(new Path("/testDir/dir/file1")); + fs.create(new Path("/testDir/dir/file2")); + + fs.create(new Path("/testDir/dirSamePrefix/file1")); + fs.create(new Path("/testDir/dirSamePrefix/file2")); + + fs.rename(new Path("/testDir/dir"), new Path("/testDir/dir2")); + + Assertions.assertThat(fs.exists(new Path("/testDir/dirSamePrefix/file1"))) + .isTrue(); + Assertions.assertThat(fs.exists(new Path("/testDir/dir/file1"))) + .isFalse(); + Assertions.assertThat(fs.exists(new Path("/testDir/dir/file2"))) + .isFalse(); + Assertions.assertThat(fs.exists(new Path("/testDir/dir/"))) + .isFalse(); + } + + @Test + public void testBlobRenameCancelRenewTimerForLeaseTakenInAtomicRename() + throws Exception { + AzureBlobFileSystem fs = Mockito.spy( + (AzureBlobFileSystem) FileSystem.newInstance(getRawConfiguration())); + assumeNonHnsAccountBlobEndpoint(fs); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + + fs.mkdirs(new Path("/hbase/dir")); + fs.create(new Path("/hbase/dir/file1")); + fs.create(new Path("/hbase/dir/file2")); + + final List leases = new ArrayList<>(); + Mockito.doAnswer(answer -> { + AbfsBlobLease lease = Mockito.spy( + (AbfsBlobLease) answer.callRealMethod()); + leases.add(lease); + return lease; + }) + .when(store) + .getBlobLease(Mockito.anyString(), Mockito.nullable(Integer.class), + Mockito.any(TracingContext.class)); + + fs.rename(new Path("/hbase/dir"), new Path("/hbase/dir2")); + + Assertions.assertThat(leases).hasSize(3); + for (AbfsBlobLease lease : leases) { + Mockito.verify(lease, Mockito.times(1)).cancelTimer(); + } + } + + @Test + public void testBlobRenameServerReturnsOneBlobPerList() throws Exception { + assumeNonHnsAccountBlobEndpoint(getFileSystem()); + AzureBlobFileSystem fs = (AzureBlobFileSystem) Mockito.spy(FileSystem.newInstance(getRawConfiguration())); + fs.mkdirs(new Path("/testDir/")); + fs.create(new Path("/testDir/file1")); + fs.create(new Path("/testDir/file2")); + + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = fs.getAbfsClient(); + AbfsClient spiedClient = Mockito.spy(client); + store.setClient(spiedClient); + Mockito.doReturn(store).when(fs).getAbfsStore(); + Mockito.doAnswer(answer -> { + String marker = answer.getArgument(0); + String prefix = answer.getArgument(1); + String delimeter = answer.getArgument(2); + Integer count = answer.getArgument(3); + TracingContext tracingContext = answer.getArgument(4); + AbfsRestOperation op = client.getListBlobs(marker, prefix, delimeter, 1, tracingContext); + return op; + }).when(spiedClient).getListBlobs(Mockito.nullable(String.class), + Mockito.nullable(String.class), Mockito.nullable(String.class), + Mockito.nullable(Integer.class), Mockito.any(TracingContext.class)); + + fs.rename(new Path("/testDir"), new Path("/testDir1")); + Assertions.assertThat(fs.exists(new Path("/testDir"))).isFalse(); + Assertions.assertThat(fs.exists(new Path("/testDir1"))).isTrue(); + Assertions.assertThat(fs.exists(new Path("/testDir1/file1"))).isTrue(); + Assertions.assertThat(fs.exists(new Path("/testDir1/file2"))).isTrue(); + } + + @Test + public void testBlobAtomicRenameSrcAndDstAreNotLeftLeased() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + Assume.assumeFalse(getConfiguration().shouldIngressFallbackToDfs()); + fs.setWorkingDirectory(new Path("/")); + fs.create(new Path("/hbase/dir1/blob1")); + fs.create(new Path("/hbase/dir1/blob2")); + fs.rename(new Path("/hbase/dir1/"), new Path("/hbase/dir2")); + fs.create(new Path("/hbase/dir1/blob1")); + byte[] bytes = new byte[4 * ONE_MB]; + new Random().nextBytes(bytes); + try (FSDataOutputStream os = fs.append(new Path("hbase/dir2/blob1"))) { + os.write(bytes); + } + } + + /** + * Test to assert that the CID in src marker blob copy and delete contains the + * total number of blobs operated in the rename directory. + * Also, to assert that all operations in the rename-directory flow have same + * primaryId and opType. + */ + @Test + public void testRenameSrcDirDeleteEmitDeletionCountInClientRequestId() + throws Exception { + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + Assume.assumeTrue(getPrefixMode(fs) == PrefixMode.BLOB); + String dirPathStr = "/testDir/dir1"; + fs.mkdirs(new Path(dirPathStr)); + ExecutorService executorService = Executors.newFixedThreadPool(5); + List futures = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + final int iter = i; + Future future = executorService.submit(() -> { + return fs.create(new Path("/testDir/dir1/file" + iter)); + }); + futures.add(future); + } + + for (Future future : futures) { + future.get(); + } + executorService.shutdown(); + + + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + AbfsClient client = Mockito.spy(store.getClient()); + store.setClient(client); + + final TracingHeaderValidator tracingHeaderValidator + = new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.RENAME, true, 0); + fs.registerListener(tracingHeaderValidator); + + Mockito.doAnswer(answer -> { + Mockito.doAnswer(copyAnswer -> { + if (dirPathStr.equalsIgnoreCase( + ((Path) copyAnswer.getArgument(0)).toUri().getPath())) { + tracingHeaderValidator.setOperatedBlobCount(11); + return copyAnswer.callRealMethod(); + } + return copyAnswer.callRealMethod(); + }) + .when(client) + .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + + Mockito.doAnswer(deleteAnswer -> { + if (dirPathStr.equalsIgnoreCase( + ((Path) deleteAnswer.getArgument(0)).toUri().getPath())) { + Object result = deleteAnswer.callRealMethod(); + tracingHeaderValidator.setOperatedBlobCount(null); + return result; + } + return deleteAnswer.callRealMethod(); + }) + .when(client) + .deleteBlobPath(Mockito.any(Path.class), + Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + + return answer.callRealMethod(); + }) + .when(store) + .rename(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.any( + RenameAtomicityUtils.class), Mockito.any(TracingContext.class)); + + fs.rename(new Path(dirPathStr), new Path("/dst/")); + } + + /** + * Test to assert that the rename resume from FileStatus uses the same + * {@link TracingContext} object used by the initial ListStatus call. + * Also assert that last rename's copy and delete API call would push count + * of blobs operated in resume flow in clientRequestId. + */ + @Test + public void testBlobRenameResumeWithListStatus() throws Exception { + assumeNonHnsAccountBlobEndpoint(getFileSystem()); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + store.setClient(client); + + renameFailureSetup(fs, client, false); + AtomicInteger copied = assertTracingContextOnRenameResumeProcess(fs, store, + client, FSOperationType.LISTSTATUS); + + fs.listStatus(new Path("/hbase")); + fs.registerListener(null); + Assertions.assertThat(fs.exists(new Path("/hbase/testDir2"))).isTrue(); + Assertions.assertThat(copied.get()).isGreaterThan(0); + } + + /** + * Test to assert that the rename resume from FileStatus uses the same + * {@link TracingContext} object used by the initial GetFileStatus call. + * Also assert that last rename's copy and delete API call would push count + * of blobs operated in resume flow in clientRequestId. + */ + @Test + public void testBlobRenameResumeWithGetFileStatus() throws Exception { + assumeNonHnsAccountBlobEndpoint(getFileSystem()); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + store.setClient(client); + + renameFailureSetup(fs, client, false); + AtomicInteger copied = assertTracingContextOnRenameResumeProcess(fs, store, + client, FSOperationType.GET_FILESTATUS); + + intercept(FileNotFoundException.class, () -> { + fs.getFileStatus(new Path("/hbase/testDir")); + }); + Assertions.assertThat(fs.exists(new Path("/hbase/testDir2"))).isTrue(); + Assertions.assertThat(copied.get()).isGreaterThan(0); + } + + private void renameFailureSetup(final AzureBlobFileSystem fs, + final AbfsClient client, final Boolean srcMarkerToBeDeleted) + throws Exception { + fs.mkdirs(new Path("/hbase/testDir")); + ExecutorService executorService = Executors.newFixedThreadPool(5); + List futures = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + final int iter = i; + futures.add(executorService.submit( + () -> fs.create(new Path("/hbase/testDir/file" + iter)))); + } + + for (Future future : futures) { + future.get(); + } + + if (srcMarkerToBeDeleted) { + client.deleteBlobPath(new Path("/hbase/testDir"), null, + getTestTracingContext(fs, true)); + } + + AbfsRestOperation op = client.acquireBlobLease("/hbase/testDir/file5", -1, + getTestTracingContext(fs, true)); + String leaseId = op.getResult() + .getResponseHeader(HttpHeaderConfigurations.X_MS_LEASE_ID); + + Map pathLeaseIdMap = new ConcurrentHashMap<>(); + AtomicBoolean leaseCanBeTaken = new AtomicBoolean(true); + Mockito.doAnswer(answer -> { + if (!leaseCanBeTaken.get()) { + throw new RuntimeException(); + } + AbfsRestOperation abfsRestOperation + = (AbfsRestOperation) answer.callRealMethod(); + pathLeaseIdMap.put(answer.getArgument(0), + abfsRestOperation.getResult().getResponseHeader(X_MS_LEASE_ID)); + return abfsRestOperation; + }) + .when(client) + .acquireBlobLease(Mockito.anyString(), Mockito.anyInt(), + Mockito.any(TracingContext.class)); + + intercept(Exception.class, () -> { + fs.rename(new Path("/hbase/testDir"), new Path("/hbase/testDir2")); + }); + + leaseCanBeTaken.set(false); + for (Map.Entry entry : pathLeaseIdMap.entrySet()) { + try { + client.releaseBlobLease(entry.getKey(), entry.getValue(), + getTestTracingContext(fs, true)); + } catch (Exception e) { + + } + } + client.releaseBlobLease("/hbase/testDir/file5", leaseId, + getTestTracingContext(fs, true)); + leaseCanBeTaken.set(true); + } + + private AtomicInteger assertTracingContextOnRenameResumeProcess(final AzureBlobFileSystem fs, + final AzureBlobFileSystemStore store, + final AbfsClient client, final FSOperationType fsOperationType) + throws IOException { + final TracingHeaderValidator tracingHeaderValidator + = new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), fsOperationType, true, 0); + fs.registerListener(tracingHeaderValidator); + + AtomicInteger copied = new AtomicInteger(0); + Mockito.doAnswer(answer -> { + copied.incrementAndGet(); + Path path = answer.getArgument(0); + if ("/hbase/testDir".equalsIgnoreCase(path.toUri().getPath())) { + tracingHeaderValidator.setOperatedBlobCount(copied.get()); + } + return answer.callRealMethod(); + }) + .when(store) + .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + + /* + * RenameAtomicUtil internally calls Filesystem's API of read and delete + * which would have different primaryIds. But once renameAtomicUtil has read + * the RenamePending JSON, all the operations will use the same tracingContext + * which was started by ListStatus or GetFileStatus operation. + * This is the reason why the validation is disabled until the RenameAtomicUtil + * object reads the JSON. + * The filesystem's delete API called in RenameAtomicUtils.cleanup create a + * new TracingContext object with a new primaryRequestId and also updates the + * new primaryRequestId in the listener object of FileSystem. Therefore, once, + * cleanup method is completed, the listener is explicitly updated with the + * primaryRequestId it was using before the RenameAtomicUtils object was created. + */ + Mockito.doAnswer(answer -> { + final String primaryRequestId = ((TracingContext) answer.getArgument( + 1)).getPrimaryRequestId(); + tracingHeaderValidator.setDisableValidation(true); + RenameAtomicityUtils renameAtomicityUtils = Mockito.spy( + (RenameAtomicityUtils) answer.callRealMethod()); + Mockito.doAnswer(cleanupAnswer -> { + tracingHeaderValidator.setDisableValidation(true); + cleanupAnswer.callRealMethod(); + tracingHeaderValidator.setDisableValidation(false); + tracingHeaderValidator.updatePrimaryRequestID(primaryRequestId); + return null; + }).when(renameAtomicityUtils).cleanup(Mockito.any(Path.class)); + tracingHeaderValidator.setDisableValidation(false); + return renameAtomicityUtils; + }) + .when(fs) + .getRenameAtomicityUtilsForRedo(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.anyString(), Mockito.any( + AbfsInputStream.class)); + + Mockito.doAnswer(answer -> { + answer.callRealMethod(); + tracingHeaderValidator.setOperatedBlobCount(null); + return null; + }) + .when(client) + .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + return copied; + } + + @Test + public void testProducerStopOnRenameFailure() throws Exception { + assumeNonHnsAccountBlobEndpoint(getFileSystem()); + Configuration configuration = Mockito.spy(getRawConfiguration()); + AzureBlobFileSystem fs = Mockito.spy( + (AzureBlobFileSystem) FileSystem.get(configuration)); + + fs.mkdirs(new Path("/src")); + ExecutorService executorService = Executors.newFixedThreadPool(10); + List futureList = new ArrayList<>(); + for (int i = 0; i < 20; i++) { + int iter = i; + Future future = executorService.submit(() -> { + try { + fs.create(new Path("/src/file" + iter)); + } catch (IOException ex) {} + }); + futureList.add(future); + } + + for (Future future : futureList) { + future.get(); + } + + AbfsClient client = fs.getAbfsClient(); + AbfsClient spiedClient = Mockito.spy(client); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + store.setClient(spiedClient); + Mockito.doReturn(store).when(fs).getAbfsStore(); + + ListBlobProducer[] producers = new ListBlobProducer[1]; + Mockito.doAnswer(answer -> { + producers[0] = (ListBlobProducer) answer.callRealMethod(); + return producers[0]; + }).when(store).getListBlobProducer(Mockito.anyString(), Mockito.any( + ListBlobQueue.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + + AtomicInteger listCounter = new AtomicInteger(0); + AtomicBoolean hasConsumerStarted = new AtomicBoolean(false); + + Mockito.doAnswer(answer -> { + String marker = answer.getArgument(0); + String prefix = answer.getArgument(1); + String delimiter = answer.getArgument(2); + TracingContext tracingContext = answer.getArgument(4); + int counter = listCounter.incrementAndGet(); + if (counter > 1 && producers[0] != null) { + while (!hasConsumerStarted.get()) { + Thread.sleep(1_000L); + } + } + Object result = client.getListBlobs(marker, prefix, delimiter, 1, + tracingContext); + return result; + }) + .when(spiedClient) + .getListBlobs(Mockito.nullable(String.class), + Mockito.nullable(String.class), Mockito.nullable(String.class), + Mockito.nullable(Integer.class), Mockito.any(TracingContext.class)); + + Mockito.doAnswer(answer -> { + spiedClient.acquireBlobLease( + ((Path) answer.getArgument(0)).toUri().getPath(), -1, + answer.getArgument(2)); + hasConsumerStarted.set(true); + return answer.callRealMethod(); + }) + .when(spiedClient) + .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + + intercept(Exception.class, () -> { + fs.rename(new Path("/src"), new Path("/dst")); + }); + + producers[0].waitForProcessCompletion(); + + Mockito.verify(spiedClient, Mockito.atMost(3)) + .getListBlobs(Mockito.nullable(String.class), + Mockito.nullable(String.class), Mockito.nullable(String.class), + Mockito.nullable(Integer.class), Mockito.any(TracingContext.class)); + } + + @Test + public void testRenameResumeThroughListStatusWithSrcDirDeleted() throws Exception { + assumeNonHnsAccountBlobEndpoint(getFileSystem()); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + store.setClient(client); + + renameFailureSetup(fs, client, false); + + fs.delete(new Path("/hbase/testDir"), true); + AtomicInteger copied = assertTracingContextOnRenameResumeProcess(fs, store, + client, FSOperationType.LISTSTATUS); + Assertions.assertThat(fs.listStatus(new Path("/hbase"))).hasSize(1); + Assertions.assertThat(copied.get()).isEqualTo(0); + } + + @Test + public void testRenameResumeThroughListStatusWithSrcDirDeletedJustBeforeResume() throws Exception { + assumeNonHnsAccountBlobEndpoint(getFileSystem()); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + store.setClient(client); + + renameFailureSetup(fs, client, false); + + AtomicInteger copied = assertTracingContextOnRenameResumeProcess(fs, store, + client, FSOperationType.LISTSTATUS); + Mockito.doAnswer(answer -> { + String path = answer.getArgument(0); + if("/hbase/testDir".equalsIgnoreCase(path)) { + fs.delete(new Path(path), true); + } + return answer.callRealMethod(); + }).when(client).acquireBlobLease(Mockito.anyString(), Mockito.anyInt(), Mockito.any(TracingContext.class)); + Assertions.assertThat(fs.listStatus(new Path("/hbase"))).hasSize(1); + Assertions.assertThat(copied.get()).isEqualTo(0); + } + + @Test + public void testRenameResumeThroughListStatusWhenSrcDirectoryETagIsChanged() + throws Exception { + assumeNonHnsAccountBlobEndpoint(getFileSystem()); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + store.setClient(client); + + renameFailureSetup(fs, client, false); + + fs.delete(new Path("/hbase/testDir"), true); + fs.mkdirs(new Path("/hbase/testDir")); + AtomicInteger copied = assertTracingContextOnRenameResumeProcess(fs, store, + client, FSOperationType.LISTSTATUS); + Assertions.assertThat(fs.listStatus(new Path("/hbase"))).hasSize(2); + Assertions.assertThat(copied.get()).isEqualTo(0); + } + + @Test + public void testRenameResumeThroughGetStatusWhenSrcDirectoryETagIsChanged() + throws Exception { + assumeNonHnsAccountBlobEndpoint(getFileSystem()); + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + store.setClient(client); + + renameFailureSetup(fs, client, false); + + fs.delete(new Path("/hbase/testDir"), true); + fs.mkdirs(new Path("/hbase/testDir")); + AtomicInteger copied = assertTracingContextOnRenameResumeProcess(fs, store, + client, FSOperationType.GET_FILESTATUS); + fs.getFileStatus(new Path("/hbase/testDir")); + Assertions.assertThat(copied.get()).isEqualTo(0); + } + + @Test + public void testRenameResumeThroughGetStatusWhenSrcDirMakerOnRenameCreatedOnDfsEndpoint() + throws Exception { + assumeNonHnsAccountBlobEndpoint(getFileSystem()); + Configuration configuration = Mockito.spy(getRawConfiguration()); + configuration.set(FS_AZURE_MKDIRS_FALLBACK_TO_DFS, "true"); + commonTestRenameResumeThroughGetStatusWhenSrcDirMarkerCreatedOnRename( + configuration); + } + + @Test + public void testRenameResumeThroughGetStatusWhenSrcDirMakerOnRenameCreatedOnBlobEndpoint() + throws Exception { + assumeNonHnsAccountBlobEndpoint(getFileSystem()); + Configuration configuration = Mockito.spy(getRawConfiguration()); + configuration.set(FS_AZURE_MKDIRS_FALLBACK_TO_DFS, "false"); + commonTestRenameResumeThroughGetStatusWhenSrcDirMarkerCreatedOnRename( + configuration); + } + + private void commonTestRenameResumeThroughGetStatusWhenSrcDirMarkerCreatedOnRename( + final Configuration configuration) throws Exception { + AzureBlobFileSystem fs = Mockito.spy( + (AzureBlobFileSystem) FileSystem.newInstance( + configuration)); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + store.setClient(client); + + renameFailureSetup(fs, client, true); + intercept(FileNotFoundException.class, () -> { + fs.getFileStatus(new Path("/hbase/testDir")); + }); + FileStatus[] fileStatuses = fs.listStatus(new Path("/hbase/testDir2")); + Assertions.assertThat(fileStatuses).hasSize(10); + } + + @Test + public void testRenameResumeThroughListStatusWhenSrcDirMakerOnRenameCreatedOnDfsEndpoint() + throws Exception { + assumeNonHnsAccountBlobEndpoint(getFileSystem()); + Configuration configuration = Mockito.spy(getRawConfiguration()); + configuration.set(FS_AZURE_MKDIRS_FALLBACK_TO_DFS, "true"); + commonTestRenameResumeThroughListStatusWhenSrcDirMarkerCreatedOnRename( + configuration); + } + + @Test + public void testRenameResumeThroughListStatusWhenSrcDirMakerOnRenameCreatedOnBlobEndpoint() + throws Exception { + assumeNonHnsAccountBlobEndpoint(getFileSystem()); + Configuration configuration = Mockito.spy(getRawConfiguration()); + configuration.set(FS_AZURE_MKDIRS_FALLBACK_TO_DFS, "false"); + commonTestRenameResumeThroughListStatusWhenSrcDirMarkerCreatedOnRename( + configuration); + } + + private void commonTestRenameResumeThroughListStatusWhenSrcDirMarkerCreatedOnRename( + final Configuration configuration) throws Exception { + AzureBlobFileSystem fs = Mockito.spy( + (AzureBlobFileSystem) FileSystem.newInstance( + configuration)); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + store.setClient(client); + + renameFailureSetup(fs, client, true); + Assertions.assertThat(fs.listStatus(new Path("/hbase/"))).hasSize(1); + FileStatus[] fileStatuses = fs.listStatus(new Path("/hbase/testDir2")); + Assertions.assertThat(fileStatuses).hasSize(10); + } + + @Test + public void renameBlobDirParallelThreadToRenameOnDifferentTracingContext() + throws Exception { + Assume.assumeTrue(getPrefixMode(getFileSystem()) == PrefixMode.BLOB); + Configuration configuration = getRawConfiguration(); + AzureBlobFileSystem fs = Mockito.spy( + (AzureBlobFileSystem) FileSystem.newInstance(configuration)); + AzureBlobFileSystemStore spiedStore = Mockito.spy(fs.getAbfsStore()); + AbfsClient spiedClient = Mockito.spy(fs.getAbfsClient()); + + Mockito.doReturn(spiedStore).when(fs).getAbfsStore(); + spiedStore.setClient(spiedClient); + + fs.mkdirs(new Path("/testDir")); + fs.create(new Path("/testDir/file1")); + fs.create(new Path("/testDir/file2")); + + AbfsClientTestUtil.hookOnRestOpsForTracingContextSingularity(spiedClient); + fs.rename(new Path("/testDir"), new Path("/testDir2")); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java index 044c325c8c8dc2..a4b668b5e48353 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java @@ -20,12 +20,14 @@ import java.util.Arrays; +import org.junit.Assume; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile; @@ -76,6 +78,8 @@ public ITestAzureBlobFileSystemRenameUnicode() throws Exception { @Test public void testRenameFileUsingUnicode() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); + Assume.assumeTrue(fs.getAbfsStore().getAbfsConfiguration().getPrefixMode() + == PrefixMode.DFS || !destDir.contains(":")); Path folderPath1 = new Path(srcDir); assertMkdirs(fs, folderPath1); assertIsDirectory(fs, folderPath1); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java index 2c0bd31bf8eeb8..8ef5f1d4516703 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java @@ -202,7 +202,7 @@ public void testAppendWithCPK() throws Exception { // Trying to append with correct CPK headers AppendRequestParameters appendRequestParameters = new AppendRequestParameters( - 0, 0, 5, Mode.APPEND_MODE, false, null); + 0, 0, 5, Mode.APPEND_MODE, false, null, true); byte[] buffer = getRandomBytesArray(5); AbfsClient abfsClient = fs.getAbfsClient(); AbfsRestOperation abfsRestOperation = abfsClient @@ -247,7 +247,7 @@ public void testAppendWithoutCPK() throws Exception { // Trying to append without CPK headers AppendRequestParameters appendRequestParameters = new AppendRequestParameters( - 0, 0, 5, Mode.APPEND_MODE, false, null); + 0, 0, 5, Mode.APPEND_MODE, false, null, true); byte[] buffer = getRandomBytesArray(5); AbfsClient abfsClient = fs.getAbfsClient(); AbfsRestOperation abfsRestOperation = abfsClient diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java index 8b60dd801cb30e..58a2d99a336e34 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java @@ -20,13 +20,25 @@ import java.net.URI; +import org.assertj.core.api.Assertions; import org.junit.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.services.AuthType; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; + +import static org.apache.hadoop.fs.CommonPathCapabilities.ETAGS_AVAILABLE; +import static org.apache.hadoop.fs.CommonPathCapabilities.ETAGS_PRESERVED_IN_RENAME; +import static org.apache.hadoop.fs.CommonPathCapabilities.FS_ACLS; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DNS_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.InternalConstants.CAPABILITY_SAFE_READAHEAD; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test AzureBlobFileSystem initialization. @@ -39,11 +51,12 @@ public ITestFileSystemInitialization() throws Exception { @Test public void ensureAzureBlobFileSystemIsInitialized() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); - final String accountName = getAccountName(); + String accountName = getAccountName(); final String filesystem = getFileSystemName(); String scheme = this.getAuthType() == AuthType.SharedKey ? FileSystemUriSchemes.ABFS_SCHEME : FileSystemUriSchemes.ABFS_SECURE_SCHEME; + assertEquals(fs.getUri(), new URI(scheme, filesystem + "@" + accountName, @@ -55,8 +68,13 @@ public void ensureAzureBlobFileSystemIsInitialized() throws Exception { @Test public void ensureSecureAzureBlobFileSystemIsInitialized() throws Exception { - final String accountName = getAccountName(); + String accountName = getAccountName(); final String filesystem = getFileSystemName(); + if (getFileSystem().getAbfsStore().getAbfsConfiguration().shouldEnableBlobEndPoint()) { + if (accountName.contains(ABFS_DNS_PREFIX)) { + accountName = accountName.replace(ABFS_DNS_PREFIX, WASB_DNS_PREFIX); + } + } final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, @@ -74,4 +92,41 @@ public void ensureSecureAzureBlobFileSystemIsInitialized() throws Exception { assertNotNull("working directory", fs.getWorkingDirectory()); } } + + @Test + public void testFileSystemCapabilities() throws Throwable { + final AzureBlobFileSystem fs = getFileSystem(); + + final Path p = new Path("}"); + // etags always present + Assertions.assertThat(fs.hasPathCapability(p, ETAGS_AVAILABLE)) + .describedAs("path capability %s in %s", ETAGS_AVAILABLE, fs) + .isTrue(); + // readahead always correct + Assertions.assertThat(fs.hasPathCapability(p, CAPABILITY_SAFE_READAHEAD)) + .describedAs("path capability %s in %s", CAPABILITY_SAFE_READAHEAD, fs) + .isTrue(); + + // etags-over-rename and ACLs are either both true or both false. + final boolean etagsAcrossRename = fs.hasPathCapability(p, ETAGS_PRESERVED_IN_RENAME); + final boolean acls = fs.hasPathCapability(p, FS_ACLS); + Assertions.assertThat(etagsAcrossRename) + .describedAs("capabilities %s=%s and %s=%s in %s", + ETAGS_PRESERVED_IN_RENAME, etagsAcrossRename, + FS_ACLS, acls, fs) + .isEqualTo(acls); + } + + @Test + public void testCreateContainerOnFileSystemPath() throws Exception{ + final AzureBlobFileSystem fs = getFileSystem(); + // assert that createContainer fails for already existing fileSystem. + intercept(AbfsRestOperationException.class, + () -> fs.getAbfsStore().createFilesystem(getTestTracingContext(fs, true), + fs.getAbfsStore().getPrefixMode() == PrefixMode.BLOB)); + + fs.getAbfsStore().deleteFilesystem(getTestTracingContext(fs, true)); + intercept(AbfsRestOperationException.class, + () -> fs.getAbfsStore().getFilesystemProperties(getTestTracingContext(fs, true))); + } } 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 01227691c31393..f42cd37879b11d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestGetNameSpaceEnabled.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.UUID; +import org.apache.hadoop.fs.azurebfs.services.OperativeEndpoint; import org.junit.Assume; import org.junit.Test; import org.assertj.core.api.Assertions; @@ -32,8 +33,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.enums.Trilean; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DNS_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doReturn; @@ -58,8 +62,16 @@ public class ITestGetNameSpaceEnabled extends AbstractAbfsIntegrationTest { private static final String FALSE_STR = "false"; private boolean isUsingXNSAccount; + private boolean useBlobEndpoint; public ITestGetNameSpaceEnabled() throws Exception { isUsingXNSAccount = getConfiguration().getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false); + super.setup(); + AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem()); + PrefixMode prefixMode = abfsStore.getPrefixMode(); + AbfsConfiguration abfsConfiguration = abfsStore.getAbfsConfiguration(); + useBlobEndpoint = !(OperativeEndpoint.isIngressEnabledOnDFS(prefixMode, abfsConfiguration) || + OperativeEndpoint.isMkdirEnabledOnDFS(abfsConfiguration) || + OperativeEndpoint.isReadEnabledOnDFS(abfsConfiguration)); } @Test @@ -118,12 +130,15 @@ private AzureBlobFileSystem getNewFSWithHnsConf( rawConfig .setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true); rawConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, - getNonExistingUrl()); + getNonExistingUrl(isNamespaceEnabledAccount)); return (AzureBlobFileSystem) FileSystem.get(rawConfig); } - private String getNonExistingUrl() { + private String getNonExistingUrl(String isNamespaceEnabled) { String testUri = this.getTestUrl(); + if (Boolean.parseBoolean(isNamespaceEnabled)) { + testUri = testUri.replace(WASB_DNS_PREFIX, ABFS_DNS_PREFIX); + } return getAbfsScheme() + "://" + UUID.randomUUID() + testUri .substring(testUri.indexOf("@")); } @@ -137,11 +152,20 @@ public void testFailedRequestWhenFSNotExist() throws Exception { + testUri.substring(testUri.indexOf("@")); AzureBlobFileSystem fs = this.getFileSystem(nonExistingFsUrl); - intercept(FileNotFoundException.class, - "\"The specified filesystem does not exist.\", 404", - ()-> { - fs.getFileStatus(new Path("/")); // Run a dummy FS call - }); + if (useBlobEndpoint) { + intercept(FileNotFoundException.class, + "\"The specified container does not exist.\", 404", + ()-> { + fs.getFileStatus(new Path("/")); // Run a dummy FS call + }); + } + else { + intercept(FileNotFoundException.class, + "\"The specified filesystem does not exist.\", 404", + ()-> { + fs.getFileStatus(new Path("/")); // Run a dummy FS call + }); + } } @Test diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestListBlob.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestListBlob.java new file mode 100644 index 00000000000000..72a652e0ce4f5c --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestListBlob.java @@ -0,0 +1,200 @@ +/** + * 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 java.io.IOException; +import java.util.List; + +import org.assertj.core.api.Assertions; +import org.junit.Assume; +import org.junit.Test; +import org.mockito.Mockito; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.BlobProperty; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +public class ITestListBlob extends + AbstractAbfsIntegrationTest { + + public ITestListBlob() throws Exception { + super(); + } + + @Test + public void testListBlob() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + createBlob(fs, "/dir/"); + List blobProperties; + /* + * Call getListBlob for a path with isDefinitiveDirSearch = false. Should give + * results including the directory blob(hdi_isfolder=true). + */ + blobProperties = fs.getAbfsStore() + .getListBlobs(new Path("dir"), null, null, + getTestTracingContext(fs, true), null, false); + Assertions.assertThat(blobProperties) + .describedAs( + "BlobList should match the number of files created in tests + the directory itself") + .hasSize(11); + + /* + * Call getListBlob for a path with isDefinitiveDirSearch = false. Should give + * results excluding the directory blob(hdi_isfolder=true). + */ + blobProperties = fs.getAbfsStore() + .getListBlobs(new Path("dir"), null, null, + getTestTracingContext(fs, true), null, true); + Assertions.assertThat(blobProperties) + .describedAs( + "BlobList should match the number of files created in tests") + .hasSize(10); + + /* + * Call getListBlob for a path with isDefinitiveDirSearch = false with + * maxResult more than the number of exact blobs. Should give results including + * the directory blob(hdi_isfolder=true). + */ + blobProperties = fs.getAbfsStore() + .getListBlobs(new Path("dir"), null, null, + getTestTracingContext(fs, true), 13, false); + Assertions.assertThat(blobProperties) + .describedAs( + "BlobList should match the number of files created in tests + the directory itself") + .hasSize(11); + + /* + * Call getListBlob for a path with isDefinitiveDirSearch = false with + * maxResult lesser than the number of exact blobs. Should give result size + * same as the maxResult + */ + blobProperties = fs.getAbfsStore() + .getListBlobs(new Path("dir"), null, null, + getTestTracingContext(fs, true), 5, false); + Assertions.assertThat(blobProperties) + .describedAs( + "BlobList should match the number of maxResult given") + .hasSize(5); + } + + @Test + public void testListBlobWithMarkers() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + createBlob(fs, "/dir/"); + AbfsClient client = fs.getAbfsClient(); + AbfsClient spiedClient = Mockito.spy(client); + fs.getAbfsStore().setClient(spiedClient); + + /* + * Server can give lesser number of results. In this case, server will give + * nextMarker. + * In this case, server will return one object, expectation is that the client + * uses nextMarker to make calls for the remaining blobs. + */ + int count[] = new int[1]; + count[0] = 0; + Mockito.doAnswer(answer -> { + String marker = answer.getArgument(0); + String prefix = answer.getArgument(1); + String delimiter = answer.getArgument(2); + TracingContext tracingContext = answer.getArgument(4); + count[0]++; + return client.getListBlobs(marker, prefix, delimiter, 1, tracingContext); + }).when(spiedClient).getListBlobs(Mockito.nullable(String.class), + Mockito.anyString(), Mockito.anyString(), Mockito.nullable(Integer.class), + Mockito.any(TracingContext.class)); + + List blobProperties = fs.getAbfsStore() + .getListBlobs(new Path("dir"), null, null, + getTestTracingContext(fs, true), 5, false); + Assertions.assertThat(blobProperties) + .describedAs( + "BlobList should match the number of maxResult given") + .hasSize(5); + Assertions.assertThat(count[0]) + .describedAs( + "Number of calls to backend should be equal to maxResult given") + .isEqualTo(5); + } + + @Test + public void testListBlobWithDelimiter() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + fs.setWorkingDirectory(new Path("/")); + assumeNonHnsAccountBlobEndpoint(fs); + List blobProperties; + + // Create three levels of hierarchy. + Path level0 = new Path("a"); + Path level1 = new Path("a/b"); + Path level2 = new Path("a/b/c"); + fs.mkdirs(level0); + fs.mkdirs(level1); + fs.mkdirs(level2); + + // Without delimiter, recursive listing will return all the children and sub children + // There will be no BlobPrefix element and only explicit blobs will be returned + blobProperties = fs.getAbfsStore() + .getListBlobs(level0.getParent(), null, null, + getTestTracingContext(fs, true), null, true); + Assertions.assertThat(blobProperties) + .describedAs( + "BlobList should return all blobs in hierarchy") + .hasSize(3); + + // With delimiter, non-recursive listing will return only the immediate children + // There will be repetition of marker blobs. + blobProperties = fs.getAbfsStore() + .getListBlobs(level0.getParent(), null, "/", + getTestTracingContext(fs, true), null, true); + Assertions.assertThat(blobProperties) + .describedAs( + "BlobList should return only immediate Children") + .hasSize(2); + + // ABFS Listing With delimiter, non-recursive listing will return only the immediate children + // There will be no repetition of marker blobs. + FileStatus[] fileStatuses = fs.listStatus(level0.getParent()); + Assertions.assertThat(fileStatuses) + .describedAs( + "BlobList should return only immediate Children") + .hasSize(1); + } + + private void assumeNonHnsAccountBlobEndpoint(final AzureBlobFileSystem fs) { + Assume.assumeTrue("To work on only on non-HNS Blob endpoint", + fs.getAbfsStore().getAbfsConfiguration().getPrefixMode() + == PrefixMode.BLOB); + } + + private void createBlob(final AzureBlobFileSystem fs, final String pathString) + throws IOException { + int i = 0; + while (i < 10) { + fs.create(new Path(pathString + i)); + i++; + } + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestListBlobProducer.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestListBlobProducer.java new file mode 100644 index 00000000000000..eae5d43d4e696b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestListBlobProducer.java @@ -0,0 +1,242 @@ +/** + * 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 java.io.IOException; +import java.net.HttpURLConnection; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; + +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Test; +import org.mockito.Mockito; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil; +import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; +import org.apache.hadoop.fs.azurebfs.services.ListBlobConsumer; +import org.apache.hadoop.fs.azurebfs.services.ListBlobProducer; +import org.apache.hadoop.fs.azurebfs.services.ListBlobQueue; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_PRODUCER_QUEUE_MAX_SIZE; + +public class ITestListBlobProducer extends AbstractAbfsIntegrationTest { + + public ITestListBlobProducer() throws Exception { + super(); + } + + @Override + public void setup() throws Exception { + super.setup(); + Assume.assumeTrue( + getFileSystem().getAbfsStore().getAbfsConfiguration().getPrefixMode() + == PrefixMode.BLOB); + } + + @Test + public void testProducerWaitingForConsumerLagToGoDown() throws Exception { + Configuration configuration = Mockito.spy(getRawConfiguration()); + configuration.set(FS_AZURE_PRODUCER_QUEUE_MAX_SIZE, "10"); + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance( + configuration); + AbfsClient client = Mockito.spy(fs.getAbfsClient()); + AbfsClient spiedClient = Mockito.spy(client); + fs.getAbfsStore().setClient(spiedClient); + fs.setWorkingDirectory(new Path("/")); + fs.mkdirs(new Path("/src")); + ExecutorService executor = Executors.newFixedThreadPool(5); + List futureList = new ArrayList<>(); + for (int i = 0; i < 20; i++) { + int iter = i; + futureList.add(executor.submit(() -> { + return fs.create(new Path("/src/file" + iter)); + })); + } + for(Future future : futureList) { + future.get(); + } + + AtomicInteger producedBlobs = new AtomicInteger(0); + AtomicInteger listBlobInvoked = new AtomicInteger(0); + + final ITestListBlobProducer testObj = this; + final ListBlobQueue queue = new ListBlobQueue( + fs.getAbfsStore().getAbfsConfiguration().getProducerQueueMaxSize(), + 1); + final CountDownLatch latch = new CountDownLatch(10); + + Mockito.doAnswer(answer -> { + synchronized (testObj) { + listBlobInvoked.incrementAndGet(); + AbfsRestOperation op = client.getListBlobs(answer.getArgument(0), + answer.getArgument(1), answer.getArgument(2), 1, answer.getArgument(4)); + producedBlobs.incrementAndGet(); + latch.countDown(); + if(producedBlobs.get() > 10) { + Assert.assertTrue(queue.availableSize() > 0); + } + return op; + } + }) + .when(spiedClient) + .getListBlobs(Mockito.nullable(String.class), + Mockito.nullable(String.class), Mockito.nullable(String.class), + Mockito.nullable(Integer.class), + Mockito.nullable(TracingContext.class)); + + AbfsClientTestUtil.hookOnRestOpsForTracingContextSingularity(client); + TracingContext tracingContext = new TracingContext("clientCorrelationId", + "fileSystemId", FSOperationType.TEST_OP, + getConfiguration().getTracingHeaderFormat(), + null); + ListBlobProducer producer = new ListBlobProducer("src/", spiedClient, queue, + null, getTestTracingContext(fs, true)); + ListBlobConsumer consumer = new ListBlobConsumer(queue); + latch.await(); + + int oldInvocation = listBlobInvoked.get(); + Assert.assertTrue(listBlobInvoked.get() == oldInvocation); + + while (!consumer.isCompleted()) { + synchronized (testObj) { + consumer.consume(); + Assert.assertTrue(queue.availableSize() > 0); + } + } + + Assert.assertTrue(producedBlobs.get() == 20); + } + + @Test + public void testConsumerWhenProducerThrowException() throws Exception { + Configuration configuration = Mockito.spy(getRawConfiguration()); + configuration.set(FS_AZURE_PRODUCER_QUEUE_MAX_SIZE, "10"); + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance( + configuration); + AbfsClient client = fs.getAbfsClient(); + AbfsClient spiedClient = Mockito.spy(client); + fs.getAbfsStore().setClient(spiedClient); + fs.setWorkingDirectory(new Path("/")); + fs.mkdirs(new Path("/src")); + for (int i = 0; i < 20; i++) { + fs.create(new Path("/src/file" + i)); + } + + Mockito.doAnswer(answer -> { + throw new AbfsRestOperationException(HttpURLConnection.HTTP_CONFLICT, "", + "", new Exception("")); + + }) + .when(spiedClient) + .getListBlobs(Mockito.nullable(String.class), + Mockito.nullable(String.class), Mockito.nullable(String.class), + Mockito.nullable(Integer.class), + Mockito.nullable(TracingContext.class)); + + ListBlobQueue queue = new ListBlobQueue(getConfiguration().getProducerQueueMaxSize(), + getConfiguration().getProducerQueueMaxSize()); + ListBlobProducer producer = new ListBlobProducer("src/", spiedClient, queue, + null, getTestTracingContext(fs, true)); + ListBlobConsumer consumer = new ListBlobConsumer(queue); + + Boolean exceptionCaught = false; + try { + while (!consumer.isCompleted()) { + consumer.consume(); + } + } catch (AzureBlobFileSystemException e) { + exceptionCaught = true; + } + + Assert.assertTrue(exceptionCaught); + } + + @Test + public void testProducerStopOnConsumerFailure() throws Exception { + Configuration configuration = Mockito.spy(getRawConfiguration()); + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance( + configuration); + + fs.setWorkingDirectory(new Path("/")); + fs.mkdirs(new Path("/src")); + ExecutorService executorService = Executors.newFixedThreadPool(10); + List futureList = new ArrayList<>(); + for (int i = 0; i < 20; i++) { + int iter = i; + Future future = executorService.submit(() -> { + try { + fs.create(new Path("/src/file" + iter)); + } catch (IOException ex) {} + }); + futureList.add(future); + } + + for (Future future : futureList) { + future.get(); + } + + AbfsClient client = fs.getAbfsClient(); + AbfsClient spiedClient = Mockito.spy(client); + fs.getAbfsStore().setClient(spiedClient); + + ListBlobQueue queue = new ListBlobQueue( + getConfiguration().getProducerQueueMaxSize(), + getConfiguration().getProducerQueueMaxSize()); + ListBlobConsumer consumer = new ListBlobConsumer(queue); + + Mockito.doAnswer(answer -> { + String marker = answer.getArgument(0); + String prefix = answer.getArgument(1); + String delimiter = answer.getArgument(2); + TracingContext tracingContext = answer.getArgument(4); + Object result = client.getListBlobs(marker, prefix, delimiter, 1, + tracingContext); + consumer.fail(); + return result; + }) + .when(spiedClient) + .getListBlobs(Mockito.nullable(String.class), + Mockito.nullable(String.class), Mockito.nullable(String.class), + Mockito.nullable(Integer.class), Mockito.any(TracingContext.class)); + + ListBlobProducer producer = new ListBlobProducer("src/", spiedClient, queue, + null, getTestTracingContext(fs, true)); + + producer.waitForProcessCompletion(); + Mockito.verify(spiedClient, Mockito.times(1)) + .getListBlobs(Mockito.nullable(String.class), + Mockito.nullable(String.class), Mockito.nullable(String.class), + Mockito.nullable(Integer.class), Mockito.any(TracingContext.class)); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java index fedddcc4b16fbf..250b463cd2ec5f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSharedKeyAuth.java @@ -17,16 +17,20 @@ */ package org.apache.hadoop.fs.azurebfs; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; import org.junit.Assume; import org.junit.Test; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; -import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AuthType; +import java.io.IOException; + import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DNS_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX; import static org.apache.hadoop.test.LambdaTestUtils.intercept; public class ITestSharedKeyAuth extends AbstractAbfsIntegrationTest { @@ -40,23 +44,23 @@ public void testWithWrongSharedKey() throws Exception { Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); Configuration config = this.getRawConfiguration(); config.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, - false); + true); + AzureBlobFileSystemStore abfsStore = getAbfsStore(getFileSystem()); String accountName = this.getAccountName(); + String configkKey = FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + "." + accountName; // a wrong sharedKey String secret = "XjUjsGherkDpljuyThd7RpljhR6uhsFjhlxRpmhgD12lnj7lhfRn8kgPt5" + "+MJHS7UJNDER+jn6KP6Jnm2ONQlm=="; config.set(configkKey, secret); - AbfsClient abfsClient = this.getFileSystem(config).getAbfsClient(); - intercept(AbfsRestOperationException.class, - "\"Server failed to authenticate the request. Make sure the value of " - + "Authorization header is formed correctly including the " - + "signature.\", 403", - () -> { - abfsClient - .getAclStatus("/", getTestTracingContext(getFileSystem(), false)); - }); + intercept(IOException.class, + "\"Server failed to authenticate the request. Make sure the value of " + + "Authorization header is formed correctly including the " + + "signature.\", 403", + () -> { + FileSystem.newInstance(config); + }); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSmallWriteOptimization.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSmallWriteOptimization.java index fce2b682f580a6..667defc4d2abaf 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSmallWriteOptimization.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestSmallWriteOptimization.java @@ -24,6 +24,8 @@ import java.util.Map; import java.io.IOException; +import org.apache.hadoop.fs.azurebfs.services.OperativeEndpoint; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.runners.Parameterized; @@ -71,6 +73,7 @@ public class ITestSmallWriteOptimization extends AbstractAbfsScaleTest { private static final int HALF_TEST_BUFFER_SIZE = TWO_MB / 2; private static final int QUARTER_TEST_BUFFER_SIZE = TWO_MB / 4; private static final int TEST_FLUSH_ITERATION = 2; + private PrefixMode prefixMode = PrefixMode.DFS; @Parameterized.Parameter public String testScenario; @@ -305,13 +308,16 @@ public void testSmallWriteOptimization() // Tests with Optimization should only run if service has the feature on by // default. Default settings will be turned on when server support is // available on all store prod regions. - if (enableSmallWriteOptimization) { - Assume.assumeTrue(serviceDefaultOptmSettings); - } final AzureBlobFileSystem currentfs = this.getFileSystem(); Configuration config = currentfs.getConf(); boolean isAppendBlobTestSettingEnabled = (config.get(FS_AZURE_TEST_APPENDBLOB_ENABLED) == "true"); + prefixMode = currentfs.getAbfsStore().getPrefixMode(); + + if (enableSmallWriteOptimization) { + Assume.assumeTrue(serviceDefaultOptmSettings); + Assume.assumeTrue(prefixMode == PrefixMode.DFS); + } // This optimization doesnt take effect when append blob is on. Assume.assumeFalse(isAppendBlobTestSettingEnabled); @@ -426,7 +432,9 @@ private void formulateSmallWriteTestAppendPattern(final AzureBlobFileSystem fs, ? 1 // 1 append (with flush and close param) : (wasDataPendingToBeWrittenToServer) ? 2 // 1 append + 1 flush (with close) - : 1); // 1 flush (with close) + : (recurringWriteSize == 0 && !OperativeEndpoint.isIngressEnabledOnDFS(getPrefixMode(fs), fs.getAbfsStore().getAbfsConfiguration())) + ? 0 // no flush or close on prefix mode blob + : 1); //1 flush (with close) expectedTotalRequestsMade += totalAppendFlushCalls; expectedRequestsMadeWithData += totalAppendFlushCalls; @@ -445,10 +453,19 @@ private void formulateSmallWriteTestAppendPattern(final AzureBlobFileSystem fs, testIteration--; } + /** + * Above test iteration loop executes one of the below two patterns + * 1. Append + Close (triggers flush) + * 2. Append + Flush + * For both patters PutBlobList is complete in the iteration loop itself + * Hence with PrefixMode Blob, below close won't trigger any network call + */ opStream.close(); - expectedTotalRequestsMade += 1; - expectedRequestsMadeWithData += 1; + if (OperativeEndpoint.isIngressEnabledOnDFS(getPrefixMode(fs), fs.getAbfsStore().getAbfsConfiguration())) { + expectedTotalRequestsMade += 1; + expectedRequestsMadeWithData += 1; + } // no change in expectedBytesSent assertOpStats(fs.getInstrumentationMap(), expectedTotalRequestsMade, expectedRequestsMadeWithData, expectedBytesSent); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 006004850d0df5..1d0701125341e9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -26,17 +26,20 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.azurebfs.services.PrefixMode; import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.AssumptionViolatedException; import org.junit.Ignore; import org.junit.Test; +import org.mockito.Mockito; import org.apache.hadoop.fs.CommonPathCapabilities; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.enums.Trilean; +import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; @@ -106,9 +109,16 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, : null; //request should not fail for invalid clientCorrelationID - AbfsRestOperation op = fs.getAbfsClient() - .createPath(path, false, true, permission, umask, false, null, - tracingContext); + AbfsRestOperation op; + if (getPrefixMode(fs) == PrefixMode.DFS) { + op = fs.getAbfsClient() + .createPath(path, false, true, permission, umask, false, null, + tracingContext); + } else { + op = fs.getAbfsClient() + .createPathBlob(path, false, true, new HashMap(), null, + tracingContext); + } int statusCode = op.getResult().getStatusCode(); Assertions.assertThat(statusCode).describedAs("Request should not fail") @@ -118,7 +128,7 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, .replace("]", ""); Assertions.assertThat(requestHeader) .describedAs("Client Request Header should match TracingContext") - .isEqualTo(tracingContext.getHeader()); + .isEqualTo(op.getLastTracingContext().getHeader()); } @Ignore @@ -130,10 +140,10 @@ public void runCorrelationTestForAllMethods() throws Exception { testClasses.put(new ITestAzureBlobFileSystemListStatus(), //liststatus ITestAzureBlobFileSystemListStatus.class.getMethod("testListPath")); - testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE), //open, + testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, true), //open, // read, write ITestAbfsReadWriteAndSeek.class.getMethod("testReadAheadRequestID")); - testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE), //read (bypassreadahead) + testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, false), //read (bypassreadahead) ITestAbfsReadWriteAndSeek.class .getMethod("testReadAndWriteWithDifferentBufferSizesAndSeek")); testClasses.put(new ITestAzureBlobFileSystemAppend(), //append @@ -198,4 +208,74 @@ public void testExternalOps() throws Exception { fs.getAbfsStore().setNamespaceEnabled(Trilean.TRUE); fs.access(new Path("/"), FsAction.READ); } + + @Test + public void testRetryPrimaryRequestIdWhenInitiallySuppliedEmpty() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final String fileSystemId = fs.getFileSystemId(); + final String clientCorrelationId = fs.getClientCorrelationId(); + final TracingHeaderFormat tracingHeaderFormat = TracingHeaderFormat.ALL_ID_FORMAT; + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.CREATE_FILESYSTEM, false, + 0)); + AbfsHttpOperation abfsHttpOperation = Mockito.mock(AbfsHttpOperation.class); + Mockito.doNothing().when(abfsHttpOperation).setRequestProperty(Mockito.anyString(), Mockito.anyString()); + tracingContext.constructHeader(abfsHttpOperation, null); + String header = tracingContext.getHeader(); + String clientRequestIdUsed = header.split(":")[1]; + String[] clientRequestIdUsedParts = clientRequestIdUsed.split("-"); + String assertionPrimaryId = clientRequestIdUsedParts[clientRequestIdUsedParts.length - 1]; + + tracingContext.setRetryCount(1); + tracingContext.setListener(new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.CREATE_FILESYSTEM, false, + 1)); + + tracingContext.constructHeader(abfsHttpOperation, "RT"); + header = tracingContext.getHeader(); + String primaryRequestId = header.split(":")[3]; + + Assertions.assertThat(primaryRequestId) + .describedAs("PrimaryRequestId in a retried request's " + + "tracingContext should be equal to last part of original " + + "request's clientRequestId UUID") + .isEqualTo(assertionPrimaryId); + } + + @Test + public void testRetryPrimaryRequestIdWhenInitiallySuppliedNonEmpty() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final String fileSystemId = fs.getFileSystemId(); + final String clientCorrelationId = fs.getClientCorrelationId(); + final TracingHeaderFormat tracingHeaderFormat = TracingHeaderFormat.ALL_ID_FORMAT; + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.CREATE_FILESYSTEM, false, + 0)); + tracingContext.setPrimaryRequestID(); + AbfsHttpOperation abfsHttpOperation = Mockito.mock(AbfsHttpOperation.class); + Mockito.doNothing().when(abfsHttpOperation).setRequestProperty(Mockito.anyString(), Mockito.anyString()); + tracingContext.constructHeader(abfsHttpOperation, null); + String header = tracingContext.getHeader(); + String assertionPrimaryId = header.split(":")[3]; + + tracingContext.setRetryCount(1); + tracingContext.setListener(new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.CREATE_FILESYSTEM, false, + 1)); + + tracingContext.constructHeader(abfsHttpOperation, "RT"); + header = tracingContext.getHeader(); + String primaryRequestId = header.split(":")[3]; + + Assertions.assertThat(primaryRequestId) + .describedAs("PrimaryRequestId in a retried request's tracingContext " + + "should be equal to PrimaryRequestId in the original request.") + .isEqualTo(assertionPrimaryId); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java index 565eb38c4f70a7..9e40f22d231b05 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java @@ -24,6 +24,9 @@ public final class TestConfigurationKeys { public static final String FS_AZURE_ACCOUNT_NAME = "fs.azure.account.name"; public static final String FS_AZURE_ABFS_ACCOUNT_NAME = "fs.azure.abfs.account.name"; + public static final String FS_AZURE_ABFS_ACCOUNT1_NAME = "fs.azure.abfs.account1.name"; + public static final String FS_AZURE_ENABLE_AUTOTHROTTLING = "fs.azure.enable.autothrottling"; + public static final String FS_AZURE_ANALYSIS_PERIOD = "fs.azure.analysis.period"; public static final String FS_AZURE_ACCOUNT_KEY = "fs.azure.account.key"; public static final String FS_AZURE_CONTRACT_TEST_URI = "fs.contract.test.fs.abfs"; public static final String FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT = "fs.azure.test.namespace.enabled"; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractEtag.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractEtag.java index d498ae71a4b6f0..10e8302bdc6c33 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractEtag.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractEtag.java @@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.contract.AbstractContractEtagTest; import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.junit.Assume; /** * Contract test for etag support. @@ -38,6 +39,7 @@ public ITestAbfsFileSystemContractEtag() throws Exception { public void setup() throws Exception { binding.setup(); super.setup(); + Assume.assumeTrue(binding.isNamespaceEnabled(binding.getFileSystem())); // Base rename contract test class re-uses the test folder // This leads to failures when the test is re-run as same ABFS test // containers are re-used for test run and creation of source and diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java new file mode 100644 index 00000000000000..eba6746dcd36dc --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java @@ -0,0 +1,173 @@ +/** + * 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.IOException; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.ReentrantLock; + +import org.assertj.core.api.Assertions; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.util.functional.BiFunctionRaisingIOE; +import org.apache.hadoop.util.functional.FunctionRaisingIOE; + +import static java.net.HttpURLConnection.HTTP_OK; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_GET; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; +import static org.apache.hadoop.fs.azurebfs.services.AuthType.OAuth; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.nullable; + +public final class AbfsClientTestUtil { + + private AbfsClientTestUtil() { + + } + + public static void setMockAbfsRestOperationForCopyBlobOperation(final AbfsClient spiedClient, + final BiFunctionRaisingIOE functionRaisingIOE) { + + Mockito.doAnswer(answer -> { + final AbfsRestOperation spiedRestOp = Mockito.spy(new AbfsRestOperation( + AbfsRestOperationType.CopyBlob, + spiedClient, + HTTP_METHOD_PUT, + answer.getArgument(0), + answer.getArgument(1) + )); + final AbfsRestOperation actualCallMakerOp = new AbfsRestOperation( + AbfsRestOperationType.CopyBlob, + spiedClient, + HTTP_METHOD_PUT, + answer.getArgument(0), + answer.getArgument(1) + ); + return functionRaisingIOE.apply(spiedRestOp, actualCallMakerOp); + }) + .when(spiedClient) + .getCopyBlobOperation(Mockito.any(URL.class), Mockito.anyList()); + } + + public static void setMockAbfsRestOperationForListBlobOperation( + final AbfsClient spiedClient, + FunctionRaisingIOE functionRaisingIOE) + throws Exception { + ExponentialRetryPolicy retryPolicy = Mockito.mock(ExponentialRetryPolicy.class); + AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class); + AbfsRestOperation abfsRestOperation = Mockito.spy(new AbfsRestOperation( + AbfsRestOperationType.GetListBlobProperties, + spiedClient, + HTTP_METHOD_GET, + null, + new ArrayList<>() + )); + + Mockito.doReturn(abfsRestOperation).when(spiedClient).getListBlobOperation(any(), any()); + + addMockBehaviourToAbfsClient(spiedClient, retryPolicy); + addMockBehaviourToRestOpAndHttpOp(abfsRestOperation, httpOperation); + + functionRaisingIOE.apply(httpOperation); + } + + public static void addMockBehaviourToRestOpAndHttpOp(final AbfsRestOperation abfsRestOperation, + final AbfsHttpOperation httpOperation) throws IOException { + HttpURLConnection httpURLConnection = Mockito.mock(HttpURLConnection.class); + Mockito.doNothing().when(httpURLConnection) + .setRequestProperty(nullable(String.class), nullable(String.class)); + Mockito.doReturn(httpURLConnection).when(httpOperation).getConnection(); + Mockito.doReturn("").when(abfsRestOperation).getClientLatency(); + Mockito.doReturn(httpOperation).when(abfsRestOperation).createHttpOperation(); + } + + public static void addMockBehaviourToAbfsClient(final AbfsClient abfsClient, + final ExponentialRetryPolicy retryPolicy) throws IOException { + Mockito.doReturn(OAuth).when(abfsClient).getAuthType(); + Mockito.doReturn("").when(abfsClient).getAccessToken(); + AbfsThrottlingIntercept intercept = Mockito.mock( + AbfsThrottlingIntercept.class); + Mockito.doReturn(intercept).when(abfsClient).getIntercept(); + Mockito.doNothing() + .when(intercept) + .sendingRequest(any(), nullable(AbfsCounters.class)); + Mockito.doNothing().when(intercept).updateMetrics(any(), any()); + + Mockito.doReturn(retryPolicy).when(abfsClient).getRetryPolicy(); + Mockito.doReturn(true) + .when(retryPolicy) + .shouldRetry(nullable(Integer.class), nullable(Integer.class)); + Mockito.doReturn(false).when(retryPolicy).shouldRetry(0, HTTP_OK); + Mockito.doReturn(false).when(retryPolicy).shouldRetry(1, HTTP_OK); + Mockito.doReturn(false).when(retryPolicy).shouldRetry(2, HTTP_OK); + } + + public static void populateBlobListHelper(BlobList list, BlobProperty blob, String nextMarker) { + list.addBlobProperty(blob); + list.setNextMarker("nextMarker"); + } + + public static void hookOnRestOpsForTracingContextSingularity(AbfsClient client) { + Set tracingContextSet = new HashSet<>(); + ReentrantLock lock = new ReentrantLock(); + Answer answer = new Answer() { + @Override + public Object answer(final InvocationOnMock invocationOnMock) + throws Throwable { + AbfsRestOperation op = Mockito.spy((AbfsRestOperation) invocationOnMock.callRealMethod()); + Mockito.doAnswer(completeExecuteInvocation -> { + lock.lock(); + try { + TracingContext context = completeExecuteInvocation.getArgument(0); + Assertions.assertThat(tracingContextSet).doesNotContain(context); + tracingContextSet.add(context); + } finally { + lock.unlock(); + } + return completeExecuteInvocation.callRealMethod(); + }).when(op).completeExecute(Mockito.any(TracingContext.class)); + return op; + } + }; + + Mockito.doAnswer(answer) + .when(client) + .getAbfsRestOperation(Mockito.any(AbfsRestOperationType.class), + Mockito.anyString(), Mockito.any(URL.class), Mockito.anyList(), + Mockito.nullable(byte[].class), Mockito.anyInt(), Mockito.anyInt(), + Mockito.nullable(String.class)); + Mockito.doAnswer(answer) + .when(client) + .getAbfsRestOperation(Mockito.any(AbfsRestOperationType.class), + Mockito.anyString(), Mockito.any(URL.class), Mockito.anyList()); + Mockito.doAnswer(answer) + .when(client) + .getAbfsRestOperation(Mockito.any(AbfsRestOperationType.class), + Mockito.anyString(), Mockito.any(URL.class), Mockito.anyList(), + Mockito.nullable(String.class)); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOpTestUtil.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOpTestUtil.java new file mode 100644 index 00000000000000..575b674d7ab148 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOpTestUtil.java @@ -0,0 +1,30 @@ +/** + * 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; + + +public final class AbfsHttpOpTestUtil { + private AbfsHttpOpTestUtil() { + + } + + public static void setConnection(AbfsHttpOperation op, AbfsHttpOperation copyFrom) { + op.setConnection(copyFrom.getConnection()); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationTestUtil.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationTestUtil.java new file mode 100644 index 00000000000000..fb11f6ae618354 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationTestUtil.java @@ -0,0 +1,52 @@ +/** + * 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.IOException; + +import org.mockito.Mockito; + +import org.apache.hadoop.util.functional.BiFunctionRaisingIOE; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.times; + +public final class AbfsRestOperationTestUtil { + + private AbfsRestOperationTestUtil() { + + } + + public static void addAbfsHttpOpProcessResponseMock(final AbfsRestOperation spiedRestOp, + BiFunctionRaisingIOE functionRaisingIOE) + throws IOException { + Mockito.doAnswer(answer -> { + AbfsHttpOperation op = new AbfsHttpOperation(spiedRestOp.getUrl(), + spiedRestOp.getMethod(), spiedRestOp.getRequestHeaders()); + AbfsHttpOperation actualOp = new AbfsHttpOperation(spiedRestOp.getUrl(), + spiedRestOp.getMethod(), spiedRestOp.getRequestHeaders()); + AbfsHttpOperation spiedOp = Mockito.spy(op); + return functionRaisingIOE.apply(spiedOp, actualOp); + }).when(spiedRestOp).createHttpOperation(); + } + + public static void setResult(final AbfsRestOperation op, final AbfsHttpOperation result) { + op.setResult(result); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java similarity index 58% rename from hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java rename to hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java index a725bf3175a5c1..d98f58d236ab1c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java @@ -20,20 +20,45 @@ import java.io.IOException; import java.lang.reflect.Field; +import java.net.HttpURLConnection; +import java.net.ProtocolException; import java.net.URL; import java.util.List; +import java.util.Random; import java.util.regex.Pattern; +import org.junit.Ignore; +import org.assertj.core.api.Assertions; import org.junit.Test; +import org.mockito.Mockito; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.TestAbfsConfigurationFieldsValidation; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; -import static org.assertj.core.api.Assertions.assertThat; +import static java.net.HttpURLConnection.HTTP_NOT_FOUND; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_DELETE; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PATCH; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_POSITION; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -59,14 +84,20 @@ * Test useragent of abfs client. * */ -public final class TestAbfsClient { +public final class ITestAbfsClient extends AbstractAbfsIntegrationTest { private static final String ACCOUNT_NAME = "bogusAccountName.dfs.core.windows.net"; private static final String FS_AZURE_USER_AGENT_PREFIX = "Partner Service"; + private static final String HUNDRED_CONTINUE_USER_AGENT = SINGLE_WHITE_SPACE + HUNDRED_CONTINUE + SEMICOLON; + private static final String TEST_PATH = "/testfile"; + public static final int REDUCED_RETRY_COUNT = 2; + public static final int REDUCED_BACKOFF_INTERVAL = 100; + public static final int BUFFER_LENGTH = 5; + public static final int BUFFER_OFFSET = 0; private final Pattern userAgentStringPattern; - public TestAbfsClient(){ + public ITestAbfsClient() throws Exception { StringBuilder regEx = new StringBuilder(); regEx.append("^"); regEx.append(APN_VERSION); @@ -124,7 +155,7 @@ public void verifybBasicInfo() throws Exception { } private void verifybBasicInfo(String userAgentStr) { - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string [" + userAgentStr + "] should be of the pattern: " + this.userAgentStringPattern.pattern()) .matches(this.userAgentStringPattern) @@ -142,6 +173,7 @@ private void verifybBasicInfo(String userAgentStr) { .contains(System.getProperty(OS_ARCH)); } + @Ignore @Test public void verifyUserAgentPrefix() throws IOException, IllegalAccessException { @@ -153,7 +185,7 @@ public void verifyUserAgentPrefix() String userAgentStr = getUserAgentString(abfsConfiguration, false); verifybBasicInfo(userAgentStr); - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string should contain " + FS_AZURE_USER_AGENT_PREFIX) .contains(FS_AZURE_USER_AGENT_PREFIX); @@ -163,11 +195,38 @@ public void verifyUserAgentPrefix() userAgentStr = getUserAgentString(abfsConfiguration, false); verifybBasicInfo(userAgentStr); - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string should not contain " + FS_AZURE_USER_AGENT_PREFIX) .doesNotContain(FS_AZURE_USER_AGENT_PREFIX); } + @Test + public void verifyUserAgentExpectHeader() + throws IOException, IllegalAccessException { + final Configuration configuration = new Configuration(); + configuration.addResource(TEST_CONFIGURATION_FILE_NAME); + configuration.set(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, FS_AZURE_USER_AGENT_PREFIX); + configuration.setBoolean(ConfigurationKeys.FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED, true); + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, + ACCOUNT_NAME); + String userAgentStr = getUserAgentString(abfsConfiguration, false); + + verifybBasicInfo(userAgentStr); + Assertions.assertThat(userAgentStr) + .describedAs("User-Agent string should contain " + HUNDRED_CONTINUE_USER_AGENT) + .contains(HUNDRED_CONTINUE_USER_AGENT); + + configuration.setBoolean(ConfigurationKeys.FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED, false); + abfsConfiguration = new AbfsConfiguration(configuration, + ACCOUNT_NAME); + userAgentStr = getUserAgentString(abfsConfiguration, false); + + verifybBasicInfo(userAgentStr); + Assertions.assertThat(userAgentStr) + .describedAs("User-Agent string should not contain " + HUNDRED_CONTINUE_USER_AGENT) + .doesNotContain(HUNDRED_CONTINUE_USER_AGENT); + } + @Test public void verifyUserAgentWithoutSSLProvider() throws Exception { final Configuration configuration = new Configuration(); @@ -179,14 +238,14 @@ public void verifyUserAgentWithoutSSLProvider() throws Exception { String userAgentStr = getUserAgentString(abfsConfiguration, true); verifybBasicInfo(userAgentStr); - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string should contain sslProvider") .contains(DelegatingSSLSocketFactory.getDefaultFactory().getProviderName()); userAgentStr = getUserAgentString(abfsConfiguration, false); verifybBasicInfo(userAgentStr); - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string should not contain sslProvider") .doesNotContain(DelegatingSSLSocketFactory.getDefaultFactory().getProviderName()); } @@ -202,7 +261,7 @@ public void verifyUserAgentClusterName() throws Exception { String userAgentStr = getUserAgentString(abfsConfiguration, false); verifybBasicInfo(userAgentStr); - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string should contain cluster name") .contains(clusterName); @@ -212,7 +271,7 @@ public void verifyUserAgentClusterName() throws Exception { userAgentStr = getUserAgentString(abfsConfiguration, false); verifybBasicInfo(userAgentStr); - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string should not contain cluster name") .doesNotContain(clusterName) .describedAs("User-Agent string should contain UNKNOWN as cluster name config is absent") @@ -230,7 +289,7 @@ public void verifyUserAgentClusterType() throws Exception { String userAgentStr = getUserAgentString(abfsConfiguration, false); verifybBasicInfo(userAgentStr); - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string should contain cluster type") .contains(clusterType); @@ -240,7 +299,7 @@ public void verifyUserAgentClusterType() throws Exception { userAgentStr = getUserAgentString(abfsConfiguration, false); verifybBasicInfo(userAgentStr); - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string should not contain cluster type") .doesNotContain(clusterType) .describedAs("User-Agent string should contain UNKNOWN as cluster type config is absent") @@ -304,26 +363,34 @@ public static AbfsClient getMockAbfsClient(AbfsClient baseAbfsClientInstance, when(client.createDefaultUriQueryBuilder()).thenCallRealMethod(); when(client.createRequestUrl(any(), any())).thenCallRealMethod(); when(client.getAccessToken()).thenCallRealMethod(); + when(client.getAbfsRestOperation( + Mockito.any(AbfsRestOperationType.class), Mockito.anyString(), + Mockito.any(URL.class), + Mockito.anyList())).thenCallRealMethod(); when(client.getSharedKeyCredentials()).thenCallRealMethod(); when(client.createDefaultHeaders()).thenCallRealMethod(); - + when(client.getAbfsConfiguration()).thenReturn(abfsConfig); + when(client.getIntercept()).thenReturn( + AbfsThrottlingInterceptFactory.getInstance( + abfsConfig.getAccountName().substring(0, + abfsConfig.getAccountName().indexOf(DOT)), abfsConfig)); // override baseurl - client = TestAbfsClient.setAbfsClientField(client, "abfsConfiguration", + client = ITestAbfsClient.setAbfsClientField(client, "abfsConfiguration", abfsConfig); // override baseurl - client = TestAbfsClient.setAbfsClientField(client, "baseUrl", + client = ITestAbfsClient.setAbfsClientField(client, "baseUrl", baseAbfsClientInstance.getBaseUrl()); // override auth provider if (currentAuthType == AuthType.SharedKey) { - client = TestAbfsClient.setAbfsClientField(client, "sharedKeyCredentials", + client = ITestAbfsClient.setAbfsClientField(client, "sharedKeyCredentials", new SharedKeyCredentials( abfsConfig.getAccountName().substring(0, abfsConfig.getAccountName().indexOf(DOT)), abfsConfig.getStorageAccountKey())); } else { - client = TestAbfsClient.setAbfsClientField(client, "tokenProvider", + client = ITestAbfsClient.setAbfsClientField(client, "tokenProvider", abfsConfig.getTokenProvider()); } @@ -331,7 +398,7 @@ public static AbfsClient getMockAbfsClient(AbfsClient baseAbfsClientInstance, String userAgent = "APN/1.0 Azure Blob FS/3.4.0-SNAPSHOT (PrivateBuild " + "JavaJRE 1.8.0_252; Linux 5.3.0-59-generic/amd64; openssl-1.0; " + "UNKNOWN/UNKNOWN) MSFT"; - client = TestAbfsClient.setAbfsClientField(client, "userAgent", userAgent); + client = ITestAbfsClient.setAbfsClientField(client, "userAgent", userAgent); return client; } @@ -395,4 +462,160 @@ public static AbfsRestOperation getRestOp(AbfsRestOperationType type, url, requestHeaders); } + + public static AccessTokenProvider getAccessTokenProvider(AbfsClient client) { + return client.getTokenProvider(); + } + + /** + * Test helper method to get random bytes array. + * @param length The length of byte buffer. + * @return byte buffer. + */ + private byte[] getRandomBytesArray(int length) { + final byte[] b = new byte[length]; + new Random().nextBytes(b); + return b; + } + + /** + * Test to verify that client retries append request without + * expect header enabled if append with expect header enabled fails + * with 4xx kind of error. + * @throws Exception + */ + @Test + public void testExpectHundredContinue() throws Exception { + // Get the filesystem. + final AzureBlobFileSystem fs = getFileSystem(); + + final Configuration configuration = new Configuration(); + configuration.addResource(TEST_CONFIGURATION_FILE_NAME); + AbfsClient abfsClient = getClient(fs); + + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, + configuration.get(FS_AZURE_ABFS_ACCOUNT_NAME)); + + // Update the configuration with reduced retry count and reduced backoff interval. + AbfsConfiguration abfsConfig + = TestAbfsConfigurationFieldsValidation.updateRetryConfigs( + abfsConfiguration, + REDUCED_RETRY_COUNT, REDUCED_BACKOFF_INTERVAL); + + // Gets the client. + AbfsClient testClient = Mockito.spy( + ITestAbfsClient.createTestClientFromCurrentContext( + abfsClient, + abfsConfig)); + + // Create the append request params with expect header enabled initially. + AppendRequestParameters appendRequestParameters + = new AppendRequestParameters( + BUFFER_OFFSET, BUFFER_OFFSET, BUFFER_LENGTH, + AppendRequestParameters.Mode.APPEND_MODE, false, null, true); + + byte[] buffer = getRandomBytesArray(BUFFER_LENGTH); + + // Create a test container to upload the data. + Path testPath = path(TEST_PATH); + fs.create(testPath); + String finalTestPath = testPath.toString() + .substring(testPath.toString().lastIndexOf("/")); + + // Creates a list of request headers. + final List requestHeaders + = ITestAbfsClient.getTestRequestHeaders(testClient); + requestHeaders.add( + new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); + if (appendRequestParameters.isExpectHeaderEnabled()) { + requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); + } + + // Updates the query parameters. + final AbfsUriQueryBuilder abfsUriQueryBuilder + = testClient.createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, APPEND_ACTION); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, + Long.toString(appendRequestParameters.getPosition())); + + // Creates the url for the specified path. + URL url = testClient.createRequestUrl(finalTestPath, abfsUriQueryBuilder.toString()); + + // Create a mock of the AbfsRestOperation to set the urlConnection in the corresponding httpOperation. + AbfsRestOperation op = Mockito.spy(new AbfsRestOperation( + AbfsRestOperationType.Append, + testClient, + HTTP_METHOD_PUT, + url, + requestHeaders, buffer, + appendRequestParameters.getoffset(), + appendRequestParameters.getLength(), null)); + + AbfsHttpOperation abfsHttpOperation = Mockito.spy(new AbfsHttpOperation(url, + HTTP_METHOD_PUT, requestHeaders)); + + // Sets the expect request property if expect header is enabled. + if (appendRequestParameters.isExpectHeaderEnabled()) { + Mockito.doReturn(HUNDRED_CONTINUE).when(abfsHttpOperation) + .getConnProperty(EXPECT); + } + + HttpURLConnection urlConnection = mock(HttpURLConnection.class); + Mockito.doNothing().when(urlConnection).setRequestProperty(Mockito + .any(), Mockito.any()); + Mockito.doReturn(HTTP_METHOD_PUT).when(urlConnection).getRequestMethod(); + Mockito.doReturn(url).when(urlConnection).getURL(); + Mockito.doReturn(urlConnection).when(abfsHttpOperation).getConnection(); + + Mockito.doNothing().when(abfsHttpOperation).setRequestProperty(Mockito + .any(), Mockito.any()); + Mockito.doReturn(url).when(abfsHttpOperation).getConnUrl(); + + // Give user error code 404 when processResponse is called. + Mockito.doReturn(HTTP_METHOD_PUT).when(abfsHttpOperation).getConnRequestMethod(); + Mockito.doReturn(HTTP_NOT_FOUND).when(abfsHttpOperation).getConnResponseCode(); + Mockito.doReturn("Resource Not Found") + .when(abfsHttpOperation) + .getConnResponseMessage(); + + // Make the getOutputStream throw IOException to see it returns from the sendRequest correctly. + Mockito.doThrow(new ProtocolException("Server rejected Operation")) + .when(abfsHttpOperation) + .getConnOutputStream(); + + // Sets the httpOperation for the rest operation. + Mockito.doReturn(abfsHttpOperation) + .when(op) + .createHttpOperation(); + + // Mock the restOperation for the client. + Mockito.doReturn(op) + .when(testClient) + .getAbfsRestOperationForAppend(Mockito.any(), + Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.nullable(int.class), Mockito.nullable(int.class), + Mockito.any()); + + TracingContext tracingContext = Mockito.spy(new TracingContext("abcd", + "abcde", FSOperationType.APPEND, + TracingHeaderFormat.ALL_ID_FORMAT, null)); + + // Check that expect header is enabled before the append call. + Assertions.assertThat(appendRequestParameters.isExpectHeaderEnabled()) + .describedAs("The expect header is not true before the append call") + .isTrue(); + + intercept(AzureBlobFileSystemException.class, + () -> testClient.append(finalTestPath, buffer, appendRequestParameters, null, tracingContext)); + + // Verify that the request was not exponentially retried because of user error. + Assertions.assertThat(tracingContext.getRetryCount()) + .describedAs("The retry count is incorrect") + .isEqualTo(0); + + // Verify that the same request was retried with expect header disabled. + Assertions.assertThat(appendRequestParameters.isExpectHeaderEnabled()) + .describedAs("The expect header is not false") + .isFalse(); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java new file mode 100644 index 00000000000000..5f166a5ee1eb76 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java @@ -0,0 +1,359 @@ +/** + * 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.IOException; +import java.io.OutputStream; +import java.net.HttpURLConnection; +import java.net.ProtocolException; +import java.net.URL; +import java.util.Arrays; +import java.util.List; +import java.util.Random; + +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.mockito.Mockito; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.TestAbfsConfigurationFieldsValidation; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; + +import static java.net.HttpURLConnection.HTTP_NOT_FOUND; +import static java.net.HttpURLConnection.HTTP_OK; +import static java.net.HttpURLConnection.HTTP_UNAVAILABLE; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PATCH; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_POSITION; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; + +@RunWith(Parameterized.class) +public class ITestAbfsRestOperation extends AbstractAbfsIntegrationTest { + + // Specifies whether getOutputStream() or write() throws IOException. + public enum ErrorType {OUTPUTSTREAM, WRITE}; + + private static final int HTTP_EXPECTATION_FAILED = 417; + private static final int HTTP_ERROR = 0; + private static final int ZERO = 0; + private static final int REDUCED_RETRY_COUNT = 2; + private static final int REDUCED_BACKOFF_INTERVAL = 100; + private static final int BUFFER_LENGTH = 5; + private static final int BUFFER_OFFSET = 0; + private static final String TEST_PATH = "/testfile"; + + // Specifies whether the expect header is enabled or not. + @Parameterized.Parameter + public boolean expectHeaderEnabled; + + // Gives the http response code. + @Parameterized.Parameter(1) + public int responseCode; + + // Gives the http response message. + @Parameterized.Parameter(2) + public String responseMessage; + + // Gives the errorType based on the enum. + @Parameterized.Parameter(3) + public ErrorType errorType; + + // The intercept. + private AbfsThrottlingIntercept intercept; + + /* + HTTP_OK = 200, + HTTP_UNAVAILABLE = 503, + HTTP_NOT_FOUND = 404, + HTTP_EXPECTATION_FAILED = 417, + HTTP_ERROR = 0. + */ + @Parameterized.Parameters(name = "expect={0}-code={1}-ErrorType={3}") + public static Iterable params() { + return Arrays.asList(new Object[][]{ + {true, HTTP_OK, "OK", ErrorType.WRITE}, + {false, HTTP_OK, "OK", ErrorType.WRITE}, + {true, HTTP_UNAVAILABLE, "ServerBusy", ErrorType.OUTPUTSTREAM}, + {true, HTTP_NOT_FOUND, "Resource Not Found", ErrorType.OUTPUTSTREAM}, + {true, HTTP_EXPECTATION_FAILED, "Expectation Failed", ErrorType.OUTPUTSTREAM}, + {true, HTTP_ERROR, "Error", ErrorType.OUTPUTSTREAM} + }); + } + + public ITestAbfsRestOperation() throws Exception { + super(); + } + + /** + * Test helper method to get random bytes array. + * @param length The length of byte buffer + * @return byte buffer + */ + private byte[] getRandomBytesArray(int length) { + final byte[] b = new byte[length]; + new Random().nextBytes(b); + return b; + } + + /** + * Gives the AbfsRestOperation. + * @return abfsRestOperation. + */ + private AbfsRestOperation getRestOperation() throws Exception { + // Get the filesystem. + final AzureBlobFileSystem fs = getFileSystem(); + + final Configuration configuration = new Configuration(); + configuration.addResource(TEST_CONFIGURATION_FILE_NAME); + AbfsClient abfsClient = getClient(fs); + + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, + configuration.get(FS_AZURE_ABFS_ACCOUNT_NAME)); + + // Update the configuration with reduced retry count and reduced backoff interval. + AbfsConfiguration abfsConfig + = TestAbfsConfigurationFieldsValidation.updateRetryConfigs( + abfsConfiguration, + REDUCED_RETRY_COUNT, REDUCED_BACKOFF_INTERVAL); + + intercept = Mockito.mock(AbfsThrottlingIntercept.class); + Mockito.doNothing().when(intercept).updateMetrics(Mockito.any(), Mockito.any()); + + // Gets the client. + AbfsClient testClient = Mockito.spy(ITestAbfsClient.createTestClientFromCurrentContext( + abfsClient, + abfsConfig)); + + Mockito.doReturn(intercept).when(testClient).getIntercept(); + + // Expect header is enabled or not based on the parameter. + AppendRequestParameters appendRequestParameters + = new AppendRequestParameters( + BUFFER_OFFSET, BUFFER_OFFSET, BUFFER_LENGTH, + AppendRequestParameters.Mode.APPEND_MODE, false, null, + expectHeaderEnabled); + + byte[] buffer = getRandomBytesArray(5); + + // Create a test container to upload the data. + Path testPath = path(TEST_PATH); + fs.create(testPath); + String finalTestPath = testPath.toString().substring(testPath.toString().lastIndexOf("/")); + + // Creates a list of request headers. + final List requestHeaders = ITestAbfsClient.getTestRequestHeaders(testClient); + requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); + if (appendRequestParameters.isExpectHeaderEnabled()) { + requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); + } + + // Updates the query parameters. + final AbfsUriQueryBuilder abfsUriQueryBuilder = testClient.createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, APPEND_ACTION); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(appendRequestParameters.getPosition())); + + // Creates the url for the specified path. + URL url = testClient.createRequestUrl(finalTestPath, abfsUriQueryBuilder.toString()); + + // Create a mock of the AbfsRestOperation to set the urlConnection in the corresponding httpOperation. + AbfsRestOperation op = Mockito.spy(new AbfsRestOperation( + AbfsRestOperationType.Append, + testClient, + HTTP_METHOD_PUT, + url, + requestHeaders, buffer, + appendRequestParameters.getoffset(), + appendRequestParameters.getLength(), null)); + + AbfsHttpOperation abfsHttpOperation = Mockito.spy(new AbfsHttpOperation(url, HTTP_METHOD_PUT, requestHeaders)); + + // Sets the expect request property if expect header is enabled. + if (expectHeaderEnabled) { + Mockito.doReturn(HUNDRED_CONTINUE) + .when(abfsHttpOperation) + .getConnProperty(EXPECT); + } + + HttpURLConnection urlConnection = mock(HttpURLConnection.class); + Mockito.doNothing().when(urlConnection).setRequestProperty(Mockito + .any(), Mockito.any()); + Mockito.doReturn(HTTP_METHOD_PUT).when(urlConnection).getRequestMethod(); + Mockito.doReturn(url).when(urlConnection).getURL(); + Mockito.doReturn(urlConnection).when(abfsHttpOperation).getConnection(); + + Mockito.doNothing().when(abfsHttpOperation).setRequestProperty(Mockito + .any(), Mockito.any()); + Mockito.doReturn(url).when(abfsHttpOperation).getConnUrl(); + Mockito.doReturn(HTTP_METHOD_PUT).when(abfsHttpOperation).getConnRequestMethod(); + + switch (errorType) { + case OUTPUTSTREAM: + // If the getOutputStream() throws IOException and Expect Header is + // enabled, it returns back to processResponse and hence we have + // mocked the response code and the response message to check different + // behaviour based on response code. + Mockito.doReturn(responseCode).when(abfsHttpOperation).getConnResponseCode(); + Mockito.doReturn(responseMessage) + .when(abfsHttpOperation) + .getConnResponseMessage(); + Mockito.doThrow(new ProtocolException("Server rejected Operation")) + .when(abfsHttpOperation) + .getConnOutputStream(); + break; + case WRITE: + // If write() throws IOException and Expect Header is + // enabled or not, it should throw back the exception. + OutputStream outputStream = Mockito.spy(new OutputStream() { + @Override + public void write(final int i) throws IOException { + } + }); + Mockito.doReturn(outputStream).when(abfsHttpOperation).getConnOutputStream(); + Mockito.doThrow(new IOException()) + .when(outputStream) + .write(buffer, appendRequestParameters.getoffset(), + appendRequestParameters.getLength()); + break; + default: + break; + } + + // Sets the httpOperation for the rest operation. + Mockito.doReturn(abfsHttpOperation) + .when(op) + .createHttpOperation(); + return op; + } + + void assertTraceContextState(int retryCount, int assertRetryCount, int bytesSent, int assertBytesSent, + int expectedBytesSent, int assertExpectedBytesSent) { + // Assert that the request is retried or not. + Assertions.assertThat(retryCount) + .describedAs("The retry count is incorrect") + .isEqualTo(assertRetryCount); + + // Assert that metrics will be updated correctly. + Assertions.assertThat(bytesSent) + .describedAs("The bytes sent is incorrect") + .isEqualTo(assertBytesSent); + Assertions.assertThat(expectedBytesSent) + .describedAs("The expected bytes sent is incorrect") + .isEqualTo(assertExpectedBytesSent); + } + + /** + * Test the functionalities based on whether getOutputStream() or write() + * throws exception and what is the corresponding response code. + */ + @Test + public void testExpectHundredContinue() throws Exception { + // Gets the AbfsRestOperation. + AbfsRestOperation op = getRestOperation(); + AbfsHttpOperation httpOperation = op.createHttpOperation(); + + TracingContext tracingContext = Mockito.spy(new TracingContext("abcd", + "abcde", FSOperationType.APPEND, + TracingHeaderFormat.ALL_ID_FORMAT, null)); + Mockito.doReturn(tracingContext).when(op).createNewTracingContext(Mockito.any()); + + switch (errorType) { + case WRITE: + // If write() throws IOException and Expect Header is + // enabled or not, it should throw back the exception + // which is caught and exponential retry logic comes into place. + intercept(IOException.class, + () -> op.execute(tracingContext)); + + // Asserting update of metrics and retries. + assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, httpOperation.getBytesSent(), BUFFER_LENGTH, + 0, 0); + break; + case OUTPUTSTREAM: + switch (responseCode) { + case HTTP_UNAVAILABLE: + // In the case of 503 i.e. throttled case, we should retry. + intercept(IOException.class, + () -> op.execute(tracingContext)); + + // Asserting update of metrics and retries. + assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, httpOperation.getBytesSent(), ZERO, + httpOperation.getExpectedBytesToBeSent(), BUFFER_LENGTH); + + // Verifies that update Metrics call is made for throttle case and for the first without retry + + // for the retried cases as well. + Mockito.verify(intercept, times(REDUCED_RETRY_COUNT + 1)) + .updateMetrics(Mockito.any(), Mockito.any()); + break; + case HTTP_ERROR: + // In the case of http status code 0 i.e. ErrorType case, we should retry. + intercept(IOException.class, + () -> op.execute(tracingContext)); + + // Asserting update of metrics and retries. + assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, httpOperation.getBytesSent(), + ZERO, 0, 0); + + // Verifies that update Metrics call is made for ErrorType case and for the first without retry + + // for the retried cases as well. + Mockito.verify(intercept, times(REDUCED_RETRY_COUNT + 1)) + .updateMetrics(Mockito.any(), Mockito.any()); + break; + case HTTP_NOT_FOUND: + case HTTP_EXPECTATION_FAILED: + // In the case of 4xx ErrorType. i.e. user ErrorType, retry should not happen. + intercept(AzureBlobFileSystemException.class, + () -> op.execute(tracingContext)); + + // Asserting update of metrics and retries. + assertTraceContextState(tracingContext.getRetryCount(), ZERO, 0, + 0, 0, 0); + + // Verifies that update Metrics call is not made for user ErrorType case. + Mockito.verify(intercept, never()) + .updateMetrics(Mockito.any(), Mockito.any()); + break; + default: + break; + } + break; + default: + break; + } + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestBlobOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestBlobOperation.java new file mode 100644 index 00000000000000..77072871e8cba8 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestBlobOperation.java @@ -0,0 +1,597 @@ +package org.apache.hadoop.fs.azurebfs.services; + +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; +import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; +import org.assertj.core.api.Assertions; +import org.junit.Assume; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.IOException; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Base64; +import java.util.List; +import java.util.Random; + +import static java.net.HttpURLConnection.HTTP_BAD_REQUEST; +import static java.net.HttpURLConnection.HTTP_CREATED; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK_BLOB_TYPE; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DNS_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.CONTENT_LENGTH; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.CONTENT_MD5; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_BLOB_TYPE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_COMP; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +public class ITestBlobOperation extends AbstractAbfsIntegrationTest { + private static final int BUFFER_LENGTH = 5; + private static final int BUFFER_OFFSET = 0; + private static final String TEST_PATH = "/testfile"; + AzureBlobFileSystem fs; + private final Path testPath = new Path("/testfile"); + + public ITestBlobOperation() throws Exception { + super.setup(); + fs = getFileSystem(); + PrefixMode prefixMode = getPrefixMode(fs); + Assume.assumeTrue(prefixMode == PrefixMode.BLOB); + } + + /** + * Test helper method to get random bytes array. + * + * @param length The length of byte buffer + * @return byte buffer + */ + private byte[] getRandomBytesArray(int length) { + final byte[] b = new byte[length]; + new Random().nextBytes(b); + return b; + } + + private String computeMd5(byte[] data) throws NoSuchAlgorithmException { + MessageDigest md = MessageDigest.getInstance("MD5"); + byte[] digest = md.digest(data); + + String md5Base64 = Base64.getEncoder().encodeToString(digest); + return md5Base64; + } + + /** + * Tests the putblob success scenario. + */ + @Test + public void testPutBlob() throws Exception { + // Get the filesystem. + final AzureBlobFileSystem fs = getFileSystem(); + final Configuration configuration = new Configuration(); + configuration.addResource(TEST_CONFIGURATION_FILE_NAME); + AbfsClient abfsClient = getClient(fs); + + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, + configuration.get(FS_AZURE_ABFS_ACCOUNT_NAME)); + + // Gets the client. + AbfsClient testClient = Mockito.spy(ITestAbfsClient.createTestClientFromCurrentContext( + abfsClient, + abfsConfiguration)); + + byte[] buffer = getRandomBytesArray(5); + + // Create a test container to upload the data. + Path testPath = path(TEST_PATH); + String finalTestPath = testPath.toString().substring(testPath.toString().lastIndexOf("/")); + + // Creates a list of request headers. + final List requestHeaders = ITestAbfsClient.getTestRequestHeaders(testClient); + requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, String.valueOf(buffer.length))); + requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_TYPE, BLOCK_BLOB_TYPE)); + String ContentMD5 = computeMd5(buffer); + // Updates the query parameters. + final AbfsUriQueryBuilder abfsUriQueryBuilder = testClient.createDefaultUriQueryBuilder(); + + // Creates the url for the specified path. + URL url = testClient.createRequestUrl(finalTestPath, abfsUriQueryBuilder.toString()); + /* + * Since the AbfsRestOperation object is not similar to what is there in + * client.createBlobPath (we dont send buffer in client.createBlobPath), + * the url would be replaced for `.dfs.` to .blob.` + */ + url = new URL(url.toString().replace(ABFS_DNS_PREFIX, WASB_DNS_PREFIX)); + + // Create a mock of the AbfsRestOperation to set the urlConnection in the corresponding httpOperation. + AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.PutBlob, + testClient, + HTTP_METHOD_PUT, + url, + requestHeaders, buffer, + BUFFER_OFFSET, + BUFFER_LENGTH, null); + + TracingContext tracingContext = new TracingContext("abcd", + "abcde", FSOperationType.CREATE, + TracingHeaderFormat.ALL_ID_FORMAT, null); + + op.execute(tracingContext); + + // Validate the content by comparing the md5 computed and the value obtained from server + Assertions.assertThat(op.getResult().getResponseHeader(CONTENT_MD5)) + .describedAs("The content md5 value is not correct") + .isEqualTo(ContentMD5); + Assertions.assertThat(op.getResult().getStatusCode()) + .describedAs("The creation failed") + .isEqualTo(HTTP_CREATED); + } + + private static String generateBlockListXml(List blockIds) { + StringBuilder stringBuilder = new StringBuilder(); + stringBuilder.append("\n"); + stringBuilder.append("\n"); + for (String blockId : blockIds) { + String blockId1 = Base64.getEncoder().encodeToString(blockId.getBytes()); + stringBuilder.append(String.format("%s\n", blockId1)); + } + stringBuilder.append("\n"); + return stringBuilder.toString(); + } + + @Test + public void testPutBlockWithNullData() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Configuration configuration = new Configuration(); + configuration.addResource(TEST_CONFIGURATION_FILE_NAME); + AbfsClient abfsClient = getClient(fs); + + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, + configuration.get(FS_AZURE_ABFS_ACCOUNT_NAME)); + + AbfsClient testClient = Mockito.spy(ITestAbfsClient.createTestClientFromCurrentContext( + abfsClient, + abfsConfiguration)); + + String blockId = "block1"; + byte[] data = new byte[0]; + Path testPath = path(TEST_PATH); + String finalTestPath = testPath.toString() + .substring(testPath.toString().lastIndexOf("/")); + + AbfsRestOperation[] op = new AbfsRestOperation[1]; + Mockito.doAnswer(answer -> { + AbfsRestOperation answerOp + = (AbfsRestOperation) answer.callRealMethod(); + op[0] = answerOp; + return answerOp; + }) + .when(testClient) + .getPutBlockOperation(Mockito.any(byte[].class), + Mockito.any(AppendRequestParameters.class), Mockito.anyList(), + Mockito.nullable(String.class), Mockito.any(URL.class)); + + TracingContext tracingContext = Mockito.spy(new TracingContext("abcd", + "abcde", FSOperationType.APPEND, + TracingHeaderFormat.ALL_ID_FORMAT, null)); + String blockId1 = Base64.getEncoder().encodeToString(blockId.getBytes()); + AppendRequestParameters appendRequestParameters = Mockito.mock( + AppendRequestParameters.class); + Mockito.doReturn(0).when(appendRequestParameters).getoffset(); + Mockito.doReturn(0).when(appendRequestParameters).getLength(); + intercept(IOException.class, () -> { + testClient.append(blockId1, finalTestPath, data, + appendRequestParameters, null, tracingContext, ""); + }); + } + + @Test + public void testPutBlockWithDifferentLengthBlockIds() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Configuration configuration = new Configuration(); + configuration.addResource(TEST_CONFIGURATION_FILE_NAME); + AbfsClient abfsClient = getClient(fs); + + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, + configuration.get(FS_AZURE_ABFS_ACCOUNT_NAME)); + List blockIds = new ArrayList<>(Arrays.asList( + "block-1", + "block-2122", + "block-312234" + )); + List blockData = new ArrayList<>(Arrays.asList( + "hello".getBytes(), + "world".getBytes(), + "!".getBytes() + )); + AbfsClient testClient = Mockito.spy(ITestAbfsClient.createTestClientFromCurrentContext( + abfsClient, + abfsConfiguration)); + + Path testPath = path(TEST_PATH); + String finalTestPath = testPath.toString().substring(testPath.toString().lastIndexOf("/")); + + AbfsRestOperation[] op = new AbfsRestOperation[1]; + Mockito.doAnswer(answer -> { + AbfsRestOperation answerOp + = (AbfsRestOperation) answer.callRealMethod(); + op[0] = answerOp; + return answerOp; + }) + .when(testClient) + .getPutBlockOperation(Mockito.any(byte[].class), + Mockito.any(AppendRequestParameters.class), Mockito.anyList(), + Mockito.nullable(String.class), Mockito.any(URL.class)); + TracingContext tracingContext = Mockito.spy(new TracingContext("abcd", + "abcde", FSOperationType.APPEND, + TracingHeaderFormat.ALL_ID_FORMAT, null)); + List encodedBlockIds = new ArrayList<>(); + for (int i = 0; i < blockIds.size(); i++) { + String blockId1 = Base64.getEncoder() + .encodeToString(blockIds.get(i).getBytes()); + byte[] data = blockData.get(i); + AppendRequestParameters appendRequestParameters = Mockito.mock( + AppendRequestParameters.class); + Mockito.doReturn(0).when(appendRequestParameters).getoffset(); + Mockito.doReturn(data.length).when(appendRequestParameters).getLength(); + if (i >= 1) { + intercept(IOException.class, () -> { + testClient.append(blockId1, finalTestPath, data, + appendRequestParameters, null, tracingContext, "abcd"); + encodedBlockIds.add(blockId1); + }); + Assertions.assertThat(op[0].getResult().getStatusCode()) + .describedAs("The error code is not correct") + .isEqualTo(HTTP_BAD_REQUEST); + } else { + testClient.append(blockId1, finalTestPath, data, + appendRequestParameters, null, tracingContext, "abcd"); + encodedBlockIds.add(blockId1); + } + } + } + + @Test + public void testGetCommittedBlockList() throws IOException, IllegalAccessException { + final AzureBlobFileSystem fs = getFileSystem(); + final Configuration configuration = new Configuration(); + configuration.addResource(TEST_CONFIGURATION_FILE_NAME); + AbfsClient abfsClient = getClient(fs); + + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, + configuration.get(FS_AZURE_ABFS_ACCOUNT_NAME)); + List blockIds = new ArrayList<>(Arrays.asList( + "block-1", + "block-2", + "block-3" + )); + List blockData = new ArrayList<>(Arrays.asList( + "hello".getBytes(), + "world".getBytes(), + "!".getBytes() + )); + AbfsClient testClient = Mockito.spy(ITestAbfsClient.createTestClientFromCurrentContext( + abfsClient, + abfsConfiguration)); + Path testPath = path(TEST_PATH); + String finalTestPath = testPath.toString().substring(testPath.toString().lastIndexOf("/")); + + TracingContext tracingContext = Mockito.spy(new TracingContext("abcd", + "abcde", FSOperationType.APPEND, + TracingHeaderFormat.ALL_ID_FORMAT, null)); + List encodedBlockIds = new ArrayList<>(); + for (int i = 0; i < blockIds.size(); i++) { + String blockId1 = Base64.getEncoder() + .encodeToString(blockIds.get(i).getBytes()); + byte[] data = blockData.get(i); + AppendRequestParameters appendRequestParameters = Mockito.mock( + AppendRequestParameters.class); + Mockito.doReturn(0).when(appendRequestParameters).getoffset(); + Mockito.doReturn(data.length).when(appendRequestParameters).getLength(); + testClient.append(blockId1, finalTestPath, data, + appendRequestParameters, null, tracingContext, ""); + encodedBlockIds.add(blockId1); + } + byte[] bufferString = generateBlockListXml(blockIds).getBytes( + StandardCharsets.UTF_8); + + tracingContext = Mockito.spy(new TracingContext("abcd", + "abcde", FSOperationType.APPEND, + TracingHeaderFormat.ALL_ID_FORMAT, null)); + testClient.flush(bufferString, finalTestPath, false, null, null, "", + tracingContext); + + /* Validates that all blocks are committed and fetched */ + AbfsRestOperation op1 = testClient.getBlockList(finalTestPath, tracingContext); + List committedBlockList = op1.getResult().getBlockIdList(); + assertEquals(encodedBlockIds, committedBlockList); + } + + @Test + public void testPutBlockListForAdditionalBlockId() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Configuration configuration = new Configuration(); + configuration.addResource(TEST_CONFIGURATION_FILE_NAME); + AbfsClient abfsClient = getClient(fs); + + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, + configuration.get(FS_AZURE_ABFS_ACCOUNT_NAME)); + List blockIds = new ArrayList<>(Arrays.asList( + "block-1", + "block-2", + "block-3", + "block-4" + )); + List blockData = new ArrayList<>(Arrays.asList( + "hello".getBytes(), + "world".getBytes(), + "!".getBytes() + )); + AbfsClient testClient = Mockito.spy(ITestAbfsClient.createTestClientFromCurrentContext( + abfsClient, + abfsConfiguration)); + Path testPath = path(TEST_PATH); + String finalTestPath = testPath.toString().substring(testPath.toString().lastIndexOf("/")); + final List requestHeaders = ITestAbfsClient.getTestRequestHeaders(testClient); + final AbfsUriQueryBuilder abfsUriQueryBuilder = testClient.createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCK); + + TracingContext tracingContext = Mockito.spy(new TracingContext("abcd", + "abcde", FSOperationType.APPEND, + TracingHeaderFormat.ALL_ID_FORMAT, null)); + List encodedBlockIds = new ArrayList<>(); + AbfsRestOperation op1 = null; + for (int i = 0; i < blockIds.size() - 1; i++) { + String blockId1 = Base64.getEncoder() + .encodeToString(blockIds.get(i).getBytes()); + byte[] data = blockData.get(i); + AppendRequestParameters appendRequestParameters = Mockito.mock( + AppendRequestParameters.class); + Mockito.doReturn(0).when(appendRequestParameters).getoffset(); + Mockito.doReturn(data.length).when(appendRequestParameters).getLength(); + op1 = testClient.append(blockId1, finalTestPath, data, + appendRequestParameters, null, tracingContext, ""); + encodedBlockIds.add(blockId1); + } + byte[] bufferString = generateBlockListXml(blockIds).getBytes( + StandardCharsets.UTF_8); + + final TracingContext blockListTc = Mockito.spy(new TracingContext("abcd", + "abcde", FSOperationType.APPEND, + TracingHeaderFormat.ALL_ID_FORMAT, null)); + + /* Verify that an additional blockId which is not staged if we try to commit, it throws an exception */ + AbfsRestOperation[] op = new AbfsRestOperation[1]; + Mockito.doAnswer(answer -> { + AbfsRestOperation answerOp + = (AbfsRestOperation) answer.callRealMethod(); + op[0] = answerOp; + return answerOp; + }) + .when(testClient) + .getPutBlockListOperation(Mockito.any(byte[].class), + Mockito.anyList(), Mockito.nullable(String.class), + Mockito.any(URL.class)); + AbfsRestOperation finalOp = op1; + intercept(IOException.class, () -> { + testClient.flush(bufferString, finalTestPath, false, null, null, "", + blockListTc); + }); + Assertions.assertThat(op[0].getResult().getStatusCode()) + .describedAs("The error code is not correct") + .isEqualTo(HTTP_BAD_REQUEST); + } + + /* + * Helper method that creates test data of size provided by the + * "size" parameter. + */ + private static byte[] getTestData(int size) { + byte[] testData = new byte[size]; + System.arraycopy(RandomStringUtils.randomAlphabetic(size).getBytes(), 0, testData, 0, size); + return testData; + } + + // Helper method to create file and write fileSize bytes of data on it. + private byte[] createBaseFileWithData(int fileSize, Path testPath) throws Throwable { + // To create versions + try (FSDataOutputStream createStream = fs.create(testPath)) { + } + fs.delete(testPath, false); + try (FSDataOutputStream createStream = fs.create(testPath)) { + byte[] fileData = null; + if (fileSize != 0) { + fileData = getTestData(fileSize); + createStream.write(fileData); + } + assertTrue(fs.exists(testPath)); + return fileData; + } + } + + /* + * Helper method to verify a file data equal to "dataLength" parameter + */ + private boolean verifyFileData(int dataLength, byte[] testData, int testDataIndex, + FSDataInputStream srcStream) { + try { + byte[] fileBuffer = new byte[dataLength]; + byte[] testDataBuffer = new byte[dataLength]; + int fileBytesRead = srcStream.read(fileBuffer); + if (fileBytesRead < dataLength) { + return false; + } + System.arraycopy(testData, testDataIndex, testDataBuffer, 0, dataLength); + if (!Arrays.equals(fileBuffer, testDataBuffer)) { + return false; + } + return true; + } catch (Exception ex) { + return false; + } + } + + /* + * Helper method to verify Append on a testFile. + */ + private boolean verifyAppend(byte[] testData, Path testFile) { + try (FSDataInputStream srcStream = fs.open(testFile)) { + int baseBufferSize = 2048; + int testDataSize = testData.length; + int testDataIndex = 0; + while (testDataSize > baseBufferSize) { + if (!verifyFileData(baseBufferSize, testData, testDataIndex, srcStream)) { + return false; + } + testDataIndex += baseBufferSize; + testDataSize -= baseBufferSize; + } + if (!verifyFileData(testDataSize, testData, testDataIndex, srcStream)) { + return false; + } + return true; + } catch (Exception ex) { + return false; + } + } + + /* + * Test case to verify if an append on small size data works. This tests + * append E2E + */ + @Test + public void testSingleAppend() throws Throwable { + FSDataOutputStream appendStream = null; + try { + int baseDataSize = 50; + byte[] baseDataBuffer = createBaseFileWithData(baseDataSize, testPath); + + int appendDataSize = 20; + byte[] appendDataBuffer = getTestData(appendDataSize); + appendStream = fs.append(testPath, 10); + appendStream.write(appendDataBuffer); + appendStream.close(); + byte[] testData = new byte[baseDataSize + appendDataSize]; + System.arraycopy(baseDataBuffer, 0, testData, 0, baseDataSize); + System.arraycopy(appendDataBuffer, 0, testData, baseDataSize, appendDataSize); + + assertTrue(verifyAppend(testData, testPath)); + } finally { + if (appendStream != null) { + appendStream.close(); + } + } + } + + /* + * Test case to verify append to an empty file. + */ + @Test + public void testSingleAppendOnEmptyFile() throws Throwable { + FSDataOutputStream appendStream = null; + try { + createBaseFileWithData(0, testPath); + int appendDataSize = 20; + byte[] appendDataBuffer = getTestData(appendDataSize); + appendStream = fs.append(testPath, 10); + appendStream.write(appendDataBuffer); + appendStream.close(); + assertTrue(verifyAppend(appendDataBuffer, testPath)); + } finally { + if (appendStream != null) { + appendStream.close(); + } + } + } + + /* + * Tests to verify multiple appends on a Blob. + */ + @Test + public void testMultipleAppends() throws Throwable { + int baseDataSize = 50; + byte[] baseDataBuffer = createBaseFileWithData(baseDataSize, testPath); + int appendDataSize = 100; + int targetAppendCount = 50; + byte[] testData = new byte[baseDataSize + (appendDataSize * targetAppendCount)]; + int testDataIndex = 0; + System.arraycopy(baseDataBuffer, 0, testData, testDataIndex, baseDataSize); + testDataIndex += baseDataSize; + int appendCount = 0; + FSDataOutputStream appendStream = null; + try { + while (appendCount < targetAppendCount) { + byte[] appendDataBuffer = getTestData(appendDataSize); + appendStream = fs.append(testPath, 30); + appendStream.write(appendDataBuffer); + appendStream.close(); + + System.arraycopy(appendDataBuffer, 0, testData, testDataIndex, appendDataSize); + testDataIndex += appendDataSize; + appendCount++; + } + assertTrue(verifyAppend(testData, testPath)); + } finally { + if (appendStream != null) { + appendStream.close(); + } + } + } + + /* + * Test to verify we multiple appends on the same stream. + */ + @Test + public void testMultipleAppendsOnSameStream() throws Throwable { + int baseDataSize = 50; + byte[] baseDataBuffer = createBaseFileWithData(baseDataSize, testPath); + int appendDataSize = 100; + int targetAppendCount = 50; + byte[] testData = new byte[baseDataSize + (appendDataSize * targetAppendCount)]; + int testDataIndex = 0; + System.arraycopy(baseDataBuffer, 0, testData, testDataIndex, baseDataSize); + testDataIndex += baseDataSize; + int appendCount = 0; + FSDataOutputStream appendStream = null; + try { + while (appendCount < targetAppendCount) { + appendStream = fs.append(testPath, 50); + int singleAppendChunkSize = 20; + int appendRunSize = 0; + while (appendRunSize < appendDataSize) { + byte[] appendDataBuffer = getTestData(singleAppendChunkSize); + appendStream.write(appendDataBuffer); + System.arraycopy(appendDataBuffer, 0, testData, + testDataIndex + appendRunSize, singleAppendChunkSize); + + appendRunSize += singleAppendChunkSize; + } + appendStream.close(); + testDataIndex += appendDataSize; + appendCount++; + } + assertTrue(verifyAppend(testData, testPath)); + } finally { + if (appendStream != null) { + appendStream.close(); + } + } + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java index 705cc2530d335d..a57430fa808cc8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java @@ -25,6 +25,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -43,9 +44,24 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; +import static org.apache.hadoop.fs.azurebfs.constants.InternalConstants.CAPABILITY_SAFE_READAHEAD; +import static org.apache.hadoop.test.LambdaTestUtils.eventually; public class ITestReadBufferManager extends AbstractAbfsIntegrationTest { + /** + * Time before the JUnit test times out for eventually() clauses + * to fail. This copes with slow network connections and debugging + * sessions, yet still allows for tests to fail with meaningful + * messages. + */ + public static final int TIMEOUT_OFFSET = 5 * 60_000; + + /** + * Interval between eventually preobes. + */ + public static final int PROBE_INTERVAL_MILLIS = 1_000; + public ITestReadBufferManager() throws Exception { } @@ -60,6 +76,11 @@ public void testPurgeBufferManagerForParallelStreams() throws Exception { } ExecutorService executorService = Executors.newFixedThreadPool(4); AzureBlobFileSystem fs = getABFSWithReadAheadConfig(); + // verify that the fs has the capability to validate the fix + Assertions.assertThat(fs.hasPathCapability(new Path("/"), CAPABILITY_SAFE_READAHEAD)) + .describedAs("path capability %s in %s", CAPABILITY_SAFE_READAHEAD, fs) + .isTrue(); + try { for (int i = 0; i < 4; i++) { final String fileName = methodName.getMethodName() + i; @@ -74,17 +95,16 @@ public void testPurgeBufferManagerForParallelStreams() throws Exception { } } finally { executorService.shutdown(); + // wait for all tasks to finish + executorService.awaitTermination(1, TimeUnit.MINUTES); } ReadBufferManager bufferManager = ReadBufferManager.getBufferManager(); - assertListEmpty("CompletedList", bufferManager.getCompletedReadListCopy()); - assertListEmpty("InProgressList", bufferManager.getInProgressCopiedList()); + // readahead queue is empty assertListEmpty("ReadAheadQueue", bufferManager.getReadAheadQueueCopy()); - Assertions.assertThat(bufferManager.getFreeListCopy()) - .describedAs("After closing all streams free list contents should match with " + freeList) - .hasSize(numBuffers) - .containsExactlyInAnyOrderElementsOf(freeList); - + // verify the in progress list eventually empties out. + eventually(getTestTimeoutMillis() - TIMEOUT_OFFSET, PROBE_INTERVAL_MILLIS, () -> + assertListEmpty("InProgressList", bufferManager.getInProgressCopiedList())); } private void assertListEmpty(String listName, List list) { @@ -116,22 +136,18 @@ public void testPurgeBufferManagerForSequentialStream() throws Exception { try { iStream2 = (AbfsInputStream) fs.open(testFilePath).getWrappedStream(); iStream2.read(); - // After closing stream1, none of the buffers associated with stream1 should be present. - assertListDoesnotContainBuffersForIstream(bufferManager.getInProgressCopiedList(), iStream1); - assertListDoesnotContainBuffersForIstream(bufferManager.getCompletedReadListCopy(), iStream1); + // After closing stream1, no queued buffers of stream1 should be present + // assertions can't be made about the state of the other lists as it is + // too prone to race conditions. assertListDoesnotContainBuffersForIstream(bufferManager.getReadAheadQueueCopy(), iStream1); } finally { // closing the stream later. IOUtils.closeStream(iStream2); } - // After closing stream2, none of the buffers associated with stream2 should be present. - assertListDoesnotContainBuffersForIstream(bufferManager.getInProgressCopiedList(), iStream2); - assertListDoesnotContainBuffersForIstream(bufferManager.getCompletedReadListCopy(), iStream2); + // After closing stream2, no queued buffers of stream2 should be present. assertListDoesnotContainBuffersForIstream(bufferManager.getReadAheadQueueCopy(), iStream2); - // After closing both the streams, all lists should be empty. - assertListEmpty("CompletedList", bufferManager.getCompletedReadListCopy()); - assertListEmpty("InProgressList", bufferManager.getInProgressCopiedList()); + // After closing both the streams, read queue should be empty. assertListEmpty("ReadAheadQueue", bufferManager.getReadAheadQueueCopy()); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClientThrottlingAnalyzer.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClientThrottlingAnalyzer.java index 3f680e499300dd..bf3bd203a62ed9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClientThrottlingAnalyzer.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClientThrottlingAnalyzer.java @@ -18,9 +18,15 @@ package org.apache.hadoop.fs.azurebfs.services; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.junit.Test; +import java.io.IOException; + +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ANALYSIS_PERIOD; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -33,6 +39,15 @@ public class TestAbfsClientThrottlingAnalyzer { + ANALYSIS_PERIOD / 10; private static final long MEGABYTE = 1024 * 1024; private static final int MAX_ACCEPTABLE_PERCENT_DIFFERENCE = 20; + private AbfsConfiguration abfsConfiguration; + + public TestAbfsClientThrottlingAnalyzer() throws IOException, IllegalAccessException { + final Configuration configuration = new Configuration(); + configuration.addResource(TEST_CONFIGURATION_FILE_NAME); + configuration.setInt(FS_AZURE_ANALYSIS_PERIOD, 1000); + this.abfsConfiguration = new AbfsConfiguration(configuration, + "dummy"); + } private void sleep(long milliseconds) { try { @@ -82,8 +97,7 @@ private void validateLessThanOrEqual(long maxExpected, long actual) { @Test public void testNoMetricUpdatesThenNoWaiting() { AbfsClientThrottlingAnalyzer analyzer = new AbfsClientThrottlingAnalyzer( - "test", - ANALYSIS_PERIOD); + "test", abfsConfiguration); validate(0, analyzer.getSleepDuration()); sleep(ANALYSIS_PERIOD_PLUS_10_PERCENT); validate(0, analyzer.getSleepDuration()); @@ -96,8 +110,7 @@ public void testNoMetricUpdatesThenNoWaiting() { @Test public void testOnlySuccessThenNoWaiting() { AbfsClientThrottlingAnalyzer analyzer = new AbfsClientThrottlingAnalyzer( - "test", - ANALYSIS_PERIOD); + "test", abfsConfiguration); analyzer.addBytesTransferred(8 * MEGABYTE, false); validate(0, analyzer.getSleepDuration()); sleep(ANALYSIS_PERIOD_PLUS_10_PERCENT); @@ -112,8 +125,7 @@ public void testOnlySuccessThenNoWaiting() { @Test public void testOnlyErrorsAndWaiting() { AbfsClientThrottlingAnalyzer analyzer = new AbfsClientThrottlingAnalyzer( - "test", - ANALYSIS_PERIOD); + "test", abfsConfiguration); validate(0, analyzer.getSleepDuration()); analyzer.addBytesTransferred(4 * MEGABYTE, true); sleep(ANALYSIS_PERIOD_PLUS_10_PERCENT); @@ -132,8 +144,7 @@ public void testOnlyErrorsAndWaiting() { @Test public void testSuccessAndErrorsAndWaiting() { AbfsClientThrottlingAnalyzer analyzer = new AbfsClientThrottlingAnalyzer( - "test", - ANALYSIS_PERIOD); + "test", abfsConfiguration); validate(0, analyzer.getSleepDuration()); analyzer.addBytesTransferred(8 * MEGABYTE, false); analyzer.addBytesTransferred(2 * MEGABYTE, true); @@ -157,8 +168,7 @@ public void testSuccessAndErrorsAndWaiting() { @Test public void testManySuccessAndErrorsAndWaiting() { AbfsClientThrottlingAnalyzer analyzer = new AbfsClientThrottlingAnalyzer( - "test", - ANALYSIS_PERIOD); + "test", abfsConfiguration); validate(0, analyzer.getSleepDuration()); final int numberOfRequests = 20; for (int i = 0; i < numberOfRequests; i++) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsHttpOperation.java index 36914a4e4f3658..6806cd955e86e5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsHttpOperation.java @@ -18,16 +18,23 @@ package org.apache.hadoop.fs.azurebfs.services; +import java.io.ByteArrayInputStream; import java.io.UnsupportedEncodingException; import java.net.MalformedURLException; import java.net.URL; import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import org.assertj.core.api.Assertions; import org.junit.Test; +import org.mockito.Mockito; import org.apache.hadoop.fs.azurebfs.utils.UriUtils; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_POST; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; + public class TestAbfsHttpOperation { @Test @@ -96,6 +103,77 @@ public void testUrlWithNullValues() "http://www.testurl.net?abc=xyz&pqr=&mnop="); } + @Test + public void testParseStorageErrorStreamBlob() throws Exception { + AbfsHttpOperation op = Mockito.spy(new AbfsHttpOperation( + new URL("https://account.blob.core.windows.net/container/path"), + HTTP_METHOD_PUT, new ArrayList<>())); + String xmlString = "\n" + + "\n" + + " string-value-code\n" + + " string-value-message\n" + + ""; + ByteArrayInputStream inputStream = new ByteArrayInputStream( + xmlString.getBytes( + StandardCharsets.UTF_8)); + Mockito.doReturn(inputStream).when(op).getConnectionErrorStream(); + op.processServerErrorResponse(); + Assertions.assertThat(op.getStorageErrorCode()) + .isEqualTo("string-value-code"); + Assertions.assertThat(op.getStorageErrorMessage()) + .isEqualTo("string-value-message"); + + op = Mockito.spy(new AbfsHttpOperation( + new URL("https://account.blob.core.windows.net/container/path"), + HTTP_METHOD_PUT, new ArrayList<>())); + xmlString = "\n" + + "\n" + + " string-value-code\n" + + " string-value-message\n" + + ""; + inputStream = new ByteArrayInputStream(xmlString.getBytes( + StandardCharsets.UTF_8)); + Mockito.doReturn(inputStream).when(op).getConnectionErrorStream(); + op.processServerErrorResponse(); + Assertions.assertThat(op.getStorageErrorCode()).isEmpty(); + Assertions.assertThat(op.getStorageErrorMessage()) + .isEqualTo("string-value-message"); + + op = Mockito.spy(new AbfsHttpOperation( + new URL("https://account.blob.core.windows.net/container/path"), + HTTP_METHOD_PUT, new ArrayList<>())); + xmlString = "\n" + + "\n" + + " string-value-code\n" + + " string-value-message\n" + + ""; + inputStream = new ByteArrayInputStream(xmlString.getBytes( + StandardCharsets.UTF_8)); + Mockito.doReturn(inputStream).when(op).getConnectionErrorStream(); + op.processServerErrorResponse(); + Assertions.assertThat(op.getStorageErrorCode()) + .isEqualTo("string-value-code"); + Assertions.assertThat(op.getStorageErrorMessage()).isEmpty(); + } + + @Test + public void testParseStorageErrorStreamDfs() throws Exception { + AbfsHttpOperation op = Mockito.spy(new AbfsHttpOperation( + new URL("https://account.dfs.core.windows.net/container/path"), + HTTP_METHOD_PUT, new ArrayList<>())); + String xmlString + = "{\"error\":{\"code\":\"errorCode\", \"message\":\"errorMessage\"}}"; + ByteArrayInputStream inputStream = new ByteArrayInputStream( + xmlString.getBytes( + StandardCharsets.UTF_8)); + Mockito.doReturn(inputStream).when(op).getConnectionErrorStream(); + op.processServerErrorResponse(); + Assertions.assertThat(op.getStorageErrorCode()) + .isEqualTo("errorCode"); + Assertions.assertThat(op.getStorageErrorMessage()) + .isEqualTo("errorMessage"); + } + private void testIfMaskAndEncodeSuccessful(final String scenario, final String url, final String expectedMaskedUrl) throws UnsupportedEncodingException, MalformedURLException { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index 5e73d8424ba8b3..adf2b6676ca20c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -19,31 +19,40 @@ package org.apache.hadoop.fs.azurebfs.services; import java.io.IOException; - -import org.junit.Assert; -import org.junit.Test; import java.util.Arrays; +import java.util.Optional; +import java.util.Random; +import java.util.concurrent.ExecutionException; import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FutureDataInputStreamBuilder; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TimeoutException; import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; import org.apache.hadoop.fs.azurebfs.utils.TestCachedSASToken; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.impl.OpenFileParameters; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -73,6 +82,12 @@ public class TestAbfsInputStream extends REDUCED_READ_BUFFER_AGE_THRESHOLD * 10; // 30 sec private static final int ALWAYS_READ_BUFFER_SIZE_TEST_FILE_SIZE = 16 * ONE_MB; + @Override + public void teardown() throws Exception { + super.teardown(); + ReadBufferManager.getBufferManager().testResetReadBufferManager(); + } + private AbfsRestOperation getMockRestOp() { AbfsRestOperation op = mock(AbfsRestOperation.class); AbfsHttpOperation httpOp = mock(AbfsHttpOperation.class); @@ -192,6 +207,108 @@ public TestAbfsInputStream() throws Exception { ReadBufferManager.getBufferManager().setThresholdAgeMilliseconds(REDUCED_READ_BUFFER_AGE_THRESHOLD); } + private void writeBufferToNewFile(Path testFile, byte[] buffer) throws IOException { + AzureBlobFileSystem fs = getFileSystem(); + fs.create(testFile); + FSDataOutputStream out = fs.append(testFile); + out.write(buffer); + out.close(); + } + + private void verifyOpenWithProvidedStatus(Path path, FileStatus fileStatus, + byte[] buf, AbfsRestOperationType source) + throws IOException, ExecutionException, InterruptedException { + byte[] readBuf = new byte[buf.length]; + AzureBlobFileSystem fs = getFileSystem(); + FutureDataInputStreamBuilder builder = fs.openFile(path); + builder.withFileStatus(fileStatus); + FSDataInputStream in = builder.build().get(); + assertEquals(String.format( + "Open with fileStatus [from %s result]: Incorrect number of bytes read", + source), buf.length, in.read(readBuf)); + assertArrayEquals(String + .format("Open with fileStatus [from %s result]: Incorrect read data", + source), readBuf, buf); + } + + private void checkGetPathStatusCalls(Path testFile, FileStatus fileStatus, + AzureBlobFileSystemStore abfsStore, AbfsClient mockClient, + AbfsRestOperationType source, TracingContext tracingContext) + throws IOException { + + // verify GetPathStatus not invoked when FileStatus is provided + abfsStore.openFileForRead(testFile, Optional + .ofNullable(new OpenFileParameters().withStatus(fileStatus)), null, tracingContext); + verify(mockClient, times(0).description((String.format( + "FileStatus [from %s result] provided, GetFileStatus should not be invoked", + source)))).getPathStatus(anyString(), anyBoolean(), any(TracingContext.class)); + + // verify GetPathStatus invoked when FileStatus not provided + abfsStore.openFileForRead(testFile, + Optional.empty(), null, + tracingContext); + if (getPrefixMode(getFileSystem()) == PrefixMode.DFS) { + verify(mockClient, times(1).description( + "GetPathStatus should be invoked when FileStatus not provided")) + .getPathStatus(anyString(), anyBoolean(), any(TracingContext.class)); + } + + Mockito.reset(mockClient); //clears invocation count for next test case + } + + @Test + public void testOpenFileWithOptions() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + String testFolder = "/testFolder"; + Path smallTestFile = new Path(testFolder + "/testFile0"); + Path largeTestFile = new Path(testFolder + "/testFile1"); + fs.mkdirs(new Path(testFolder)); + int readBufferSize = getConfiguration().getReadBufferSize(); + byte[] smallBuffer = new byte[5]; + byte[] largeBuffer = new byte[readBufferSize + 5]; + new Random().nextBytes(smallBuffer); + new Random().nextBytes(largeBuffer); + writeBufferToNewFile(smallTestFile, smallBuffer); + writeBufferToNewFile(largeTestFile, largeBuffer); + + FileStatus[] getFileStatusResults = {fs.getFileStatus(smallTestFile), + fs.getFileStatus(largeTestFile)}; + FileStatus[] listStatusResults = fs.listStatus(new Path(testFolder)); + + // open with fileStatus from GetPathStatus + verifyOpenWithProvidedStatus(smallTestFile, getFileStatusResults[0], + smallBuffer, AbfsRestOperationType.GetPathStatus); + verifyOpenWithProvidedStatus(largeTestFile, getFileStatusResults[1], + largeBuffer, AbfsRestOperationType.GetPathStatus); + + // open with fileStatus from ListStatus + verifyOpenWithProvidedStatus(smallTestFile, listStatusResults[0], smallBuffer, + AbfsRestOperationType.ListPaths); + verifyOpenWithProvidedStatus(largeTestFile, listStatusResults[1], largeBuffer, + AbfsRestOperationType.ListPaths); + + // verify number of GetPathStatus invocations + AzureBlobFileSystemStore abfsStore = getAbfsStore(fs); + AbfsClient mockClient = spy(getClient(fs)); + setAbfsClient(abfsStore, mockClient); + TracingContext tracingContext = getTestTracingContext(fs, false); + checkGetPathStatusCalls(smallTestFile, getFileStatusResults[0], + abfsStore, mockClient, AbfsRestOperationType.GetPathStatus, tracingContext); + checkGetPathStatusCalls(largeTestFile, getFileStatusResults[1], + abfsStore, mockClient, AbfsRestOperationType.GetPathStatus, tracingContext); + checkGetPathStatusCalls(smallTestFile, listStatusResults[0], + abfsStore, mockClient, AbfsRestOperationType.ListPaths, tracingContext); + checkGetPathStatusCalls(largeTestFile, listStatusResults[1], + abfsStore, mockClient, AbfsRestOperationType.ListPaths, tracingContext); + + // Verify with incorrect filestatus + getFileStatusResults[0].setPath(new Path("wrongPath")); + intercept(ExecutionException.class, + () -> verifyOpenWithProvidedStatus(smallTestFile, + getFileStatusResults[0], smallBuffer, + AbfsRestOperationType.GetPathStatus)); + } + /** * This test expects AbfsInputStream to throw the exception that readAhead * thread received on read. The readAhead thread must be initiated from the @@ -384,6 +501,69 @@ public void testSuccessfulReadAhead() throws Exception { checkEvictedStatus(inputStream, 0, true); } + /** + * This test expects InProgressList is not purged by the inputStream close. + */ + @Test + public void testStreamPurgeDuringReadAheadCallExecuting() throws Exception { + AbfsClient client = getMockAbfsClient(); + AbfsRestOperation successOp = getMockRestOp(); + final Long serverCommunicationMockLatency = 3_000L; + final Long readBufferTransferToInProgressProbableTime = 1_000L; + final Integer readBufferQueuedCount = 3; + + Mockito.doAnswer(invocationOnMock -> { + //sleeping thread to mock the network latency from client to backend. + Thread.sleep(serverCommunicationMockLatency); + return successOp; + }) + .when(client) + .read(any(String.class), any(Long.class), any(byte[].class), + any(Integer.class), any(Integer.class), any(String.class), + any(String.class), any(TracingContext.class)); + + final ReadBufferManager readBufferManager + = ReadBufferManager.getBufferManager(); + + final int readBufferTotal = readBufferManager.getNumBuffers(); + final int expectedFreeListBufferCount = readBufferTotal + - readBufferQueuedCount; + + try (AbfsInputStream inputStream = getAbfsInputStream(client, + "testSuccessfulReadAhead.txt")) { + // As this is try-with-resources block, the close() method of the created + // abfsInputStream object shall be called on the end of the block. + queueReadAheads(inputStream); + + //Sleeping to give ReadBufferWorker to pick the readBuffers for processing. + Thread.sleep(readBufferTransferToInProgressProbableTime); + + Assertions.assertThat(readBufferManager.getInProgressCopiedList()) + .describedAs(String.format("InProgressList should have %d elements", + readBufferQueuedCount)) + .hasSize(readBufferQueuedCount); + Assertions.assertThat(readBufferManager.getFreeListCopy()) + .describedAs(String.format("FreeList should have %d elements", + expectedFreeListBufferCount)) + .hasSize(expectedFreeListBufferCount); + Assertions.assertThat(readBufferManager.getCompletedReadListCopy()) + .describedAs("CompletedList should have 0 elements") + .hasSize(0); + } + + Assertions.assertThat(readBufferManager.getInProgressCopiedList()) + .describedAs(String.format("InProgressList should have %d elements", + readBufferQueuedCount)) + .hasSize(readBufferQueuedCount); + Assertions.assertThat(readBufferManager.getFreeListCopy()) + .describedAs(String.format("FreeList should have %d elements", + expectedFreeListBufferCount)) + .hasSize(expectedFreeListBufferCount); + Assertions.assertThat(readBufferManager.getCompletedReadListCopy()) + .describedAs("CompletedList should have 0 elements") + .hasSize(0); + } + /** * This test expects ReadAheadManager to throw exception if the read ahead * thread had failed within the last thresholdAgeMilliseconds. diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index 0673e387bfbf0b..5c4f24362381b6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -72,6 +72,7 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext( boolean isFlushEnabled, boolean disableOutputStreamFlush, boolean isAppendBlob, + boolean isExpectHeaderEnabled, AbfsClient client, String path, TracingContext tracingContext, @@ -89,6 +90,7 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext( return new AbfsOutputStreamContext(2) .withWriteBufferSize(writeBufferSize) + .enableExpectHeader(isExpectHeaderEnabled) .enableFlush(isFlushEnabled) .disableOutputStreamFlush(disableOutputStreamFlush) .withStreamStatistics(new AbfsOutputStreamStatisticsImpl()) @@ -103,6 +105,15 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext( .build(); } + private AbfsConfiguration getConf() throws IOException, IllegalAccessException { + AbfsConfiguration abfsConf; + final Configuration conf = new Configuration(); + conf.set(accountKey1, accountValue1); + abfsConf = new AbfsConfiguration(conf, accountName1); + abfsConf.setPrefixMode(PrefixMode.DFS); + return abfsConf; + } + /** * The test verifies OutputStream shortwrite case(2000bytes write followed by flush, hflush, hsync) is making correct HTTP calls to the server */ @@ -111,11 +122,10 @@ public void verifyShortWriteRequest() throws Exception { AbfsClient client = mock(AbfsClient.class); AbfsRestOperation op = mock(AbfsRestOperation.class); - AbfsConfiguration abfsConf; - final Configuration conf = new Configuration(); - conf.set(accountKey1, accountValue1); - abfsConf = new AbfsConfiguration(conf, accountName1); + AbfsConfiguration abfsConf = getConf(); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); + + when(client.getAbfsConfiguration()).thenReturn(abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(TracingContext.class))) @@ -129,6 +139,7 @@ public void verifyShortWriteRequest() throws Exception { true, false, false, + true, client, PATH, new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", @@ -149,9 +160,9 @@ public void verifyShortWriteRequest() throws Exception { out.hsync(); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, WRITE_SIZE, APPEND_MODE, false, null); + 0, 0, WRITE_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - WRITE_SIZE, 0, 2 * WRITE_SIZE, APPEND_MODE, false, null); + WRITE_SIZE, 0, 2 * WRITE_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), @@ -171,15 +182,13 @@ public void verifyWriteRequest() throws Exception { AbfsClient client = mock(AbfsClient.class); AbfsRestOperation op = mock(AbfsRestOperation.class); - AbfsConfiguration abfsConf; - final Configuration conf = new Configuration(); - conf.set(accountKey1, accountValue1); - abfsConf = new AbfsConfiguration(conf, accountName1); + AbfsConfiguration abfsConf = getConf(); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext("test-corr-id", "test-fs-id", FSOperationType.WRITE, TracingHeaderFormat.ALL_ID_FORMAT, null); + when(client.getAbfsConfiguration()).thenReturn(abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(TracingContext.class))).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull(), any(TracingContext.class))).thenReturn(op); @@ -190,6 +199,7 @@ public void verifyWriteRequest() throws Exception { true, false, false, + true, client, PATH, tracingContext, @@ -203,9 +213,9 @@ public void verifyWriteRequest() throws Exception { out.close(); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, 5*WRITE_SIZE-BUFFER_SIZE, APPEND_MODE, false, null); + BUFFER_SIZE, 0, 5*WRITE_SIZE-BUFFER_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), @@ -243,15 +253,14 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { AbfsClient client = mock(AbfsClient.class); AbfsRestOperation op = mock(AbfsRestOperation.class); AbfsHttpOperation httpOp = mock(AbfsHttpOperation.class); - AbfsConfiguration abfsConf; - final Configuration conf = new Configuration(); - conf.set(accountKey1, accountValue1); - abfsConf = new AbfsConfiguration(conf, accountName1); + AbfsConfiguration abfsConf = getConf(); + AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext( abfsConf.getClientCorrelationId(), "test-fs-id", FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(), null); + when(client.getAbfsConfiguration()).thenReturn(abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(TracingContext.class))).thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull(), any(TracingContext.class))).thenReturn(op); @@ -264,6 +273,7 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { true, false, false, + true, client, PATH, tracingContext, @@ -277,9 +287,9 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { out.close(); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); @@ -314,12 +324,11 @@ public void verifyWriteRequestOfBufferSize() throws Exception { AbfsClient client = mock(AbfsClient.class); AbfsRestOperation op = mock(AbfsRestOperation.class); AbfsHttpOperation httpOp = mock(AbfsHttpOperation.class); - AbfsConfiguration abfsConf; - final Configuration conf = new Configuration(); - conf.set(accountKey1, accountValue1); - abfsConf = new AbfsConfiguration(conf, accountName1); + AbfsConfiguration abfsConf = getConf(); + AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); + when(client.getAbfsConfiguration()).thenReturn(abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(TracingContext.class))) @@ -335,6 +344,7 @@ public void verifyWriteRequestOfBufferSize() throws Exception { true, false, false, + true, client, PATH, new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", @@ -350,9 +360,9 @@ public void verifyWriteRequestOfBufferSize() throws Exception { Thread.sleep(1000); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); @@ -371,12 +381,10 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { AbfsClient client = mock(AbfsClient.class); AbfsRestOperation op = mock(AbfsRestOperation.class); - AbfsConfiguration abfsConf; - final Configuration conf = new Configuration(); - conf.set(accountKey1, accountValue1); - abfsConf = new AbfsConfiguration(conf, accountName1); + AbfsConfiguration abfsConf = getConf(); AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); + when(client.getAbfsConfiguration()).thenReturn(abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(TracingContext.class))) @@ -390,6 +398,7 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { true, false, true, + true, client, PATH, new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", @@ -405,9 +414,9 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { Thread.sleep(1000); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, true, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, true, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, true, null); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, true, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); @@ -426,16 +435,18 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { AbfsClient client = mock(AbfsClient.class); AbfsRestOperation op = mock(AbfsRestOperation.class); + AbfsHttpOperation abfsHttpOperation = mock(AbfsHttpOperation.class); when(op.getSasToken()).thenReturn(""); - AbfsConfiguration abfsConf; - final Configuration conf = new Configuration(); - conf.set(accountKey1, accountValue1); - abfsConf = new AbfsConfiguration(conf, accountName1); + AbfsConfiguration abfsConf = getConf(); + AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); TracingContext tracingContext = new TracingContext( abfsConf.getClientCorrelationId(), "test-fs-id", FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(), null); + when(op.getResult()).thenReturn(abfsHttpOperation); + + when(client.getAbfsConfiguration()).thenReturn(abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(TracingContext.class))) @@ -449,6 +460,7 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { true, false, false, + true, client, PATH, new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", @@ -464,9 +476,9 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { out.hflush(); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); @@ -500,11 +512,10 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { AbfsClient client = mock(AbfsClient.class); AbfsRestOperation op = mock(AbfsRestOperation.class); - AbfsConfiguration abfsConf; - final Configuration conf = new Configuration(); - conf.set(accountKey1, accountValue1); - abfsConf = new AbfsConfiguration(conf, accountName1); + AbfsConfiguration abfsConf = getConf(); + AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); + when(client.getAbfsConfiguration()).thenReturn(abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(TracingContext.class))) @@ -518,6 +529,7 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { true, false, false, + true, client, PATH, new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", @@ -535,9 +547,9 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { Thread.sleep(1000); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStreamBlob.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStreamBlob.java new file mode 100644 index 00000000000000..e6ea4e0d43b9fd --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStreamBlob.java @@ -0,0 +1,439 @@ +/** + * 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.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.Random; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; + +import org.junit.Test; + +import org.mockito.ArgumentCaptor; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; +import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.store.DataBlocks; +import org.apache.hadoop.util.BlockingThreadPoolExecutorService; +import org.apache.hadoop.util.SemaphoredDelegatingExecutor; +import org.mockito.Mockito; + +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.DATA_BLOCKS_BUFFER; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BLOCK_UPLOAD_BUFFER_DIR; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_UPLOAD_ACTIVE_BLOCKS_DEFAULT; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DATA_BLOCKS_BUFFER_DEFAULT; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.isNull; +import static org.mockito.ArgumentMatchers.refEq; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode.APPEND_MODE; +import static org.mockito.Mockito.*; + +public final class TestAbfsOutputStreamBlob { + + private static final int BUFFER_SIZE = 4096; + private static final int WRITE_SIZE = 1000; + private static final String PATH = "~/testpath"; + private final String globalKey = "fs.azure.configuration"; + private final String accountName1 = "account1"; + private final String accountKey1 = globalKey + "." + accountName1; + private final String accountValue1 = "one"; + + private AbfsOutputStreamContext populateAbfsOutputStreamContext( + int writeBufferSize, + boolean isFlushEnabled, + boolean disableOutputStreamFlush, + boolean isAppendBlob, + boolean isExpectHeaderEnabled, + AbfsClient client, + String path, + TracingContext tracingContext, + ExecutorService executorService) throws IOException, + IllegalAccessException { + AbfsConfiguration abfsConf = new AbfsConfiguration(new Configuration(), + accountName1); + String blockFactoryName = + abfsConf.getRawConfiguration().getTrimmed(DATA_BLOCKS_BUFFER, + DATA_BLOCKS_BUFFER_DEFAULT); + DataBlocks.BlockFactory blockFactory = + DataBlocks.createFactory(FS_AZURE_BLOCK_UPLOAD_BUFFER_DIR, + abfsConf.getRawConfiguration(), + blockFactoryName); + + return new AbfsOutputStreamContext(2) + .withWriteBufferSize(writeBufferSize) + .enableExpectHeader(isExpectHeaderEnabled) + .enableFlush(isFlushEnabled) + .disableOutputStreamFlush(disableOutputStreamFlush) + .withStreamStatistics(new AbfsOutputStreamStatisticsImpl()) + .withAppendBlob(isAppendBlob) + .withWriteMaxConcurrentRequestCount(abfsConf.getWriteMaxConcurrentRequestCount()) + .withMaxWriteRequestsToQueue(abfsConf.getMaxWriteRequestsToQueue()) + .withClient(client) + .withPath(path) + .withTracingContext(tracingContext) + .withExecutorService(executorService) + .withBlockFactory(blockFactory) + .build(); + } + + public AbfsConfiguration getConf() throws IOException, IllegalAccessException { + AbfsConfiguration abfsConf; + final Configuration conf = new Configuration(); + conf.set(accountKey1, accountValue1); + abfsConf = new AbfsConfiguration(conf, accountName1); + abfsConf.setPrefixMode(PrefixMode.BLOB); + return abfsConf; + } + + public AbfsClient getClient() throws IOException, IllegalAccessException { + AbfsClient client = mock(AbfsClient.class); + AbfsRestOperation op = mock(AbfsRestOperation.class); + when(op.getSasToken()).thenReturn("abcd"); + AbfsHttpOperation httpOperation = mock(AbfsHttpOperation.class); + + when(op.getResult()).thenReturn(httpOperation); + when(op.getResult().getBlockIdList()).thenReturn(new ArrayList()); + when(client.getBlockList(anyString(), any())).thenReturn(op); + AbfsConfiguration abfsConf = getConf(); + AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); + TracingContext tracingContext = new TracingContext( + abfsConf.getClientCorrelationId(), "test-fs-id", + FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(), null); + + when(client.getAbfsConfiguration()).thenReturn(abfsConf); + when(client.getAbfsPerfTracker()).thenReturn(tracker); + when(client.append(anyString(), anyString(), any(byte[].class), + any(AppendRequestParameters.class), any(), any(TracingContext.class), any())) + .thenReturn(op); + when(client.flush(any(byte[].class), anyString(), anyBoolean(), isNull(), isNull(), any(), + any(TracingContext.class))).thenReturn(op); + return client; + } + + public AbfsOutputStream getOutputStream(AbfsClient client, AbfsConfiguration abfsConf) throws IOException, IllegalAccessException { + AbfsOutputStream out = new AbfsOutputStream( + populateAbfsOutputStreamContext( + BUFFER_SIZE, + true, + false, + false, + true, + client, + PATH, + new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", + FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(), + null), + createExecutorService(abfsConf))); + return out; + } + + + /** + * The test verifies OutputStream shortwrite case(2000bytes write followed by flush, hflush, hsync) is making correct HTTP calls to the server + */ + @Test + public void verifyShortWriteRequest() throws Exception { + AbfsClient client = getClient(); + AbfsOutputStream out = getOutputStream(client, getConf()); + + final byte[] b = new byte[WRITE_SIZE]; + new Random().nextBytes(b); + out.write(b); + out.hsync(); + + final byte[] b1 = new byte[2 * WRITE_SIZE]; + new Random().nextBytes(b1); + out.write(b1); + out.flush(); + out.hflush(); + + out.hsync(); + + AppendRequestParameters firstReqParameters = new AppendRequestParameters( + 0, 0, WRITE_SIZE, APPEND_MODE, false, null, true); + AppendRequestParameters secondReqParameters = new AppendRequestParameters( + WRITE_SIZE, 0, 2 * WRITE_SIZE, APPEND_MODE, false, null, true); + + verify(client, times(1)).append(any(), + eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), + any(TracingContext.class), any()); + verify(client, times(1)).append(any(), + eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class), any()); + // confirm there were only 2 invocations in all + verify(client, times(2)).append(any(), + eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class), any()); + + } + + /** + * The test verifies OutputStream Write of WRITE_SIZE(1000 bytes) followed by a close is making correct HTTP calls to the server + */ + @Test + public void verifyWriteRequest() throws Exception { + AbfsClient client = getClient(); + AbfsOutputStream out = getOutputStream(client, getConf()); + + final byte[] b = new byte[WRITE_SIZE]; + new Random().nextBytes(b); + + for (int i = 0; i < 5; i++) { + out.write(b); + } + out.close(); + + AppendRequestParameters firstReqParameters = new AppendRequestParameters( + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); + AppendRequestParameters secondReqParameters = new AppendRequestParameters( + BUFFER_SIZE, 0, 5 * WRITE_SIZE - BUFFER_SIZE, APPEND_MODE, false, null, true); + + verify(client, times(1)).append(any(), + eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), + any(TracingContext.class), any()); + verify(client, times(1)).append(any(), + eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), + any(TracingContext.class), any()); + // confirm there were only 2 invocations in all + verify(client, times(2)).append(any(), + eq(PATH), any(byte[].class), any(), any(), + any(TracingContext.class), any()); + + ArgumentCaptor acByte = ArgumentCaptor.forClass(byte[].class); + ArgumentCaptor acFlushPath = ArgumentCaptor.forClass(String.class); + ArgumentCaptor acTracingContext = ArgumentCaptor + .forClass(TracingContext.class); + ArgumentCaptor acFlushClose = ArgumentCaptor.forClass(Boolean.class); + ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); + ArgumentCaptor acEtag = ArgumentCaptor.forClass(String.class); + + verify(client, times(1)).flush(acByte.capture(), acFlushPath.capture(), acFlushClose.capture(), + acFlushSASToken.capture(), isNull(), acEtag.capture(), acTracingContext.capture()); + assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushPath.getAllValues()); + assertThat(Arrays.asList(true)).describedAs("Close flag").isEqualTo(acFlushClose.getAllValues()); + } + + /** + * The test verifies OutputStream Write of BUFFER_SIZE(4KB) followed by a close is making correct HTTP calls to the server + */ + @Test + public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { + + AbfsClient client = getClient(); + AbfsOutputStream out = getOutputStream(client, getConf()); + + final byte[] b = new byte[BUFFER_SIZE]; + new Random().nextBytes(b); + + for (int i = 0; i < 2; i++) { + out.write(b); + } + out.close(); + + AppendRequestParameters firstReqParameters = new AppendRequestParameters( + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); + AppendRequestParameters secondReqParameters = new AppendRequestParameters( + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); + + verify(client, times(1)).append(any(), + eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class), any()); + verify(client, times(1)).append(any(), + eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class), any()); + // confirm there were only 2 invocations in all + verify(client, times(2)).append(any(), + eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class), any()); + + ArgumentCaptor acByte = ArgumentCaptor.forClass(byte[].class); + ArgumentCaptor acFlushPath = ArgumentCaptor.forClass(String.class); + ArgumentCaptor acTracingContext = ArgumentCaptor + .forClass(TracingContext.class); + ArgumentCaptor acFlushClose = ArgumentCaptor.forClass(Boolean.class); + ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); + ArgumentCaptor acEtag = ArgumentCaptor.forClass(String.class); + + verify(client, times(1)).flush(acByte.capture(), acFlushPath.capture(), acFlushClose.capture(), + acFlushSASToken.capture(), isNull(), acEtag.capture(), acTracingContext.capture()); + assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushPath.getAllValues()); + assertThat(Arrays.asList(true)).describedAs("Close flag").isEqualTo(acFlushClose.getAllValues()); + } + + /** + * The test verifies OutputStream Write of BUFFER_SIZE(4KB) is making correct HTTP calls to the server + */ + @Test + public void verifyWriteRequestOfBufferSize() throws Exception { + AbfsClient client = getClient(); + AbfsOutputStream out = getOutputStream(client, getConf()); + + final byte[] b = new byte[BUFFER_SIZE]; + new Random().nextBytes(b); + + for (int i = 0; i < 2; i++) { + out.write(b); + } + Thread.sleep(1000); + + AppendRequestParameters firstReqParameters = new AppendRequestParameters( + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); + AppendRequestParameters secondReqParameters = new AppendRequestParameters( + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); + + verify(client, times(1)).append(any(), + eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class), any()); + verify(client, times(1)).append(any(), + eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class), any()); + // confirm there were only 2 invocations in all + verify(client, times(2)).append(any(), + eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class), any()); + } + + /** + * The test verifies OutputStream Write of BUFFER_SIZE(4KB) on a AppendBlob based stream is making correct HTTP calls to the server + */ + @Test + public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { + AbfsClient client = getClient(); + AbfsConfiguration abfsConf = getConf(); + AbfsOutputStream out = Mockito.spy(new AbfsOutputStream( + populateAbfsOutputStreamContext( + BUFFER_SIZE, + true, + false, + true, + true, + client, + PATH, + new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", + FSOperationType.OPEN, abfsConf.getTracingHeaderFormat(), + null), + createExecutorService(abfsConf)))); + + LinkedHashMap map = Mockito.mock(LinkedHashMap.class); + Mockito.when(map.put(Mockito.any(), Mockito.any())).thenReturn(BlockStatus.SUCCESS); + Mockito.when(out.getMap()).thenReturn(map); + + final byte[] b = new byte[BUFFER_SIZE]; + new Random().nextBytes(b); + + for (int i = 0; i < 2; i++) { + intercept(Exception.class , () -> out.write(b)); + } + } + + /** + * The test verifies OutputStream Write of BUFFER_SIZE(4KB) followed by a hflush call is making correct HTTP calls to the server + */ + @Test + public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { + AbfsClient client = getClient(); + AbfsOutputStream out = getOutputStream(client, getConf()); + + final byte[] b = new byte[BUFFER_SIZE]; + new Random().nextBytes(b); + + for (int i = 0; i < 2; i++) { + out.write(b); + } + out.hflush(); + + AppendRequestParameters firstReqParameters = new AppendRequestParameters( + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); + AppendRequestParameters secondReqParameters = new AppendRequestParameters( + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); + + verify(client, times(1)).append(any(), + eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class), any()); + verify(client, times(1)).append(any(), + eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class), any()); + // confirm there were only 2 invocations in all + verify(client, times(2)).append(any(), + eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class), any()); + + ArgumentCaptor acByte = ArgumentCaptor.forClass(byte[].class); + ArgumentCaptor acFlushPath = ArgumentCaptor.forClass(String.class); + ArgumentCaptor acTracingContext = ArgumentCaptor + .forClass(TracingContext.class); + ArgumentCaptor acFlushClose = ArgumentCaptor.forClass(Boolean.class); + ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); + ArgumentCaptor acEtag = ArgumentCaptor.forClass(String.class); + + verify(client, times(1)).flush(acByte.capture(), acFlushPath.capture(), acFlushClose.capture(), + acFlushSASToken.capture(), isNull(), acEtag.capture(), acTracingContext.capture()); + assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushPath.getAllValues()); + assertThat(Arrays.asList(false)).describedAs("Close flag").isEqualTo(acFlushClose.getAllValues()); + } + + /** + * The test verifies OutputStream Write of BUFFER_SIZE(4KB) followed by a flush call is making correct HTTP calls to the server + */ + @Test + public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { + AbfsClient client = getClient(); + AbfsOutputStream out = getOutputStream(client, getConf()); + + final byte[] b = new byte[BUFFER_SIZE]; + new Random().nextBytes(b); + + for (int i = 0; i < 2; i++) { + out.write(b); + } + Thread.sleep(1000); + out.flush(); + Thread.sleep(1000); + + AppendRequestParameters firstReqParameters = new AppendRequestParameters( + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); + AppendRequestParameters secondReqParameters = new AppendRequestParameters( + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); + + verify(client, times(1)).append(any(), + eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class), any()); + verify(client, times(1)).append(any(), + eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class), any()); + // confirm there were only 2 invocations in all + verify(client, times(2)).append(any(), + eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class), any()); + } + + /** + * Method to create an executor Service for AbfsOutputStream. + * + * @param abfsConf Configuration. + * @return ExecutorService. + */ + private ExecutorService createExecutorService( + AbfsConfiguration abfsConf) { + ExecutorService executorService = + new SemaphoredDelegatingExecutor(BlockingThreadPoolExecutorService.newInstance( + abfsConf.getWriteMaxConcurrentRequestCount(), + abfsConf.getMaxWriteRequestsToQueue(), + 10L, TimeUnit.SECONDS, + "abfs-test-bounded"), + BLOCK_UPLOAD_ACTIVE_BLOCKS_DEFAULT, true); + return executorService; + } +} + diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsPerfTracker.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsPerfTracker.java index 191d6e77ae09b2..ef52f244f7e490 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsPerfTracker.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsPerfTracker.java @@ -113,7 +113,7 @@ public void verifyTrackingForSingletonLatencyRecords() throws Exception { assertThat(latencyDetails).describedAs("AbfsPerfTracker should return non-null record").isNotNull(); assertThat(latencyDetails).describedAs("Latency record should be in the correct format") .containsPattern("h=[^ ]* t=[^ ]* a=bogusFilesystemName c=bogusAccountName cr=oneOperationCaller" - + " ce=oneOperationCallee r=Succeeded l=[0-9]+ s=0 e= ci=[^ ]* ri=[^ ]* bs=0 br=0 m=GET" + + " ce=oneOperationCallee r=Succeeded l=[0-9]+ s=0 e= ci=[^ ]* ri=[^ ]* ct=[^ ]* st=[^ ]* rt=[^ ]* bs=0 br=0 m=GET" + " u=http%3A%2F%2Fwww.microsoft.com%2FbogusFile"); } @@ -154,7 +154,7 @@ public void verifyTrackingForAggregateLatencyRecords() throws Exception { assertThat(latencyDetails).describedAs("Latency record should be in the correct format") .containsPattern("h=[^ ]* t=[^ ]* a=bogusFilesystemName c=bogusAccountName cr=oneOperationCaller" + " ce=oneOperationCallee r=Succeeded l=[0-9]+ ls=[0-9]+ lc=" + TEST_AGGREGATE_COUNT - + " s=0 e= ci=[^ ]* ri=[^ ]* bs=0 br=0 m=GET u=http%3A%2F%2Fwww.microsoft.com%2FbogusFile"); + + " s=0 e= ci=[^ ]* ri=[^ ]* ct=[^ ]* st=[^ ]* rt=[^ ]* bs=0 br=0 m=GET u=http%3A%2F%2Fwww.microsoft.com%2FbogusFile"); } latencyDetails = abfsPerfTracker.getClientLatency(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperationMockFailures.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperationMockFailures.java new file mode 100644 index 00000000000000..c3b08cd73c10f3 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperationMockFailures.java @@ -0,0 +1,272 @@ +/** + * 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.InterruptedIOException; +import java.net.SocketException; +import java.net.SocketTimeoutException; +import java.net.UnknownHostException; +import java.util.ArrayList; + +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.stubbing.Stubber; + +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +import static java.net.HttpURLConnection.HTTP_BAD_REQUEST; +import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR; +import static java.net.HttpURLConnection.HTTP_OK; +import static java.net.HttpURLConnection.HTTP_UNAVAILABLE; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.EGRESS_OVER_ACCOUNT_LIMIT; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.INGRESS_OVER_ACCOUNT_LIMIT; +import static org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil.addMockBehaviourToAbfsClient; +import static org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil.addMockBehaviourToRestOpAndHttpOp; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_RESET_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_RESET_MESSAGE; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_JDK_MESSAGE; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.EGRESS_LIMIT_BREACH_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.INGRESS_LIMIT_BREACH_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.IO_EXCEPTION_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.OPERATION_BREACH_MESSAGE; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.OPERATION_LIMIT_BREACH_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.READ_TIMEOUT_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.READ_TIMEOUT_JDK_MESSAGE; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.SOCKET_EXCEPTION_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.UNKNOWN_HOST_EXCEPTION_ABBREVIATION; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.nullable; + +public class TestAbfsRestOperationMockFailures { + + @Test + public void testClientRequestIdForConnectTimeoutRetry() throws Exception { + Exception[] exceptions = new Exception[1]; + String[] abbreviations = new String[1]; + exceptions[0] = new SocketTimeoutException(CONNECTION_TIMEOUT_JDK_MESSAGE); + abbreviations[0] = CONNECTION_TIMEOUT_ABBREVIATION; + testClientRequestIdForTimeoutRetry(exceptions, abbreviations, 1); + } + + @Test + public void testClientRequestIdForConnectAndReadTimeoutRetry() + throws Exception { + Exception[] exceptions = new Exception[2]; + String[] abbreviations = new String[2]; + exceptions[0] = new SocketTimeoutException(CONNECTION_TIMEOUT_JDK_MESSAGE); + abbreviations[0] = CONNECTION_TIMEOUT_ABBREVIATION; + exceptions[1] = new SocketTimeoutException(READ_TIMEOUT_JDK_MESSAGE); + abbreviations[1] = READ_TIMEOUT_ABBREVIATION; + testClientRequestIdForTimeoutRetry(exceptions, abbreviations, 1); + } + + @Test + public void testClientRequestIdForReadTimeoutRetry() throws Exception { + Exception[] exceptions = new Exception[1]; + String[] abbreviations = new String[1]; + exceptions[0] = new SocketTimeoutException(READ_TIMEOUT_JDK_MESSAGE); + abbreviations[0] = READ_TIMEOUT_ABBREVIATION; + testClientRequestIdForTimeoutRetry(exceptions, abbreviations, 1); + } + + @Test + public void testClientRequestIdForUnknownHostRetry() throws Exception { + Exception[] exceptions = new Exception[1]; + String[] abbreviations = new String[1]; + exceptions[0] = new UnknownHostException(); + abbreviations[0] = UNKNOWN_HOST_EXCEPTION_ABBREVIATION; + testClientRequestIdForTimeoutRetry(exceptions, abbreviations, 1); + } + + @Test + public void testClientRequestIdForConnectionResetRetry() throws Exception { + Exception[] exceptions = new Exception[1]; + String[] abbreviations = new String[1]; + exceptions[0] = new SocketTimeoutException(CONNECTION_RESET_MESSAGE + " by peer"); + abbreviations[0] = CONNECTION_RESET_ABBREVIATION; + testClientRequestIdForTimeoutRetry(exceptions, abbreviations, 1); + } + + @Test + public void testClientRequestIdForUnknownSocketExRetry() throws Exception { + Exception[] exceptions = new Exception[1]; + String[] abbreviations = new String[1]; + exceptions[0] = new SocketException("unknown"); + abbreviations[0] = SOCKET_EXCEPTION_ABBREVIATION; + testClientRequestIdForTimeoutRetry(exceptions, abbreviations, 1); + } + + @Test + public void testClientRequestIdForIOERetry() throws Exception { + Exception[] exceptions = new Exception[1]; + String[] abbreviations = new String[1]; + exceptions[0] = new InterruptedIOException(); + abbreviations[0] = IO_EXCEPTION_ABBREVIATION; + testClientRequestIdForTimeoutRetry(exceptions, abbreviations, 1); + } + + @Test + public void testClientRequestIdFor400Retry() throws Exception { + testClientRequestIdForStatusRetry(HTTP_BAD_REQUEST, "", "400"); + } + + @Test + public void testClientRequestIdFor500Retry() throws Exception { + testClientRequestIdForStatusRetry(HTTP_INTERNAL_ERROR, "", "500"); + } + + @Test + public void testClientRequestIdFor503INGRetry() throws Exception { + testClientRequestIdForStatusRetry(HTTP_UNAVAILABLE, + INGRESS_OVER_ACCOUNT_LIMIT.getErrorMessage(), + INGRESS_LIMIT_BREACH_ABBREVIATION); + } + + @Test + public void testClientRequestIdFor503egrRetry() throws Exception { + testClientRequestIdForStatusRetry(HTTP_UNAVAILABLE, + EGRESS_OVER_ACCOUNT_LIMIT.getErrorMessage(), + EGRESS_LIMIT_BREACH_ABBREVIATION); + } + + @Test + public void testClientRequestIdFor503OPRRetry() throws Exception { + testClientRequestIdForStatusRetry(HTTP_UNAVAILABLE, + OPERATION_BREACH_MESSAGE, OPERATION_LIMIT_BREACH_ABBREVIATION); + } + + @Test + public void testClientRequestIdFor503OtherRetry() throws Exception { + testClientRequestIdForStatusRetry(HTTP_UNAVAILABLE, "Other.", "503"); + } + + private void testClientRequestIdForStatusRetry(int status, + String serverErrorMessage, + String keyExpected) throws Exception { + + AbfsClient abfsClient = Mockito.mock(AbfsClient.class); + ExponentialRetryPolicy retryPolicy = Mockito.mock( + ExponentialRetryPolicy.class); + addMockBehaviourToAbfsClient(abfsClient, retryPolicy); + + + AbfsRestOperation abfsRestOperation = Mockito.spy(new AbfsRestOperation( + AbfsRestOperationType.ReadFile, + abfsClient, + "PUT", + null, + new ArrayList<>() + )); + + AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class); + addMockBehaviourToRestOpAndHttpOp(abfsRestOperation, httpOperation); + + Mockito.doNothing() + .doNothing() + .when(httpOperation) + .processResponse(nullable(byte[].class), nullable(int.class), + nullable(int.class)); + + int[] statusCount = new int[1]; + statusCount[0] = 0; + Mockito.doAnswer(answer -> { + if (statusCount[0] <= 5) { + statusCount[0]++; + return status; + } + return HTTP_OK; + }).when(httpOperation).getStatusCode(); + + Mockito.doReturn(serverErrorMessage) + .when(httpOperation) + .getStorageErrorMessage(); + + TracingContext tracingContext = Mockito.mock(TracingContext.class); + Mockito.doNothing().when(tracingContext).setRetryCount(nullable(int.class)); + Mockito.doReturn(tracingContext).when(abfsRestOperation).createNewTracingContext(any()); + + int[] count = new int[1]; + count[0] = 0; + Mockito.doAnswer(invocationOnMock -> { + if (count[0] == 1) { + Assertions.assertThat((String) invocationOnMock.getArgument(1)) + .isEqualTo(keyExpected); + } + count[0]++; + return null; + }).when(tracingContext).constructHeader(any(), any()); + + abfsRestOperation.execute(tracingContext); + Assertions.assertThat(count[0]).isEqualTo(2); + + } + + private void testClientRequestIdForTimeoutRetry(Exception[] exceptions, + String[] abbreviationsExpected, + int len) throws Exception { + AbfsClient abfsClient = Mockito.mock(AbfsClient.class); + ExponentialRetryPolicy retryPolicy = Mockito.mock( + ExponentialRetryPolicy.class); + addMockBehaviourToAbfsClient(abfsClient, retryPolicy); + + + AbfsRestOperation abfsRestOperation = Mockito.spy(new AbfsRestOperation( + AbfsRestOperationType.ReadFile, + abfsClient, + "PUT", + null, + new ArrayList<>() + )); + + AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class); + addMockBehaviourToRestOpAndHttpOp(abfsRestOperation, httpOperation); + + Stubber stubber = Mockito.doThrow(exceptions[0]); + for (int iteration = 1; iteration < len; iteration++) { + stubber.doThrow(exceptions[iteration]); + } + stubber + .doNothing() + .when(httpOperation) + .processResponse(nullable(byte[].class), nullable(int.class), + nullable(int.class)); + + Mockito.doReturn(HTTP_OK).when(httpOperation).getStatusCode(); + + TracingContext tracingContext = Mockito.mock(TracingContext.class); + Mockito.doNothing().when(tracingContext).setRetryCount(nullable(int.class)); + Mockito.doReturn(tracingContext).when(abfsRestOperation).createNewTracingContext(any()); + + int[] count = new int[1]; + count[0] = 0; + Mockito.doAnswer(invocationOnMock -> { + if (count[0] > 0 && count[0] <= len) { + Assertions.assertThat((String) invocationOnMock.getArgument(1)) + .isEqualTo(abbreviationsExpected[count[0] - 1]); + } + count[0]++; + return null; + }).when(tracingContext).constructHeader(any(), any()); + + abfsRestOperation.execute(tracingContext); + Assertions.assertThat(count[0]).isEqualTo(len + 1); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestBlobListXmlParser.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestBlobListXmlParser.java new file mode 100644 index 00000000000000..ec4cf3fa04ffe5 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestBlobListXmlParser.java @@ -0,0 +1,114 @@ +/** + * 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 javax.xml.parsers.ParserConfigurationException; +import javax.xml.parsers.SAXParser; +import javax.xml.parsers.SAXParserFactory; +import java.io.ByteArrayInputStream; +import java.io.InputStream; +import org.assertj.core.api.Assertions; +import java.util.List; + +import org.junit.Test; +import org.xml.sax.SAXException; + +public class TestBlobListXmlParser { + @Test + public void testXMLParser() throws Exception { + String xmlResponse = "" + + "" + + "" + + "/" + + "" + + "" + + "Splitting Example.txt" + + "" + + "Tue, 06 Jun 2023 08:33:51 GMT" + + "Tue, 06 Jun 2023 08:33:51 GMT" + + "0x8DB6668C17D3B76" + + "3844" + + "file" + + "BlockBlob" + + "Hot" + + "true" + + "unlocked" + + "available" + + "true" + + "true" + + "true" + + "Tue, 06 Jun 2023 08:33:51 GMT" + + "true" + + "true" + + "" + + "" + + "" + + "" + + "" + + "bye/" + + "" + + "" + + "" + + ""; + byte[] bytes = xmlResponse.getBytes(); + final InputStream stream = new ByteArrayInputStream(bytes);; + final SAXParser saxParser = saxParserThreadLocal.get(); + saxParser.reset(); + BlobList blobList = new BlobList(); + saxParser.parse(stream, new BlobListXmlParser(blobList, "https://sample.url")); + List prop = blobList.getBlobPropertyList(); + Assertions.assertThat(prop.size()).isEqualTo(2); + Assertions.assertThat(prop.get(0).getIsDirectory()).isEqualTo(false); + Assertions.assertThat(prop.get(1).getIsDirectory()).isEqualTo(true); + } + + @Test + public void testEmptyBlobList() throws Exception { + String xmlResponse = "" + + "<" + + "EnumerationResults ServiceEndpoint=\"https://anujtestfns.blob.core.windows.net/\" ContainerName=\"manualtest\">" + + "abc/" + + "/" + + "" + + ""; + byte[] bytes = xmlResponse.getBytes(); + final InputStream stream = new ByteArrayInputStream(bytes);; + final SAXParser saxParser = saxParserThreadLocal.get(); + saxParser.reset(); + BlobList blobList = new BlobList(); + saxParser.parse(stream, new BlobListXmlParser(blobList, "https://sample.url")); + List prop = blobList.getBlobPropertyList(); + } + + private static final ThreadLocal saxParserThreadLocal + = new ThreadLocal() { + @Override + public SAXParser initialValue() { + SAXParserFactory factory = SAXParserFactory.newInstance(); + factory.setNamespaceAware(true); + try { + return factory.newSAXParser(); + } catch (SAXException e) { + throw new RuntimeException("Unable to create SAXParser", e); + } catch (ParserConfigurationException e) { + throw new RuntimeException("Check parser configuration", e); + } + } + }; +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java index 0f8dc55aa14a4c..511e110c2887c8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java @@ -18,20 +18,38 @@ package org.apache.hadoop.fs.azurebfs.services; +import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR; +import static org.apache.hadoop.fs.azure.integration.AzureTestConstants.TEST_CONFIGURATION_FILE_NAME; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_BACKOFF_INTERVAL; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_BACKOFF_INTERVAL; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_IO_RETRIES; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MIN_BACKOFF_INTERVAL; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT1_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ENABLE_AUTOTHROTTLING; +import static org.junit.Assume.assumeTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import java.net.URI; import java.util.Random; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.assertj.core.api.Assertions; import org.junit.Assert; +import org.junit.Assume; import org.junit.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; +import org.mockito.Mockito; /** * Unit test TestExponentialRetryPolicy. @@ -41,6 +59,9 @@ public class TestExponentialRetryPolicy extends AbstractAbfsIntegrationTest { private final int noRetryCount = 0; private final int retryCount = new Random().nextInt(maxRetryCount); private final int retryCountBeyondMax = maxRetryCount + 1; + private static final String TEST_PATH = "/testfile"; + private static final double MULTIPLYING_FACTOR = 1.5; + private static final int ANALYSIS_PERIOD = 10000; public TestExponentialRetryPolicy() throws Exception { @@ -67,6 +88,172 @@ public void testDefaultMaxIORetryCount() throws Exception { testMaxIOConfig(abfsConfig); } + @Test + public void testThrottlingIntercept() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + final Configuration configuration = new Configuration(); + configuration.addResource(TEST_CONFIGURATION_FILE_NAME); + configuration.setBoolean(FS_AZURE_ENABLE_AUTOTHROTTLING, false); + + // On disabling throttling AbfsNoOpThrottlingIntercept object is returned + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, + "dummy.dfs.core.windows.net"); + AbfsThrottlingIntercept intercept; + AbfsClient abfsClient = ITestAbfsClient.createTestClientFromCurrentContext(getClient(fs), abfsConfiguration); + intercept = abfsClient.getIntercept(); + Assertions.assertThat(intercept) + .describedAs("AbfsNoOpThrottlingIntercept instance expected") + .isInstanceOf(AbfsNoOpThrottlingIntercept.class); + + configuration.setBoolean(FS_AZURE_ENABLE_AUTOTHROTTLING, true); + configuration.setBoolean(FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED, true); + // On disabling throttling AbfsClientThrottlingIntercept object is returned + AbfsConfiguration abfsConfiguration1 = new AbfsConfiguration(configuration, + "dummy1.dfs.core.windows.net"); + AbfsClient abfsClient1 = ITestAbfsClient.createTestClientFromCurrentContext(getClient(fs), abfsConfiguration1); + intercept = abfsClient1.getIntercept(); + Assertions.assertThat(intercept) + .describedAs("AbfsClientThrottlingIntercept instance expected") + .isInstanceOf(AbfsClientThrottlingIntercept.class); + } + + @Test + public void testCreateMultipleAccountThrottling() throws Exception { + Configuration config = new Configuration(getRawConfiguration()); + String accountName = config.get(FS_AZURE_ACCOUNT_NAME); + if (accountName == null) { + // check if accountName is set using different config key + accountName = config.get(FS_AZURE_ABFS_ACCOUNT1_NAME); + } + assumeTrue("Not set: " + FS_AZURE_ABFS_ACCOUNT1_NAME, + accountName != null && !accountName.isEmpty()); + + Configuration rawConfig1 = new Configuration(); + rawConfig1.addResource(TEST_CONFIGURATION_FILE_NAME); + + AbfsRestOperation successOp = mock(AbfsRestOperation.class); + AbfsHttpOperation http500Op = mock(AbfsHttpOperation.class); + when(http500Op.getStatusCode()).thenReturn(HTTP_INTERNAL_ERROR); + when(successOp.getResult()).thenReturn(http500Op); + + AbfsConfiguration configuration = Mockito.mock(AbfsConfiguration.class); + when(configuration.getAnalysisPeriod()).thenReturn(ANALYSIS_PERIOD); + when(configuration.isAutoThrottlingEnabled()).thenReturn(true); + when(configuration.accountThrottlingEnabled()).thenReturn(false); + + AbfsThrottlingIntercept instance1 = AbfsThrottlingInterceptFactory.getInstance(accountName, configuration); + String accountName1 = config.get(FS_AZURE_ABFS_ACCOUNT1_NAME); + + assumeTrue("Not set: " + FS_AZURE_ABFS_ACCOUNT1_NAME, + accountName1 != null && !accountName1.isEmpty()); + + AbfsThrottlingIntercept instance2 = AbfsThrottlingInterceptFactory.getInstance(accountName1, configuration); + //if singleton is enabled, for different accounts both the instances should return same value + Assertions.assertThat(instance1) + .describedAs( + "if singleton is enabled, for different accounts both the instances should return same value") + .isEqualTo(instance2); + + when(configuration.accountThrottlingEnabled()).thenReturn(true); + AbfsThrottlingIntercept instance3 = AbfsThrottlingInterceptFactory.getInstance(accountName, configuration); + AbfsThrottlingIntercept instance4 = AbfsThrottlingInterceptFactory.getInstance(accountName1, configuration); + AbfsThrottlingIntercept instance5 = AbfsThrottlingInterceptFactory.getInstance(accountName, configuration); + //if singleton is not enabled, for different accounts instances should return different value + Assertions.assertThat(instance3) + .describedAs( + "iff singleton is not enabled, for different accounts instances should return different value") + .isNotEqualTo(instance4); + + //if singleton is not enabled, for same accounts instances should return same value + Assertions.assertThat(instance3) + .describedAs( + "if singleton is not enabled, for same accounts instances should return same value") + .isEqualTo(instance5); + } + + @Test + public void testOperationOnAccountIdle() throws Exception { + //Get the filesystem. + AzureBlobFileSystem fs = getFileSystem(); + AbfsClient client = getClient(fs); + AbfsConfiguration configuration1 = client.getAbfsConfiguration(); + Assume.assumeTrue(configuration1.isAutoThrottlingEnabled()); + Assume.assumeTrue(configuration1.accountThrottlingEnabled()); + + AbfsClientThrottlingIntercept accountIntercept + = (AbfsClientThrottlingIntercept) client.getIntercept(); + final byte[] b = new byte[2 * MIN_BUFFER_SIZE]; + new Random().nextBytes(b); + + Path testPath = path(TEST_PATH); + + //Do an operation on the filesystem. + try (FSDataOutputStream stream = fs.create(testPath)) { + stream.write(b); + } + + //Don't perform any operation on the account. + int sleepTime = (int) ((getAbfsConfig().getAccountOperationIdleTimeout()) * MULTIPLYING_FACTOR); + Thread.sleep(sleepTime); + + try (FSDataInputStream streamRead = fs.open(testPath)) { + streamRead.read(b); + } + + //Perform operations on another account. + AzureBlobFileSystem fs1 = new AzureBlobFileSystem(); + Configuration config = new Configuration(getRawConfiguration()); + String accountName1 = config.get(FS_AZURE_ABFS_ACCOUNT1_NAME); + assumeTrue("Not set: " + FS_AZURE_ABFS_ACCOUNT1_NAME, + accountName1 != null && !accountName1.isEmpty()); + final String abfsUrl1 = this.getFileSystemName() + "12" + "@" + accountName1; + URI defaultUri1 = null; + defaultUri1 = new URI("abfss", abfsUrl1, null, null, null); + fs1.initialize(defaultUri1, getRawConfiguration()); + AbfsClient client1 = getClient(fs1); + AbfsClientThrottlingIntercept accountIntercept1 + = (AbfsClientThrottlingIntercept) client1.getIntercept(); + try (FSDataOutputStream stream1 = fs1.create(testPath)) { + stream1.write(b); + } + + //Verify the write analyzer for first account is idle but the read analyzer is not idle. + Assertions.assertThat(accountIntercept.getWriteThrottler() + .getIsOperationOnAccountIdle() + .get()) + .describedAs("Write analyzer for first account should be idle the first time") + .isTrue(); + + Assertions.assertThat( + accountIntercept.getReadThrottler() + .getIsOperationOnAccountIdle() + .get()) + .describedAs("Read analyzer for first account should not be idle") + .isFalse(); + + //Verify the write analyzer for second account is not idle. + Assertions.assertThat( + accountIntercept1.getWriteThrottler() + .getIsOperationOnAccountIdle() + .get()) + .describedAs("Write analyzer for second account should not be idle") + .isFalse(); + + //Again perform an operation on the first account. + try (FSDataOutputStream stream2 = fs.create(testPath)) { + stream2.write(b); + } + + //Verify the write analyzer on first account is not idle. + Assertions.assertThat( + accountIntercept.getWriteThrottler() + .getIsOperationOnAccountIdle() + .get()) + .describedAs( + "Write analyzer for first account should not be idle second time") + .isFalse(); + } + @Test public void testAbfsConfigConstructor() throws Exception { // Ensure we choose expected values that are not defaults diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestRetryReason.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestRetryReason.java new file mode 100644 index 00000000000000..76fcc6dc2c8a45 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestRetryReason.java @@ -0,0 +1,134 @@ +/** + * 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.IOException; +import java.net.SocketException; +import java.net.SocketTimeoutException; +import java.net.UnknownHostException; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import static java.net.HttpURLConnection.HTTP_FORBIDDEN; +import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR; +import static java.net.HttpURLConnection.HTTP_UNAVAILABLE; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.EGRESS_OVER_ACCOUNT_LIMIT; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.INGRESS_OVER_ACCOUNT_LIMIT; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_RESET_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_RESET_MESSAGE; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_JDK_MESSAGE; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.EGRESS_LIMIT_BREACH_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.INGRESS_LIMIT_BREACH_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.IO_EXCEPTION_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.OPERATION_BREACH_MESSAGE; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.OPERATION_LIMIT_BREACH_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.READ_TIMEOUT_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.READ_TIMEOUT_JDK_MESSAGE; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.SOCKET_EXCEPTION_ABBREVIATION; +import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.UNKNOWN_HOST_EXCEPTION_ABBREVIATION; + +public class TestRetryReason { + + @Test + public void test4xxStatusRetryReason() { + Assertions.assertThat(RetryReason.getAbbreviation(null, HTTP_FORBIDDEN, null)) + .describedAs("Abbreviation for 4xx should be equal to 4xx") + .isEqualTo(HTTP_FORBIDDEN + ""); + } + + @Test + public void testConnectionResetRetryReason() { + SocketException connReset = new SocketException(CONNECTION_RESET_MESSAGE.toUpperCase()); + Assertions.assertThat(RetryReason.getAbbreviation(connReset, null, null)).isEqualTo(CONNECTION_RESET_ABBREVIATION); + } + + @Test + public void testConnectionTimeoutRetryReason() { + SocketTimeoutException connectionTimeoutException = new SocketTimeoutException(CONNECTION_TIMEOUT_JDK_MESSAGE); + Assertions.assertThat(RetryReason.getAbbreviation(connectionTimeoutException, null, null)).isEqualTo( + CONNECTION_TIMEOUT_ABBREVIATION + ); + } + + @Test + public void testReadTimeoutRetryReason() { + SocketTimeoutException connectionTimeoutException = new SocketTimeoutException(READ_TIMEOUT_JDK_MESSAGE); + Assertions.assertThat(RetryReason.getAbbreviation(connectionTimeoutException, null, null)).isEqualTo( + READ_TIMEOUT_ABBREVIATION + ); + } + + @Test + public void testEgressLimitRetryReason() { + Assertions.assertThat(RetryReason.getAbbreviation(null, HTTP_UNAVAILABLE, EGRESS_OVER_ACCOUNT_LIMIT.getErrorMessage())).isEqualTo( + EGRESS_LIMIT_BREACH_ABBREVIATION + ); + } + + @Test + public void testIngressLimitRetryReason() { + Assertions.assertThat(RetryReason.getAbbreviation(null, HTTP_UNAVAILABLE, INGRESS_OVER_ACCOUNT_LIMIT.getErrorMessage())).isEqualTo( + INGRESS_LIMIT_BREACH_ABBREVIATION + ); + } + + @Test + public void testOperationLimitRetryReason() { + Assertions.assertThat(RetryReason.getAbbreviation(null, HTTP_UNAVAILABLE, OPERATION_BREACH_MESSAGE)).isEqualTo( + OPERATION_LIMIT_BREACH_ABBREVIATION + ); + } + + @Test + public void test503UnknownRetryReason() { + Assertions.assertThat(RetryReason.getAbbreviation(null, HTTP_UNAVAILABLE, null)).isEqualTo( + "503" + ); + } + + @Test + public void test500RetryReason() { + Assertions.assertThat(RetryReason.getAbbreviation(null, HTTP_INTERNAL_ERROR, null)).isEqualTo( + "500" + ); + } + + @Test + public void testUnknownHostRetryReason() { + Assertions.assertThat(RetryReason.getAbbreviation(new UnknownHostException(), null, null)).isEqualTo( + UNKNOWN_HOST_EXCEPTION_ABBREVIATION + ); + } + + @Test + public void testUnknownIOExceptionRetryReason() { + Assertions.assertThat(RetryReason.getAbbreviation(new IOException(), null, null)).isEqualTo( + IO_EXCEPTION_ABBREVIATION + ); + } + + @Test + public void testUnknownSocketException() { + Assertions.assertThat(RetryReason.getAbbreviation(new SocketException(), null, null)).isEqualTo( + SOCKET_EXCEPTION_ABBREVIATION + ); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanupTestContainers.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanupTestContainers.java new file mode 100644 index 00000000000000..b8272319ab8516 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanupTestContainers.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs.utils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.microsoft.azure.storage.CloudStorageAccount; +import com.microsoft.azure.storage.blob.CloudBlobClient; +import com.microsoft.azure.storage.blob.CloudBlobContainer; +import com.microsoft.azure.storage.StorageCredentials; +import com.microsoft.azure.storage.StorageCredentialsAccountAndKey; + +import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; + +/** + * This looks like a test, but it is really a command to invoke to + * clean up containers created in other test runs. + * + */ +public class CleanupTestContainers extends AbstractAbfsIntegrationTest { + private static final Logger LOG = LoggerFactory.getLogger(CleanupTestContainers.class); + private static final String CONTAINER_PREFIX = "abfs-testcontainer-"; + + public CleanupTestContainers() throws Exception { + } + + @org.junit.Test + public void testDeleteContainers() throws Throwable { + int count = 0; + AbfsConfiguration abfsConfig = getAbfsStore(getFileSystem()).getAbfsConfiguration(); + String accountName = abfsConfig.getAccountName().split("\\.")[0]; + LOG.debug("Deleting test containers in account - {}", abfsConfig.getAccountName()); + + String accountKey = abfsConfig.getStorageAccountKey(); + if ((accountKey == null) || (accountKey.isEmpty())) { + LOG.debug("Clean up not possible. Account ket not present in config"); + } + final StorageCredentials credentials; + credentials = new StorageCredentialsAccountAndKey( + accountName, accountKey); + CloudStorageAccount storageAccount = new CloudStorageAccount(credentials, true); + CloudBlobClient blobClient = storageAccount.createCloudBlobClient(); + Iterable containers + = blobClient.listContainers(CONTAINER_PREFIX); + for (CloudBlobContainer container : containers) { + LOG.info("Container {} URI {}", + container.getName(), + container.getUri()); + if (container.deleteIfExists()) { + count++; + LOG.info("Current deleted test containers count - #{}", count); + } + } + LOG.info("Summary: Deleted {} test containers", count); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java index 6f2209a6e8cedc..39799fb162e969 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/DelegationSASGenerator.java @@ -58,13 +58,21 @@ public String getDelegationSAS(String accountName, String containerName, String switch (operation) { case SASTokenProvider.CREATE_FILE_OPERATION: case SASTokenProvider.CREATE_DIRECTORY_OPERATION: + case SASTokenProvider.CREATE_CONTAINER_OPERATION: case SASTokenProvider.WRITE_OPERATION: case SASTokenProvider.SET_PROPERTIES_OPERATION: + case SASTokenProvider.LEASE_OPERATION: + case SASTokenProvider.SET_BLOB_METADATA_OPERATION: + case SASTokenProvider.SET_CONTAINER_METADATA_OPERATION: sp = "w"; break; case SASTokenProvider.DELETE_OPERATION: sp = "d"; break; + case SASTokenProvider.DELETE_BLOB_OPERATION: + case SASTokenProvider.DELETE_CONTAINER_OPERATION: + sp = "d"; + break; case SASTokenProvider.DELETE_RECURSIVE_OPERATION: sp = "d"; sr = "d"; @@ -78,7 +86,22 @@ public String getDelegationSAS(String accountName, String containerName, String case SASTokenProvider.LIST_OPERATION: sp = "l"; break; + case SASTokenProvider.LIST_BLOB_OPERATION: + sp = "l"; + sr = "c"; + break; + case SASTokenProvider.COPY_BLOB_DESTINATION: + sp = "w"; + break; + case SASTokenProvider.COPY_BLOB_SOURCE: + sp = "r"; + break; case SASTokenProvider.GET_PROPERTIES_OPERATION: + case SASTokenProvider.GET_BLOCK_LIST: + case SASTokenProvider.GET_BLOB_PROPERTIES_OPERATION: + case SASTokenProvider.GET_CONTAINER_PROPERTIES_OPERATION: + case SASTokenProvider.GET_BLOB_METADATA_OPERATION: + case SASTokenProvider.GET_CONTAINER_METADATA_OPERATION: case SASTokenProvider.READ_OPERATION: sp = "r"; break; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java index e195f1c381a940..37fb2db1de0ccc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java @@ -38,10 +38,16 @@ public class TracingHeaderValidator implements Listener { private TracingHeaderFormat format; private static final String GUID_PATTERN = "^[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}$"; + private Integer operatedBlobCount = null; + + private Boolean disableValidation = false; @Override public void callTracingHeaderValidator(String tracingContextHeader, TracingHeaderFormat format) { + if (disableValidation) { + return; + } this.format = format; validateTracingHeader(tracingContextHeader); } @@ -52,6 +58,9 @@ public TracingHeaderValidator getClone() { clientCorrelationId, fileSystemId, operation, needsPrimaryRequestId, retryNum, streamID); tracingHeaderValidator.primaryRequestId = primaryRequestId; + if (disableValidation) { + tracingHeaderValidator.setDisableValidation(true); + } return tracingHeaderValidator; } @@ -78,6 +87,13 @@ private void validateTracingHeader(String tracingContextHeader) { if (format != TracingHeaderFormat.ALL_ID_FORMAT) { return; } + if (idList.length >= 9) { + if (operatedBlobCount != null) { + Assertions.assertThat(Integer.parseInt(idList[8])) + .describedAs("OperatedBlobCount is incorrect") + .isEqualTo(operatedBlobCount); + } + } if (!primaryRequestId.isEmpty() && !idList[3].isEmpty()) { Assertions.assertThat(idList[3]) .describedAs("PrimaryReqID should be common for these requests") @@ -93,7 +109,8 @@ private void validateTracingHeader(String tracingContextHeader) { private void validateBasicFormat(String[] idList) { if (format == TracingHeaderFormat.ALL_ID_FORMAT) { Assertions.assertThat(idList) - .describedAs("header should have 7 elements").hasSize(7); + .describedAs("header should have 8 or 9 elements") + .hasSizeBetween(8, 9); } else if (format == TracingHeaderFormat.TWO_ID_FORMAT) { Assertions.assertThat(idList) .describedAs("header should have 2 elements").hasSize(2); @@ -130,6 +147,9 @@ private void validateBasicFormat(String[] idList) { } Assertions.assertThat(idList[5]).describedAs("Operation name incorrect") .isEqualTo(operation.toString()); + if (idList[6].contains("_")) { + idList[6] = idList[6].split("_")[0]; + } int retryCount = Integer.parseInt(idList[6]); Assertions.assertThat(retryCount) .describedAs("Retry was required due to issue on server side") @@ -149,4 +169,12 @@ public void setOperation(FSOperationType operation) { public void updatePrimaryRequestID(String primaryRequestId) { this.primaryRequestId = primaryRequestId; } + + public void setOperatedBlobCount(Integer operatedBlobCount) { + this.operatedBlobCount = operatedBlobCount; + } + + public void setDisableValidation(Boolean disableValidation) { + this.disableValidation = disableValidation; + } } diff --git a/hadoop-tools/hadoop-azure/src/test/resources/accountSettings/accountName_settings.xml.template b/hadoop-tools/hadoop-azure/src/test/resources/accountSettings/accountName_settings.xml.template new file mode 100644 index 00000000000000..062b2f4bf3ad1d --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/resources/accountSettings/accountName_settings.xml.template @@ -0,0 +1,185 @@ + + + + + + + + fs.azure.abfs.account.name + ACCOUNTNAME.dfs.core.windows.net + + + fs.contract.test.fs.abfs + abfs://CONTAINER_NAME@ACCOUNTNAME.dfs.core.windows.net + + + fs.contract.test.fs.abfss + abfss://CONTAINER_NAME@ACCOUNTNAME.dfs.core.windows.net + + + fs.contract.test.fs.wasb + wasb://CONTAINER_NAME@ACCOUNTNAME.blob.core.windows.net + + + fs.azure.wasb.account.name + ACCOUNTNAME.blob.core.windows.net + + + fs.azure.scale.test.enabled + true + + + fs.azure.test.namespace.enabled.ACCOUNTNAME.dfs.core.windows.net + IS_NAMESPACE_ENABLED + + + fs.azure.test.namespace.enabled + IS_NAMESPACE_ENABLED + + + fs.azure.account.hns.enabled.ACCOUNTNAME.dfs.core.windows.net + IS_NAMESPACE_ENABLED + + + fs.azure.account.hns.enabled + IS_NAMESPACE_ENABLED + + + fs.azure.account.key.ACCOUNTNAME.dfs.core.windows.net + ACCOUNT_KEY + + + fs.azure.account.key.ACCOUNTNAME.blob.core.windows.net + ACCOUNT_KEY + + + fs.azure.account.key.ACCOUNTNAME.dfs.core.windows.net + ACCOUNT_KEY + + + fs.azure.account.key.ACCOUNTNAME.blob.core.windows.net + ACCOUNT_KEY + + + + + fs.azure.account.oauth2.client.endpoint.ACCOUNTNAME.dfs.core.windows.net + https://login.microsoftonline.com/SUPERUSER_TENANT_ID/oauth2/token + + + fs.azure.account.oauth2.client.endpoint + https://login.microsoftonline.com/SUPERUSER_TENANT_ID/oauth2/token + + + fs.azure.account.oauth.provider.type.ACCOUNTNAME.dfs.core.windows.net + org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider + + + fs.azure.account.oauth.provider.type + org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider + + + fs.azure.account.oauth2.client.id.ACCOUNTNAME.dfs.core.windows.net + SUPERUSER_CLIENT_ID + + + fs.azure.account.oauth2.client.id + SUPERUSER_CLIENT_ID + + + fs.azure.account.oauth2.client.secret.ACCOUNTNAME.dfs.core.windows.net + SUPERUSER_CLIENT_SECRET + + + fs.azure.account.oauth2.client.secret + SUPERUSER_CLIENT_SECRET + + + + + fs.azure.enable.check.access + true + + + fs.azure.account.test.oauth2.client.id + NO_RBAC_USER_CLIENT_ID + + + fs.azure.account.test.oauth2.client.secret + NO_RBAC_USER_CLIENT_SECRET + + + fs.azure.check.access.testuser.guid + NO_RBAC_USER_OID + + + + + fs.azure.account.oauth2.contributor.client.id + CONTRIBUTOR_RBAC_USER_CLIENT_ID + + + fs.azure.account.oauth2.contributor.client.secret + CONTRIBUTOR_RBAC_USER_CLIENT_SECRET + + + + + fs.azure.account.oauth2.reader.client.id + READER_RBAC_USER_CLIENT_ID + + + fs.azure.account.oauth2.reader.client.secret + READER_RBAC_USER_CLIENT_ID + + diff --git a/hadoop-tools/hadoop-azure/src/test/resources/azure-auth-keys.xml.template b/hadoop-tools/hadoop-azure/src/test/resources/azure-auth-keys.xml.template index 12dbbfab479707..636816551ddf7e 100644 --- a/hadoop-tools/hadoop-azure/src/test/resources/azure-auth-keys.xml.template +++ b/hadoop-tools/hadoop-azure/src/test/resources/azure-auth-keys.xml.template @@ -14,162 +14,16 @@ --> + - - - - - - fs.azure.account.auth.type - SharedKey - - - - - - fs.azure.account.key.{ABFS_ACCOUNT_NAME}.dfs.core.windows.net - {ACCOUNT_ACCESS_KEY} - Account access key - - - - fs.azure.account.oauth.provider.type.{ABFS_ACCOUNT_NAME}.dfs.core.windows.net - - org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider - OAuth token provider implementation class - - - - fs.azure.account.oauth2.client.endpoint.{ABFS_ACCOUNT_NAME}.dfs.core.windows.net - - https://login.microsoftonline.com/{TENANTID}/oauth2/token - Token end point, this can be found through Azure portal - - - - - fs.azure.account.oauth2.client.id.{ABFS_ACCOUNT_NAME}.dfs.core.windows.net - - {client id} - AAD client id. - - - - fs.azure.account.oauth2.client.secret.{ABFS_ACCOUNT_NAME}.dfs.core.windows.net - - {client secret} - AAD client secret - - - - - fs.contract.test.fs.abfs - abfs://{CONTAINER_NAME}@{ACCOUNT_NAME}.dfs.core.windows.net - - - fs.contract.test.fs.abfss - abfss://{CONTAINER_NAME}@{ACCOUNT_NAME}.dfs.core.windows.net - - - - - fs.azure.wasb.account.name - {WASB_ACCOUNT_NAME}.blob.core.windows.net - - - fs.azure.account.key.{WASB_ACCOUNT_NAME}.blob.core.windows.net - WASB account key - - - fs.contract.test.fs.wasb - wasb://{WASB_FILESYSTEM}@{WASB_ACCOUNT_NAME}.blob.core.windows.net - - - - - - fs.azure.account.oauth2.contributor.client.id - {Client id of SP with RBAC Storage Blob Data Contributor} - - - fs.azure.account.oauth2.contributor.client.secret - {Client secret of SP with RBAC Storage Blob Data Contributor} - - - fs.azure.account.oauth2.reader.client.id - {Client id of SP with RBAC Storage Blob Data Reader} - - - fs.azure.account.oauth2.reader.client.secret - {Client secret of SP with RBAC Storage Blob Data Reader} - - - - - - - fs.azure.account.test.oauth2.client.id - {client id} - The client id(app id) for the app created on step 1 - - - - fs.azure.account.test.oauth2.client.secret - {client secret} - -The client secret(application's secret) for the app created on step 1 - - - - fs.azure.check.access.testuser.guid - {guid} - The guid fetched on step 2 - - fs.azure.account.oauth2.client.endpoint.{account name}.dfs.core -.windows.net - https://login.microsoftonline.com/{TENANTID}/oauth2/token - -Token end point. This can be found through Azure portal. As part of CheckAccess -test cases. The access will be tested for an FS instance created with the -above mentioned client credentials. So this configuration is necessary to -create the test FS instance. - + fs.azure.hnsTestAccountName + - - fs.azure.test.appendblob.enabled - false - If made true, tests will be running under the assumption that - append blob is enabled and the root directory and contract test root - directory will be part of the append blob directories. Should be false for - non-HNS accounts. - + fs.azure.nonHnsTestAccountName + diff --git a/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml b/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml index 24ffeb5d107a04..6730021974e5f9 100644 --- a/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml +++ b/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml @@ -28,11 +28,8 @@ false - - fs.azure.test.namespace.enabled - true - - + + fs.azure.abfs.latency.track false @@ -43,9 +40,6 @@ true - - - fs.azure.user.agent.prefix @@ -59,7 +53,40 @@ STORE THE CONFIGURATION PROPERTIES WITHIN IT. TO PREVENT ACCIDENTAL LEAKS OF YOUR STORAGE ACCOUNT CREDENTIALS, THIS FILE IS LISTED IN .gitignore TO PREVENT YOU FROM INCLUDING - IT IN PATCHES OR COMMITS. --> + IT IN PATCHES OR COMMITS. + + TEST SCRIPT RUNS: + ================ + FOR EASIER TEST RUNS, TEST RUNS FOR VARIOUS COMBINATIONS CAN BE + TRIGGERED OVER SCRIPT: + ./dev-support/testrun-scripts/runtests.sh + (FROM hadoop-azure ROOT PROJECT PATH) + + TO USE THE TEST SCRIPT, + 1. COPY + ./src/test/resources/azure-auth-keys.xml.template + TO + ./src/test/resources/azure-auth-keys.xml + UPDATE ACCOUNT NAMES THAT SHOULD BE USED IN THE TEST RUN + FOR HNS AND NON-HNS COMBINATIONS IN THE 2 PROPERTIES + PRESENT IN THE XML, NAMELY + fs.azure.hnsTestAccountName and + fs.azure.nonHnsTestAccountName + (ACCOUNT NAME SHOULD BE WITHOUT DOMAIN) + + azure-auth-keys.xml IS LISTED IN .gitignore, SO ANY + ACCIDENTAL ACCOUNT NAME LEAK IS PREVENTED. + + 2. CREATE ACCOUNT CONFIG FILES (ONE CONFIG FILE + PER ACCOUNT) IN FOLDER: + ./src/test/resources/accountSettings/ + + FOLLOW INSTRUCTIONS IN THE START OF THE TEMPLATE FILE + accountName_settings.xml.template + WITHIN accountSettings FOLDER WHILE CREATING ACCOUNT CONFIG FILE. + + NEW FILES CREATED IN FOLDER accountSettings IS LISTED IN .gitignore + TO PREVENT ACCIDENTAL CRED LEAKS. --> diff --git a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties index 9f72d036533064..8adcccc4717b32 100644 --- a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties @@ -26,7 +26,8 @@ log4j.logger.org.apache.hadoop.fs.azure.AzureFileSystemThreadPoolExecutor=DEBUG log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG log4j.logger.org.apache.hadoop.fs.azurebfs.contracts.services.TracingService=TRACE log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsClient=DEBUG - +log4j.logger.org.apache.hadoop.fs.azure.AzureNativeFileSystemStore=DEBUG +log4j.logger.org.apache.hadoop.fs.azurebfs.oauth2.AzureADAuthenticator=DEBUG # after here: turn off log messages from other parts of the system # which only clutter test reports. log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR