From f044deedbbfee0812316d587139cb828f27172e9 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 15 Jun 2018 18:14:13 +0100 Subject: [PATCH 01/29] HADOOP-15407. HADOOP-15540. Support Windows Azure Storage - Blob file system "ABFS" in Hadoop: Core Commit. Contributed by Shane Mainali, Thomas Marquardt, Zichen Sun, Georgi Chalakov, Esfandiar Manii, Amit Singh, Dana Kaban, Da Zhou, Junhua Gu, Saher Ahwal, Saurabh Pant, James Baker, Shaoyu Zhang, Lawrence Chen, Kevin Chen and Steve Loughran --- .gitignore | 1 + .../src/main/resources/core-default.xml | 10 + .../conf/TestCommonConfigurationFields.java | 3 + hadoop-project/pom.xml | 11 + hadoop-tools/hadoop-azure/pom.xml | 61 +- .../src/config/checkstyle-suppressions.xml | 47 ++ .../org/apache/hadoop/fs/azurebfs/Abfs.java | 48 ++ .../org/apache/hadoop/fs/azurebfs/Abfss.java | 48 ++ .../fs/azurebfs/AzureBlobFileSystem.java | 612 ++++++++++++++++ .../azurebfs/SecureAzureBlobFileSystem.java | 41 ++ .../azurebfs/constants/AbfsHttpConstants.java | 76 ++ .../azurebfs/constants/ConfigurationKeys.java | 57 ++ .../constants/FileSystemConfigurations.java | 59 ++ .../constants/FileSystemUriSchemes.java | 42 ++ .../constants/HttpHeaderConfigurations.java | 57 ++ .../azurebfs/constants/HttpQueryParams.java | 40 + .../fs/azurebfs/constants/package-info.java | 22 + .../ConfigurationValidationAnnotations.java | 104 +++ .../contracts/annotations/package-info.java | 22 + .../diagnostics/ConfigurationValidator.java | 37 + .../contracts/diagnostics/package-info.java | 22 + .../AbfsRestOperationException.java | 84 +++ .../AzureBlobFileSystemException.java | 56 ++ ...onfigurationPropertyNotFoundException.java | 32 + ...FileSystemOperationUnhandledException.java | 33 + .../InvalidAbfsRestOperationException.java | 40 + .../InvalidConfigurationValueException.java | 37 + .../InvalidFileSystemPropertyException.java | 33 + .../InvalidUriAuthorityException.java | 33 + .../exceptions/InvalidUriException.java | 33 + .../ServiceResolutionException.java | 36 + .../exceptions/TimeoutException.java | 33 + .../contracts/exceptions/package-info.java | 22 + .../fs/azurebfs/contracts/package-info.java | 22 + .../services/AbfsHttpClientFactory.java | 39 + .../contracts/services/AbfsHttpService.java | 162 ++++ .../services/AbfsServiceProvider.java | 40 + .../services/AzureServiceErrorCode.java | 112 +++ .../services/ConfigurationService.java | 143 ++++ .../contracts/services/InjectableService.java | 30 + .../services/ListResultEntrySchema.java | 160 ++++ .../contracts/services/ListResultSchema.java | 58 ++ .../contracts/services/ReadBufferStatus.java | 29 + .../contracts/services/TracingService.java | 66 ++ .../contracts/services/package-info.java | 22 + ...se64StringConfigurationBasicValidator.java | 50 ++ .../BooleanConfigurationBasicValidator.java | 50 ++ .../ConfigurationBasicValidator.java | 67 ++ .../IntegerConfigurationBasicValidator.java | 68 ++ .../LongConfigurationBasicValidator.java | 65 ++ .../StringConfigurationBasicValidator.java | 45 ++ .../fs/azurebfs/diagnostics/package-info.java | 22 + .../apache/hadoop/fs/azurebfs/package.html | 31 + .../fs/azurebfs/services/AbfsClient.java | 402 ++++++++++ .../services/AbfsHttpClientFactoryImpl.java | 116 +++ .../fs/azurebfs/services/AbfsHttpHeader.java | 40 + .../azurebfs/services/AbfsHttpOperation.java | 430 +++++++++++ .../services/AbfsHttpServiceImpl.java | 693 ++++++++++++++++++ .../fs/azurebfs/services/AbfsInputStream.java | 382 ++++++++++ .../azurebfs/services/AbfsOutputStream.java | 335 +++++++++ .../azurebfs/services/AbfsRestOperation.java | 178 +++++ .../services/AbfsServiceInjectorImpl.java | 81 ++ .../services/AbfsServiceProviderImpl.java | 96 +++ .../services/AbfsUriQueryBuilder.java | 58 ++ .../services/ConfigurationServiceImpl.java | 317 ++++++++ .../services/ExponentialRetryPolicy.java | 141 ++++ .../azurebfs/services/LoggerSpanReceiver.java | 74 ++ .../fs/azurebfs/services/ReadBuffer.java | 139 ++++ .../azurebfs/services/ReadBufferManager.java | 391 ++++++++++ .../azurebfs/services/ReadBufferWorker.java | 72 ++ .../services/SharedKeyCredentials.java | 507 +++++++++++++ .../azurebfs/services/TracingServiceImpl.java | 134 ++++ .../fs/azurebfs/services/package-info.java | 22 + .../hadoop/fs/azurebfs/utils/UriUtils.java | 77 ++ .../fs/azurebfs/utils/package-info.java | 22 + .../fs/azurebfs/DependencyInjectedTest.java | 225 ++++++ .../ITestAzureBlobFileSystemAppend.java | 78 ++ .../ITestAzureBlobFileSystemBackCompat.java | 83 +++ .../ITestAzureBlobFileSystemCopy.java | 100 +++ .../ITestAzureBlobFileSystemCreate.java | 110 +++ .../ITestAzureBlobFileSystemDelete.java | 119 +++ .../azurebfs/ITestAzureBlobFileSystemE2E.java | 150 ++++ .../ITestAzureBlobFileSystemE2EScale.java | 133 ++++ .../ITestAzureBlobFileSystemFileStatus.java | 64 ++ .../ITestAzureBlobFileSystemFlush.java | 204 ++++++ ...ITestAzureBlobFileSystemInitAndCreate.java | 50 ++ .../ITestAzureBlobFileSystemListStatus.java | 132 ++++ .../ITestAzureBlobFileSystemMkDir.java | 88 +++ .../ITestAzureBlobFileSystemOpen.java | 41 ++ .../ITestAzureBlobFileSystemRandomRead.java | 582 +++++++++++++++ .../ITestAzureBlobFileSystemRename.java | 152 ++++ .../ITestFileSystemInitialization.java | 78 ++ .../azurebfs/ITestFileSystemRegistration.java | 82 +++ .../azurebfs/ITestWasbAbfsCompatibility.java | 202 +++++ .../constants/TestConfigurationKeys.java | 37 + .../fs/azurebfs/constants/package-info.java | 22 + .../DependencyInjectedContractTest.java | 63 ++ .../contract/ITestAbfsFileSystemContract.java | 54 ++ .../ITestAbfsFileSystemContractAppend.java | 70 ++ .../ITestAbfsFileSystemContractConcat.java | 62 ++ .../ITestAbfsFileSystemContractCreate.java | 63 ++ .../ITestAbfsFileSystemContractDelete.java | 63 ++ .../ITestAbfsFileSystemContractDistCp.java | 44 ++ ...stAbfsFileSystemContractGetFileStatus.java | 62 ++ .../ITestAbfsFileSystemContractMkdir.java | 63 ++ .../ITestAbfsFileSystemContractOpen.java | 63 ++ .../ITestAbfsFileSystemContractRename.java | 63 ++ ...stAbfsFileSystemContractRootDirectory.java | 67 ++ ...estAbfsFileSystemContractSecureDistCp.java | 44 ++ .../ITestAbfsFileSystemContractSeek.java | 63 ++ .../ITestAbfsFileSystemContractSetTimes.java | 62 ++ .../ITestAzureBlobFileSystemBasics.java | 91 +++ .../fs/azurebfs/contract/package-info.java | 22 + .../TestConfigurationValidators.java | 124 ++++ .../fs/azurebfs/diagnostics/package-info.java | 22 + .../hadoop/fs/azurebfs/package-info.java | 22 + .../services/ITestAbfsHttpServiceImpl.java | 122 +++ .../services/ITestReadWriteAndSeek.java | 80 ++ .../services/ITestTracingServiceImpl.java | 79 ++ .../MockAbfsHttpClientFactoryImpl.java | 69 ++ .../services/MockAbfsServiceInjectorImpl.java | 50 ++ .../services/MockServiceProviderImpl.java | 36 + ...tConfigurationServiceFieldsValidation.java | 149 ++++ .../fs/azurebfs/services/package-info.java | 22 + .../fs/azurebfs/utils/TestUriUtils.java | 48 ++ .../fs/azurebfs/utils/package-info.java | 22 + .../hadoop-azure/src/test/resources/abfs.xml | 64 ++ .../src/test/resources/azure-bfs-test.xml | 76 ++ .../src/test/resources/log4j.properties | 2 + 129 files changed, 12606 insertions(+), 12 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemUriSchemes.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsRestOperationException.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ConfigurationPropertyNotFoundException.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/FileSystemOperationUnhandledException.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidConfigurationValueException.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidFileSystemPropertyException.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriAuthorityException.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/TimeoutException.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ReadBufferStatus.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/package.html create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpServiceImpl.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ConfigurationServiceImpl.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/UriUtils.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOpen.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TestUriUtils.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/test/resources/abfs.xml create mode 100644 hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml diff --git a/.gitignore b/.gitignore index d5550364b8f..ac245c7575c 100644 --- a/.gitignore +++ b/.gitignore @@ -22,6 +22,7 @@ make-build-debug # Filesystem contract test options and credentials auth-keys.xml azure-auth-keys.xml +azure-bfs-auth-keys.xml # External tool builders */.externalToolBuilders 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 81502dc24c6..e15fab8ab16 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 @@ -1639,6 +1639,16 @@ SAS keys to communicate with Azure storage. + + fs.abfs.impl + org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem + The implementation class of the Azure Blob Filesystem + + + fs.abfss.impl + org.apache.hadoop.fs.azurebfs.SecureAzureBlobFileSystem + The implementation class of the Secure Azure Blob Filesystem + fs.azure.local.sas.key.mode false diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java index 023c83109e5..e10617daaba 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java @@ -113,6 +113,9 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase { xmlPrefixToSkipCompare.add("fs.wasb.impl"); xmlPrefixToSkipCompare.add("fs.wasbs.impl"); xmlPrefixToSkipCompare.add("fs.azure."); + xmlPrefixToSkipCompare.add("fs.abfs.impl"); + xmlPrefixToSkipCompare.add("fs.abfss.impl"); + // ADL properties are in a different subtree // - org.apache.hadoop.hdfs.web.ADLConfKeys diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 20a27232bef..184cb3d1ac8 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -1209,6 +1209,11 @@ jsch 0.1.54 + + org.apache.htrace + htrace-core + 3.1.0-incubating + org.apache.htrace htrace-core4 @@ -1344,6 +1349,12 @@ 7.0.0 + + org.threadly + threadly + 4.9.0 + + com.aliyun.oss aliyun-sdk-oss diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml index 52b5b726a13..d4046ef2dd7 100644 --- a/hadoop-tools/hadoop-azure/pom.xml +++ b/hadoop-tools/hadoop-azure/pom.xml @@ -67,6 +67,7 @@ src/config/checkstyle.xml + src/config/checkstyle-suppressions.xml @@ -148,11 +149,23 @@ provided + + org.threadly + threadly + compile + + com.fasterxml.jackson.core jackson-core compile + + + com.fasterxml.jackson.core + jackson-databind + compile + org.apache.httpcomponents @@ -172,9 +185,39 @@ + + org.apache.htrace + htrace-core + compile + + + + org.apache.htrace + htrace-core4 + compile + + + + com.google.inject + guice + compile + + + + com.google.guava + guava + + + + com.google.guava guava + + + + joda-time + joda-time compile @@ -183,15 +226,8 @@ jetty-util-ajax compile - - - - - commons-io - commons-io - test - + junit junit @@ -229,10 +265,6 @@ mockito-all test - - com.fasterxml.jackson.core - jackson-databind - @@ -398,8 +430,10 @@ **/ITestNativeAzureFileSystemConcurrencyLive.java **/ITestNativeAzureFileSystemLive.java **/ITestNativeAzureFSPageBlobLive.java + **/ITestAzureBlobFileSystemRandomRead.java **/ITestWasbRemoteCallHelper.java **/ITestBlockBlobInputStream.java + **/ITestWasbAbfsCompatibility.java @@ -424,12 +458,15 @@ ${fs.azure.scale.test.list.performance.files} + **/ITestWasbAbfsCompatibility.java **/ITestFileSystemOperationsExceptionHandlingMultiThreaded.java **/ITestFileSystemOperationsWithThreads.java **/ITestOutOfBandAzureBlobOperationsLive.java **/ITestNativeAzureFileSystemAuthorizationWithOwner.java **/ITestNativeAzureFileSystemConcurrencyLive.java **/ITestNativeAzureFileSystemLive.java + **/ITestNativeAzureFSPageBlobLive.java + **/ITestAzureBlobFileSystemRandomRead.java **/ITestWasbRemoteCallHelper.java **/ITestBlockBlobInputStream.java diff --git a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml new file mode 100644 index 00000000000..02043552992 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml @@ -0,0 +1,47 @@ + + + + + + + + + + + diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java new file mode 100644 index 00000000000..707e264a365 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.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; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.DelegateToFileSystem; +import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; + +/** + * Azure Blob File System implementation of AbstractFileSystem. + * This impl delegates to the old FileSystem + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class Abfs extends DelegateToFileSystem { + + Abfs(final URI theUri, final Configuration conf) throws IOException, + URISyntaxException { + super(theUri, new AzureBlobFileSystem(), conf, FileSystemUriSchemes.ABFS_SCHEME, false); + } + + @Override + public int getUriDefaultPort() { + return -1; + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java new file mode 100644 index 00000000000..19c0f7a7d62 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.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; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.DelegateToFileSystem; +import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; + +/** + * Azure Blob File System implementation of AbstractFileSystem. + * This impl delegates to the old FileSystem + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class Abfss extends DelegateToFileSystem { + + Abfss(final URI theUri, final Configuration conf) throws IOException, + URISyntaxException { + super(theUri, new SecureAzureBlobFileSystem(), conf, FileSystemUriSchemes.ABFS_SECURE_SCHEME, false); + } + + @Override + public int getUriDefaultPort() { + return -1; + } +} \ No newline at end of file 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 new file mode 100644 index 00000000000..707c81ec768 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -0,0 +1,612 @@ +/** + * 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.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.HttpURLConnection; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.EnumSet; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.threadly.util.ExceptionUtils; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl; + +import org.apache.commons.lang.ArrayUtils; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.BlockLocation; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations; +import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; +import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService; +import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; +import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService; +import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsServiceProvider; +import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.FileSystemOperationUnhandledException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.Progressable; +import org.apache.htrace.core.TraceScope; + +/** + * A {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on Windows Azure + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class AzureBlobFileSystem extends FileSystem { + public static final Logger LOG = LoggerFactory.getLogger(AzureBlobFileSystem.class); + private URI uri; + private Path workingDir; + private UserGroupInformation userGroupInformation; + private String user; + private String primaryUserGroup; + private AbfsServiceProvider abfsServiceProvider; + private TracingService tracingService; + private AbfsHttpService abfsHttpService; + private ConfigurationService configurationService; + private boolean isClosed; + + @Override + public void initialize(URI uri, Configuration configuration) + throws IOException { + uri = ensureAuthority(uri, configuration); + super.initialize(uri, configuration); + + setConf(configuration); + + try { + this.abfsServiceProvider = AbfsServiceProviderImpl.create(configuration); + this.tracingService = abfsServiceProvider.get(TracingService.class); + this.abfsHttpService = abfsServiceProvider.get(AbfsHttpService.class); + this.configurationService = abfsServiceProvider.get(ConfigurationService.class); + } catch (AzureBlobFileSystemException exception) { + throw new IOException(exception); + } + + this.LOG.debug( + "Initializing AzureBlobFileSystem for {}", uri); + + this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority()); + this.userGroupInformation = UserGroupInformation.getCurrentUser(); + this.user = userGroupInformation.getUserName(); + this.primaryUserGroup = userGroupInformation.getPrimaryGroupName(); + + this.LOG.debug( + "Initializing NativeAzureFileSystem for {}", uri); + + this.setWorkingDirectory(this.getHomeDirectory()); + + if (this.configurationService.getCreateRemoteFileSystemDuringInitialization()) { + this.createFileSystem(); + } + + this.mkdirs(this.workingDir); + } + + public boolean isSecure() { + return false; + } + + @Override + public URI getUri() { + return this.uri; + } + + @Override + public FSDataInputStream open(final Path path, final int bufferSize) throws IOException { + this.LOG.debug( + "AzureBlobFileSystem.open path: {} bufferSize: {}", path.toString(), bufferSize); + + try { + InputStream inputStream = abfsHttpService.openFileForRead(this, makeQualified(path), statistics); + return new FSDataInputStream(inputStream); + } catch(AzureBlobFileSystemException ex) { + checkException(path, ex); + return null; + } + } + + @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 { + this.LOG.debug( + "AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}", + f.toString(), + permission, + overwrite, + blockSize); + + try { + OutputStream outputStream = abfsHttpService.createFile(this, makeQualified(f), overwrite); + return new FSDataOutputStream(outputStream, statistics); + } catch(AzureBlobFileSystemException ex) { + checkException(f, ex); + return null; + } + } + + @Override + @SuppressWarnings("deprecation") + public FSDataOutputStream createNonRecursive(final Path f, final FsPermission permission, + final boolean overwrite, final int bufferSize, final short replication, final long blockSize, + final Progressable progress) throws IOException { + + final Path parent = f.getParent(); + final FileStatus parentFileStatus = tryGetFileStatus(parent); + + if (parentFileStatus == null) { + throw new FileNotFoundException("Cannot create file " + + f.getName() + " because parent folder does not exist."); + } + + return create(f, permission, overwrite, bufferSize, replication, blockSize, progress); + } + + @Override + @SuppressWarnings("deprecation") + public FSDataOutputStream createNonRecursive(final Path f, final FsPermission permission, + final EnumSet flags, final int bufferSize, final short replication, final long blockSize, + final Progressable progress) throws IOException { + + // Check if file should be appended or overwritten. Assume that the file + // is overwritten on if the CREATE and OVERWRITE create flags are set. + final EnumSet createflags = + EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE); + final boolean overwrite = flags.containsAll(createflags); + + // Delegate the create non-recursive call. + return this.createNonRecursive(f, permission, overwrite, + bufferSize, replication, blockSize, progress); + } + + @Override + @SuppressWarnings("deprecation") + public FSDataOutputStream createNonRecursive(final Path f, + final boolean overwrite, final int bufferSize, final short replication, final long blockSize, + final Progressable progress) throws IOException { + return this.createNonRecursive(f, FsPermission.getFileDefault(), + overwrite, bufferSize, replication, blockSize, progress); + } + + @Override + public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress) throws IOException { + this.LOG.debug( + "AzureBlobFileSystem.append path: {} bufferSize: {}", + f.toString(), + bufferSize); + + try { + OutputStream outputStream = abfsHttpService.openFileForWrite(this, makeQualified(f), false); + return new FSDataOutputStream(outputStream, statistics); + } catch(AzureBlobFileSystemException ex) { + checkException(f, ex); + return null; + } + } + + public boolean rename(final Path src, final Path dst) throws IOException { + this.LOG.debug( + "AzureBlobFileSystem.rename src: {} dst: {}", src.toString(), dst.toString()); + + Path parentFolder = src.getParent(); + if (parentFolder == null) { + return false; + } + + final FileStatus dstFileStatus = tryGetFileStatus(dst); + try { + String sourceFileName = src.getName(); + Path adjustedDst = dst; + + if (dstFileStatus != null) { + if (!dstFileStatus.isDirectory()) { + return src.equals(dst); + } + + adjustedDst = new Path(dst, sourceFileName); + } + + abfsHttpService.rename(this, makeQualified(src), makeQualified(adjustedDst)); + return true; + } catch(AzureBlobFileSystemException ex) { + checkException( + src, + ex, + AzureServiceErrorCode.PATH_ALREADY_EXISTS, + AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH, + AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND, + AzureServiceErrorCode.INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE, + AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND); + return false; + } + + } + + @Override + public boolean delete(final Path f, final boolean recursive) throws IOException { + this.LOG.debug( + "AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), recursive); + + if (f.isRoot()) { + if (!recursive) { + return false; + } + + return deleteRoot(); + } + + try { + abfsHttpService.delete(this, makeQualified(f), recursive); + return true; + } catch (AzureBlobFileSystemException ex) { + checkException(f, ex, AzureServiceErrorCode.PATH_NOT_FOUND); + return false; + } + + } + + @Override + public FileStatus[] listStatus(final Path f) throws IOException { + this.LOG.debug( + "AzureBlobFileSystem.listStatus path: {}", f.toString()); + + try { + FileStatus[] result = abfsHttpService.listStatus(this, makeQualified(f)); + return result; + } catch (AzureBlobFileSystemException ex) { + checkException(f, ex); + return null; + } + } + + @Override + public boolean mkdirs(final Path f, final FsPermission permission) throws IOException { + this.LOG.debug( + "AzureBlobFileSystem.mkdirs path: {} permissions: {}", f.toString(), permission); + + final Path parentFolder = f.getParent(); + if (parentFolder == null) { + // Cannot create root + return true; + } + + try { + abfsHttpService.createDirectory(this, makeQualified(f)); + return true; + } catch (AzureBlobFileSystemException ex) { + checkException(f, ex, AzureServiceErrorCode.PATH_ALREADY_EXISTS); + return true; + } + } + + @Override + public synchronized void close() throws IOException { + if (isClosed) { + return; + } + + super.close(); + this.LOG.debug("AzureBlobFileSystem.close"); + + try { + abfsHttpService.closeFileSystem(this); + } catch (AzureBlobFileSystemException ex) { + checkException(null, ex); + this.isClosed = true; + } + } + + @Override + public FileStatus getFileStatus(final Path f) throws IOException { + this.LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f.toString()); + + try { + return abfsHttpService.getFileStatus(this, makeQualified(f)); + } catch(AzureBlobFileSystemException ex) { + checkException(f, ex); + return null; + } + } + + @Override + public Path getWorkingDirectory() { + return this.workingDir; + } + + @Override + public void setWorkingDirectory(final Path newDir) { + if (newDir.isAbsolute()) { + this.workingDir = newDir; + } else { + this.workingDir = new Path(workingDir, newDir); + } + } + + @Override + public String getScheme() { + return FileSystemUriSchemes.ABFS_SCHEME; + } + + @Override + public Path getHomeDirectory() { + return makeQualified(new Path( + FileSystemConfigurations.USER_HOME_DIRECTORY_PREFIX + "/" + this.userGroupInformation.getShortUserName())); + } + + /** + * Return an array containing hostnames, offset and size of + * portions of the given file. For ABFS we'll just lie and give + * fake hosts to make sure we get many splits in MR jobs. + */ + @Override + public BlockLocation[] getFileBlockLocations(FileStatus file, + long start, long len) throws IOException { + if (file == null) { + return null; + } + + if ((start < 0) || (len < 0)) { + throw new IllegalArgumentException("Invalid start or len parameter"); + } + + if (file.getLen() < start) { + return new BlockLocation[0]; + } + final String blobLocationHost = this.configurationService.getAzureBlockLocationHost(); + + final String[] name = { blobLocationHost }; + final String[] host = { blobLocationHost }; + long blockSize = file.getBlockSize(); + if (blockSize <= 0) { + throw new IllegalArgumentException( + "The block size for the given file is not a positive number: " + + blockSize); + } + int numberOfLocations = (int) (len / blockSize) + + ((len % blockSize == 0) ? 0 : 1); + BlockLocation[] locations = new BlockLocation[numberOfLocations]; + for (int i = 0; i < locations.length; i++) { + long currentOffset = start + (i * blockSize); + long currentLength = Math.min(blockSize, start + len - currentOffset); + locations[i] = new BlockLocation(name, host, currentOffset, currentLength); + } + + return locations; + } + + public String getOwnerUser() { + return user; + } + + public String getOwnerUserPrimaryGroup() { + return primaryUserGroup; + } + + private boolean deleteRoot() throws IOException { + this.LOG.debug("Deleting root content"); + + final ExecutorService executorService = Executors.newFixedThreadPool(10); + + try { + final FileStatus[] ls = listStatus(makeQualified(new Path(File.separator))); + final ArrayList deleteTasks = new ArrayList<>(); + for (final FileStatus fs : ls) { + final Future deleteTask = executorService.submit(new Callable() { + @Override + public Void call() throws Exception { + delete(fs.getPath(), fs.isDirectory()); + return null; + } + }); + deleteTasks.add(deleteTask); + } + + for (final Future deleteTask : deleteTasks) { + execute("deleteRoot", new Callable() { + @Override + public Void call() throws Exception { + deleteTask.get(); + return null; + } + }); + } + } + finally { + executorService.shutdownNow(); + } + + return true; + } + + private FileStatus tryGetFileStatus(final Path f) { + try { + return getFileStatus(f); + } + catch (IOException ex) { + this.LOG.debug("File not found {}", f.toString()); + return null; + } + } + + private void createFileSystem() throws IOException { + this.LOG.debug( + "AzureBlobFileSystem.createFileSystem uri: {}", uri); + try { + abfsHttpService.createFilesystem(this); + } catch (AzureBlobFileSystemException ex) { + checkException(null, ex, AzureServiceErrorCode.FILE_SYSTEM_ALREADY_EXISTS); + } + + + } + + private URI ensureAuthority(URI uri, final Configuration conf) { + + Preconditions.checkNotNull(uri, "uri"); + + if (uri.getAuthority() == null) { + final URI defaultUri = FileSystem.getDefaultUri(conf); + + if (defaultUri != null && isAbfsScheme(defaultUri.getScheme())) { + try { + // Reconstruct the URI with the authority from the default URI. + uri = new URI( + uri.getScheme(), + defaultUri.getAuthority(), + uri.getPath(), + uri.getQuery(), + uri.getFragment()); + } catch (URISyntaxException e) { + // This should never happen. + throw new IllegalArgumentException(new InvalidUriException(uri.toString())); + } + } + } + + if (uri.getAuthority() == null) { + throw new IllegalArgumentException(new InvalidUriAuthorityException(uri.toString())); + } + + return uri; + } + + private boolean isAbfsScheme(final String scheme) { + if (scheme == null) { + return false; + } + + if (scheme.equals(FileSystemUriSchemes.ABFS_SCHEME) || scheme.equals(FileSystemUriSchemes.ABFS_SECURE_SCHEME)) { + return true; + } + + return false; + } + + @VisibleForTesting + FileSystemOperation execute( + final String scopeDescription, + final Callable callableFileOperation) throws IOException { + return execute(scopeDescription, callableFileOperation, null); + } + + @VisibleForTesting + FileSystemOperation execute( + final String scopeDescription, + final Callable callableFileOperation, + T defaultResultValue) throws IOException { + + final TraceScope traceScope = tracingService.traceBegin(scopeDescription); + try { + final T executionResult = callableFileOperation.call(); + return new FileSystemOperation(executionResult, null); + } catch (AbfsRestOperationException abfsRestOperationException) { + return new FileSystemOperation(defaultResultValue, abfsRestOperationException); + } catch (AzureBlobFileSystemException azureBlobFileSystemException) { + tracingService.traceException(traceScope, azureBlobFileSystemException); + throw new IOException(azureBlobFileSystemException); + } catch (Exception exception) { + if (exception instanceof ExecutionException) { + exception = (Exception) ExceptionUtils.getRootCause(exception); + } + + final FileSystemOperationUnhandledException fileSystemOperationUnhandledException = new FileSystemOperationUnhandledException(exception); + tracingService.traceException(traceScope, fileSystemOperationUnhandledException); + throw new IOException(fileSystemOperationUnhandledException); + } finally { + tracingService.traceEnd(traceScope); + } + } + + private void checkException(final Path path, + final AzureBlobFileSystemException exception, + final AzureServiceErrorCode... allowedErrorCodesList) throws IOException { + if (exception instanceof AbfsRestOperationException) { + AbfsRestOperationException ere = (AbfsRestOperationException) exception; + + if (ArrayUtils.contains(allowedErrorCodesList, ere.getErrorCode())) { + return; + } + int statusCode = ere.getStatusCode(); + + //AbfsRestOperationException.getMessage() contains full error info including path/uri. + if (statusCode == HttpURLConnection.HTTP_NOT_FOUND) { + throw new FileNotFoundException(ere.getMessage()); + } else if (statusCode == HttpURLConnection.HTTP_CONFLICT) { + throw new FileAlreadyExistsException(ere.getMessage()); + } else { + throw ere; + } + } else { + if (path == null) { + throw exception; + } + // record info of path + throw new PathIOException(path.toString(), exception); + } + } + + @VisibleForTesting + FileSystem.Statistics getFsStatistics() { + return this.statistics; + } + + @VisibleForTesting + static class FileSystemOperation { + private final T result; + private final AbfsRestOperationException exception; + + FileSystemOperation(final T result, final AbfsRestOperationException exception) { + this.result = result; + this.exception = exception; + } + + public boolean failed() { + return this.exception != null; + } + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.java new file mode 100644 index 00000000000..a6ad8299231 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.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; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; + +/** + * A secure {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on Windows Azure + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class SecureAzureBlobFileSystem extends AzureBlobFileSystem { + @Override + public boolean isSecure() { + return true; + } + + @Override + public String getScheme() { + return FileSystemUriSchemes.ABFS_SECURE_SCHEME; + } +} \ No newline at end of file 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 new file mode 100644 index 00000000000..2ec4db0197c --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java @@ -0,0 +1,76 @@ +/** + * 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; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Responsible to keep all constant keys used in abfs rest client here + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class AbfsHttpConstants { + // Abfs Http client constants + public static final String FILESYSTEM = "filesystem"; + public static final String FILE = "file"; + public static final String DIRECTORY = "directory"; + public static final String APPEND_ACTION = "append"; + public static final String FLUSH_ACTION = "flush"; + public static final String SET_PROPERTIES_ACTION = "setProperties"; + public static final String DEFAULT_TIMEOUT = "90"; + + public static final String JAVA_VERSION = "java.version"; + public static final String OS_NAME = "os.name"; + public static final String OS_VERSION = "os.version"; + + public static final String CLIENT_VERSION = "Azure Blob FS/1.0"; + + // Abfs Http Verb + public static final String HTTP_METHOD_DELETE = "DELETE"; + public static final String HTTP_METHOD_GET = "GET"; + public static final String HTTP_METHOD_HEAD = "HEAD"; + public static final String HTTP_METHOD_PATCH = "PATCH"; + public static final String HTTP_METHOD_POST = "POST"; + public static final String HTTP_METHOD_PUT = "PUT"; + + // Abfs generic constants + public static final String SINGLE_WHITE_SPACE = " "; + public static final String EMPTY_STRING = ""; + public static final String FORWARD_SLASH = "/"; + public static final String DOT = "."; + public static final String PLUS = "+"; + public static final String STAR = "*"; + public static final String COMMA = ","; + public static final String EQUAL = "="; + public static final String QUESTION_MARK = "?"; + public static final String AND_MARK = "&"; + public static final String SEMICOLON = ";"; + public static final String HTTP_HEADER_PREFIX = "x-ms-"; + + public static final String PLUS_ENCODE = "%20"; + public static final String FORWARD_SLASH_ENCODE = "%2F"; + public static final String AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER = "@"; + public static final String UTF_8 = "utf-8"; + 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"; + + 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 new file mode 100644 index 00000000000..ead1003982b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -0,0 +1,57 @@ +/** + * 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; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Responsible to keep all the Azure Blob File System configurations keys in Hadoop configuration file. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class ConfigurationKeys { + 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"; + + // Retry strategy defined by the user + public static final String AZURE_MIN_BACKOFF_INTERVAL = "fs.azure.io.retry.min.backoff.interval"; + public static final String AZURE_MAX_BACKOFF_INTERVAL = "fs.azure.io.retry.max.backoff.interval"; + public static final String AZURE_BACKOFF_INTERVAL = "fs.azure.io.retry.backoff.interval"; + public static final String AZURE_MAX_IO_RETRIES = "fs.azure.io.retry.max.retries"; + + // Remove this and use common azure storage emulator property for public release. + public static final String FS_AZURE_EMULATOR_ENABLED = "fs.azure.abfs.emulator.enabled"; + + // Read and write buffer sizes defined by the user + public static final String AZURE_WRITE_BUFFER_SIZE = "fs.azure.write.request.size"; + public static final String AZURE_READ_BUFFER_SIZE = "fs.azure.read.request.size"; + public static final String AZURE_BLOCK_SIZE_PROPERTY_NAME = "fs.azure.block.size"; + public static final String AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME = "fs.azure.block.location.impersonatedhost"; + public static final String AZURE_CONCURRENT_CONNECTION_VALUE_OUT = "fs.azure.concurrentRequestCount.out"; + public static final String AZURE_CONCURRENT_CONNECTION_VALUE_IN = "fs.azure.concurrentRequestCount.in"; + public static final String AZURE_TOLERATE_CONCURRENT_APPEND = "fs.azure.io.read.tolerate.concurrent.append"; + public static final String AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION = "fs.azure.createRemoteFileSystemDuringInitialization"; + public static final String FS_AZURE_AUTOTHROTTLING_ENABLE = "fs.azure.autothrottling.enable"; + public static final String FS_AZURE_ATOMIC_RENAME_KEY = "fs.azure.atomic.rename.key"; + public static final String FS_AZURE_READ_AHEAD_QUEUE_DEPTH = "fs.azure.readaheadqueue.depth"; + + 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 new file mode 100644 index 00000000000..482158c1d73 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -0,0 +1,59 @@ +/** + * 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; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Responsible to keep all the Azure Blob File System related configurations. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class FileSystemConfigurations { + public static final String USER_HOME_DIRECTORY_PREFIX = "/user"; + + // Retry parameter defaults. + public static final int DEFAULT_MIN_BACKOFF_INTERVAL = 3 * 1000; // 3s + public static final int DEFAULT_MAX_BACKOFF_INTERVAL = 30 * 1000; // 30s + public static final int DEFAULT_BACKOFF_INTERVAL = 3 * 1000; // 3s + public static final int DEFAULT_MAX_RETRY_ATTEMPTS = 30; + + private static final int ONE_KB = 1024; + private static final int ONE_MB = ONE_KB * ONE_KB; + + // Default upload and download buffer size + public static final int DEFAULT_WRITE_BUFFER_SIZE = 4 * ONE_MB; // 4 MB + public static final int DEFAULT_READ_BUFFER_SIZE = 4 * ONE_MB; // 4 MB + public static final int MIN_BUFFER_SIZE = 16 * ONE_KB; // 16 KB + public static final int MAX_BUFFER_SIZE = 100 * ONE_MB; // 100 MB + public static final long MAX_AZURE_BLOCK_SIZE = 512 * 1024 * 1024L; + public static final String AZURE_BLOCK_LOCATION_HOST_DEFAULT = "localhost"; + + public static final int MAX_CONCURRENT_READ_THREADS = 12; + public static final int MAX_CONCURRENT_WRITE_THREADS = 8; + public static final boolean DEFAULT_READ_TOLERATE_CONCURRENT_APPEND = false; + public static final boolean DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION = false; + + public static final String DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES = "/hbase"; + + public static final int DEFAULT_READ_AHEAD_QUEUE_DEPTH = -1; + + private FileSystemConfigurations() {} +} \ No newline at end of file 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 new file mode 100644 index 00000000000..c7a0cdad605 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemUriSchemes.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.constants; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Responsible to keep all Azure Blob File System valid URI schemes. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +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 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"; + + 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 new file mode 100644 index 00000000000..9b7f9bc6ce7 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java @@ -0,0 +1,57 @@ +/** + * 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; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Responsible to keep all abfs http headers here + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class HttpHeaderConfigurations { + public static final String ACCEPT = "Accept"; + public static final String ACCEPT_CHARSET = "Accept-Charset"; + public static final String AUTHORIZATION = "Authorization"; + public static final String IF_MODIFIED_SINCE = "If-Modified-Since"; + public static final String IF_UNMODIFIED_SINCE = "If-Unmodified-Since"; + public static final String IF_MATCH = "If-Match"; + public static final String IF_NONE_MATCH = "If-None-Match"; + public static final String CONTENT_LENGTH = "Content-Length"; + public static final String CONTENT_ENCODING = "Content-Encoding"; + public static final String CONTENT_LANGUAGE = "Content-Language"; + public static final String CONTENT_MD5 = "Content-MD5"; + public static final String CONTENT_TYPE = "Content-Type"; + public static final String RANGE = "Range"; + public static final String TRANSFER_ENCODING = "Transfer-Encoding"; + public static final String USER_AGENT = "User-Agent"; + public static final String X_HTTP_METHOD_OVERRIDE = "X-HTTP-Method-Override"; + public static final String X_MS_CLIENT_REQUEST_ID = "x-ms-client-request-id"; + public static final String X_MS_DATE = "x-ms-date"; + public static final String X_MS_REQUEST_ID = "x-ms-request-id"; + public static final String X_MS_VERSION = "x-ms-version"; + public static final String X_MS_RESOURCE_TYPE = "x-ms-resource-type"; + public static final String X_MS_CONTINUATION = "x-ms-continuation"; + public static final String ETAG = "ETag"; + public static final String X_MS_PROPERTIES = "x-ms-properties"; + public static final String X_MS_RENAME_SOURCE = "x-ms-rename-source"; + public static final String LAST_MODIFIED = "Last-Modified"; + + 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 new file mode 100644 index 00000000000..a9f7d3350b7 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java @@ -0,0 +1,40 @@ +/** + * 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; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Responsible to keep all Http Query params here + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class HttpQueryParams { + public static final String QUERY_PARAM_RESOURCE = "resource"; + public static final String QUERY_PARAM_DIRECTORY = "directory"; + public static final String QUERY_PARAM_CONTINUATION = "continuation"; + public static final String QUERY_PARAM_RECURSIVE = "recursive"; + public static final String QUERY_PARAM_MAXRESULTS = "maxResults"; + public static final String QUERY_PARAM_ACTION = "action"; + public static final String QUERY_PARAM_POSITION = "position"; + public static final String QUERY_PARAM_TIMEOUT = "timeout"; + public static final String QUERY_PARAM_RETAIN_UNCOMMITTED_DATA = "retainUncommittedData"; + + private HttpQueryParams() {} +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java new file mode 100644 index 00000000000..e6a471bca8d --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +package org.apache.hadoop.fs.azurebfs.constants; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java new file mode 100644 index 00000000000..462ebbc88f6 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java @@ -0,0 +1,104 @@ +/** + * 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.contracts.annotations; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Definitions of Annotations for all types of the validators + */ +@InterfaceStability.Evolving +public class ConfigurationValidationAnnotations { + /** + * Describes the requirements when validating the annotated int field + */ + @Target({ ElementType.FIELD }) + @Retention(RetentionPolicy.RUNTIME) + public @interface IntegerConfigurationValidatorAnnotation { + String ConfigurationKey(); + + int MaxValue() default Integer.MAX_VALUE; + + int MinValue() default Integer.MIN_VALUE; + + int DefaultValue(); + + boolean ThrowIfInvalid() default false; + } + + /** + * Describes the requirements when validating the annotated long field + */ + @Target({ ElementType.FIELD }) + @Retention(RetentionPolicy.RUNTIME) + public @interface LongConfigurationValidatorAnnotation { + String ConfigurationKey(); + + long MaxValue() default Long.MAX_VALUE; + + long MinValue() default Long.MIN_VALUE; + + long DefaultValue(); + + boolean ThrowIfInvalid() default false; + } + + /** + * Describes the requirements when validating the annotated String field + */ + @Target({ ElementType.FIELD }) + @Retention(RetentionPolicy.RUNTIME) + public @interface StringConfigurationValidatorAnnotation { + String ConfigurationKey(); + + String DefaultValue(); + + boolean ThrowIfInvalid() default false; + } + + /** + * Describes the requirements when validating the annotated String field + */ + @Target({ ElementType.FIELD }) + @Retention(RetentionPolicy.RUNTIME) + public @interface Base64StringConfigurationValidatorAnnotation { + String ConfigurationKey(); + + String DefaultValue(); + + boolean ThrowIfInvalid() default false; + } + + /** + * Describes the requirements when validating the annotated boolean field + */ + @Target({ ElementType.FIELD }) + @Retention(RetentionPolicy.RUNTIME) + public @interface BooleanConfigurationValidatorAnnotation { + String ConfigurationKey(); + + boolean DefaultValue(); + + boolean ThrowIfInvalid() default false; + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/package-info.java new file mode 100644 index 00000000000..0fc4deb3a2b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +package org.apache.hadoop.fs.azurebfs.contracts.annotations; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java new file mode 100644 index 00000000000..796f7859567 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.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.contracts.diagnostics; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; + +/** + * ConfigurationValidator to validate the value of a configuration key + * @param the type of the validator and the validated value + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface ConfigurationValidator { + /** + * Validates the configValue + * @return validated value of type T + */ + T validate(String configValue) throws InvalidConfigurationValueException; +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/package-info.java new file mode 100644 index 00000000000..f8d27b28bf4 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +package org.apache.hadoop.fs.azurebfs.contracts.diagnostics; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file 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 new file mode 100644 index 00000000000..f0b69ef91de --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsRestOperationException.java @@ -0,0 +1,84 @@ +/** + * 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.contracts.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; +import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; + +/** + * Exception to wrap Azure service error responses. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class AbfsRestOperationException extends AzureBlobFileSystemException { + private final int statusCode; + private final AzureServiceErrorCode errorCode; + private final String errorMessage; + + public AbfsRestOperationException( + final int statusCode, + final String errorCode, + final String errorMessage, + final Exception innerException) { + super("Status code: " + statusCode + " error code: " + errorCode + " error message: " + errorMessage, innerException); + + this.statusCode = statusCode; + this.errorCode = AzureServiceErrorCode.getAzureServiceCode(this.statusCode, errorCode); + this.errorMessage = errorMessage; + } + + public AbfsRestOperationException( + final int statusCode, + final String errorCode, + final String errorMessage, + final Exception innerException, + final AbfsHttpOperation abfsHttpOperation) { + super(formatMessage(abfsHttpOperation)); + + this.statusCode = statusCode; + this.errorCode = AzureServiceErrorCode.getAzureServiceCode(this.statusCode, errorCode); + this.errorMessage = errorMessage; + } + + public int getStatusCode() { + return this.statusCode; + } + + public AzureServiceErrorCode getErrorCode() { + return this.errorCode; + } + + public String getErrorMessage() { + return this.errorMessage; + } + + private static String formatMessage(final AbfsHttpOperation abfsHttpOperation) { + return String.format( + "%1$s %2$s%nStatusCode=%3$s%nStatusDescription=%4$s%nErrorCode=%5$s%nErrorMessage=%6$s", + abfsHttpOperation.getMethod(), + abfsHttpOperation.getUrl().toString(), + abfsHttpOperation.getStatusCode(), + abfsHttpOperation.getStatusDescription(), + abfsHttpOperation.getStorageErrorCode(), + abfsHttpOperation.getStorageErrorMessage()); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java new file mode 100644 index 00000000000..f31c680628b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java @@ -0,0 +1,56 @@ +/** + * 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.contracts.exceptions; + +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Base exception for any Azure Blob File System driver exceptions. All the exceptions must inherit this class. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public abstract class AzureBlobFileSystemException extends IOException { + public AzureBlobFileSystemException(final String message) { + super(message); + } + + public AzureBlobFileSystemException(final String message, final Exception innerException) { + super(message, innerException); + } + + @Override + public String toString() { + if (this.getMessage() == null && this.getCause() == null) { + return ""; + } + + if (this.getCause() == null) { + return this.getMessage(); + } + + if (this.getMessage() == null) { + return this.getCause().toString(); + } + + return this.getMessage() + this.getCause().toString(); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ConfigurationPropertyNotFoundException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ConfigurationPropertyNotFoundException.java new file mode 100644 index 00000000000..bf3b2f34d6e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ConfigurationPropertyNotFoundException.java @@ -0,0 +1,32 @@ +/** + * 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.contracts.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Thrown when a searched for element is not found + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class ConfigurationPropertyNotFoundException extends AzureBlobFileSystemException { + public ConfigurationPropertyNotFoundException(String property) { + super("Configuration property " + property + "not found."); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/FileSystemOperationUnhandledException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/FileSystemOperationUnhandledException.java new file mode 100644 index 00000000000..484c8385b35 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/FileSystemOperationUnhandledException.java @@ -0,0 +1,33 @@ +/** + * 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.contracts.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Thrown when an unhandled exception is occurred during a file system operation. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class FileSystemOperationUnhandledException extends AzureBlobFileSystemException { + public FileSystemOperationUnhandledException(Exception innerException) { + super("An unhandled file operation exception", innerException); + } +} \ No newline at end of file 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 new file mode 100644 index 00000000000..aba1d8c1efa --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java @@ -0,0 +1,40 @@ +/** + * 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.contracts.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; + +/** + * Exception to wrap invalid Azure service error responses. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class InvalidAbfsRestOperationException extends AbfsRestOperationException { + public InvalidAbfsRestOperationException( + final Exception innerException) { + super( + AzureServiceErrorCode.UNKNOWN.getStatusCode(), + AzureServiceErrorCode.UNKNOWN.getErrorCode(), + "InvalidAbfsRestOperationException", + innerException); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidConfigurationValueException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidConfigurationValueException.java new file mode 100644 index 00000000000..7591bac59e2 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidConfigurationValueException.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.contracts.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Thrown when a configuration value is invalid + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class InvalidConfigurationValueException extends AzureBlobFileSystemException { + public InvalidConfigurationValueException(String configKey, Exception innerException) { + super("Invalid configuration value detected for " + configKey, innerException); + } + + public InvalidConfigurationValueException(String configKey) { + super("Invalid configuration value detected for " + configKey); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidFileSystemPropertyException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidFileSystemPropertyException.java new file mode 100644 index 00000000000..5823fd2c589 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidFileSystemPropertyException.java @@ -0,0 +1,33 @@ +/** + * 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.contracts.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Thrown when a file system property is invalid. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class InvalidFileSystemPropertyException extends AzureBlobFileSystemException { + public InvalidFileSystemPropertyException(String property) { + super(String.format("%s is invalid.", property)); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriAuthorityException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriAuthorityException.java new file mode 100644 index 00000000000..7aa319c90c8 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriAuthorityException.java @@ -0,0 +1,33 @@ +/** + * 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.contracts.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Thrown when URI authority is invalid. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class InvalidUriAuthorityException extends AzureBlobFileSystemException { + public InvalidUriAuthorityException(String url) { + super(String.format("%s has invalid authority.", url)); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.java new file mode 100644 index 00000000000..a84495afc61 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.java @@ -0,0 +1,33 @@ +/** + * 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.contracts.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Thrown when URI is invalid. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class InvalidUriException extends AzureBlobFileSystemException { + public InvalidUriException(String url) { + super(String.format("%s is invalid.", url)); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.java new file mode 100644 index 00000000000..694d9023c59 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.java @@ -0,0 +1,36 @@ +/** + * 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.contracts.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl; + +/** + * Thrown a service is either not configured to be injected or the service is not existing. + * For service registration + * @see AbfsServiceProviderImpl + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class ServiceResolutionException extends AzureBlobFileSystemException { + public ServiceResolutionException(String serviceName, Exception innerException) { + super(String.format("%s cannot be resolved.", serviceName), innerException); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/TimeoutException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/TimeoutException.java new file mode 100644 index 00000000000..8dd5d71d683 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/TimeoutException.java @@ -0,0 +1,33 @@ +/** + * 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.contracts.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Thrown when a timeout happens. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class TimeoutException extends AzureBlobFileSystemException { + public TimeoutException(String message) { + super(message); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/package-info.java new file mode 100644 index 00000000000..e4c75f460f9 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +package org.apache.hadoop.fs.azurebfs.contracts.exceptions; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/package-info.java new file mode 100644 index 00000000000..67f5633c3a7 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +package org.apache.hadoop.fs.azurebfs.contracts; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.java new file mode 100644 index 00000000000..c433f9a216f --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.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.contracts.services; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; + +/** + * AbfsClient factory. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface AbfsHttpClientFactory extends InjectableService { + /** + * Creates and configures an instance of new AbfsClient + * @return AbfsClient instance + */ + AbfsClient create(AzureBlobFileSystem fs) throws AzureBlobFileSystemException; +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java new file mode 100644 index 00000000000..3107fa3b01f --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java @@ -0,0 +1,162 @@ +/** + * 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.contracts.services; + +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Hashtable; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FileStatus; +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.AzureBlobFileSystemException; + +/** + * File System http service to provide network calls for file system operations. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface AbfsHttpService extends InjectableService { + /** + * Gets filesystem properties on the Azure service. + * @param azureBlobFileSystem filesystem to get the properties. + * @return Hashtable hash table containing all the filesystem properties. + */ + Hashtable getFilesystemProperties(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException; + + + /** + * Sets filesystem properties on the Azure service. + * @param azureBlobFileSystem filesystem to get the properties. + * @param properties file system properties to set. + */ + void setFilesystemProperties(AzureBlobFileSystem azureBlobFileSystem, Hashtable properties) throws + AzureBlobFileSystemException; + + /** + * Gets path properties on the Azure service. + * @param azureBlobFileSystem filesystem to get the properties of the path. + * @param path path to get properties. + * @return Hashtable hash table containing all the path properties. + */ + Hashtable getPathProperties(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException; + + /** + * Sets path properties on the Azure service. + * @param azureBlobFileSystem filesystem to get the properties of the path. + * @param path path to set properties. + * @param properties hash table containing all the path properties. + */ + void setPathProperties(AzureBlobFileSystem azureBlobFileSystem, Path path, Hashtable properties) throws + AzureBlobFileSystemException; + + /** + * Creates filesystem on the Azure service. + * @param azureBlobFileSystem filesystem to be created. + */ + void createFilesystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException; + + /** + * Deletes filesystem on the Azure service. + * @param azureBlobFileSystem filesystem to be deleted. + */ + void deleteFilesystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException; + + /** + * Creates a file on the Azure service. + * @param azureBlobFileSystem filesystem to create file or directory. + * @param path path of the file to be created. + * @param overwrite should overwrite. + * @return OutputStream stream to the file. + */ + OutputStream createFile(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean overwrite) throws AzureBlobFileSystemException; + + /** + * Creates a directory on the Azure service. + * @param azureBlobFileSystem filesystem to create file or directory. + * @param path path of the directory to be created. + * @return OutputStream stream to the file. + */ + Void createDirectory(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException; + + /** + * Opens a file to read and returns the stream. + * @param azureBlobFileSystem filesystem to read a file from. + * @param path file path to read. + * @return InputStream a stream to the file to read. + */ + InputStream openFileForRead(AzureBlobFileSystem azureBlobFileSystem, Path path, FileSystem.Statistics statistics) throws AzureBlobFileSystemException; + + /** + * Opens a file to write and returns the stream. + * @param azureBlobFileSystem filesystem to write a file to. + * @param path file path to write. + * @param overwrite should overwrite. + * @return OutputStream a stream to the file to write. + */ + OutputStream openFileForWrite(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean overwrite) throws AzureBlobFileSystemException; + + /** + * Renames a file or directory from source to destination. + * @param azureBlobFileSystem filesystem to rename a path. + * @param source source path. + * @param destination destination path. + */ + void rename(AzureBlobFileSystem azureBlobFileSystem, Path source, Path destination) throws AzureBlobFileSystemException; + + /** + * Deletes a file or directory. + * @param azureBlobFileSystem filesystem to delete the path. + * @param path file path to be deleted. + * @param recursive true if path is a directory and recursive deletion is desired. + */ + void delete(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean recursive) throws AzureBlobFileSystemException; + + /** + * Gets path's status under the provided path on the Azure service. + * @param azureBlobFileSystem filesystem to perform the get file status operation. + * @param path path delimiter. + * @return FileStatus FileStatus of the path in the file system. + */ + FileStatus getFileStatus(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException; + + /** + * Lists all the paths under the provided path on the Azure service. + * @param azureBlobFileSystem filesystem to perform the list operation. + * @param path path delimiter. + * @return FileStatus[] list of all paths in the file system. + */ + FileStatus[] listStatus(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException; + + /** + * Closes the client to filesystem to Azure service. + * @param azureBlobFileSystem filesystem to perform the list operation. + */ + void closeFileSystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException; + + /** + * Checks for the given path if it is marked as atomic rename directory or not. + * @param key + * @return True if the given path is listed under atomic rename property otherwise False. + */ + boolean isAtomicRenameKey(String key); +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java new file mode 100644 index 00000000000..bd98baec4e7 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java @@ -0,0 +1,40 @@ +/** + * 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.contracts.services; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ServiceResolutionException; + +/** + * Dependency injected Azure Storage services provider interface. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface AbfsServiceProvider { + /** + * Returns an instance of resolved injectable service by class name. + * The injectable service must be configured first to be resolvable. + * @param clazz the injectable service which is expected to be returned. + * @param The type of injectable service. + * @return T instance + * @throws ServiceResolutionException if the service is not resolvable. + */ + T get(Class clazz) throws ServiceResolutionException; +} \ No newline at end of file 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 new file mode 100644 index 00000000000..90e580f9a68 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java @@ -0,0 +1,112 @@ +/** + * 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.contracts.services; + +import java.net.HttpURLConnection; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Azure service error codes. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public enum AzureServiceErrorCode { + FILE_SYSTEM_ALREADY_EXISTS("FilesystemAlreadyExists", HttpURLConnection.HTTP_CONFLICT, null), + PATH_ALREADY_EXISTS("PathAlreadyExists", HttpURLConnection.HTTP_CONFLICT, null), + PATH_CONFLICT("PathConflict", HttpURLConnection.HTTP_CONFLICT, null), + FILE_SYSTEM_NOT_FOUND("FilesystemNotFound", HttpURLConnection.HTTP_NOT_FOUND, null), + PATH_NOT_FOUND("PathNotFound", HttpURLConnection.HTTP_NOT_FOUND, null), + PRE_CONDITION_FAILED("PreconditionFailed", HttpURLConnection.HTTP_PRECON_FAILED, null), + SOURCE_PATH_NOT_FOUND("SourcePathNotFound", HttpURLConnection.HTTP_NOT_FOUND, null), + INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE("InvalidSourceOrDestinationResourceType", HttpURLConnection.HTTP_CONFLICT, null), + RENAME_DESTINATION_PARENT_PATH_NOT_FOUND("RenameDestinationParentPathNotFound", HttpURLConnection.HTTP_NOT_FOUND, null), + INVALID_RENAME_SOURCE_PATH("InvalidRenameSourcePath", HttpURLConnection.HTTP_CONFLICT, null), + INGRESS_OVER_ACCOUNT_LIMIT(null, HttpURLConnection.HTTP_UNAVAILABLE, "Ingress is over the account limit."), + EGRESS_OVER_ACCOUNT_LIMIT(null, HttpURLConnection.HTTP_UNAVAILABLE, "Egress is over the account limit."), + UNKNOWN(null, -1, null); + + private final String errorCode; + private final int httpStatusCode; + private final String errorMessage; + AzureServiceErrorCode(String errorCode, int httpStatusCodes, String errorMessage) { + this.errorCode = errorCode; + this.httpStatusCode = httpStatusCodes; + this.errorMessage = errorMessage; + } + + public int getStatusCode() { + return this.httpStatusCode; + } + + public String getErrorCode() { + return this.errorCode; + } + + public static List getAzureServiceCode(int httpStatusCode) { + List errorCodes = new ArrayList<>(); + if (httpStatusCode == UNKNOWN.httpStatusCode) { + errorCodes.add(UNKNOWN); + return errorCodes; + } + + for (AzureServiceErrorCode azureServiceErrorCode : AzureServiceErrorCode.values()) { + if (azureServiceErrorCode.httpStatusCode == httpStatusCode) { + errorCodes.add(azureServiceErrorCode); + } + } + + return errorCodes; + } + + public static AzureServiceErrorCode getAzureServiceCode(int httpStatusCode, String errorCode) { + if (errorCode == null || errorCode.isEmpty() || httpStatusCode == UNKNOWN.httpStatusCode) { + return UNKNOWN; + } + + for (AzureServiceErrorCode azureServiceErrorCode : AzureServiceErrorCode.values()) { + if (errorCode.equalsIgnoreCase(azureServiceErrorCode.errorCode) + && azureServiceErrorCode.httpStatusCode == httpStatusCode) { + return azureServiceErrorCode; + } + } + + return UNKNOWN; + } + + public static AzureServiceErrorCode getAzureServiceCode(int httpStatusCode, String errorCode, final String errorMessage) { + if (errorCode == null || errorCode.isEmpty() || httpStatusCode == UNKNOWN.httpStatusCode || errorMessage == null || errorMessage.isEmpty()) { + return UNKNOWN; + } + + for (AzureServiceErrorCode azureServiceErrorCode : AzureServiceErrorCode.values()) { + if (azureServiceErrorCode.httpStatusCode == httpStatusCode + && errorCode.equalsIgnoreCase(azureServiceErrorCode.errorCode) + && errorMessage.equalsIgnoreCase(azureServiceErrorCode.errorMessage) + ) { + return azureServiceErrorCode; + } + } + + return UNKNOWN; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java new file mode 100644 index 00000000000..ee40c9d4951 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java @@ -0,0 +1,143 @@ +/** + * 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.contracts.services; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException; + +/** + * Configuration service collects required Azure Hadoop configurations and provides it to the consumers. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface ConfigurationService extends InjectableService { + /** + * Checks if ABFS is running from Emulator; + * @return is emulator mode. + */ + boolean isEmulator(); + + /** + * Retrieves storage secure mode from Hadoop configuration; + * @return storage secure mode; + */ + boolean isSecureMode(); + + /** + * Retrieves storage account key for provided account name from Hadoop configuration. + * @param accountName the account name to retrieve the key. + * @return storage account key; + */ + String getStorageAccountKey(String accountName) throws ConfigurationPropertyNotFoundException; + + /** + * Returns Hadoop configuration. + * @return Hadoop configuration. + */ + Configuration getConfiguration(); + + /** + * Retrieves configured write buffer size + * @return the size of the write buffer + */ + int getWriteBufferSize(); + + /** + * Retrieves configured read buffer size + * @return the size of the read buffer + */ + int getReadBufferSize(); + + /** + * Retrieves configured min backoff interval + * @return min backoff interval + */ + int getMinBackoffIntervalMilliseconds(); + + /** + * Retrieves configured max backoff interval + * @return max backoff interval + */ + int getMaxBackoffIntervalMilliseconds(); + + /** + * Retrieves configured backoff interval + * @return backoff interval + */ + int getBackoffIntervalMilliseconds(); + + /** + * Retrieves configured num of retries + * @return num of retries + */ + int getMaxIoRetries(); + + /** + * Retrieves configured azure block size + * @return azure block size + */ + long getAzureBlockSize(); + + /** + * Retrieves configured azure block location host + * @return azure block location host + */ + String getAzureBlockLocationHost(); + + /** + * Retrieves configured number of concurrent threads + * @return number of concurrent write threads + */ + int getMaxConcurrentWriteThreads(); + + /** + * Retrieves configured number of concurrent threads + * @return number of concurrent read threads + */ + int getMaxConcurrentReadThreads(); + + /** + * Retrieves configured boolean for tolerating out of band writes to files + * @return configured boolean for tolerating out of band writes to files + */ + boolean getTolerateOobAppends(); + + /** + * Retrieves the comma-separated list of directories to receive special treatment so that folder + * rename is made atomic. The default value for this setting is just '/hbase'. + * Example directories list : /hbase,/data + * @see AtomicRenameProperty + * @return atomic rename directories + */ + String getAzureAtomicRenameDirs(); + + /** + * Retrieves configured boolean for creating remote file system during initialization + * @return configured boolean for creating remote file system during initialization + */ + boolean getCreateRemoteFileSystemDuringInitialization(); + + /** + * Retrieves configured value of read ahead queue + * @return depth of read ahead + */ + int getReadAheadQueueDepth(); +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.java new file mode 100644 index 00000000000..8b3801f8e41 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.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.contracts.services; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Marker interface for all the injectable services. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface InjectableService { +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java new file mode 100644 index 00000000000..02a7ac9b01c --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java @@ -0,0 +1,160 @@ +/** + * 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.contracts.services; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.apache.hadoop.classification.InterfaceStability; + +/** + * The ListResultEntrySchema model. + */ +@InterfaceStability.Evolving +public class ListResultEntrySchema { + /** + * The name property. + */ + @JsonProperty(value = "name") + private String name; + + /** + * The isDirectory property. + */ + @JsonProperty(value = "isDirectory") + private Boolean isDirectory; + + /** + * The lastModified property. + */ + @JsonProperty(value = "lastModified") + private String lastModified; + + /** + * The eTag property. + */ + @JsonProperty(value = "etag") + private String eTag; + + /** + * The contentLength property. + */ + @JsonProperty(value = "contentLength") + private Long contentLength; + + /** + * Get the name value. + * + * @return the name value + */ + public String name() { + return this.name; + } + + /** + * Set the name value. + * + * @param name the name value to set + * @return the ListEntrySchema object itself. + */ + public ListResultEntrySchema withName(String name) { + this.name = name; + return this; + } + + /** + * Get the isDirectory value. + * + * @return the isDirectory value + */ + public Boolean isDirectory() { + return this.isDirectory; + } + + /** + * Set the isDirectory value. + * + * @param isDirectory the isDirectory value to set + * @return the ListEntrySchema object itself. + */ + public ListResultEntrySchema withIsDirectory(final Boolean isDirectory) { + this.isDirectory = isDirectory; + return this; + } + + /** + * Get the lastModified value. + * + * @return the lastModified value + */ + public String lastModified() { + return this.lastModified; + } + + /** + * Set the lastModified value. + * + * @param lastModified the lastModified value to set + * @return the ListEntrySchema object itself. + */ + public ListResultEntrySchema withLastModified(String lastModified) { + this.lastModified = lastModified; + return this; + } + + /** + * Get the etag value. + * + * @return the etag value + */ + public String eTag() { + return this.eTag; + } + + /** + * Set the eTag value. + * + * @param eTag the eTag value to set + * @return the ListEntrySchema object itself. + */ + public ListResultEntrySchema withETag(final String eTag) { + this.eTag = eTag; + return this; + } + + /** + * Get the contentLength value. + * + * @return the contentLength value + */ + public Long contentLength() { + return this.contentLength; + } + + /** + * Set the contentLength value. + * + * @param contentLength the contentLength value to set + * @return the ListEntrySchema object itself. + */ + public ListResultEntrySchema withContentLength(final Long contentLength) { + this.contentLength = contentLength; + return this; + } + +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java new file mode 100644 index 00000000000..baf06dca250 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java @@ -0,0 +1,58 @@ +/** + * 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.contracts.services; + +import java.util.List; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.apache.hadoop.classification.InterfaceStability; + +/** + * The ListResultSchema model. + */ +@InterfaceStability.Evolving +public class ListResultSchema { + /** + * The paths property. + */ + @JsonProperty(value = "paths") + private List paths; + + /** + * * Get the paths value. + * + * @return the paths value + */ + public List paths() { + return this.paths; + } + + /** + * Set the paths value. + * + * @param paths the paths value to set + * @return the ListSchema object itself. + */ + public ListResultSchema withPaths(final List paths) { + this.paths = paths; + return this; + } + +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ReadBufferStatus.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ReadBufferStatus.java new file mode 100644 index 00000000000..ad750c87a5d --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ReadBufferStatus.java @@ -0,0 +1,29 @@ +/** + * 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.contracts.services; + +/** + * The ReadBufferStatus for Rest AbfsClient + */ +public enum ReadBufferStatus { + NOT_AVAILABLE, // buffers sitting in readaheadqueue have this stats + READING_IN_PROGRESS, // reading is in progress on this buffer. Buffer should be in inProgressList + AVAILABLE, // data is available in buffer. It should be in completedList + READ_FAILED // read completed, but failed. +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.java new file mode 100644 index 00000000000..267d11f1144 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.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.contracts.services; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.htrace.core.SpanId; +import org.apache.htrace.core.TraceScope; + +/** + * Azure Blob File System tracing service. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public interface TracingService extends InjectableService { + /** + * Creates a {@link TraceScope} object with the provided description. + * @param description the trace description. + * @return created traceScope. + */ + TraceScope traceBegin(String description); + + /** + * Creates a {@link TraceScope} object with the provided description. + * @param description the trace description. + * @param parentSpanId the span id of the parent trace scope. + * @return create traceScope + */ + TraceScope traceBegin(String description, SpanId parentSpanId); + + /** + * Gets current thread latest generated traceScope id. + * @return current thread latest generated traceScope id. + */ + SpanId getCurrentTraceScopeSpanId(); + + /** + * Appends the provided exception to the trace scope. + * @param traceScope the scope which exception needs to be attached to. + * @param azureBlobFileSystemException the exception to be attached to the scope. + */ + void traceException(TraceScope traceScope, AzureBlobFileSystemException azureBlobFileSystemException); + + /** + * Ends the provided traceScope. + * @param traceScope the scope that needs to be ended. + */ + void traceEnd(TraceScope traceScope); +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/package-info.java new file mode 100644 index 00000000000..8b8a597cd24 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +package org.apache.hadoop.fs.azurebfs.contracts.services; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java new file mode 100644 index 00000000000..69288c5198f --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java @@ -0,0 +1,50 @@ +/** + * 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.diagnostics; + + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; +import org.apache.commons.codec.binary.Base64; + +/** +* String Base64 configuration value Validator +*/ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class Base64StringConfigurationBasicValidator extends ConfigurationBasicValidator implements ConfigurationValidator{ + + public Base64StringConfigurationBasicValidator(final String configKey, final String defaultVal, final boolean throwIfInvalid){ + super(configKey, defaultVal, throwIfInvalid); + } + + public String validate(final String configValue) throws InvalidConfigurationValueException { + String result = super.validate((configValue)); + if (result != null) { + return result; + } + + if (!Base64.isBase64(configValue)) { + throw new InvalidConfigurationValueException(getConfigKey()); + } + return configValue; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java new file mode 100644 index 00000000000..c9927ff0856 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java @@ -0,0 +1,50 @@ +/** + * 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.diagnostics; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; + +/** + * Boolean configuration value validator + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class BooleanConfigurationBasicValidator extends ConfigurationBasicValidator { + private static final String TRUE = "true"; + private static final String FALSE = "false"; + + public BooleanConfigurationBasicValidator(final String configKey, final boolean defaultVal, final boolean throwIfInvalid) { + super(configKey, defaultVal, throwIfInvalid); + } + + public Boolean validate(final String configValue) throws InvalidConfigurationValueException { + Boolean result = super.validate(configValue); + if (result != null) { + return result; + } + + if (configValue.equalsIgnoreCase(TRUE) || configValue.equalsIgnoreCase(FALSE)) { + return Boolean.valueOf(configValue); + } + + throw new InvalidConfigurationValueException(getConfigKey()); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java new file mode 100644 index 00000000000..7da809cdcad --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.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.diagnostics; + +import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; + +/** + * ConfigurationBasicValidator covers the base case of missing user defined configuration value + * @param the type of the validated value + */ +abstract class ConfigurationBasicValidator implements ConfigurationValidator { + private final T defaultVal; + private final String configKey; + private final boolean throwIfInvalid; + + ConfigurationBasicValidator(final String configKey, final T defaultVal, final boolean throwIfInvalid) { + this.configKey = configKey; + this.defaultVal = defaultVal; + this.throwIfInvalid = throwIfInvalid; + } + + /** + * This method handles the base case where the configValue is null, based on the throwIfInvalid it either throws or returns the defaultVal, + * otherwise it returns null indicating that the configValue needs to be validated further + * @param configValue the configuration value set by the user + * @return the defaultVal in case the configValue is null and not required to be set, null in case the configValue not null + * @throws InvalidConfigurationValueException in case the configValue is null and required to be set + */ + public T validate(final String configValue) throws InvalidConfigurationValueException { + if (configValue == null) { + if (this.throwIfInvalid) { + throw new InvalidConfigurationValueException(this.configKey); + } + return this.defaultVal; + } + return null; + } + + public T getDefaultVal() { + return this.defaultVal; + } + + public String getConfigKey() { + return this.configKey; + } + + public boolean getThrowIfInvalid() { + return this.throwIfInvalid; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java new file mode 100644 index 00000000000..ec38cd8ea47 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java @@ -0,0 +1,68 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

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

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs.diagnostics; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; + +/** + * Integer configuration value Validator + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class IntegerConfigurationBasicValidator extends ConfigurationBasicValidator implements ConfigurationValidator { + private final int min; + private final int max; + + public IntegerConfigurationBasicValidator(final int min, final int max, final int defaultVal, final String configKey, final boolean throwIfInvalid) { + super(configKey, defaultVal, throwIfInvalid); + this.min = min; + this.max = max; + } + + public Integer validate(final String configValue) throws InvalidConfigurationValueException { + Integer result = super.validate(configValue); + if (result != null) { + return result; + } + + try { + result = Integer.parseInt(configValue); + // throw an exception if a 'within bounds' value is missing + if (getThrowIfInvalid() && (result < this.min || result > this.max)) { + throw new InvalidConfigurationValueException(getConfigKey()); + } + + // set the value to the nearest bound if it's out of bounds + if (result < this.min) { + return this.min; + } + + if (result > this.max) { + return this.max; + } + } catch (NumberFormatException ex) { + throw new InvalidConfigurationValueException(getConfigKey(), ex); + } + + return result; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java new file mode 100644 index 00000000000..559dbc0c49b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java @@ -0,0 +1,65 @@ +/** + * 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.diagnostics; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; + +/** + * Long configuration value Validator + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class LongConfigurationBasicValidator extends ConfigurationBasicValidator implements ConfigurationValidator { + private final long min; + private final long max; + + public LongConfigurationBasicValidator(final long min, final long max, final long defaultVal, final String configKey, final boolean throwIfInvalid) { + super(configKey, defaultVal, throwIfInvalid); + this.min = min; + this.max = max; + } + + public Long validate(final String configValue) throws InvalidConfigurationValueException { + Long result = super.validate(configValue); + if (result != null) { + return result; + } + + try { + result = Long.parseLong(configValue); + // throw an exception if a 'within bounds' value is missing + if (getThrowIfInvalid() && (result < this.min || result > this.max)) { + throw new InvalidConfigurationValueException(getConfigKey()); + } + + // set the value to the nearest bound if it's out of bounds + if (result < this.min) { + return this.min; + } else if (result > this.max) { + return this.max; + } + } catch (NumberFormatException ex) { + throw new InvalidConfigurationValueException(getConfigKey(), ex); + } + + return result; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java new file mode 100644 index 00000000000..d6f9c59e5d2 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.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.diagnostics; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; + +/** + * String configuration value Validator + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class StringConfigurationBasicValidator extends ConfigurationBasicValidator implements ConfigurationValidator{ + + public StringConfigurationBasicValidator(final String configKey, final String defaultVal, final boolean throwIfInvalid){ + super(configKey, defaultVal, throwIfInvalid); + } + + public String validate(final String configValue) throws InvalidConfigurationValueException { + String result = super.validate((configValue)); + if (result != null) { + return result; + } + + return configValue; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java new file mode 100644 index 00000000000..c3434acfc2c --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +package org.apache.hadoop.fs.azurebfs.diagnostics; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/package.html b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/package.html new file mode 100644 index 00000000000..5333cec2d58 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/package.html @@ -0,0 +1,31 @@ + + + + + + +

+A distributed implementation of {@link +org.apache.hadoop.fs.FileSystem} for reading and writing files on +Azure Storage. +This implementation stores files on Azure in their native form for +interoperability with other Azure tools. +

+ + + 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 new file mode 100644 index 00000000000..c17a5c18dfc --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -0,0 +1,402 @@ +/** + * 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.UnsupportedEncodingException; +import java.net.MalformedURLException; +import java.net.URL; +import java.net.URLEncoder; +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; + +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; +import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * AbfsClient + */ +public class AbfsClient { + public static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); + private final URL baseUrl; + private final SharedKeyCredentials sharedKeyCredentials; + private final String xMsVersion = "2018-03-28"; + private final ExponentialRetryPolicy retryPolicy; + private final String filesystem; + private final ConfigurationService configurationService; + private final String userAgent; + + public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, + final ConfigurationService configurationService, + final ExponentialRetryPolicy exponentialRetryPolicy) { + this.baseUrl = baseUrl; + this.sharedKeyCredentials = sharedKeyCredentials; + String baseUrlString = baseUrl.toString(); + this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(AbfsHttpConstants.FORWARD_SLASH) + 1); + this.configurationService = configurationService; + this.retryPolicy = exponentialRetryPolicy; + this.userAgent = initializeUserAgent(); + } + + public String getFileSystem() { + return filesystem; + } + + ExponentialRetryPolicy getRetryPolicy() { + return retryPolicy; + } + + SharedKeyCredentials getSharedKeyCredentials() { + return sharedKeyCredentials; + } + + List createDefaultHeaders() { + final List requestHeaders = new ArrayList(); + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_VERSION, xMsVersion)); + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.ACCEPT, AbfsHttpConstants.APPLICATION_JSON + + AbfsHttpConstants.COMMA + AbfsHttpConstants.SINGLE_WHITE_SPACE + AbfsHttpConstants.APPLICATION_OCTET_STREAM)); + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.ACCEPT_CHARSET, + AbfsHttpConstants.UTF_8)); + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.CONTENT_TYPE, AbfsHttpConstants.EMPTY_STRING)); + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.USER_AGENT, userAgent)); + return requestHeaders; + } + + AbfsUriQueryBuilder createDefaultUriQueryBuilder() { + final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_TIMEOUT, AbfsHttpConstants.DEFAULT_TIMEOUT); + return abfsUriQueryBuilder; + } + + public AbfsRestOperation createFilesystem() throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM); + + final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + this, + AbfsHttpConstants.HTTP_METHOD_PUT, + url, + requestHeaders); + op.execute(); + return op; + } + + public AbfsRestOperation setFilesystemProperties(final String properties) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + // JDK7 does not support PATCH, so to workaround the issue we will use + // PUT and specify the real method in the X-Http-Method-Override header. + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE, + AbfsHttpConstants.HTTP_METHOD_PATCH)); + + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PROPERTIES, + properties)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM); + + final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + this, + AbfsHttpConstants.HTTP_METHOD_PUT, + url, + requestHeaders); + op.execute(); + return op; + } + + public AbfsRestOperation listPath(final String relativePath, final boolean recursive, final int listMaxResults, + final String continuation) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_DIRECTORY, relativePath == null ? "" : urlEncode(relativePath)); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RECURSIVE, String.valueOf(recursive)); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_MAXRESULTS, String.valueOf(listMaxResults)); + + final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + this, + AbfsHttpConstants.HTTP_METHOD_GET, + url, + requestHeaders); + op.execute(); + return op; + } + + public AbfsRestOperation getFilesystemProperties() throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM); + + final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + this, + AbfsHttpConstants.HTTP_METHOD_HEAD, + url, + requestHeaders); + op.execute(); + return op; + } + + public AbfsRestOperation deleteFilesystem() throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM); + + final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + this, + AbfsHttpConstants.HTTP_METHOD_DELETE, + url, + requestHeaders); + op.execute(); + return op; + } + + public AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + if (!overwrite) { + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_NONE_MATCH, "*")); + } + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, isFile ? AbfsHttpConstants.FILE : AbfsHttpConstants.DIRECTORY); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + this, + AbfsHttpConstants.HTTP_METHOD_PUT, + url, + requestHeaders); + op.execute(); + return op; + } + + public AbfsRestOperation renamePath(final String source, final String destination, final String continuation) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final String encodedRenameSource = urlEncode(AbfsHttpConstants.FORWARD_SLASH + this.getFileSystem() + source); + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_RENAME_SOURCE, encodedRenameSource)); + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_NONE_MATCH, AbfsHttpConstants.STAR)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation); + + final URL url = createRequestUrl(destination, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + this, + AbfsHttpConstants.HTTP_METHOD_PUT, + url, + requestHeaders); + op.execute(); + return op; + } + + public AbfsRestOperation append(final String path, final long position, final byte[] buffer, final int offset, + final int length) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + // JDK7 does not support PATCH, so to workaround the issue we will use + // PUT and specify the real method in the X-Http-Method-Override header. + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE, + AbfsHttpConstants.HTTP_METHOD_PATCH)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.APPEND_ACTION); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_POSITION, Long.toString(position)); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + this, + AbfsHttpConstants.HTTP_METHOD_PUT, + url, + requestHeaders, buffer, offset, length); + op.execute(); + return op; + } + + + public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + // JDK7 does not support PATCH, so to workaround the issue we will use + // PUT and specify the real method in the X-Http-Method-Override header. + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE, + AbfsHttpConstants.HTTP_METHOD_PATCH)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.FLUSH_ACTION); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_POSITION, Long.toString(position)); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RETAIN_UNCOMMITTED_DATA, String.valueOf(retainUncommittedData)); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + this, + AbfsHttpConstants.HTTP_METHOD_PUT, + url, + requestHeaders); + op.execute(); + return op; + } + + public AbfsRestOperation setPathProperties(final String path, final String properties) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + // JDK7 does not support PATCH, so to workaround the issue we will use + // PUT and specify the real method in the X-Http-Method-Override header. + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE, + AbfsHttpConstants.HTTP_METHOD_PATCH)); + + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PROPERTIES, properties)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_PROPERTIES_ACTION); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + this, + AbfsHttpConstants.HTTP_METHOD_PUT, + url, + requestHeaders); + op.execute(); + return op; + } + + public AbfsRestOperation getPathProperties(final String path) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + this, + AbfsHttpConstants.HTTP_METHOD_HEAD, + url, + requestHeaders); + op.execute(); + return op; + } + + public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset, + final int bufferLength, final String eTag) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.RANGE, + String.format("bytes=%d-%d", position, position + bufferLength - 1))); + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_MATCH, eTag)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + + final AbfsRestOperation op = new AbfsRestOperation( + this, + AbfsHttpConstants.HTTP_METHOD_GET, + url, + requestHeaders, + buffer, + bufferOffset, + bufferLength); + op.execute(); + + return op; + } + + public AbfsRestOperation deletePath(final String path, final boolean recursive, final String continuation) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RECURSIVE, String.valueOf(recursive)); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + this, + AbfsHttpConstants.HTTP_METHOD_DELETE, + url, + requestHeaders); + op.execute(); + return op; + } + + private URL createRequestUrl(final String query) throws AzureBlobFileSystemException { + return createRequestUrl(AbfsHttpConstants.EMPTY_STRING, query); + } + + private URL createRequestUrl(final String path, final String query) + throws AzureBlobFileSystemException { + final String base = baseUrl.toString(); + String encodedPath = path; + try { + encodedPath = urlEncode(path); + } catch (AzureBlobFileSystemException ex) { + this.LOG.debug( + "Unexpected error.", ex); + } + + final StringBuilder sb = new StringBuilder(); + sb.append(base); + sb.append(encodedPath); + sb.append(query); + + final URL url; + try { + url = new URL(sb.toString()); + } catch (MalformedURLException ex) { + throw new InvalidUriException(sb.toString()); + } + return url; + } + + private static String urlEncode(final String value) throws AzureBlobFileSystemException { + String encodedString = null; + try { + encodedString = URLEncoder.encode(value, AbfsHttpConstants.UTF_8) + .replace(AbfsHttpConstants.PLUS, AbfsHttpConstants.PLUS_ENCODE) + .replace(AbfsHttpConstants.FORWARD_SLASH_ENCODE, AbfsHttpConstants.FORWARD_SLASH); + } catch (UnsupportedEncodingException ex) { + throw new InvalidUriException(value); + } + + return encodedString; + } + + private String initializeUserAgent() { + final String userAgentComment = String.format(Locale.ROOT, + "(JavaJRE %s; %s %s)", + System.getProperty(AbfsHttpConstants.JAVA_VERSION), + System.getProperty(AbfsHttpConstants.OS_NAME) + .replaceAll(AbfsHttpConstants.SINGLE_WHITE_SPACE, AbfsHttpConstants.EMPTY_STRING), + System.getProperty(AbfsHttpConstants.OS_VERSION)); + + return String.format(AbfsHttpConstants.CLIENT_VERSION + " %s", userAgentComment); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.java new file mode 100644 index 00000000000..9e4c27bdcfc --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.java @@ -0,0 +1,116 @@ +/** + * 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.net.MalformedURLException; +import java.net.URI; +import java.net.URL; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.inject.Inject; +import com.google.inject.Singleton; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory; +import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; +import org.apache.http.client.utils.URIBuilder; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; + +@Singleton +@InterfaceAudience.Private +@InterfaceStability.Evolving +class AbfsHttpClientFactoryImpl implements AbfsHttpClientFactory { + private final ConfigurationService configurationService; + + @Inject + AbfsHttpClientFactoryImpl( + final ConfigurationService configurationService) { + + Preconditions.checkNotNull(configurationService, "configurationService"); + + this.configurationService = configurationService; + } + + @VisibleForTesting + URIBuilder getURIBuilder(final String hostName, final FileSystem fs) { + final AzureBlobFileSystem abfs = (AzureBlobFileSystem) fs; + + String scheme = FileSystemUriSchemes.HTTP_SCHEME; + + if (abfs.isSecure()) { + scheme = FileSystemUriSchemes.HTTPS_SCHEME; + } + + final URIBuilder uriBuilder = new URIBuilder(); + uriBuilder.setScheme(scheme); + uriBuilder.setHost(hostName); + + return uriBuilder; + } + + public AbfsClient create(final AzureBlobFileSystem fs) throws AzureBlobFileSystemException { + final URI uri = fs.getUri(); + final String authority = uri.getRawAuthority(); + if (null == authority) { + throw new InvalidUriAuthorityException(uri.toString()); + } + + if (!authority.contains(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER)) { + throw new InvalidUriAuthorityException(uri.toString()); + } + + final String[] authorityParts = authority.split(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2); + + if (authorityParts.length < 2 || "".equals(authorityParts[0])) { + final String errMsg = String + .format("URI '%s' has a malformed authority, expected container name. " + + "Authority takes the form "+ FileSystemUriSchemes.ABFS_SCHEME + "://[@]", + uri.toString()); + throw new InvalidUriException(errMsg); + } + + final String fileSystemName = authorityParts[0]; + final String accountName = authorityParts[1]; + + final URIBuilder uriBuilder = getURIBuilder(accountName, fs); + + final String url = uriBuilder.toString() + AbfsHttpConstants.FORWARD_SLASH + fileSystemName; + + URL baseUrl; + try { + baseUrl = new URL(url); + } catch (MalformedURLException e) { + throw new InvalidUriException(String.format("URI '%s' is malformed", uri.toString())); + } + + SharedKeyCredentials creds = + new SharedKeyCredentials(accountName.substring(0, accountName.indexOf(AbfsHttpConstants.DOT)), + this.configurationService.getStorageAccountKey(accountName)); + + return new AbfsClient(baseUrl, creds, configurationService, new ExponentialRetryPolicy()); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java new file mode 100644 index 00000000000..46b4c6d8442 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java @@ -0,0 +1,40 @@ +/** + * 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; + +/** + * The Http Request / Response Headers for Rest AbfsClient + */ +public class AbfsHttpHeader { + private final String name; + private final String value; + + public AbfsHttpHeader(final String name, final String value) { + this.name = name; + this.value = value; + } + + public String getName() { + return name; + } + + public String getValue() { + return value; + } +} 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 new file mode 100644 index 00000000000..0ea936569b9 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -0,0 +1,430 @@ +/** + * 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.InputStream; +import java.io.OutputStream; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.List; +import java.util.UUID; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonToken; +import com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Represents an HTTP operation. + */ +public class AbfsHttpOperation { + private static final Logger LOG = LoggerFactory.getLogger(AbfsHttpOperation.class); + + private static final int CONNECT_TIMEOUT = 30 * 1000; + private static final int READ_TIMEOUT = 30 * 1000; + + private static final int CLEAN_UP_BUFFER_SIZE = 64 * 1024; + + private static final int ONE_THOUSAND = 1000; + private static final int ONE_MILLION = ONE_THOUSAND * ONE_THOUSAND; + + private final String method; + private final URL url; + + private HttpURLConnection connection; + private int statusCode; + private String statusDescription; + private String storageErrorCode = ""; + private String storageErrorMessage = ""; + private String clientRequestId = ""; + private String requestId = ""; + private ListResultSchema listResultSchema = null; + + // metrics + private int bytesSent; + private long bytesReceived; + + // optional trace enabled metrics + private final boolean isTraceEnabled; + private long connectionTimeMs; + private long sendRequestTimeMs; + private long recvResponseTimeMs; + + protected HttpURLConnection getConnection() { + return connection; + } + + public String getMethod() { + return method; + } + + public URL getUrl() { + return url; + } + + public int getStatusCode() { + return statusCode; + } + + public String getStatusDescription() { + return statusDescription; + } + + public String getStorageErrorCode() { + return storageErrorCode; + } + + public String getStorageErrorMessage() { + return storageErrorMessage; + } + + public String getClientRequestId() { + return clientRequestId; + } + + public String getRequestId() { + return requestId; + } + + public int getBytesSent() { + return bytesSent; + } + + public long getBytesReceived() { + return bytesReceived; + } + + public ListResultSchema getListResultSchema() { + return listResultSchema; + } + + public String getResponseHeader(String httpHeader) { + return connection.getHeaderField(httpHeader); + } + + // Returns a trace message for the request + @Override + public String toString() { + final String urlStr = url.toString(); + final StringBuilder sb = new StringBuilder(); + sb.append(statusCode); + sb.append(","); + sb.append(storageErrorCode); + sb.append(",cid="); + sb.append(clientRequestId); + sb.append(",rid="); + sb.append(requestId); + if (isTraceEnabled) { + sb.append(",connMs="); + sb.append(connectionTimeMs); + sb.append(",sendMs="); + sb.append(sendRequestTimeMs); + sb.append(",recvMs="); + sb.append(recvResponseTimeMs); + } + sb.append(",sent="); + sb.append(bytesSent); + sb.append(",recv="); + sb.append(bytesReceived); + sb.append(","); + sb.append(method); + sb.append(","); + sb.append(urlStr); + return sb.toString(); + } + + /** + * Initializes a new HTTP request and opens the connection. + * + * @param url The full URL including query string parameters. + * @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE). + * @param requestHeaders The HTTP request headers.READ_TIMEOUT + * + * @throws IOException if an error occurs. + */ + public AbfsHttpOperation(final URL url, final String method, final List requestHeaders) + throws IOException { + this.isTraceEnabled = this.LOG.isTraceEnabled(); + this.url = url; + this.method = method; + this.clientRequestId = UUID.randomUUID().toString(); + + this.connection = openConnection(); + + this.connection.setConnectTimeout(CONNECT_TIMEOUT); + this.connection.setReadTimeout(READ_TIMEOUT); + + this.connection.setRequestMethod(method); + + for (AbfsHttpHeader header : requestHeaders) { + this.connection.setRequestProperty(header.getName(), header.getValue()); + } + + this.connection.setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, clientRequestId); + } + + /** + * Sends the HTTP request. Note that HttpUrlConnection requires that an + * empty buffer be sent in order to set the "Content-Length: 0" header, which + * is required by our endpoint. + * + * @param buffer the request entity body. + * @param offset an offset into the buffer where the data beings. + * @param length the length of the data in the buffer. + * + * @throws IOException if an error occurs. + */ + public void sendRequest(byte[] buffer, int offset, int length) throws IOException { + this.connection.setDoOutput(true); + this.connection.setFixedLengthStreamingMode(length); + if (buffer == null) { + // An empty buffer is sent to set the "Content-Length: 0" header, which + // is required by our endpoint. + buffer = new byte[]{}; + offset = 0; + length = 0; + } + + // send the request body + + long startTime = 0; + if (this.isTraceEnabled) { + startTime = System.nanoTime(); + } + try (OutputStream outputStream = this.connection.getOutputStream()) { + // update bytes sent before they are sent so we may observe + // attempted sends as well as successful sends via the + // accompanying statusCode + this.bytesSent = length; + outputStream.write(buffer, offset, length); + } finally { + if (this.isTraceEnabled) { + this.sendRequestTimeMs = elapsedTimeMs(startTime); + } + } + } + + /** + * Gets and processes the HTTP response. + * + * @throws IOException if an error occurs. + */ + public void processResponse(final byte[] buffer, final int offset, final int length) throws IOException { + + // get the response + long startTime = 0; + if (this.isTraceEnabled) { + startTime = System.nanoTime(); + } + + this.statusCode = this.connection.getResponseCode(); + + if (this.isTraceEnabled) { + this.recvResponseTimeMs = elapsedTimeMs(startTime); + } + + this.statusDescription = this.connection.getResponseMessage(); + + this.requestId = this.connection.getHeaderField(HttpHeaderConfigurations.X_MS_REQUEST_ID); + if (this.requestId == null) { + this.requestId = AbfsHttpConstants.EMPTY_STRING; + } + + if (AbfsHttpConstants.HTTP_METHOD_HEAD.equals(this.method)) { + // If it is HEAD, and it is ERROR + return; + } + + if (this.isTraceEnabled) { + startTime = System.nanoTime(); + } + + if (statusCode >= HttpURLConnection.HTTP_BAD_REQUEST) { + processStorageErrorResponse(); + if (this.isTraceEnabled) { + this.recvResponseTimeMs += elapsedTimeMs(startTime); + } + this.bytesReceived = this.connection.getHeaderFieldLong(HttpHeaderConfigurations.CONTENT_LENGTH, 0); + } else { + // consume the input stream to release resources + int totalBytesRead = 0; + + try (InputStream stream = this.connection.getInputStream()) { + if (isNullInputStream(stream)) { + return; + } + boolean endOfStream = false; + + // 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); + } else { + if (buffer != null) { + while (totalBytesRead < length) { + int bytesRead = stream.read(buffer, offset + totalBytesRead, length - totalBytesRead); + if (bytesRead == -1) { + endOfStream = true; + break; + } + totalBytesRead += bytesRead; + } + } + if (!endOfStream && stream.read() != -1) { + // read and discard + int bytesRead = 0; + byte[] b = new byte[CLEAN_UP_BUFFER_SIZE]; + while ((bytesRead = stream.read(b)) >= 0) { + totalBytesRead += bytesRead; + } + } + } + } catch (IOException ex) { + this.LOG.error("UnexpectedError: ", ex); + throw ex; + } finally { + if (this.isTraceEnabled) { + this.recvResponseTimeMs += elapsedTimeMs(startTime); + } + this.bytesReceived = totalBytesRead; + } + } + } + + + /** + * Open the HTTP connection. + * + * @throws IOException if an error occurs. + */ + private HttpURLConnection openConnection() throws IOException { + if (!isTraceEnabled) { + return (HttpURLConnection) url.openConnection(); + } + long start = System.nanoTime(); + try { + return (HttpURLConnection) url.openConnection(); + } finally { + connectionTimeMs = elapsedTimeMs(start); + } + } + + /** + * When the request fails, this function is used to parse the responseAbfsHttpClient.LOG.debug("ExpectedError: ", ex); + * and extract the storageErrorCode and storageErrorMessage. Any errors + * encountered while attempting to process the error response are logged, + * but otherwise ignored. + * + * For storage errors, the response body *usually* has the following format: + * + * { + * "error": + * { + * "code": "string", + * "message": "string" + * } + * } + * + */ + private void processStorageErrorResponse() { + try (InputStream stream = connection.getErrorStream()) { + if (stream == null) { + return; + } + JsonFactory jf = new JsonFactory(); + try (JsonParser jp = jf.createParser(stream)) { + String fieldName, fieldValue; + jp.nextToken(); // START_OBJECT - { + jp.nextToken(); // FIELD_NAME - "error": + jp.nextToken(); // START_OBJECT - { + jp.nextToken(); + while (jp.hasCurrentToken()) { + if (jp.getCurrentToken() == JsonToken.FIELD_NAME) { + fieldName = jp.getCurrentName(); + jp.nextToken(); + fieldValue = jp.getText(); + switch (fieldName) { + case "code": + storageErrorCode = fieldValue; + break; + case "message": + storageErrorMessage = fieldValue; + break; + default: + break; + } + } + jp.nextToken(); + } + } + } catch (IOException ex) { + // Ignore errors that occur while attempting to parse the storage + // error, since the response may have been handled by the HTTP driver + // or for other reasons have an unexpected + this.LOG.debug("ExpectedError: ", ex); + } + } + + /** + * Returns the elapsed time in milliseconds. + */ + private long elapsedTimeMs(final long startTime) { + return (System.nanoTime() - startTime) / ONE_MILLION; + } + + /** + * Parse the list file response + * + * @param stream InputStream contains the list results. + * @throws IOException + */ + private void parseListFilesResponse(final InputStream stream) throws IOException { + if (stream == null) { + return; + } + + if (listResultSchema != null) { + // already parse the response + return; + } + + try { + final ObjectMapper objectMapper = new ObjectMapper(); + this.listResultSchema = objectMapper.readValue(stream, ListResultSchema.class); + } catch (IOException ex) { + this.LOG.error("Unable to deserialize list results", ex); + throw ex; + } + } + + /** + * Check null stream, this is to pass findbugs's redundant check for NULL + * @param stream InputStream + */ + private boolean isNullInputStream(InputStream stream) { + return stream == null ? true : false; + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpServiceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpServiceImpl.java new file mode 100644 index 00000000000..06e1a8a5cad --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpServiceImpl.java @@ -0,0 +1,693 @@ +/** + * 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.bind.DatatypeConverter; +import java.io.File; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.URI; +import java.net.URISyntaxException; +import java.nio.ByteBuffer; +import java.nio.CharBuffer; +import java.nio.charset.CharacterCodingException; +import java.nio.charset.Charset; +import java.nio.charset.CharsetDecoder; +import java.nio.charset.CharsetEncoder; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Set; +import java.util.HashSet; +import java.util.Hashtable; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import com.google.common.base.Preconditions; +import com.google.inject.Inject; +import com.google.inject.Singleton; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException; +import org.joda.time.DateTime; +import org.joda.time.format.DateTimeFormat; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; + +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +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.exceptions.InvalidFileSystemPropertyException; +import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService; +import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory; +import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; +import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService; +import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema; +import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; +import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.permission.FsPermission; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.hadoop.util.Time.now; + +@Singleton +@InterfaceAudience.Private +@InterfaceStability.Evolving +final class AbfsHttpServiceImpl implements AbfsHttpService { + public static final Logger LOG = LoggerFactory.getLogger(AbfsHttpService.class); + private static final String DATE_TIME_PATTERN = "E, dd MMM yyyy HH:mm:ss 'GMT'"; + private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1"; + private static final int LIST_MAX_RESULTS = 5000; + private static final int DELETE_DIRECTORY_TIMEOUT_MILISECONDS = 180000; + private static final int RENAME_TIMEOUT_MILISECONDS = 180000; + + private final AbfsHttpClientFactory abfsHttpClientFactory; + private final ConcurrentHashMap clientCache; + private final ConfigurationService configurationService; + private final Set azureAtomicRenameDirSet; + + @Inject + AbfsHttpServiceImpl( + final ConfigurationService configurationService, + final AbfsHttpClientFactory abfsHttpClientFactory, + final TracingService tracingService) { + Preconditions.checkNotNull(abfsHttpClientFactory, "abfsHttpClientFactory"); + Preconditions.checkNotNull(configurationService, "configurationService"); + Preconditions.checkNotNull(tracingService, "tracingService"); + + this.configurationService = configurationService; + this.clientCache = new ConcurrentHashMap<>(); + this.abfsHttpClientFactory = abfsHttpClientFactory; + this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(configurationService.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA))); + } + + @Override + public Hashtable getFilesystemProperties(final AzureBlobFileSystem azureBlobFileSystem) + throws AzureBlobFileSystemException{ + final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); + + this.LOG.debug( + "getFilesystemProperties for filesystem: {}", + client.getFileSystem()); + + final Hashtable parsedXmsProperties; + + final AbfsRestOperation op = client.getFilesystemProperties(); + final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); + + parsedXmsProperties = parseCommaSeparatedXmsProperties(xMsProperties); + + return parsedXmsProperties; + } + + @Override + public void setFilesystemProperties(final AzureBlobFileSystem azureBlobFileSystem, final Hashtable properties) throws + AzureBlobFileSystemException { + if (properties == null || properties.size() == 0) { + return; + } + + final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); + + this.LOG.debug( + "setFilesystemProperties for filesystem: {} with properties: {}", + client.getFileSystem(), + properties); + + final String commaSeparatedProperties; + try { + commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties); + } catch (CharacterCodingException ex) { + throw new InvalidAbfsRestOperationException(ex); + } + client.setFilesystemProperties(commaSeparatedProperties); + } + + @Override + public Hashtable getPathProperties(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws + AzureBlobFileSystemException { + final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); + + this.LOG.debug( + "getPathProperties for filesystem: {} path: {}", + client.getFileSystem(), + path.toString()); + + final Hashtable parsedXmsProperties; + final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); + + final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); + + parsedXmsProperties = parseCommaSeparatedXmsProperties(xMsProperties); + + return parsedXmsProperties; + } + + @Override + public void setPathProperties(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final Hashtable properties) throws + AzureBlobFileSystemException { + final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); + + this.LOG.debug( + "setFilesystemProperties for filesystem: {} path: {} with properties: {}", + client.getFileSystem(), + path.toString(), + properties); + + final String commaSeparatedProperties; + try { + commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties); + } catch (CharacterCodingException ex) { + throw new InvalidAbfsRestOperationException(ex); + } + client.setPathProperties("/" + getRelativePath(path), commaSeparatedProperties); + } + + @Override + public void createFilesystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException { + final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); + + this.LOG.debug( + "createFilesystem for filesystem: {}", + client.getFileSystem()); + + client.createFilesystem(); + } + + @Override + public void deleteFilesystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException { + final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); + + this.LOG.debug( + "deleteFilesystem for filesystem: {}", + client.getFileSystem()); + + client.deleteFilesystem(); + } + + @Override + public OutputStream createFile(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean overwrite) throws + AzureBlobFileSystemException { + final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); + + this.LOG.debug( + "createFile filesystem: {} path: {} overwrite: {}", + client.getFileSystem(), + path.toString(), + overwrite); + + client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), true, overwrite); + + final OutputStream outputStream; + outputStream = new FSDataOutputStream( + new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), 0, + configurationService.getWriteBufferSize()), null); + return outputStream; + } + + @Override + public Void createDirectory(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException { + final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); + + this.LOG.debug( + "createDirectory filesystem: {} path: {} overwrite: {}", + client.getFileSystem(), + path.toString()); + + client.createPath("/" + getRelativePath(path), false, true); + + return null; + } + + @Override + public InputStream openFileForRead(final AzureBlobFileSystem azureBlobFileSystem, final Path path, + final FileSystem.Statistics statistics) throws AzureBlobFileSystemException { + final AbfsClient client = getOrCreateClient(azureBlobFileSystem); + + this.LOG.debug( + "openFileForRead filesystem: {} path: {}", + client.getFileSystem(), + path.toString()); + + final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); + + 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); + + if (parseIsDirectory(resourceType)) { + throw new AbfsRestOperationException( + AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), + AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), + "openFileForRead must be used with files and not directories", + null); + } + + // Add statistics for InputStream + return new FSDataInputStream( + new AbfsInputStream(client, statistics, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength, + configurationService.getReadBufferSize(), configurationService.getReadAheadQueueDepth(), eTag)); + } + + @Override + public OutputStream openFileForWrite(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean overwrite) throws + AzureBlobFileSystemException { + final AbfsClient client = getOrCreateClient(azureBlobFileSystem); + + this.LOG.debug( + "openFileForWrite filesystem: {} path: {} overwrite: {}", + client.getFileSystem(), + path.toString(), + overwrite); + + final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); + + final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); + final Long contentLength = Long.valueOf(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); + + if (parseIsDirectory(resourceType)) { + throw new AbfsRestOperationException( + AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), + AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), + "openFileForRead must be used with files and not directories", + null); + } + + final long offset = overwrite ? 0 : contentLength; + + final OutputStream outputStream; + outputStream = new FSDataOutputStream( + new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), + offset, configurationService.getWriteBufferSize()), null); + return outputStream; + } + + @Override + public void rename(final AzureBlobFileSystem azureBlobFileSystem, final Path source, final Path destination) throws + AzureBlobFileSystemException { + + if (isAtomicRenameKey(source.getName())) { + this.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."); + } + + final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); + + this.LOG.debug( + "renameAsync filesystem: {} source: {} destination: {}", + client.getFileSystem(), + source.toString(), + destination.toString()); + + String continuation = null; + long deadline = now() + RENAME_TIMEOUT_MILISECONDS; + + do { + if (now() > deadline) { + LOG.debug( + "Rename {} to {} timed out.", + source, + destination); + + throw new TimeoutException("Rename timed out."); + } + + AbfsRestOperation op = client.renamePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(source), + AbfsHttpConstants.FORWARD_SLASH + getRelativePath(destination), continuation); + continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); + + } while (continuation != null && !continuation.isEmpty()); + } + + @Override + public void delete(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean recursive) throws + AzureBlobFileSystemException { + final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); + + this.LOG.debug( + "delete filesystem: {} path: {} recursive: {}", + client.getFileSystem(), + path.toString(), + String.valueOf(recursive)); + + String continuation = null; + long deadline = now() + DELETE_DIRECTORY_TIMEOUT_MILISECONDS; + + do { + if (now() > deadline) { + this.LOG.debug( + "Delete directory {} timed out.", path); + + throw new TimeoutException("Delete directory timed out."); + } + + AbfsRestOperation op = client.deletePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), recursive, continuation); + continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); + + } while (continuation != null && !continuation.isEmpty()); + } + + @Override + public FileStatus getFileStatus(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException { + final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); + + this.LOG.debug( + "getFileStatus filesystem: {} path: {}", + client.getFileSystem(), + path.toString()); + + if (path.isRoot()) { + AbfsRestOperation op = client.getFilesystemProperties(); + final long blockSize = configurationService.getAzureBlockSize(); + final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + final String lastModified = op.getResult().getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED); + return new VersionedFileStatus( + azureBlobFileSystem.getOwnerUser(), + azureBlobFileSystem.getOwnerUserPrimaryGroup(), + 0, + true, + 1, + blockSize, + parseLastModifiedTime(lastModified).getMillis(), + path, + eTag); + } else { + AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); + + final long blockSize = configurationService.getAzureBlockSize(); + final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + final String lastModified = op.getResult().getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED); + final String contentLength = op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH); + final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); + + return new VersionedFileStatus( + azureBlobFileSystem.getOwnerUser(), + azureBlobFileSystem.getOwnerUserPrimaryGroup(), + parseContentLength(contentLength), + parseIsDirectory(resourceType), + 1, + blockSize, + parseLastModifiedTime(lastModified).getMillis(), + path, + eTag); + } + } + + @Override + public FileStatus[] listStatus(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException { + final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); + + this.LOG.debug( + "listStatus filesystem: {} path: {}", + client.getFileSystem(), + path.toString()); + + String relativePath = path.isRoot() ? AbfsHttpConstants.EMPTY_STRING : getRelativePath(path); + String continuation = null; + ArrayList fileStatuses = new ArrayList<>(); + + do { + AbfsRestOperation op = client.listPath(relativePath, false, LIST_MAX_RESULTS, continuation); + continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); + ListResultSchema retrievedSchema = op.getResult().getListResultSchema(); + if (retrievedSchema == null) { + throw new AbfsRestOperationException( + AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), + AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), + "listStatusAsync path not found", + null, op.getResult()); + } + + long blockSize = configurationService.getAzureBlockSize(); + + for (ListResultEntrySchema entry : retrievedSchema.paths()) { + long lastModifiedMillis = 0; + long contentLength = entry.contentLength() == null ? 0 : entry.contentLength(); + boolean isDirectory = entry.isDirectory() == null ? false : entry.isDirectory(); + if (entry.lastModified() != null && !entry.lastModified().isEmpty()) { + final DateTime dateTime = DateTime.parse( + entry.lastModified(), + DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC()); + lastModifiedMillis = dateTime.getMillis(); + } + + fileStatuses.add( + new VersionedFileStatus( + azureBlobFileSystem.getOwnerUser(), + azureBlobFileSystem.getOwnerUserPrimaryGroup(), + contentLength, + isDirectory, + 1, + blockSize, + lastModifiedMillis, + azureBlobFileSystem.makeQualified(new Path(File.separator + entry.name())), + entry.eTag())); + } + + } while (continuation != null && !continuation.isEmpty()); + + return fileStatuses.toArray(new FileStatus[0]); + } + + @Override + public synchronized void closeFileSystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException { + this.clientCache.remove(azureBlobFileSystem); + } + + @Override + public boolean isAtomicRenameKey(String key) { + return isKeyForDirectorySet(key, azureAtomicRenameDirSet); + } + + private String getRelativePath(final Path path) { + Preconditions.checkNotNull(path, "path"); + final String relativePath = path.toUri().getPath(); + + if (relativePath.length() == 0) { + return relativePath; + } + + if (relativePath.charAt(0) == Path.SEPARATOR_CHAR) { + if (relativePath.length() == 1) { + return AbfsHttpConstants.EMPTY_STRING; + } + + return relativePath.substring(1); + } + + return relativePath; + } + + private synchronized AbfsClient getOrCreateClient(final AzureBlobFileSystem azureBlobFileSystem) throws + AzureBlobFileSystemException { + Preconditions.checkNotNull(azureBlobFileSystem, "azureBlobFileSystem"); + + AbfsClient client = this.clientCache.get(azureBlobFileSystem); + + if (client != null) { + return client; + } + + client = abfsHttpClientFactory.create(azureBlobFileSystem); + this.clientCache.put( + azureBlobFileSystem, + client); + return client; + } + + private long parseContentLength(final String contentLength) { + if (contentLength == null) { + return -1; + } + + return Long.parseLong(contentLength); + } + + private boolean parseIsDirectory(final String resourceType) { + return resourceType == null ? false : resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY); + } + + private DateTime parseLastModifiedTime(final String lastModifiedTime) { + return DateTime.parse( + lastModifiedTime, + DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC()); + } + + private String convertXmsPropertiesToCommaSeparatedString(final Hashtable properties) throws + CharacterCodingException { + StringBuilder commaSeparatedProperties = new StringBuilder(); + + final CharsetEncoder encoder = Charset.forName(XMS_PROPERTIES_ENCODING).newEncoder(); + + for (Map.Entry propertyEntry : properties.entrySet()) { + String key = propertyEntry.getKey(); + String value = propertyEntry.getValue(); + + Boolean canEncodeValue = encoder.canEncode(value); + if (!canEncodeValue) { + throw new CharacterCodingException(); + } + + String encodedPropertyValue = DatatypeConverter.printBase64Binary(encoder.encode(CharBuffer.wrap(value)).array()); + commaSeparatedProperties.append(key) + .append(AbfsHttpConstants.EQUAL) + .append(encodedPropertyValue); + + commaSeparatedProperties.append(AbfsHttpConstants.COMMA); + } + + if (commaSeparatedProperties.length() != 0) { + commaSeparatedProperties.deleteCharAt(commaSeparatedProperties.length() - 1); + } + + return commaSeparatedProperties.toString(); + } + + private Hashtable parseCommaSeparatedXmsProperties(String xMsProperties) throws + InvalidFileSystemPropertyException, InvalidAbfsRestOperationException { + Hashtable properties = new Hashtable<>(); + + final CharsetDecoder decoder = Charset.forName(XMS_PROPERTIES_ENCODING).newDecoder(); + + if (xMsProperties != null && !xMsProperties.isEmpty()) { + String[] userProperties = xMsProperties.split(AbfsHttpConstants.COMMA); + + if (userProperties.length == 0) { + return properties; + } + + for (String property : userProperties) { + if (property.isEmpty()) { + throw new InvalidFileSystemPropertyException(xMsProperties); + } + + String[] nameValue = property.split(AbfsHttpConstants.EQUAL, 2); + if (nameValue.length != 2) { + throw new InvalidFileSystemPropertyException(xMsProperties); + } + + byte[] decodedValue = DatatypeConverter.parseBase64Binary(nameValue[1]); + + final String value; + try { + value = decoder.decode(ByteBuffer.wrap(decodedValue)).toString(); + } catch (CharacterCodingException ex) { + throw new InvalidAbfsRestOperationException(ex); + } + properties.put(nameValue[0], value); + } + } + + return properties; + } + + private boolean isKeyForDirectorySet(String key, Set dirSet) { + for (String dir : dirSet) { + if (dir.isEmpty() || key.startsWith(dir + AbfsHttpConstants.FORWARD_SLASH)) { + return true; + } + + try { + URI uri = new URI(dir); + if (null == uri.getAuthority()) { + if (key.startsWith(dir + "/")){ + return true; + } + } + } catch (URISyntaxException e) { + this.LOG.info("URI syntax error creating URI for {}", dir); + } + } + + return false; + } + + private static class VersionedFileStatus extends FileStatus { + private final String version; + + VersionedFileStatus( + final String owner, final String group, + final long length, final boolean isdir, final int blockReplication, + final long blocksize, final long modificationTime, final Path path, + String version) { + super(length, isdir, blockReplication, blocksize, modificationTime, 0, + new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL), + owner, + group, + path); + + this.version = version; + } + + /** Compare if this object is equal to another object. + * @param obj the object to be compared. + * @return true if two file status has the same path name; false if not. + */ + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + + if (obj == null) { + return false; + } + + if (this.getClass() == obj.getClass()) { + VersionedFileStatus other = (VersionedFileStatus) obj; + return this.getPath().equals(other.getPath()) && this.version.equals(other.version); + } + + return false; + } + + /** + * Returns a hash code value for the object, which is defined as + * the hash code of the path name. + * + * @return a hash code value for the path name and version + */ + @Override + public int hashCode() { + int hash = getPath().hashCode(); + hash = 89 * hash + (this.version != null ? this.version.hashCode() : 0); + return hash; + } + + /** + * Returns the version of this FileStatus + * + * @return a string value for the FileStatus version + */ + public String getVersion() { + return this.version; + } + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java new file mode 100644 index 00000000000..6554380ebd0 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -0,0 +1,382 @@ +/** + * 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.EOFException; +import java.io.IOException; + +import com.google.common.base.Preconditions; + +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.fs.FileSystem.Statistics; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; + +/** + * The AbfsInputStream for AbfsClient + */ +public class AbfsInputStream extends FSInputStream { + private final AbfsClient client; + private final Statistics statistics; + private final String path; + private final long contentLength; + private final int bufferSize; // default buffer size + private final int readAheadQueueDepth; // initialized in constructor + private final String eTag; // eTag of the path when InputStream are created + private final boolean tolerateOobAppends; // whether tolerate Oob Appends + private final boolean readAheadEnabled; // whether enable readAhead; + + private byte[] buffer = null; // will be initialized on first use + + private long fCursor = 0; // cursor of buffer within file - offset of next byte to read from remote server + private long fCursorAfterLastRead = -1; + private int bCursor = 0; // cursor of read within buffer - offset of next byte to be returned from buffer + private int limit = 0; // offset of next byte to be read into buffer from service (i.e., upper marker+1 + // of valid bytes in buffer) + private boolean closed = false; + + public AbfsInputStream( + final AbfsClient client, + final Statistics statistics, + final String path, + final long contentLength, + final int bufferSize, + final int readAheadQueueDepth, + final String eTag) { + super(); + this.client = client; + this.statistics = statistics; + this.path = path; + this.contentLength = contentLength; + this.bufferSize = bufferSize; + this.readAheadQueueDepth = (readAheadQueueDepth >= 0) ? readAheadQueueDepth : 2 * Runtime.getRuntime().availableProcessors(); + this.eTag = eTag; + this.tolerateOobAppends = false; + this.readAheadEnabled = true; + } + + public String getPath() { + return path; + } + + @Override + public int read() throws IOException { + byte[] b = new byte[1]; + int numberOfBytesRead = read(b, 0, 1); + if (numberOfBytesRead < 0) { + return -1; + } else { + return (b[0] & 0xFF); + } + } + + @Override + public synchronized int read(final byte[] b, final int off, final int len) throws IOException { + int currentOff = off; + int currentLen = len; + int lastReadBytes; + int totalReadBytes = 0; + do { + lastReadBytes = readOneBlock(b, currentOff, currentLen); + if (lastReadBytes > 0) { + currentOff += lastReadBytes; + currentLen -= lastReadBytes; + totalReadBytes += lastReadBytes; + } + if (currentLen <= 0 || currentLen > b.length - currentOff) { + break; + } + } while (lastReadBytes > 0); + return totalReadBytes > 0 ? totalReadBytes : lastReadBytes; + } + + private int readOneBlock(final byte[] b, final int off, final int len) throws IOException { + if (closed) { + throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + } + + Preconditions.checkNotNull(b); + + if (len == 0) { + return 0; + } + + if (this.available() == 0) { + return -1; + } + + if (off < 0 || len < 0 || len > b.length - off) { + throw new IndexOutOfBoundsException(); + } + + //If buffer is empty, then fill the buffer. + if (bCursor == limit) { + //If EOF, then return -1 + if (fCursor >= contentLength) { + return -1; + } + + long bytesRead = 0; + //reset buffer to initial state - i.e., throw away existing data + bCursor = 0; + limit = 0; + if (buffer == null) { + buffer = new byte[bufferSize]; + } + + // Enable readAhead when reading sequentially + if (-1 == fCursorAfterLastRead || fCursorAfterLastRead == fCursor || b.length >= bufferSize) { + bytesRead = readInternal(fCursor, buffer, 0, bufferSize, false); + } else { + bytesRead = readInternal(fCursor, buffer, 0, b.length, true); + } + + if (bytesRead == -1) { + return -1; + } + + limit += bytesRead; + fCursor += bytesRead; + fCursorAfterLastRead = fCursor; + } + + //If there is anything in the buffer, then return lesser of (requested bytes) and (bytes in buffer) + //(bytes returned may be less than requested) + int bytesRemaining = limit - bCursor; + int bytesToRead = Math.min(len, bytesRemaining); + System.arraycopy(buffer, bCursor, b, off, bytesToRead); + bCursor += bytesToRead; + if (statistics != null) { + statistics.incrementBytesRead(bytesToRead); + } + return bytesToRead; + } + + + private int readInternal(final long position, final byte[] b, final int offset, final int length, + final boolean bypassReadAhead) throws IOException { + if (readAheadEnabled && !bypassReadAhead) { + // try reading from read-ahead + if (offset != 0) { + throw new IllegalArgumentException("readahead buffers cannot have non-zero buffer offsets"); + } + int receivedBytes; + + // queue read-aheads + int numReadAheads = this.readAheadQueueDepth; + long nextSize; + long nextOffset = position; + while (numReadAheads > 0 && nextOffset < contentLength) { + nextSize = Math.min((long) bufferSize, contentLength - nextOffset); + ReadBufferManager.getBufferManager().queueReadAhead(this, nextOffset, (int) nextSize); + nextOffset = nextOffset + nextSize; + numReadAheads--; + } + + // try reading from buffers first + receivedBytes = ReadBufferManager.getBufferManager().getBlock(this, position, length, b); + if (receivedBytes > 0) { + return receivedBytes; + } + + // got nothing from read-ahead, do our own read now + receivedBytes = readRemote(position, b, offset, length); + return receivedBytes; + } else { + return readRemote(position, b, offset, length); + } + } + + int readRemote(long position, byte[] b, int offset, int length) throws IOException { + if (position < 0) { + throw new IllegalArgumentException("attempting to read from negative offset"); + } + if (position >= contentLength) { + return -1; // Hadoop prefers -1 to EOFException + } + if (b == null) { + throw new IllegalArgumentException("null byte array passed in to read() method"); + } + if (offset >= b.length) { + throw new IllegalArgumentException("offset greater than length of array"); + } + if (length < 0) { + throw new IllegalArgumentException("requested read length is less than zero"); + } + if (length > (b.length - offset)) { + throw new IllegalArgumentException("requested read length is more than will fit after requested offset in buffer"); + } + final AbfsRestOperation op; + try { + op = client.read(path, position, b, offset, length, tolerateOobAppends ? "*" : eTag); + } catch (AzureBlobFileSystemException ex) { + throw new IOException(ex); + } + long bytesRead = op.getResult().getBytesReceived(); + if (bytesRead > Integer.MAX_VALUE) { + throw new IOException("Unexpected Content-Length"); + } + return (int) bytesRead; + } + + /** + * Seek to given position in stream. + * @param n position to seek to + * @throws IOException if there is an error + * @throws EOFException if attempting to seek past end of file + */ + @Override + public synchronized void seek(long n) throws IOException { + if (closed) { + throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + } + if (n < 0) { + throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK); + } + if (n > contentLength) { + throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF); + } + + if (n>=fCursor-limit && n<=fCursor) { // within buffer + bCursor = (int) (n-(fCursor-limit)); + return; + } + + // next read will read from here + fCursor = n; + + //invalidate buffer + limit = 0; + bCursor = 0; + } + + @Override + public synchronized long skip(long n) throws IOException { + if (closed) { + throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + } + long currentPos = getPos(); + if (currentPos == contentLength) { + if (n > 0) { + throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF); + } + } + long newPos = currentPos + n; + if (newPos < 0) { + newPos = 0; + n = newPos - currentPos; + } + if (newPos > contentLength) { + newPos = contentLength; + n = newPos - currentPos; + } + seek(newPos); + return n; + } + + /** + * Return the size of the remaining available bytes + * if the size is less than or equal to {@link Integer#MAX_VALUE}, + * otherwise, return {@link Integer#MAX_VALUE}. + * + * This is to match the behavior of DFSInputStream.available(), + * which some clients may rely on (HBase write-ahead log reading in + * particular). + */ + @Override + public synchronized int available() throws IOException { + if (closed) { + throw new IOException( + FSExceptionMessages.STREAM_IS_CLOSED); + } + final long remaining = this.contentLength - this.getPos(); + return remaining <= Integer.MAX_VALUE + ? (int) remaining : Integer.MAX_VALUE; + } + + /** + * Returns the length of the file that this stream refers to. Note that the length returned is the length + * as of the time the Stream was opened. Specifically, if there have been subsequent appends to the file, + * they wont be reflected in the returned length. + * + * @return length of the file. + * @throws IOException if the stream is closed + */ + public long length() throws IOException { + if (closed) { + throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + } + return contentLength; + } + + /** + * Return the current offset from the start of the file + * @throws IOException throws {@link IOException} if there is an error + */ + @Override + public synchronized long getPos() throws IOException { + if (closed) { + throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + } + return fCursor - limit + bCursor; + } + + /** + * Seeks a different copy of the data. Returns true if + * found a new source, false otherwise. + * @throws IOException throws {@link IOException} if there is an error + */ + @Override + public boolean seekToNewSource(long l) throws IOException { + return false; + } + + @Override + public synchronized void close() throws IOException { + closed = true; + buffer = null; // de-reference the buffer so it can be GC'ed sooner + } + + /** + * Not supported by this stream. Throws {@link UnsupportedOperationException} + * @param readlimit ignored + */ + @Override + public synchronized void mark(int readlimit) { + throw new UnsupportedOperationException("mark()/reset() not supported on this stream"); + } + + /** + * Not supported by this stream. Throws {@link UnsupportedOperationException} + */ + @Override + public synchronized void reset() throws IOException { + throw new UnsupportedOperationException("mark()/reset() not supported on this stream"); + } + + /** + * gets whether mark and reset are supported by {@code ADLFileInputStream}. Always returns false. + * + * @return always {@code false} + */ + @Override + public boolean markSupported() { + return false; + } +} \ No newline at end of file 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 new file mode 100644 index 00000000000..de5c934d64e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -0,0 +1,335 @@ +/** + * 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.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.Callable; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import com.google.common.base.Preconditions; + +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.Syncable; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; + +/** + * The BlobFsOutputStream for Rest AbfsClient + */ +public class AbfsOutputStream extends OutputStream implements Syncable { + private final AbfsClient client; + private final String path; + private long position; + private boolean closed; + private volatile IOException lastError; + + private long lastFlushOffset; + private long lastTotalAppendOffset = 0; + + private final int bufferSize; + private byte[] buffer; + private int bufferIndex; + private final int maxConcurrentRequestCount; + + private ConcurrentLinkedDeque writeOperations; + private final ThreadPoolExecutor threadExecutor; + private final ExecutorCompletionService completionService; + + public AbfsOutputStream( + final AbfsClient client, + final String path, + final long position, + final int bufferSize) { + this.client = client; + this.path = path; + this.position = position; + this.closed = false; + this.lastError = null; + this.lastFlushOffset = 0; + this.bufferSize = bufferSize; + this.buffer = new byte[bufferSize]; + this.bufferIndex = 0; + this.writeOperations = new ConcurrentLinkedDeque<>(); + + this.maxConcurrentRequestCount = 4 * Runtime.getRuntime().availableProcessors(); + + this.threadExecutor + = new ThreadPoolExecutor(maxConcurrentRequestCount, + maxConcurrentRequestCount, + 10L, + TimeUnit.SECONDS, + new LinkedBlockingQueue()); + this.completionService = new ExecutorCompletionService(this.threadExecutor); + } + + /** + * Writes the specified byte to this output stream. The general contract for + * write is that one byte is written to the output stream. The byte to be + * written is the eight low-order bits of the argument b. The 24 high-order + * bits of b are ignored. + * + * @param byteVal the byteValue to write. + * @throws IOException if an I/O error occurs. In particular, an IOException may be + * thrown if the output stream has been closed. + */ + @Override + public void write(final int byteVal) throws IOException { + write(new byte[]{(byte) (byteVal & 0xFF)}); + } + + /** + * Writes length bytes from the specified byte array starting at off to + * this output stream. + * + * @param data the byte array to write. + * @param off the start off in the data. + * @param length the number of bytes to write. + * @throws IOException if an I/O error occurs. In particular, an IOException may be + * thrown if the output stream has been closed. + */ + @Override + public synchronized void write(final byte[] data, final int off, final int length) + throws IOException { + if (this.lastError != null) { + throw this.lastError; + } + + Preconditions.checkArgument(data != null, "null data"); + + if (off < 0 || length < 0 || length > data.length - off) { + throw new IndexOutOfBoundsException(); + } + + int currentOffset = off; + int writableBytes = bufferSize - bufferIndex; + int numberOfBytesToWrite = length; + + while (numberOfBytesToWrite > 0) { + if (writableBytes <= numberOfBytesToWrite) { + System.arraycopy(data, currentOffset, buffer, bufferIndex, writableBytes); + bufferIndex += writableBytes; + writeCurrentBufferToService(); + currentOffset += writableBytes; + numberOfBytesToWrite = numberOfBytesToWrite - writableBytes; + } else { + System.arraycopy(data, currentOffset, buffer, bufferIndex, numberOfBytesToWrite); + bufferIndex += numberOfBytesToWrite; + numberOfBytesToWrite = 0; + } + + writableBytes = bufferSize - bufferIndex; + } + } + + /** + * Flushes this output stream and forces any buffered output bytes to be + * 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. + */ + @Override + public void flush() throws IOException { + this.flushInternalAsync(); + } + + /** 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 + */ + @Override + public void hsync() throws IOException { + this.flushInternal(); + } + + /** 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 + */ + @Override + public void hflush() throws IOException { + this.flushInternal(); + } + + /** + * Force all data in the output stream to be written to Azure storage. + * Wait to return until this is complete. Close the access to the stream and + * shutdown the upload thread pool. + * If the blob was created, its lease will be released. + * Any error encountered caught in threads and stored will be rethrown here + * after cleanup. + */ + @Override + public synchronized void close() throws IOException { + if (closed) { + return; + } + + try { + this.flushInternal(); + this.threadExecutor.shutdown(); + } finally { + this.lastError = new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + this.buffer = null; + this.bufferIndex = 0; + this.closed = true; + this.writeOperations.clear(); + if (!this.threadExecutor.isShutdown()) { + this.threadExecutor.shutdownNow(); + } + } + } + + private synchronized void flushInternal() throws IOException { + if (this.lastError != null) { + throw this.lastError; + } + this.writeCurrentBufferToService(); + this.flushWrittenBytesToService(); + } + + private synchronized void flushInternalAsync() throws IOException { + if (this.lastError != null) { + throw this.lastError; + } + this.writeCurrentBufferToService(); + this.flushWrittenBytesToServiceAsync(); + } + + private synchronized void writeCurrentBufferToService() throws IOException { + if (bufferIndex == 0) { + return; + } + + final byte[] bytes = this.buffer; + final int bytesLength = bufferIndex; + + this.buffer = new byte[bufferSize]; + this.bufferIndex = 0; + final long offset = this.position; + this.position += bytesLength; + + if (this.threadExecutor.getQueue().size() >= maxConcurrentRequestCount * 2) { + this.waitForTaskToComplete(); + } + + final Future job = this.completionService.submit(new Callable() { + @Override + public Void call() throws Exception { + client.append(path, offset, bytes, 0, + bytesLength); + return null; + } + }); + + this.writeOperations.add(new WriteOperation(job, offset, bytesLength)); + + // Try to shrink the queue + shrinkWriteOperationQueue(); + } + + private synchronized void flushWrittenBytesToService() throws IOException { + for (WriteOperation writeOperation : this.writeOperations) { + try { + writeOperation.task.get(); + } catch (Exception ex) { + if (AzureBlobFileSystemException.class.isInstance(ex.getCause())) { + ex = AzureBlobFileSystemException.class.cast(ex.getCause()); + } + this.lastError = new IOException(ex); + throw this.lastError; + } + } + flushWrittenBytesToServiceInternal(this.position, false); + } + + private synchronized void flushWrittenBytesToServiceAsync() throws IOException { + shrinkWriteOperationQueue(); + + if (this.lastTotalAppendOffset > this.lastFlushOffset) { + this.flushWrittenBytesToServiceInternal(this.lastTotalAppendOffset, true); + } + + this.lastTotalAppendOffset = 0; + } + + private synchronized void flushWrittenBytesToServiceInternal(final long offset, final boolean retainUncommitedData) throws IOException { + try { + client.flush(path, offset, retainUncommitedData); + } catch (AzureBlobFileSystemException ex) { + throw new IOException(ex); + } + this.lastFlushOffset = offset; + } + + /** + * Try to remove the completed write operations from the beginning of write + * operation FIFO queue. + */ + private synchronized void shrinkWriteOperationQueue() throws IOException { + try { + while (this.writeOperations.peek() != null && this.writeOperations.peek().task.isDone()) { + this.writeOperations.peek().task.get(); + this.lastTotalAppendOffset += this.writeOperations.peek().length; + this.writeOperations.remove(); + } + } catch (Exception e) { + if (AzureBlobFileSystemException.class.isInstance(e.getCause())) { + this.lastError = IOException.class.cast(e.getCause()); + } else { + this.lastError = new IOException(e); + } + throw this.lastError; + } + } + + private void waitForTaskToComplete() throws IOException { + boolean completed; + for (completed = false; this.completionService.poll() != null; completed = true) {} + + if (!completed) { + try { + this.completionService.take(); + } catch (InterruptedException e) { + this.lastError = new IOException(e); + throw this.lastError; + } + } + } + + private static class WriteOperation { + private final Future task; + private final long startOffset; + private final long length; + + WriteOperation(final Future task, final long startOffset, final long length) { + Preconditions.checkNotNull(task, "task"); + Preconditions.checkArgument(startOffset >= 0, "startOffset"); + Preconditions.checkArgument(length >= 0, "length"); + + this.task = task; + this.startOffset = startOffset; + this.length = length; + } + } +} \ No newline at end of file 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 new file mode 100644 index 00000000000..17fc35afcb6 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java @@ -0,0 +1,178 @@ +/** + * 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.List; + +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException; + +import org.slf4j.Logger; + +/** + * The AbfsRestOperation for Rest AbfsClient + */ +public class AbfsRestOperation { + // Blob FS client, which has the credentials, retry policy, and logs. + private final AbfsClient client; + // the HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE) + private final String method; + // full URL including query parameters + private final URL url; + // all the custom HTTP request headers provided by the caller + private final List requestHeaders; + + // This is a simple operation class, where all the upload methods have a + // request body and all the download methods have a response body. + private final boolean hasRequestBody; + + private final Logger logger; + + // For uploads, this is the request entity body. For downloads, + // this will hold the response entity body. + private byte[] buffer; + private int bufferOffset; + private int bufferLength; + + private AbfsHttpOperation result; + + public AbfsHttpOperation getResult() { + return result; + } + + /** + * Initializes a new REST operation. + * + * @param client The Blob FS client. + * @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE). + * @param url The full URL including query string parameters. + * @param requestHeaders The HTTP request headers. + */ + AbfsRestOperation(final AbfsClient client, + final String method, + final URL url, + final List requestHeaders) { + this.client = client; + this.method = method; + this.url = url; + this.requestHeaders = requestHeaders; + this.hasRequestBody = (AbfsHttpConstants.HTTP_METHOD_PUT.equals(method) + || AbfsHttpConstants.HTTP_METHOD_PATCH.equals(method)); + this.logger = client.LOG; + } + + /** + * Initializes a new REST operation. + * + * @param client The Blob FS client. + * @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE). + * @param url The full URL including query string parameters. + * @param requestHeaders The HTTP request headers. + * @param buffer For uploads, this is the request entity body. For downloads, + * this will hold the response entity body. + * @param bufferOffset An offset into the buffer where the data beings. + * @param bufferLength The length of the data in the buffer. + */ + AbfsRestOperation(AbfsClient client, + String method, + URL url, + List requestHeaders, + byte[] buffer, + int bufferOffset, + int bufferLength) { + this(client, method, url, requestHeaders); + this.buffer = buffer; + this.bufferOffset = bufferOffset; + this.bufferLength = bufferLength; + } + + /** + * Executes the REST operation with retry, by issuing one or more + * HTTP operations. + */ + void execute() throws AzureBlobFileSystemException { + int retryCount = 0; + while (!executeHttpOperation(retryCount++)) { + try { + Thread.sleep(client.getRetryPolicy().getRetryInterval(retryCount)); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + } + } + + if (result.getStatusCode() > HttpURLConnection.HTTP_BAD_REQUEST) { + throw new AbfsRestOperationException(result.getStatusCode(), result.getStorageErrorCode(), + result.getStorageErrorMessage(), null, result); + } + } + + /** + * Executes a single HTTP operation to complete the REST operation. If it + * fails, there may be a retry. The retryCount is incremented with each + * attempt. + */ + private boolean executeHttpOperation(final int retryCount) throws AzureBlobFileSystemException { + AbfsHttpOperation httpOperation = null; + try { + // initialize the HTTP request and open the connection + httpOperation = new AbfsHttpOperation(url, method, requestHeaders); + + // sign the HTTP request + client.getSharedKeyCredentials().signRequest( + httpOperation.getConnection(), + hasRequestBody ? bufferLength : 0); + + if (hasRequestBody) { + // HttpUrlConnection requires + httpOperation.sendRequest(buffer, bufferOffset, bufferLength); + } + + httpOperation.processResponse(buffer, bufferOffset, bufferLength); + } catch (IOException ex) { + if (logger.isDebugEnabled()) { + if (httpOperation != null) { + logger.debug("HttpRequestFailure: " + httpOperation.toString(), ex); + } else { + logger.debug("HttpRequestFailure: " + method + "," + url, ex); + } + } + if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) { + throw new InvalidAbfsRestOperationException(ex); + } + return false; + } + + if (logger.isDebugEnabled()) { + logger.debug("HttpRequest: " + httpOperation.toString()); + } + + if (client.getRetryPolicy().shouldRetry(retryCount, httpOperation.getStatusCode())) { + return false; + } + + result = httpOperation; + + return true; + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java new file mode 100644 index 00000000000..1cbf6b5c3a2 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java @@ -0,0 +1,81 @@ +/** + * 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 com.google.inject.AbstractModule; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory; +import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService; +import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; +import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService; + +/** + * This class is responsible to configure all the services used by Azure Blob File System. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +class AbfsServiceInjectorImpl extends AbstractModule { + private final Configuration configuration; + private final Map providers; + private final Map instances; + + AbfsServiceInjectorImpl(Configuration configuration) { + this.providers = new HashMap<>(); + this.instances = new HashMap<>(); + this.configuration = configuration; + + this.instances.put(Configuration.class, this.configuration); + + this.providers.put(ConfigurationService.class, ConfigurationServiceImpl.class); + + this.providers.put(AbfsHttpService.class, AbfsHttpServiceImpl.class); + this.providers.put(AbfsHttpClientFactory.class, AbfsHttpClientFactoryImpl.class); + + this.providers.put(TracingService.class, TracingServiceImpl.class); + } + + @Override + protected void configure() { + for (Map.Entry entrySet : this.instances.entrySet()) { + bind(entrySet.getKey()).toInstance(entrySet.getValue()); + } + + for (Map.Entry entrySet : this.providers.entrySet()) { + bind(entrySet.getKey()).to(entrySet.getValue()); + } + } + + protected Configuration getConfiguration() { + return this.configuration; + } + + protected Map getProviders() { + return this.providers; + } + + protected Map getInstances() { + return this.instances; + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.java new file mode 100644 index 00000000000..8560620a69b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.java @@ -0,0 +1,96 @@ +/** + * 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 com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.inject.Guice; +import com.google.inject.Injector; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ServiceResolutionException; +import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsServiceProvider; +import org.apache.hadoop.fs.azurebfs.contracts.services.InjectableService; + +/** + * Dependency injected Azure Storage services provider. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class AbfsServiceProviderImpl implements AbfsServiceProvider { + private static AbfsServiceProviderImpl abfsServiceProvider; + private final Injector abfsServiceInjector; + + private AbfsServiceProviderImpl(final Configuration configuration) { + this.abfsServiceInjector = Guice.createInjector(new AbfsServiceInjectorImpl(Preconditions.checkNotNull(configuration, "configuration"))); + } + + @VisibleForTesting + private AbfsServiceProviderImpl(final Injector abfsServiceInjector) { + Preconditions.checkNotNull(abfsServiceInjector, "abfsServiceInjector"); + this.abfsServiceInjector = abfsServiceInjector; + } + + /** + * Create an instance or returns existing instance of service provider. + * This method must be marked as synchronized to ensure thread-safety. + * @param configuration hadoop configuration. + * @return AbfsServiceProvider the service provider instance. + */ + public static synchronized AbfsServiceProvider create(final Configuration configuration) { + if (abfsServiceProvider == null) { + abfsServiceProvider = new AbfsServiceProviderImpl(configuration); + } + + return abfsServiceProvider; + } + + /** + * Returns current instance of service provider. + * @return AbfsServiceProvider the service provider instance. + */ + public static AbfsServiceProvider instance() { + return abfsServiceProvider; + } + + @VisibleForTesting + static synchronized AbfsServiceProvider create(Injector serviceInjector) { + abfsServiceProvider = new AbfsServiceProviderImpl(serviceInjector); + return abfsServiceProvider; + } + + /** + * Returns an instance of resolved injectable service by class name. + * The injectable service must be configured first to be resolvable. + * @param clazz the injectable service which is expected to be returned. + * @param The type of injectable service. + * @return T instance + * @throws ServiceResolutionException if the service is not resolvable. + */ + @Override + public T get(final Class clazz) throws ServiceResolutionException { + try { + return this.abfsServiceInjector.getInstance(clazz); + } catch (Exception ex) { + throw new ServiceResolutionException(clazz.getSimpleName(), ex); + } + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java new file mode 100644 index 00000000000..bac66af8824 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java @@ -0,0 +1,58 @@ +/** + * 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 java.util.HashMap; +import java.util.Map; + +/** + * The UrlQueryBuilder for Rest AbfsClient + */ +public class AbfsUriQueryBuilder { + private Map parameters; + + public AbfsUriQueryBuilder() { + this.parameters = new HashMap<>(); + } + + public void addQuery(final String name, final String value) { + if (value != null && !value.isEmpty()) { + this.parameters.put(name, value); + } + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + boolean first = true; + + for (Map.Entry entry : parameters.entrySet()) { + if (first) { + sb.append(AbfsHttpConstants.QUESTION_MARK); + first = false; + } else { + sb.append(AbfsHttpConstants.AND_MARK); + } + sb.append(entry.getKey()).append(AbfsHttpConstants.EQUAL).append(entry.getValue()); + } + return sb.toString(); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ConfigurationServiceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ConfigurationServiceImpl.java new file mode 100644 index 00000000000..568ee5da63a --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ConfigurationServiceImpl.java @@ -0,0 +1,317 @@ +/** + * 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.lang.reflect.Field; +import java.util.Map; + +import com.google.common.annotations.VisibleForTesting; +import com.google.inject.Inject; +import com.google.inject.Singleton; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations; +import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation; +import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation; +import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation; +import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation; +import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; +import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; +import org.apache.hadoop.fs.azurebfs.diagnostics.Base64StringConfigurationBasicValidator; +import org.apache.hadoop.fs.azurebfs.diagnostics.BooleanConfigurationBasicValidator; +import org.apache.hadoop.fs.azurebfs.diagnostics.IntegerConfigurationBasicValidator; +import org.apache.hadoop.fs.azurebfs.diagnostics.LongConfigurationBasicValidator; +import org.apache.hadoop.fs.azurebfs.diagnostics.StringConfigurationBasicValidator; + +@Singleton +@InterfaceAudience.Private +@InterfaceStability.Evolving +class ConfigurationServiceImpl implements ConfigurationService { + private final Configuration configuration; + private final boolean isSecure; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_WRITE_BUFFER_SIZE, + MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE, + MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE, + DefaultValue = FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE) + private int writeBufferSize; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_READ_BUFFER_SIZE, + MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE, + MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE, + DefaultValue = FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE) + private int readBufferSize; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MIN_BACKOFF_INTERVAL, + DefaultValue = FileSystemConfigurations.DEFAULT_MIN_BACKOFF_INTERVAL) + private int minBackoffInterval; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MAX_BACKOFF_INTERVAL, + DefaultValue = FileSystemConfigurations.DEFAULT_MAX_BACKOFF_INTERVAL) + private int maxBackoffInterval; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BACKOFF_INTERVAL, + DefaultValue = FileSystemConfigurations.DEFAULT_BACKOFF_INTERVAL) + private int backoffInterval; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MAX_IO_RETRIES, + MinValue = 0, + DefaultValue = FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS) + private int maxIoRetries; + + @LongConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BLOCK_SIZE_PROPERTY_NAME, + MinValue = 0, + MaxValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE, + DefaultValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE) + private long azureBlockSize; + + @StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME, + DefaultValue = FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT) + private String azureBlockLocationHost; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CONCURRENT_CONNECTION_VALUE_OUT, + MinValue = 1, + DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_WRITE_THREADS) + private int maxConcurrentWriteThreads; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CONCURRENT_CONNECTION_VALUE_IN, + MinValue = 1, + DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_READ_THREADS) + private int maxConcurrentReadThreads; + + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_TOLERATE_CONCURRENT_APPEND, + DefaultValue = FileSystemConfigurations.DEFAULT_READ_TOLERATE_CONCURRENT_APPEND) + private boolean tolerateOobAppends; + + @StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_ATOMIC_RENAME_KEY, + DefaultValue = FileSystemConfigurations.DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES) + private String azureAtomicDirs; + + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, + DefaultValue = FileSystemConfigurations.DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION) + private boolean createRemoteFileSystemDuringInitialization; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH, + DefaultValue = FileSystemConfigurations.DEFAULT_READ_AHEAD_QUEUE_DEPTH) + private int readAheadQueueDepth; + + private Map storageAccountKeys; + + @Inject + ConfigurationServiceImpl(final Configuration configuration) throws IllegalAccessException, InvalidConfigurationValueException { + this.configuration = configuration; + this.isSecure = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_SECURE_MODE, false); + + validateStorageAccountKeys(); + Field[] fields = this.getClass().getDeclaredFields(); + for (Field field : fields) { + field.setAccessible(true); + if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) { + field.set(this, validateInt(field)); + } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) { + field.set(this, validateLong(field)); + } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) { + field.set(this, validateString(field)); + } else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) { + field.set(this, validateBase64String(field)); + } else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) { + field.set(this, validateBoolean(field)); + } + } + } + + @Override + public boolean isEmulator() { + return this.getConfiguration().getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false); + } + + @Override + public boolean isSecureMode() { + return this.isSecure; + } + + @Override + public String getStorageAccountKey(final String accountName) throws ConfigurationPropertyNotFoundException { + String accountKey = this.storageAccountKeys.get(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + accountName); + if (accountKey == null) { + throw new ConfigurationPropertyNotFoundException(accountName); + } + + return accountKey; + } + + @Override + public Configuration getConfiguration() { + return this.configuration; + } + + @Override + public int getWriteBufferSize() { + return this.writeBufferSize; + } + + @Override + public int getReadBufferSize() { + return this.readBufferSize; + } + + @Override + public int getMinBackoffIntervalMilliseconds() { + return this.minBackoffInterval; + } + + @Override + public int getMaxBackoffIntervalMilliseconds() { + return this.maxBackoffInterval; + } + + @Override + public int getBackoffIntervalMilliseconds() { + return this.backoffInterval; + } + + @Override + public int getMaxIoRetries() { + return this.maxIoRetries; + } + + @Override + public long getAzureBlockSize() { + return this.azureBlockSize; + } + + @Override + public String getAzureBlockLocationHost() { + return this.azureBlockLocationHost; + } + + @Override + public int getMaxConcurrentWriteThreads() { + return this.maxConcurrentWriteThreads; + } + + @Override + public int getMaxConcurrentReadThreads() { + return this.maxConcurrentReadThreads; + } + + @Override + public boolean getTolerateOobAppends() { + return this.tolerateOobAppends; + } + + @Override + public String getAzureAtomicRenameDirs() { + return this.azureAtomicDirs; + } + + @Override + public boolean getCreateRemoteFileSystemDuringInitialization() { + return this.createRemoteFileSystemDuringInitialization; + } + + @Override + public int getReadAheadQueueDepth() { + return this.readAheadQueueDepth; + } + + void validateStorageAccountKeys() throws InvalidConfigurationValueException { + Base64StringConfigurationBasicValidator validator = new Base64StringConfigurationBasicValidator( + ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME, "", true); + this.storageAccountKeys = this.configuration.getValByRegex(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX); + + for (Map.Entry account : this.storageAccountKeys.entrySet()) { + validator.validate(account.getValue()); + } + } + + int validateInt(Field field) throws IllegalAccessException, InvalidConfigurationValueException { + IntegerConfigurationValidatorAnnotation validator = field.getAnnotation(IntegerConfigurationValidatorAnnotation.class); + String value = this.configuration.get(validator.ConfigurationKey()); + + // validate + return new IntegerConfigurationBasicValidator( + validator.MinValue(), + validator.MaxValue(), + validator.DefaultValue(), + validator.ConfigurationKey(), + validator.ThrowIfInvalid()).validate(value); + } + + long validateLong(Field field) throws IllegalAccessException, InvalidConfigurationValueException { + LongConfigurationValidatorAnnotation validator = field.getAnnotation(LongConfigurationValidatorAnnotation.class); + String value = this.configuration.get(validator.ConfigurationKey()); + + // validate + return new LongConfigurationBasicValidator( + validator.MinValue(), + validator.MaxValue(), + validator.DefaultValue(), + validator.ConfigurationKey(), + validator.ThrowIfInvalid()).validate(value); + } + + String validateString(Field field) throws IllegalAccessException, InvalidConfigurationValueException { + StringConfigurationValidatorAnnotation validator = field.getAnnotation(StringConfigurationValidatorAnnotation.class); + String value = this.configuration.get(validator.ConfigurationKey()); + + // validate + return new StringConfigurationBasicValidator( + validator.ConfigurationKey(), + validator.DefaultValue(), + validator.ThrowIfInvalid()).validate(value); + } + + String validateBase64String(Field field) throws IllegalAccessException, InvalidConfigurationValueException { + Base64StringConfigurationValidatorAnnotation validator = field.getAnnotation((Base64StringConfigurationValidatorAnnotation.class)); + String value = this.configuration.get(validator.ConfigurationKey()); + + // validate + return new Base64StringConfigurationBasicValidator( + validator.ConfigurationKey(), + validator.DefaultValue(), + validator.ThrowIfInvalid()).validate(value); + } + + boolean validateBoolean(Field field) throws IllegalAccessException, InvalidConfigurationValueException { + BooleanConfigurationValidatorAnnotation validator = field.getAnnotation(BooleanConfigurationValidatorAnnotation.class); + String value = this.configuration.get(validator.ConfigurationKey()); + + // validate + return new BooleanConfigurationBasicValidator( + validator.ConfigurationKey(), + validator.DefaultValue(), + validator.ThrowIfInvalid()).validate(value); + } + + @VisibleForTesting + void setReadBufferSize(int bufferSize) { + this.readBufferSize = bufferSize; + } + + @VisibleForTesting + void setWriteBufferSize(int bufferSize) { + this.writeBufferSize = bufferSize; + } +} \ No newline at end of file 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 new file mode 100644 index 00000000000..0c9261216ac --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java @@ -0,0 +1,141 @@ +/** + * 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.Random; +import java.net.HttpURLConnection; + +class ExponentialRetryPolicy { + /** + * Represents the default number of retry attempts. + */ + private static final int DEFAULT_CLIENT_RETRY_COUNT = 30; + + /** + * Represents the default amount of time used when calculating a random delta in the exponential + * delay between retries. + */ + private static final int DEFAULT_CLIENT_BACKOFF = 1000 * 3; + + /** + * Represents the default maximum amount of time used when calculating the exponential + * delay between retries. + */ + private static final int DEFAULT_MAX_BACKOFF = 1000 * 30; + + /** + *Represents the default minimum amount of time used when calculating the exponential + * delay between retries. + */ + private static final int DEFAULT_MIN_BACKOFF = 1000 * 3; + + /** + * The minimum random ratio used for delay interval calculation. + */ + private static final double MIN_RANDOM_RATIO = 0.8; + + /** + * The maximum random ratio used for delay interval calculation. + */ + private static final double MAX_RANDOM_RATIO = 1.2; + + /** + * Holds the random number generator used to calculate randomized backoff intervals + */ + private final Random randRef = new Random(); + + /** + * The value that will be used to calculate a random delta in the exponential delay interval + */ + private final int deltaBackoff; + + /** + * The maximum backoff time. + */ + private final int maxBackoff; + + /** + * The minimum backoff time. + */ + private final int minBackoff; + + /** + * The maximum number of retry attempts. + */ + private final int retryCount; + + /** + * Initializes a new instance of the {@link ExponentialRetryPolicy} class. + */ + ExponentialRetryPolicy() { + this(DEFAULT_CLIENT_RETRY_COUNT, DEFAULT_MIN_BACKOFF, DEFAULT_MAX_BACKOFF, DEFAULT_CLIENT_BACKOFF); + } + + /** + * Initializes a new instance of the {@link ExponentialRetryPolicy} class. + * + * @param retryCount The maximum number of retry attempts. + * @param minBackoff The minimum backoff time. + * @param maxBackoff The maximum backoff time. + * @param deltaBackoff The value that will be used to calculate a random delta in the exponential delay + * between retries. + */ + ExponentialRetryPolicy(final int retryCount, final int minBackoff, final int maxBackoff, final int deltaBackoff) { + this.retryCount = retryCount; + this.minBackoff = minBackoff; + this.maxBackoff = maxBackoff; + this.deltaBackoff = deltaBackoff; + } + + /** + * Returns if a request should be retried based on the retry count, current response, + * and the current strategy. + * + * @param retryCount The current retry attempt count. + * @param statusCode The status code of the response, or -1 for socket error. + * @return true if the request should be retried; false otherwise. + */ + public boolean shouldRetry(final int retryCount, final int statusCode) { + return retryCount < this.retryCount + && (statusCode == -1 + || statusCode == HttpURLConnection.HTTP_CLIENT_TIMEOUT + || (statusCode >= HttpURLConnection.HTTP_INTERNAL_ERROR + && statusCode != HttpURLConnection.HTTP_NOT_IMPLEMENTED + && statusCode != HttpURLConnection.HTTP_VERSION)); + } + + /** + * Returns backoff interval between 80% and 120% of the desired backoff, + * multiply by 2^n-1 for exponential. + * + * @param retryCount The current retry attempt count. + * @return backoff Interval time + */ + public long getRetryInterval(final int retryCount) { + final long boundedRandDelta = (int) (this.deltaBackoff * MIN_RANDOM_RATIO) + + this.randRef.nextInt((int) (this.deltaBackoff * MAX_RANDOM_RATIO) + - (int) (this.deltaBackoff * MIN_RANDOM_RATIO)); + + final double incrementDelta = (Math.pow(2, retryCount - 1)) * boundedRandDelta; + + final long retryInterval = (int) Math.round(Math.min(this.minBackoff + incrementDelta, maxBackoff)); + + return retryInterval; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.java new file mode 100644 index 00000000000..99190e6d4a7 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.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.services; + +import java.io.IOException; + +import com.google.common.base.Preconditions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.htrace.core.HTraceConfiguration; +import org.apache.htrace.core.Span; +import org.apache.htrace.core.SpanReceiver; +import org.apache.htrace.fasterxml.jackson.core.JsonProcessingException; +import org.apache.htrace.fasterxml.jackson.databind.ObjectMapper; +import org.apache.htrace.fasterxml.jackson.databind.ObjectWriter; +import org.apache.htrace.fasterxml.jackson.databind.SerializationFeature; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * LoggerSpanReceiver is a layer between HTrace and log4j only used for {@link org.apache.hadoop.fs.azurebfs.contracts.services.TracingService} + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class LoggerSpanReceiver extends SpanReceiver { + private static final ObjectWriter JSON_WRITER = + new ObjectMapper() + .configure(SerializationFeature.INDENT_OUTPUT, true) + .configure(SerializationFeature.WRITE_BIGDECIMAL_AS_PLAIN, true) + .configure(SerializationFeature.WRITE_EMPTY_JSON_ARRAYS, false) + .configure(SerializationFeature.USE_EQUALITY_FOR_OBJECT_ID, false) + .writer(); + + public LoggerSpanReceiver(HTraceConfiguration hTraceConfiguration) { + Preconditions.checkNotNull(hTraceConfiguration, "hTraceConfiguration"); + } + + @Override + public void receiveSpan(final Span span) { + String jsonValue; + + Logger logger = LoggerFactory.getLogger(AzureBlobFileSystem.class); + + try { + jsonValue = JSON_WRITER.writeValueAsString(span); + logger.trace(jsonValue); + } catch (JsonProcessingException e) { + logger.error("Json processing error: " + e.getMessage()); + } + } + + @Override + public void close() throws IOException { + // No-Op + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java new file mode 100644 index 00000000000..1fac13dcaa8 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.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 org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; + +import java.util.concurrent.CountDownLatch; + +class ReadBuffer { + + private AbfsInputStream stream; + private long offset; // offset within the file for the buffer + private int length; // actual length, set after the buffer is filles + private int requestedLength; // requested length of the read + private byte[] buffer; // the buffer itself + private int bufferindex = -1; // index in the buffers array in Buffer manager + private ReadBufferStatus status; // status of the buffer + private CountDownLatch latch = null; // signaled when the buffer is done reading, so any client + // waiting on this buffer gets unblocked + + // fields to help with eviction logic + private long timeStamp = 0; // tick at which buffer became available to read + private boolean isFirstByteConsumed = false; + private boolean isLastByteConsumed = false; + private boolean isAnyByteConsumed = false; + + public AbfsInputStream getStream() { + return stream; + } + + public void setStream(AbfsInputStream stream) { + this.stream = stream; + } + + public long getOffset() { + return offset; + } + + public void setOffset(long offset) { + this.offset = offset; + } + + public int getLength() { + return length; + } + + public void setLength(int length) { + this.length = length; + } + + public int getRequestedLength() { + return requestedLength; + } + + public void setRequestedLength(int requestedLength) { + this.requestedLength = requestedLength; + } + + public byte[] getBuffer() { + return buffer; + } + + public void setBuffer(byte[] buffer) { + this.buffer = buffer; + } + + public int getBufferindex() { + return bufferindex; + } + + public void setBufferindex(int bufferindex) { + this.bufferindex = bufferindex; + } + + public ReadBufferStatus getStatus() { + return status; + } + + public void setStatus(ReadBufferStatus status) { + this.status = status; + } + + public CountDownLatch getLatch() { + return latch; + } + + public void setLatch(CountDownLatch latch) { + this.latch = latch; + } + + public long getTimeStamp() { + return timeStamp; + } + + public void setTimeStamp(long timeStamp) { + this.timeStamp = timeStamp; + } + + public boolean isFirstByteConsumed() { + return isFirstByteConsumed; + } + + public void setFirstByteConsumed(boolean isFirstByteConsumed) { + this.isFirstByteConsumed = isFirstByteConsumed; + } + + public boolean isLastByteConsumed() { + return isLastByteConsumed; + } + + public void setLastByteConsumed(boolean isLastByteConsumed) { + this.isLastByteConsumed = isLastByteConsumed; + } + + public boolean isAnyByteConsumed() { + return isAnyByteConsumed; + } + + public void setAnyByteConsumed(boolean isAnyByteConsumed) { + this.isAnyByteConsumed = isAnyByteConsumed; + } + +} 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 new file mode 100644 index 00000000000..164e54992ae --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java @@ -0,0 +1,391 @@ +/** + * 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.services.ReadBufferStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.LinkedList; +import java.util.Queue; +import java.util.Stack; +import java.util.concurrent.CountDownLatch; + +/** + * The Read Buffer Manager for Rest AbfsClient + */ +final class ReadBufferManager { + private static final Logger LOGGER = LoggerFactory.getLogger(ReadBufferManager.class); + + private static final int NUM_BUFFERS = 16; + private static final int BLOCK_SIZE = 4 * 1024 * 1024; + private static final int NUM_THREADS = 8; + private static final int THRESHOLD_AGE_MILLISECONDS = 3000; // have to see if 3 seconds is a good threshold + + private Thread[] threads = new Thread[NUM_THREADS]; + private byte[][] buffers; // array of byte[] buffers, to hold the data that is read + private Stack freeList = new Stack(); // indices in buffers[] array that are available + + private Queue readAheadQueue = new LinkedList(); // queue of requests that are not picked up by any worker thread yet + private LinkedList inProgressList = new LinkedList(); // requests being processed by worker threads + private LinkedList completedReadList = new LinkedList(); // buffers available for reading + private static final ReadBufferManager BUFFER_MANAGER; // singleton, initialized in static initialization block + + static { + BUFFER_MANAGER = new ReadBufferManager(); + BUFFER_MANAGER.init(); + } + + static ReadBufferManager getBufferManager() { + return BUFFER_MANAGER; + } + + private void init() { + buffers = new byte[NUM_BUFFERS][]; + for (int i = 0; i < NUM_BUFFERS; i++) { + buffers[i] = new byte[BLOCK_SIZE]; // same buffers are reused. The byte array never goes back to GC + freeList.add(i); + } + for (int i = 0; i < NUM_THREADS; i++) { + Thread t = new Thread(new ReadBufferWorker(i)); + t.setDaemon(true); + threads[i] = t; + t.setName("ABFS-prefetch-" + i); + t.start(); + } + ReadBufferWorker.UNLEASH_WORKERS.countDown(); + } + + // hide instance constructor + private ReadBufferManager() { + } + + + /* + * + * AbfsInputStream-facing methods + * + */ + + + /** + * {@link AbfsInputStream} calls this method to queue read-aheads + * + * @param stream The {@link AbfsInputStream} for which to do the read-ahead + * @param requestedOffset The offset in the file which shoukd be read + * @param requestedLength The length to read + */ + void queueReadAhead(final AbfsInputStream stream, final long requestedOffset, final int requestedLength) { + if (LOGGER.isTraceEnabled()) { + LOGGER.trace("Start Queueing readAhead for " + stream.getPath() + " offset " + requestedOffset + + " length " + requestedLength); + } + ReadBuffer buffer; + synchronized (this) { + if (isAlreadyQueued(stream, requestedOffset)) { + return; // already queued, do not queue again + } + if (freeList.size() == 0 && !tryEvict()) { + return; // no buffers available, cannot queue anything + } + + buffer = new ReadBuffer(); + buffer.setStream(stream); + buffer.setOffset(requestedOffset); + buffer.setLength(0); + buffer.setRequestedLength(requestedLength); + buffer.setStatus(ReadBufferStatus.NOT_AVAILABLE); + buffer.setLatch(new CountDownLatch(1)); + + Integer bufferIndex = freeList.pop(); // will return a value, since we have checked size > 0 already + + buffer.setBuffer(buffers[bufferIndex]); + buffer.setBufferindex(bufferIndex); + readAheadQueue.add(buffer); + notifyAll(); + } + if (LOGGER.isTraceEnabled()) { + LOGGER.trace("Done q-ing readAhead for file " + stream.getPath() + " offset " + requestedOffset + + " buffer idx " + buffer.getBufferindex()); + } + } + + + /** + * {@link AbfsInputStream} calls this method read any bytes already available in a buffer (thereby saving a + * remote read). This returns the bytes if the data already exists in buffer. If there is a buffer that is reading + * the requested offset, then this method blocks until that read completes. If the data is queued in a read-ahead + * but not picked up by a worker thread yet, then it cancels that read-ahead and reports cache miss. This is because + * depending on worker thread availability, the read-ahead may take a while - the calling thread can do it's own + * read to get the data faster (copmared to the read waiting in queue for an indeterminate amount of time). + * + * @param stream the file to read bytes for + * @param position the offset in the file to do a read for + * @param length the length to read + * @param buffer the buffer to read data into. Note that the buffer will be written into from offset 0. + * @return the number of bytes read + */ + int getBlock(final AbfsInputStream stream, final long position, final int length, final byte[] buffer) { + // not synchronized, so have to be careful with locking + if (LOGGER.isTraceEnabled()) { + LOGGER.trace("getBlock for file " + stream.getPath() + " position " + position + " thread " + Thread.currentThread().getName()); + } + + waitForProcess(stream, position); + + int bytesRead = 0; + synchronized (this) { + bytesRead = getBlockFromCompletedQueue(stream, position, length, buffer); + } + if (bytesRead > 0) { + if (LOGGER.isTraceEnabled()) { + LOGGER.trace("Done read from Cache for " + stream.getPath() + " position " + position + " length " + bytesRead); + } + return bytesRead; + } + + // otherwise, just say we got nothing - calling thread can do it's own read + return 0; + } + + /* + * + * Internal methods + * + */ + + private void waitForProcess(final AbfsInputStream stream, final long position) { + ReadBuffer readBuf; + synchronized (this) { + clearFromReadAheadQueue(stream, position); + readBuf = getFromList(inProgressList, stream, position); + } + if (readBuf != null) { // if in in-progress queue, then block for it + try { + if (LOGGER.isTraceEnabled()) { + LOGGER.trace("got a relevant read buffer for file " + stream.getPath() + " offset " + readBuf.getOffset() + + " buffer idx " + readBuf.getBufferindex()); + } + readBuf.getLatch().await(); // blocking wait on the caller stream's thread + // Note on correctness: readBuf gets out of inProgressList only in 1 place: after worker thread + // is done processing it (in doneReading). There, the latch is set after removing the buffer from + // inProgressList. So this latch is safe to be outside the synchronized block. + // Putting it in synchronized would result in a deadlock, since this thread would be holding the lock + // while waiting, so no one will be able to change any state. If this becomes more complex in the future, + // then the latch cane be removed and replaced with wait/notify whenever inProgressList is touched. + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + } + if (LOGGER.isTraceEnabled()) { + LOGGER.trace("latch done for file " + stream.getPath() + " buffer idx " + readBuf.getBufferindex() + + " length " + readBuf.getLength()); + } + } + } + + /** + * If any buffer in the completedlist can be reclaimed then reclaim it and return the buffer to free list. + * The objective is to find just one buffer - there is no advantage to evicting more than one. + * + * @return whether the eviction succeeeded - i.e., were we able to free up one buffer + */ + private synchronized boolean tryEvict() { + ReadBuffer nodeToEvict = null; + if (completedReadList.size() <= 0) { + return false; // there are no evict-able buffers + } + + // first, try buffers where all bytes have been consumed (approximated as first and last bytes consumed) + for (ReadBuffer buf : completedReadList) { + if (buf.isFirstByteConsumed() && buf.isLastByteConsumed()) { + nodeToEvict = buf; + break; + } + } + if (nodeToEvict != null) { + return evict(nodeToEvict); + } + + // next, try buffers where any bytes have been consumed (may be a bad idea? have to experiment and see) + for (ReadBuffer buf : completedReadList) { + if (buf.isAnyByteConsumed()) { + nodeToEvict = buf; + break; + } + } + + if (nodeToEvict != null) { + return evict(nodeToEvict); + } + + // next, try any old nodes that have not been consumed + long earliestBirthday = Long.MAX_VALUE; + for (ReadBuffer buf : completedReadList) { + if (buf.getTimeStamp() < earliestBirthday) { + nodeToEvict = buf; + earliestBirthday = buf.getTimeStamp(); + } + } + if ((currentTimeMillis() - earliestBirthday > THRESHOLD_AGE_MILLISECONDS) && (nodeToEvict != null)) { + return evict(nodeToEvict); + } + + // nothing can be evicted + return false; + } + + private boolean evict(final ReadBuffer buf) { + freeList.push(buf.getBufferindex()); + completedReadList.remove(buf); + if (LOGGER.isTraceEnabled()) { + LOGGER.trace("Evicting buffer idx " + buf.getBufferindex() + "; was used for file " + buf.getStream().getPath() + + " offset " + buf.getOffset() + " length " + buf.getLength()); + } + return true; + } + + private boolean isAlreadyQueued(final AbfsInputStream stream, final long requestedOffset) { + // returns true if any part of the buffer is already queued + return (isInList(readAheadQueue, stream, requestedOffset) + || isInList(inProgressList, stream, requestedOffset) + || isInList(completedReadList, stream, requestedOffset)); + } + + private boolean isInList(final Collection list, final AbfsInputStream stream, final long requestedOffset) { + return (getFromList(list, stream, requestedOffset) != null); + } + + private ReadBuffer getFromList(final Collection list, final AbfsInputStream stream, final long requestedOffset) { + for (ReadBuffer buffer : list) { + if (buffer.getStream() == stream) { + if (buffer.getStatus() == ReadBufferStatus.AVAILABLE + && requestedOffset >= buffer.getOffset() + && requestedOffset < buffer.getOffset() + buffer.getLength()) { + return buffer; + } else if (requestedOffset >= buffer.getOffset() + && requestedOffset < buffer.getOffset() + buffer.getRequestedLength()) { + return buffer; + } + } + } + return null; + } + + private void clearFromReadAheadQueue(final AbfsInputStream stream, final long requestedOffset) { + ReadBuffer buffer = getFromList(readAheadQueue, stream, requestedOffset); + if (buffer != null) { + readAheadQueue.remove(buffer); + notifyAll(); // lock is held in calling method + freeList.push(buffer.getBufferindex()); + } + } + + private int getBlockFromCompletedQueue(final AbfsInputStream stream, final long position, final int length, + final byte[] buffer) { + ReadBuffer buf = getFromList(completedReadList, stream, position); + if (buf == null || position >= buf.getOffset() + buf.getLength()) { + return 0; + } + int cursor = (int) (position - buf.getOffset()); + int availableLengthInBuffer = buf.getLength() - cursor; + int lengthToCopy = Math.min(length, availableLengthInBuffer); + System.arraycopy(buf.getBuffer(), cursor, buffer, 0, lengthToCopy); + if (cursor == 0) { + buf.setFirstByteConsumed(true); + } + if (cursor + lengthToCopy == buf.getLength()) { + buf.setLastByteConsumed(true); + } + buf.setAnyByteConsumed(true); + return lengthToCopy; + } + + /* + * + * ReadBufferWorker-thread-facing methods + * + */ + + /** + * ReadBufferWorker thread calls this to get the next buffer that it should work on. + * + * @return {@link ReadBuffer} + * @throws InterruptedException if thread is interrupted + */ + ReadBuffer getNextBlockToRead() throws InterruptedException { + ReadBuffer buffer = null; + synchronized (this) { + //buffer = readAheadQueue.take(); // blocking method + while (readAheadQueue.size() == 0) { + wait(); + } + buffer = readAheadQueue.remove(); + notifyAll(); + if (buffer == null) { + return null; // should never happen + } + buffer.setStatus(ReadBufferStatus.READING_IN_PROGRESS); + inProgressList.add(buffer); + } + if (LOGGER.isTraceEnabled()) { + LOGGER.trace("ReadBufferWorker picked file " + buffer.getStream().getPath() + " for offset " + buffer.getOffset()); + } + return buffer; + } + + /** + * ReadBufferWorker thread calls this method to post completion + * + * @param buffer the buffer whose read was completed + * @param result the {@link ReadBufferStatus} after the read operation in the worker thread + * @param bytesActuallyRead the number of bytes that the worker thread was actually able to read + */ + void doneReading(final ReadBuffer buffer, final ReadBufferStatus result, final int bytesActuallyRead) { + if (LOGGER.isTraceEnabled()) { + LOGGER.trace("ReadBufferWorker completed file " + buffer.getStream().getPath() + " for offset " + buffer.getOffset() + + " bytes " + bytesActuallyRead); + } + synchronized (this) { + inProgressList.remove(buffer); + if (result == ReadBufferStatus.AVAILABLE && bytesActuallyRead > 0) { + buffer.setStatus(ReadBufferStatus.AVAILABLE); + buffer.setTimeStamp(currentTimeMillis()); + buffer.setLength(bytesActuallyRead); + completedReadList.add(buffer); + } else { + freeList.push(buffer.getBufferindex()); + // buffer should go out of scope after the end of the calling method in ReadBufferWorker, and eligible for GC + } + } + //outside the synchronized, since anyone receiving a wake-up from the latch must see safe-published results + buffer.getLatch().countDown(); // wake up waiting threads (if any) + } + + /** + * Similar to System.currentTimeMillis, except implemented with System.nanoTime(). + * System.currentTimeMillis can go backwards when system clock is changed (e.g., with NTP time synchronization), + * making it unsuitable for measuring time intervals. nanotime is strictly monotonically increasing, + * so it is much more suitable to measuring intervals. + * + * @return current time in milliseconds + */ + private long currentTimeMillis() { + return System.nanoTime() / 1000 / 1000; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java new file mode 100644 index 00000000000..2d0c96e15c0 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java @@ -0,0 +1,72 @@ +/** + * 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.services.ReadBufferStatus; + +import java.util.concurrent.CountDownLatch; + +class ReadBufferWorker implements Runnable { + + protected static final CountDownLatch UNLEASH_WORKERS = new CountDownLatch(1); + private int id; + + ReadBufferWorker(final int id) { + this.id = id; + } + + /** + * return the ID of ReadBufferWorker. + */ + public int getId() { + return this.id; + } + + /** + * Waits until a buffer becomes available in ReadAheadQueue. + * Once a buffer becomes available, reads the file specified in it and then posts results back to buffer manager. + * Rinse and repeat. Forever. + */ + public void run() { + try { + UNLEASH_WORKERS.await(); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + } + ReadBufferManager bufferManager = ReadBufferManager.getBufferManager(); + ReadBuffer buffer; + while (true) { + try { + buffer = bufferManager.getNextBlockToRead(); // blocks, until a buffer is available for this thread + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + return; + } + if (buffer != null) { + try { + // do the actual read, from the file. + int bytesRead = buffer.getStream().readRemote(buffer.getOffset(), buffer.getBuffer(), 0, buffer.getRequestedLength()); + bufferManager.doneReading(buffer, ReadBufferStatus.AVAILABLE, bytesRead); // post result back to ReadBufferManager + } catch (Exception ex) { + bufferManager.doneReading(buffer, ReadBufferStatus.READ_FAILED, 0); + } + } + } + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java new file mode 100644 index 00000000000..dd598921915 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java @@ -0,0 +1,507 @@ +/** + * 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.crypto.Mac; +import javax.crypto.spec.SecretKeySpec; +import java.io.UnsupportedEncodingException; +import java.net.HttpURLConnection; +import java.net.URLDecoder; +import java.text.DateFormat; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Map.Entry; +import java.util.TimeZone; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; + +import org.apache.commons.codec.binary.Base64; +import org.apache.commons.codec.Charsets; +/** + * Represents the shared key credentials used to access an Azure Storage + * account. + */ +public class SharedKeyCredentials { + private static final int EXPECTED_BLOB_QUEUE_CANONICALIZED_STRING_LENGTH = 300; + private static final Pattern CRLF = Pattern.compile("\r\n", Pattern.LITERAL); + private static final String HMAC_SHA256 = "HmacSHA256"; + private static final Base64 BASE_64 = new Base64(); + + /** + * Stores a reference to the RFC1123 date/time pattern. + */ + private static final String RFC1123_PATTERN = "EEE, dd MMM yyyy HH:mm:ss z"; + + + private String accountName; + private byte[] accountKey; + private Mac hmacSha256; + + public SharedKeyCredentials(final String accountName, + final String accountKey) { + if (accountName == null || accountName.isEmpty()) { + throw new IllegalArgumentException("Invalid account name."); + } + if (accountKey == null || accountKey.isEmpty()) { + throw new IllegalArgumentException("Invalid account key."); + } + this.accountName = accountName; + this.accountKey = BASE_64.decode(accountKey); + initializeMac(); + } + + public void signRequest(HttpURLConnection connection, final long contentLength) throws UnsupportedEncodingException { + + connection.setRequestProperty(HttpHeaderConfigurations.X_MS_DATE, getGMTTime()); + + final String stringToSign = canonicalize(connection, accountName, contentLength); + + final String computedBase64Signature = computeHmac256(stringToSign); + + connection.setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION, + String.format("%s %s:%s", "SharedKey", accountName, computedBase64Signature)); + } + + private String computeHmac256(final String stringToSign) { + byte[] utf8Bytes = null; + try { + utf8Bytes = stringToSign.getBytes(AbfsHttpConstants.UTF_8); + } catch (final UnsupportedEncodingException e) { + throw new IllegalArgumentException(e); + } + byte[] hmac; + synchronized (this) { + hmac = hmacSha256.doFinal(utf8Bytes); + } + return new String(BASE_64.encode(hmac), Charsets.UTF_8); + } + + /** + * Add x-ms- prefixed headers in a fixed order. + * + * @param conn the HttpURLConnection for the operation + * @param canonicalizedString the canonicalized string to add the canonicalized headerst to. + */ + private static void addCanonicalizedHeaders(final HttpURLConnection conn, final StringBuilder canonicalizedString) { + // Look for header names that start with + // HeaderNames.PrefixForStorageHeader + // Then sort them in case-insensitive manner. + + final Map> headers = conn.getRequestProperties(); + final ArrayList httpStorageHeaderNameArray = new ArrayList(); + + for (final String key : headers.keySet()) { + if (key.toLowerCase(Locale.ROOT).startsWith(AbfsHttpConstants.HTTP_HEADER_PREFIX)) { + httpStorageHeaderNameArray.add(key.toLowerCase(Locale.ROOT)); + } + } + + Collections.sort(httpStorageHeaderNameArray); + + // Now go through each header's values in the sorted order and append + // them to the canonicalized string. + for (final String key : httpStorageHeaderNameArray) { + final StringBuilder canonicalizedElement = new StringBuilder(key); + String delimiter = ":"; + final ArrayList values = getHeaderValues(headers, key); + + boolean appendCanonicalizedElement = false; + // Go through values, unfold them, and then append them to the + // canonicalized element string. + for (final String value : values) { + if (value != null) { + appendCanonicalizedElement = true; + } + + // Unfolding is simply removal of CRLF. + final String unfoldedValue = CRLF.matcher(value) + .replaceAll(Matcher.quoteReplacement("")); + + // Append it to the canonicalized element string. + canonicalizedElement.append(delimiter); + canonicalizedElement.append(unfoldedValue); + delimiter = ","; + } + + // Now, add this canonicalized element to the canonicalized header + // string. + if (appendCanonicalizedElement) { + appendCanonicalizedElement(canonicalizedString, canonicalizedElement.toString()); + } + } + } + + /** + * Initialie the HmacSha256 associated with the account key. + */ + private void initializeMac() { + // Initializes the HMAC-SHA256 Mac and SecretKey. + try { + hmacSha256 = Mac.getInstance(HMAC_SHA256); + hmacSha256.init(new SecretKeySpec(accountKey, HMAC_SHA256)); + } catch (final Exception e) { + throw new IllegalArgumentException(e); + } + } + + /** + * Append a string to a string builder with a newline constant + * + * @param builder the StringBuilder object + * @param element the string to append. + */ + private static void appendCanonicalizedElement(final StringBuilder builder, final String element) { + builder.append("\n"); + builder.append(element); + } + + /** + * Constructs a canonicalized string from the request's headers that will be used to construct the signature string + * for signing a Blob or Queue service request under the Shared Key Full authentication scheme. + * + * @param address the request URI + * @param accountName the account name associated with the request + * @param method the verb to be used for the HTTP request. + * @param contentType the content type of the HTTP request. + * @param contentLength the length of the content written to the outputstream in bytes, -1 if unknown + * @param date the date/time specification for the HTTP request + * @param conn the HttpURLConnection for the operation. + * @return A canonicalized string. + */ + private static String canonicalizeHttpRequest(final java.net.URL address, final String accountName, + final String method, final String contentType, final long contentLength, final String date, + final HttpURLConnection conn) throws UnsupportedEncodingException { + + // The first element should be the Method of the request. + // I.e. GET, POST, PUT, or HEAD. + final StringBuilder canonicalizedString = new StringBuilder(EXPECTED_BLOB_QUEUE_CANONICALIZED_STRING_LENGTH); + canonicalizedString.append(conn.getRequestMethod()); + + // The next elements are + // If any element is missing it may be empty. + appendCanonicalizedElement(canonicalizedString, + getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_ENCODING, AbfsHttpConstants.EMPTY_STRING)); + appendCanonicalizedElement(canonicalizedString, + getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_LANGUAGE, AbfsHttpConstants.EMPTY_STRING)); + appendCanonicalizedElement(canonicalizedString, + contentLength <= 0 ? "" : String.valueOf(contentLength)); + appendCanonicalizedElement(canonicalizedString, + getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_MD5, AbfsHttpConstants.EMPTY_STRING)); + appendCanonicalizedElement(canonicalizedString, contentType != null ? contentType : AbfsHttpConstants.EMPTY_STRING); + + final String dateString = getHeaderValue(conn, HttpHeaderConfigurations.X_MS_DATE, AbfsHttpConstants.EMPTY_STRING); + // If x-ms-date header exists, Date should be empty string + appendCanonicalizedElement(canonicalizedString, dateString.equals(AbfsHttpConstants.EMPTY_STRING) ? date + : ""); + + appendCanonicalizedElement(canonicalizedString, + getHeaderValue(conn, HttpHeaderConfigurations.IF_MODIFIED_SINCE, AbfsHttpConstants.EMPTY_STRING)); + appendCanonicalizedElement(canonicalizedString, + getHeaderValue(conn, HttpHeaderConfigurations.IF_MATCH, AbfsHttpConstants.EMPTY_STRING)); + appendCanonicalizedElement(canonicalizedString, + getHeaderValue(conn, HttpHeaderConfigurations.IF_NONE_MATCH, AbfsHttpConstants.EMPTY_STRING)); + appendCanonicalizedElement(canonicalizedString, + getHeaderValue(conn, HttpHeaderConfigurations.IF_UNMODIFIED_SINCE, AbfsHttpConstants.EMPTY_STRING)); + appendCanonicalizedElement(canonicalizedString, + getHeaderValue(conn, HttpHeaderConfigurations.RANGE, AbfsHttpConstants.EMPTY_STRING)); + + addCanonicalizedHeaders(conn, canonicalizedString); + + appendCanonicalizedElement(canonicalizedString, getCanonicalizedResource(address, accountName)); + + return canonicalizedString.toString(); + } + + /** + * Gets the canonicalized resource string for a Blob or Queue service request under the Shared Key Lite + * authentication scheme. + * + * @param address the resource URI. + * @param accountName the account name for the request. + * @return the canonicalized resource string. + */ + private static String getCanonicalizedResource(final java.net.URL address, final String accountName) throws UnsupportedEncodingException { + // Resource path + final StringBuilder resourcepath = new StringBuilder(AbfsHttpConstants.FORWARD_SLASH); + resourcepath.append(accountName); + + // Note that AbsolutePath starts with a '/'. + resourcepath.append(address.getPath()); + final StringBuilder canonicalizedResource = new StringBuilder(resourcepath.toString()); + + // query parameters + if (address.getQuery() == null || !address.getQuery().contains(AbfsHttpConstants.EQUAL)) { + //no query params. + return canonicalizedResource.toString(); + } + + final Map queryVariables = parseQueryString(address.getQuery()); + + final Map lowercasedKeyNameValue = new HashMap(); + + for (final Entry entry : queryVariables.entrySet()) { + // sort the value and organize it as comma separated values + final List sortedValues = Arrays.asList(entry.getValue()); + Collections.sort(sortedValues); + + final StringBuilder stringValue = new StringBuilder(); + + for (final String value : sortedValues) { + if (stringValue.length() > 0) { + stringValue.append(AbfsHttpConstants.COMMA); + } + + stringValue.append(value); + } + + // key turns out to be null for ?a&b&c&d + lowercasedKeyNameValue.put((entry.getKey()) == null ? null + : entry.getKey().toLowerCase(Locale.ROOT), stringValue.toString()); + } + + final ArrayList sortedKeys = new ArrayList(lowercasedKeyNameValue.keySet()); + + Collections.sort(sortedKeys); + + for (final String key : sortedKeys) { + final StringBuilder queryParamString = new StringBuilder(); + + queryParamString.append(key); + queryParamString.append(":"); + queryParamString.append(lowercasedKeyNameValue.get(key)); + + appendCanonicalizedElement(canonicalizedResource, queryParamString.toString()); + } + + return canonicalizedResource.toString(); + } + + /** + * Gets all the values for the given header in the one to many map, performs a trimStart() on each return value + * + * @param headers a one to many map of key / values representing the header values for the connection. + * @param headerName the name of the header to lookup + * @return an ArrayList of all trimmed values corresponding to the requested headerName. This may be empty + * if the header is not found. + */ + private static ArrayList getHeaderValues(final Map> headers, final String headerName) { + + final ArrayList arrayOfValues = new ArrayList(); + List values = null; + + for (final Entry> entry : headers.entrySet()) { + if (entry.getKey().toLowerCase(Locale.ROOT).equals(headerName)) { + values = entry.getValue(); + break; + } + } + if (values != null) { + for (final String value : values) { + // canonicalization formula requires the string to be left + // trimmed. + arrayOfValues.add(trimStart(value)); + } + } + return arrayOfValues; + } + + /** + * Parses a query string into a one to many hashmap. + * + * @param parseString the string to parse + * @return a HashMap of the key values. + */ + private static HashMap parseQueryString(String parseString) throws UnsupportedEncodingException { + final HashMap retVals = new HashMap(); + if (parseString == null || parseString.isEmpty()) { + return retVals; + } + + // 1. Remove ? if present + final int queryDex = parseString.indexOf(AbfsHttpConstants.QUESTION_MARK); + if (queryDex >= 0 && parseString.length() > 0) { + parseString = parseString.substring(queryDex + 1); + } + + // 2. split name value pairs by splitting on the 'c&' character + final String[] valuePairs = parseString.contains(AbfsHttpConstants.AND_MARK) + ? parseString.split(AbfsHttpConstants.AND_MARK) + : parseString.split(AbfsHttpConstants.SEMICOLON); + + // 3. for each field value pair parse into appropriate map entries + for (int m = 0; m < valuePairs.length; m++) { + final int equalDex = valuePairs[m].indexOf(AbfsHttpConstants.EQUAL); + + if (equalDex < 0 || equalDex == valuePairs[m].length() - 1) { + continue; + } + + String key = valuePairs[m].substring(0, equalDex); + String value = valuePairs[m].substring(equalDex + 1); + + key = safeDecode(key); + value = safeDecode(value); + + // 3.1 add to map + String[] values = retVals.get(key); + + if (values == null) { + values = new String[]{value}; + if (!value.equals("")) { + retVals.put(key, values); + } + } + } + + return retVals; + } + + /** + * Performs safe decoding of the specified string, taking care to preserve each + character, rather + * than replacing it with a space character. + * + * @param stringToDecode A String that represents the string to decode. + * @return A String that represents the decoded string. + *

+ * If a storage service error occurred. + */ + private static String safeDecode(final String stringToDecode) throws UnsupportedEncodingException { + if (stringToDecode == null) { + return null; + } + + if (stringToDecode.length() == 0) { + return ""; + } + + if (stringToDecode.contains(AbfsHttpConstants.PLUS)) { + final StringBuilder outBuilder = new StringBuilder(); + + int startDex = 0; + for (int m = 0; m < stringToDecode.length(); m++) { + if (stringToDecode.charAt(m) == '+') { + if (m > startDex) { + outBuilder.append(URLDecoder.decode(stringToDecode.substring(startDex, m), + AbfsHttpConstants.UTF_8)); + } + + outBuilder.append(AbfsHttpConstants.PLUS); + startDex = m + 1; + } + } + + if (startDex != stringToDecode.length()) { + outBuilder.append(URLDecoder.decode(stringToDecode.substring(startDex, stringToDecode.length()), + AbfsHttpConstants.UTF_8)); + } + + return outBuilder.toString(); + } else { + return URLDecoder.decode(stringToDecode, AbfsHttpConstants.UTF_8); + } + } + + private static String trimStart(final String value) { + int spaceDex = 0; + while (spaceDex < value.length() && value.charAt(spaceDex) == ' ') { + spaceDex++; + } + + return value.substring(spaceDex); + } + + private static String getHeaderValue(final HttpURLConnection conn, final String headerName, final String defaultValue) { + final String headerValue = conn.getRequestProperty(headerName); + return headerValue == null ? defaultValue : headerValue; + } + + + /** + * Constructs a canonicalized string for signing a request. + * + * @param conn the HttpURLConnection to canonicalize + * @param accountName the account name associated with the request + * @param contentLength the length of the content written to the outputstream in bytes, + * -1 if unknown + * @return a canonicalized string. + */ + private String canonicalize(final HttpURLConnection conn, + final String accountName, + final Long contentLength) throws UnsupportedEncodingException { + + if (contentLength < -1) { + throw new IllegalArgumentException( + "The Content-Length header must be greater than or equal to -1."); + } + + String contentType = getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_TYPE, ""); + + return canonicalizeHttpRequest(conn.getURL(), accountName, + conn.getRequestMethod(), contentType, contentLength, null, conn); + } + + /** + * Thread local for storing GMT date format. + */ + private static ThreadLocal rfc1123GmtDateTimeFormatter + = new ThreadLocal() { + @Override + protected DateFormat initialValue() { + final DateFormat formatter = new SimpleDateFormat(RFC1123_PATTERN, Locale.ROOT); + formatter.setTimeZone(GMT_ZONE); + return formatter; + } + }; + + public static final TimeZone GMT_ZONE = TimeZone.getTimeZone(AbfsHttpConstants.GMT_TIMEZONE); + + + /** + * Returns the current GMT date/time String using the RFC1123 pattern. + * + * @return A String that represents the current GMT date/time using the RFC1123 pattern. + */ + static String getGMTTime() { + return getGMTTime(new Date()); + } + + /** + * Returns the GTM date/time String for the specified value using the RFC1123 pattern. + * + * @param date + * A Date object that represents the date to convert to GMT date/time in the RFC1123 + * pattern. + * + * @return A String that represents the GMT date/time for the specified value using the RFC1123 + * pattern. + */ + static String getGMTTime(final Date date) { + return rfc1123GmtDateTimeFormatter.get().format(date); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.java new file mode 100644 index 00000000000..57b6463322b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.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.PrintWriter; +import java.io.StringWriter; +import java.util.Objects; + +import com.google.common.base.Preconditions; +import com.google.inject.Inject; +import com.google.inject.Singleton; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService; +import org.apache.htrace.core.HTraceConfiguration; +import org.apache.htrace.core.Sampler; +import org.apache.htrace.core.SpanId; +import org.apache.htrace.core.TraceScope; +import org.apache.htrace.core.Tracer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Singleton +@InterfaceAudience.Private +@InterfaceStability.Evolving +final class TracingServiceImpl implements TracingService { + private static final Logger LOG = LoggerFactory.getLogger(TracingService.class); + + private final Tracer tracer; + private final ThreadLocal currentScopeId; + + @Inject + TracingServiceImpl( + final Configuration configuration) { + Preconditions.checkNotNull(configuration, "configuration"); + + this.currentScopeId = new ThreadLocal<>(); + + this.tracer = new Tracer.Builder(TracingService.class.getSimpleName()). + conf(new HTraceConfiguration() { + @Override + public String get(String key) { + if (Objects.equals(key, Tracer.SPAN_RECEIVER_CLASSES_KEY)) { + return LoggerSpanReceiver.class.getName(); + } + return null; + } + + @Override + public String get(String key, String defaultValue) { + String value = get(key); + if (value != null) { + return value; + } + return defaultValue; + } + }). + build(); + + this.tracer.addSampler(Sampler.ALWAYS); + } + + @Override + public TraceScope traceBegin(String description) { + if (this.LOG.isTraceEnabled()) { + TraceScope traceScope = this.tracer.newScope(description); + this.currentScopeId.set(traceScope.getSpanId()); + return traceScope; + } + + return null; + } + + @Override + public TraceScope traceBegin(String description, SpanId parentSpanId) { + if (this.LOG.isTraceEnabled()) { + TraceScope traceScope = this.tracer.newScope(description, parentSpanId); + this.currentScopeId.set(traceScope.getSpanId()); + return traceScope; + } + + return null; + } + + @Override + public void traceException(TraceScope traceScope, AzureBlobFileSystemException azureBlobFileSystemException) { + if (this.LOG.isTraceEnabled()) { + Preconditions.checkNotNull(traceScope, "traceScope"); + Preconditions.checkNotNull(azureBlobFileSystemException, "azureBlobFileSystemException"); + + StringWriter stringWriter = new StringWriter(); + PrintWriter printWriter = new PrintWriter(stringWriter); + azureBlobFileSystemException.printStackTrace(printWriter); + printWriter.flush(); + + traceScope.addKVAnnotation("Exception", stringWriter.toString()); + } + } + + @Override + public SpanId getCurrentTraceScopeSpanId() { + return this.currentScopeId.get(); + } + + @Override + public void traceEnd(TraceScope traceScope) { + if (this.LOG.isTraceEnabled()) { + Preconditions.checkNotNull(traceScope, "traceScope"); + + SpanId[] parents = traceScope.getSpan().getParents(); + this.currentScopeId.set(parents != null && parents.length > 0 ? parents[parents.length - 1] : null); + traceScope.close(); + } + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/package-info.java new file mode 100644 index 00000000000..97c1d71251f --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +package org.apache.hadoop.fs.azurebfs.services; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file 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 new file mode 100644 index 00000000000..7652adf10e3 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/UriUtils.java @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs.utils; + +import java.util.regex.Pattern; + +/** + * Utility class to help with Abfs url transformation to blob urls. + */ +public final class UriUtils { + private static final String ABFS_URI_REGEX = "[^.]+\\.dfs\\.(preprod\\.){0,1}core\\.windows\\.net"; + private static final Pattern ABFS_URI_PATTERN = Pattern.compile(ABFS_URI_REGEX); + + /** + * Checks whether a string includes abfs url. + * @param string the string to check. + * @return true if string has abfs url. + */ + public static boolean containsAbfsUrl(final String string) { + if (string == null || string.isEmpty()) { + return false; + } + + return ABFS_URI_PATTERN.matcher(string).matches(); + } + + /** + * Extracts the raw account name from account name. + * @param accountName to extract the raw account name. + * @return extracted raw account name. + */ + public static String extractRawAccountFromAccountName(final String accountName) { + if (accountName == null || accountName.isEmpty()) { + return null; + } + + if (!containsAbfsUrl(accountName)) { + return null; + } + + String[] splitByDot = accountName.split("\\."); + if (splitByDot.length == 0) { + return null; + } + + return splitByDot[0]; + } + + /** + * Generate unique test path for multiple user tests. + * + * @return root test path + */ + public static String generateUniqueTestPath() { + String testUniqueForkId = System.getProperty("test.unique.fork.id"); + return testUniqueForkId == null ? "/test" : "/" + testUniqueForkId + "/test"; + } + + private UriUtils() { + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/package-info.java new file mode 100644 index 00000000000..d8cc940da1b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +package org.apache.hadoop.fs.azurebfs.utils; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java new file mode 100644 index 00000000000..5ec1e2ed424 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java @@ -0,0 +1,225 @@ +/** + * 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.net.URI; +import java.util.UUID; +import java.util.concurrent.Callable; + +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.mockito.internal.util.MockUtil; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore; +import org.apache.hadoop.fs.azure.NativeAzureFileSystem; +import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation; +import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; +import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory; +import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService; +import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; +import org.apache.hadoop.fs.azurebfs.services.MockAbfsHttpClientFactoryImpl; +import org.apache.hadoop.fs.azurebfs.services.MockAbfsServiceInjectorImpl; +import org.apache.hadoop.fs.azurebfs.services.MockServiceProviderImpl; + +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.junit.Assert.assertEquals; +import static org.junit.Assume.assumeNotNull; + +/** + * Provide dependencies for AzureBlobFileSystem tests. + */ +public abstract class DependencyInjectedTest { + private final MockAbfsServiceInjectorImpl mockServiceInjector; + private final boolean isEmulator; + private NativeAzureFileSystem wasb; + private String abfsScheme; + + private Configuration configuration; + private String fileSystemName; + private String accountName; + private String testUrl; + + public DependencyInjectedTest(final boolean secure) { + this(secure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME); + } + + public MockAbfsServiceInjectorImpl getMockServiceInjector() { + return this.mockServiceInjector; + } + + protected DependencyInjectedTest() { + this(FileSystemUriSchemes.ABFS_SCHEME); + } + + private DependencyInjectedTest(final String scheme) { + abfsScheme = scheme; + fileSystemName = UUID.randomUUID().toString(); + configuration = new Configuration(); + configuration.addResource("azure-bfs-test.xml"); + + assumeNotNull(configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME)); + assumeNotNull(configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_KEY_PREFIX + configuration.get(TestConfigurationKeys + .FS_AZURE_TEST_ACCOUNT_NAME))); + + final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName(); + URI defaultUri = null; + + try { + defaultUri = new URI(abfsScheme, abfsUrl, null, null, null); + } catch (Exception ex) { + Assert.fail(ex.getMessage()); + } + + this.testUrl = defaultUri.toString(); + configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); + configuration.setBoolean(ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true); + this.mockServiceInjector = new MockAbfsServiceInjectorImpl(configuration); + this.isEmulator = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false); + this.accountName = this.configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME); + } + + @Before + public void initialize() throws Exception { + if (this.isEmulator) { + this.mockServiceInjector.replaceProvider(AbfsHttpClientFactory.class, MockAbfsHttpClientFactoryImpl.class); + } + + MockServiceProviderImpl.create(this.mockServiceInjector); + + if (!this.isEmulator) { + final URI wasbUri = new URI(abfsUrlToWasbUrl(this.getTestUrl())); + final AzureNativeFileSystemStore azureNativeFileSystemStore = new AzureNativeFileSystemStore(); + azureNativeFileSystemStore.initialize( + wasbUri, + this.getConfiguration(), + new AzureFileSystemInstrumentation(this.getConfiguration())); + + this.wasb = new NativeAzureFileSystem(azureNativeFileSystemStore); + this.wasb.initialize(wasbUri, configuration); + } + } + + @After + public void testCleanup() throws Exception { + if (this.wasb != null) { + this.wasb.close(); + } + + FileSystem.closeAll(); + + final AzureBlobFileSystem fs = this.getFileSystem(); + final AbfsHttpService abfsHttpService = AbfsServiceProviderImpl.instance().get(AbfsHttpService.class); + abfsHttpService.deleteFilesystem(fs); + + if (!(new MockUtil().isMock(abfsHttpService))) { + AbfsRestOperationException ex = intercept( + AbfsRestOperationException.class, + new Callable() { + @Override + public Void call() throws Exception { + abfsHttpService.getFilesystemProperties(fs); + return null; + } + }); + + assertEquals(FILE_SYSTEM_NOT_FOUND.getStatusCode(), ex.getStatusCode()); + } + } + + public AzureBlobFileSystem getFileSystem() throws Exception { + final Configuration configuration = AbfsServiceProviderImpl.instance().get(ConfigurationService.class).getConfiguration(); + final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(configuration); + return fs; + } + + protected NativeAzureFileSystem getWasbFileSystem() { + return this.wasb; + } + + protected String getHostName() { + return configuration.get(TestConfigurationKeys.FS_AZURE_TEST_HOST_NAME); + } + + protected void updateTestUrl(String testUrl) { + this.testUrl = testUrl; + } + protected String getTestUrl() { + return testUrl; + } + + protected void updateFileSystemName(String fileSystemName) { + this.fileSystemName = fileSystemName; + } + protected String getFileSystemName() { + return fileSystemName; + } + + protected String getAccountName() { + return configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME); + } + + protected String getAccountKey() { + return configuration.get( + TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_KEY_PREFIX + + getAccountName()); + } + + protected Configuration getConfiguration() { + return this.configuration; + } + + protected boolean isEmulator() { + return isEmulator; + } + + protected static String wasbUrlToAbfsUrl(final String wasbUrl) { + return convertTestUrls( + wasbUrl, FileSystemUriSchemes.WASB_SCHEME, FileSystemUriSchemes.WASB_SECURE_SCHEME, FileSystemUriSchemes.WASB_DNS_PREFIX, + FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME, FileSystemUriSchemes.ABFS_DNS_PREFIX); + } + + protected static String abfsUrlToWasbUrl(final String abfsUrl) { + 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); + } + + 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) { + String data = null; + if (url.startsWith(fromNonSecureScheme + "://")) { + data = url.replace(fromNonSecureScheme + "://", toNonSecureScheme + "://"); + } else if (url.startsWith(fromSecureScheme + "://")) { + data = url.replace(fromSecureScheme + "://", toSecureScheme + "://"); + } + + data = data.replace("." + fromDnsPrefix + ".", "." + toDnsPrefix + "."); + return data; + } +} 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 new file mode 100644 index 00000000000..10d42d1399d --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAppend.java @@ -0,0 +1,78 @@ +/** + * 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.FileNotFoundException; +import java.util.Random; + +import org.junit.Test; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; + +import static org.junit.Assert.assertEquals; + +/** + * Test append operations. + */ +public class ITestAzureBlobFileSystemAppend extends DependencyInjectedTest { + private static final Path TEST_FILE_PATH = new Path("testfile"); + private static final Path TEST_FOLDER_PATH = new Path("testFolder"); + public ITestAzureBlobFileSystemAppend() { + super(); + } + + @Test(expected = FileNotFoundException.class) + public void testAppendDirShouldFail() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final Path filePath = TEST_FILE_PATH; + fs.mkdirs(filePath); + fs.append(filePath, 0); + } + + @Test + public void testAppendWithLength0() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + FSDataOutputStream stream = fs.create(TEST_FILE_PATH); + final byte[] b = new byte[1024]; + new Random().nextBytes(b); + stream.write(b, 1000, 0); + + assertEquals(0, stream.getPos()); + } + + + @Test(expected = FileNotFoundException.class) + public void testAppendFileAfterDelete() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final Path filePath = TEST_FILE_PATH; + fs.create(filePath); + fs.delete(filePath, false); + + fs.append(filePath); + } + + @Test(expected = FileNotFoundException.class) + public void testAppendDirectory() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final Path folderPath = TEST_FOLDER_PATH; + fs.mkdirs(folderPath); + fs.append(folderPath); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java new file mode 100644 index 00000000000..d107c9d008b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java @@ -0,0 +1,83 @@ +/** + * 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 com.microsoft.azure.storage.CloudStorageAccount; +import com.microsoft.azure.storage.blob.CloudBlobClient; +import com.microsoft.azure.storage.blob.CloudBlobContainer; +import com.microsoft.azure.storage.blob.CloudBlockBlob; +import org.junit.Test; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Test AzureBlobFileSystem back compatibility with WASB. + */ +public class ITestAzureBlobFileSystemBackCompat extends DependencyInjectedTest { + public ITestAzureBlobFileSystemBackCompat() { + super(); + } + + @Test + public void testBlobBackCompat() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + String storageConnectionString = getBlobConnectionString(); + CloudStorageAccount storageAccount = CloudStorageAccount.parse(storageConnectionString); + CloudBlobClient blobClient = storageAccount.createCloudBlobClient(); + CloudBlobContainer container = blobClient.getContainerReference(this.getFileSystemName()); + container.createIfNotExists(); + + CloudBlockBlob blockBlob = container.getBlockBlobReference("test/10/10/10"); + blockBlob.uploadText(""); + + blockBlob = container.getBlockBlobReference("test/10/123/3/2/1/3"); + blockBlob.uploadText(""); + + FileStatus[] fileStatuses = fs.listStatus(new Path("/test/10/")); + assertEquals(fileStatuses.length, 2); + assertEquals(fileStatuses[0].getPath().getName(), "10"); + assertTrue(fileStatuses[0].isDirectory()); + assertEquals(fileStatuses[0].getLen(), 0); + assertEquals(fileStatuses[1].getPath().getName(), "123"); + assertTrue(fileStatuses[1].isDirectory()); + assertEquals(fileStatuses[1].getLen(), 0); + } + + private String getBlobConnectionString() { + String connectionString; + if (isEmulator()) { + connectionString = "DefaultEndpointsProtocol=http;BlobEndpoint=http://" + + this.getHostName() + ":8880/" + this.getAccountName().split("\\.") [0] + + ";AccountName=" + this.getAccountName().split("\\.")[0] + + ";AccountKey=" + this.getAccountKey(); + } + else { + connectionString = "DefaultEndpointsProtocol=http;BlobEndpoint=http://" + + this.getAccountName().replaceFirst("\\.dfs\\.", ".blob.") + + ";AccountName=" + this.getAccountName().split("\\.")[0] + + ";AccountKey=" + this.getAccountKey(); + } + + return connectionString; + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java new file mode 100644 index 00000000000..c158e03e622 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java @@ -0,0 +1,100 @@ +/** + * 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.BufferedReader; +import java.io.BufferedWriter; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.OutputStreamWriter; + +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Test copy operation. + */ +public class ITestAzureBlobFileSystemCopy extends DependencyInjectedTest { + public ITestAzureBlobFileSystemCopy() { + super(); + } + + @Test + public void testCopyFromLocalFileSystem() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + Path localFilePath = new Path(System.getProperty("test.build.data", + "azure_test")); + FileSystem localFs = FileSystem.get(new Configuration()); + localFs.delete(localFilePath, true); + try { + writeString(localFs, localFilePath, "Testing"); + Path dstPath = new Path("copiedFromLocal"); + assertTrue(FileUtil.copy(localFs, localFilePath, fs, dstPath, false, + fs.getConf())); + assertTrue(fs.exists(dstPath)); + assertEquals("Testing", readString(fs, dstPath)); + fs.delete(dstPath, true); + } finally { + localFs.delete(localFilePath, true); + } + } + + private String readString(FileSystem fs, Path testFile) throws IOException { + FSDataInputStream inputStream = fs.open(testFile); + String ret = readString(inputStream); + inputStream.close(); + return ret; + } + + private String readString(FSDataInputStream inputStream) throws IOException { + BufferedReader reader = new BufferedReader(new InputStreamReader( + inputStream)); + final int bufferSize = 1024; + char[] buffer = new char[bufferSize]; + int count = reader.read(buffer, 0, bufferSize); + if (count > bufferSize) { + throw new IOException("Exceeded buffer size"); + } + inputStream.close(); + return new String(buffer, 0, count); + } + + private void writeString(FileSystem fs, Path path, String value) + throws IOException { + FSDataOutputStream outputStream = fs.create(path, true); + writeString(outputStream, value); + } + + private void writeString(FSDataOutputStream outputStream, String value) + throws IOException { + BufferedWriter writer = new BufferedWriter(new OutputStreamWriter( + outputStream)); + writer.write(value); + writer.close(); + } +} 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 new file mode 100644 index 00000000000..c9b99e6d8a2 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -0,0 +1,110 @@ +/** + * 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.FileNotFoundException; +import java.util.EnumSet; + +import org.junit.Test; + +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * Test create operation. + */ +public class ITestAzureBlobFileSystemCreate extends DependencyInjectedTest { + 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"; + public ITestAzureBlobFileSystemCreate() { + super(); + } + + @Test(expected = FileAlreadyExistsException.class) + public void testCreateFileWithExistingDir() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.mkdirs(TEST_FOLDER_PATH); + fs.create(TEST_FOLDER_PATH); + } + + @Test + public void testEnsureFileCreated() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.create(TEST_FILE_PATH); + + FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH); + assertNotNull(fileStatus); + } + + @Test + @SuppressWarnings("deprecation") + public void testCreateNonRecursive() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE); + try { + fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null); + assertTrue("Should've thrown", false); + } catch (FileNotFoundException e) { + } + fs.mkdirs(TEST_FOLDER_PATH); + fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null) + .close(); + assertTrue(fs.exists(testFile)); + } + + @Test + @SuppressWarnings("deprecation") + public void testCreateNonRecursive1() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE); + try { + fs.createNonRecursive(testFile, FsPermission.getDefault(), EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE), 1024, (short) 1, 1024, null); + assertTrue("Should've thrown", false); + } catch (FileNotFoundException e) { + } + fs.mkdirs(TEST_FOLDER_PATH); + fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null) + .close(); + assertTrue(fs.exists(testFile)); + } + + @Test + @SuppressWarnings("deprecation") + public void testCreateNonRecursive2() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + + Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE); + try { + fs.createNonRecursive(testFile, FsPermission.getDefault(), false, 1024, (short) 1, 1024, null); + assertTrue("Should've thrown", false); + } catch (FileNotFoundException e) { + } + fs.mkdirs(TEST_FOLDER_PATH); + fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null) + .close(); + assertTrue(fs.exists(testFile)); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java new file mode 100644 index 00000000000..372a0876b11 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -0,0 +1,119 @@ +/** + * 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.FileNotFoundException; +import java.util.ArrayList; +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 org.junit.Test; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import static org.junit.Assert.assertEquals; + +/** + * Test delete operation. + */ +public class ITestAzureBlobFileSystemDelete extends DependencyInjectedTest { + public ITestAzureBlobFileSystemDelete() { + super(); + } + + @Test + public void testDeleteRoot() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + + fs.mkdirs(new Path("/testFolder0")); + fs.mkdirs(new Path("/testFolder1")); + fs.mkdirs(new Path("/testFolder2")); + fs.create(new Path("/testFolder1/testfile")); + fs.create(new Path("/testFolder1/testfile2")); + fs.create(new Path("/testFolder1/testfile3")); + + FileStatus[] ls = fs.listStatus(new Path("/")); + assertEquals(4, ls.length); // and user dir + + fs.delete(new Path("/"), true); + ls = fs.listStatus(new Path("/")); + assertEquals(0, ls.length); + } + + @Test(expected = FileNotFoundException.class) + public void testOpenFileAfterDelete() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.create(new Path("/testFile")); + fs.delete(new Path("/testFile"), false); + + fs.open(new Path("/testFile")); + } + + @Test(expected = FileNotFoundException.class) + public void testEnsureFileIsDeleted() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.create(new Path("testfile")); + fs.delete(new Path("testfile"), false); + + fs.getFileStatus(new Path("testfile")); + } + + @Test(expected = FileNotFoundException.class) + public void testDeleteDirectory() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.mkdirs(new Path("testfile")); + fs.mkdirs(new Path("testfile/test1")); + fs.mkdirs(new Path("testfile/test1/test2")); + + fs.delete(new Path("testfile"), true); + fs.getFileStatus(new Path("testfile")); + } + + @Test(expected = FileNotFoundException.class) + public void testDeleteFirstLevelDirectory() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final List tasks = new ArrayList<>(); + + ExecutorService es = Executors.newFixedThreadPool(10); + for (int i = 0; i < 1000; i++) { + final Path fileName = new Path("/test/" + i); + Callable callable = new Callable() { + @Override + public Void call() throws Exception { + fs.create(fileName); + return null; + } + }; + + tasks.add(es.submit(callable)); + } + + for (Future task : tasks) { + task.get(); + } + + es.shutdownNow(); + fs.delete(new Path("/test"), true); + fs.getFileStatus(new Path("/test")); + } +} 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 new file mode 100644 index 00000000000..4985f58fdbf --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java @@ -0,0 +1,150 @@ +/** + * 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.Arrays; +import java.util.Random; + + +import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl; +import org.junit.Test; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertArrayEquals; + +/** + * Test end to end between ABFS client and ABFS server. + */ +public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest { + private static final Path TEST_FILE = new Path("testfile"); + private static final int TEST_BYTE = 100; + private static final int TEST_OFFSET = 100; + private static final int TEST_DEFAULT_BUFFER_SIZE = 4 * 1024 * 1024; + private static final int TEST_DEFAULT_READ_BUFFER_SIZE = 1023900; + + public ITestAzureBlobFileSystemE2E() { + super(); + Configuration configuration = this.getConfiguration(); + configuration.set(ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH, "0"); + this.getMockServiceInjector().replaceInstance(Configuration.class, configuration); + + } + + @Test + public void testWriteOneByteToFile() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + FSDataOutputStream stream = fs.create(TEST_FILE); + + stream.write(TEST_BYTE); + stream.close(); + + FileStatus fileStatus = fs.getFileStatus(TEST_FILE); + assertEquals(1, fileStatus.getLen()); + } + + @Test + public void testReadWriteBytesToFile() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + testWriteOneByteToFile(); + FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE); + int i = inputStream.read(); + inputStream.close(); + + assertEquals(TEST_BYTE, i); + } + + @Test (expected = IOException.class) + public void testOOBWrites() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + int readBufferSize = AbfsServiceProviderImpl.instance().get(ConfigurationService.class).getReadBufferSize(); + + fs.create(TEST_FILE); + FSDataOutputStream writeStream = fs.create(TEST_FILE); + + byte[] bytesToRead = new byte[readBufferSize]; + final byte[] b = new byte[2 * readBufferSize]; + new Random().nextBytes(b); + + writeStream.write(b); + writeStream.flush(); + writeStream.close(); + + FSDataInputStream readStream = fs.open(TEST_FILE); + readStream.read(bytesToRead, 0, readBufferSize); + + writeStream = fs.create(TEST_FILE); + writeStream.write(b); + writeStream.flush(); + writeStream.close(); + + readStream.read(bytesToRead, 0, readBufferSize); + readStream.close(); + } + + @Test + public void testWriteWithBufferOffset() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final FSDataOutputStream stream = fs.create(TEST_FILE); + + final byte[] b = new byte[1024 * 1000]; + new Random().nextBytes(b); + stream.write(b, TEST_OFFSET, b.length - TEST_OFFSET); + stream.close(); + + final byte[] r = new byte[TEST_DEFAULT_READ_BUFFER_SIZE]; + FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE); + int result = inputStream.read(r); + + assertNotEquals(-1, result); + assertArrayEquals(r, Arrays.copyOfRange(b, TEST_OFFSET, b.length)); + + inputStream.close(); + } + + @Test + public void testReadWriteHeavyBytesToFileWithSmallerChunks() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final FSDataOutputStream stream = fs.create(TEST_FILE); + + final byte[] writeBuffer = new byte[5 * 1000 * 1024]; + new Random().nextBytes(writeBuffer); + stream.write(writeBuffer); + stream.close(); + + final byte[] readBuffer = new byte[5 * 1000 * 1024]; + FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE); + int offset = 0; + while (inputStream.read(readBuffer, offset, TEST_OFFSET) > 0) { + offset += TEST_OFFSET; + } + + assertArrayEquals(readBuffer, writeBuffer); + inputStream.close(); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java new file mode 100644 index 00000000000..616253bca35 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java @@ -0,0 +1,133 @@ +/** + * 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.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +/** + * Test end to end between ABFS client and ABFS server with heavy traffic. + */ +public class ITestAzureBlobFileSystemE2EScale extends DependencyInjectedTest { + private static final int TEN = 10; + private static final int ONE_THOUSAND = 1000; + private static final int BASE_SIZE = 1024; + private static final int ONE_MB = 1024 * 1024; + private static final int DEFAULT_WRITE_TIMES = 100; + private static final Path TEST_FILE = new Path("testfile"); + + public ITestAzureBlobFileSystemE2EScale() { + super(); + } + + @Test + public void testWriteHeavyBytesToFile() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final FSDataOutputStream stream = fs.create(TEST_FILE); + ExecutorService es = Executors.newFixedThreadPool(TEN); + + int testWriteBufferSize = 2 * TEN * ONE_THOUSAND * BASE_SIZE; + final byte[] b = new byte[testWriteBufferSize]; + new Random().nextBytes(b); + List> tasks = new ArrayList<>(); + + for (int i = 0; i < DEFAULT_WRITE_TIMES; i++) { + Callable callable = new Callable() { + @Override + public Void call() throws Exception { + stream.write(b); + return null; + } + }; + + tasks.add(es.submit(callable)); + } + + for (Future task : tasks) { + task.get(); + } + + tasks.clear(); + stream.close(); + + es.shutdownNow(); + FileStatus fileStatus = fs.getFileStatus(TEST_FILE); + assertEquals(testWriteBufferSize * DEFAULT_WRITE_TIMES, fileStatus.getLen()); + } + + @Test + public void testReadWriteHeavyBytesToFile() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final FSDataOutputStream stream = fs.create(TEST_FILE); + + int testBufferSize = 5 * TEN * ONE_THOUSAND * BASE_SIZE; + final byte[] b = new byte[testBufferSize]; + new Random().nextBytes(b); + stream.write(b); + stream.close(); + + final byte[] r = new byte[testBufferSize]; + FSDataInputStream inputStream = fs.open(TEST_FILE, 4 * ONE_MB); + int result = inputStream.read(r); + inputStream.close(); + + assertNotEquals(-1, result); + assertArrayEquals(r, b); + } + + @Test + public void testReadWriteHeavyBytesToFileWithStatistics() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final FSDataOutputStream stream = fs.create(TEST_FILE); + final FileSystem.Statistics abfsStatistics = fs.getFsStatistics(); + abfsStatistics.reset(); + + int testBufferSize = 5 * TEN * ONE_THOUSAND * BASE_SIZE; + final byte[] b = new byte[testBufferSize]; + new Random().nextBytes(b); + stream.write(b); + stream.close(); + + final byte[] r = new byte[testBufferSize]; + FSDataInputStream inputStream = fs.open(TEST_FILE, 4 * ONE_MB); + inputStream.read(r); + inputStream.close(); + + Assert.assertEquals(r.length, abfsStatistics.getBytesRead()); + Assert.assertEquals(b.length, abfsStatistics.getBytesWritten()); + } +} 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 new file mode 100644 index 00000000000..bfa662d455b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs; + +import org.junit.Test; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.permission.FsPermission; + +import static org.junit.Assert.assertEquals; + +/** + * Test FileStatus. + */ +public class ITestAzureBlobFileSystemFileStatus extends DependencyInjectedTest { + private static final Path TEST_FILE = new Path("testFile"); + private static final Path TEST_FOLDER = new Path("testDir"); + public ITestAzureBlobFileSystemFileStatus() { + super(); + } + + @Test + public void testEnsureStatusWorksForRoot() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + + fs.getFileStatus(new Path("/")); + fs.listStatus(new Path("/")); + } + + @Test + public void testFileStatusPermissionsAndOwnerAndGroup() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.create(TEST_FILE); + fs.mkdirs(TEST_FOLDER); + + FileStatus fileStatus = fs.getFileStatus(TEST_FILE); + assertEquals(new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL), fileStatus.getPermission()); + assertEquals(fs.getOwnerUser(), fileStatus.getGroup()); + assertEquals(fs.getOwnerUserPrimaryGroup(), fileStatus.getOwner()); + + fileStatus = fs.getFileStatus(TEST_FOLDER); + assertEquals(new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL), fileStatus.getPermission()); + assertEquals(fs.getOwnerUser(), fileStatus.getGroup()); + assertEquals(fs.getOwnerUserPrimaryGroup(), fileStatus.getOwner()); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java new file mode 100644 index 00000000000..8c2e8ce32dd --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -0,0 +1,204 @@ +/** + * 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.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +import org.junit.Test; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +/** + * Test flush operation. + */ +public class ITestAzureBlobFileSystemFlush extends DependencyInjectedTest { + private static final int BASE_SIZE = 1024; + private static final int ONE_THOUSAND = 1000; + private static final int TEST_BUFFER_SIZE = 5 * ONE_THOUSAND * BASE_SIZE; + private static final int ONE_MB = 1024 * 1024; + private static final int FLUSH_TIMES = 200; + private static final int THREAD_SLEEP_TIME = 6000; + + private static final Path TEST_FILE_PATH = new Path("/testfile"); + + public ITestAzureBlobFileSystemFlush() { + super(); + } + + @Test + public void testAbfsOutputStreamAsyncFlushWithRetainUncommitedData() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final FSDataOutputStream stream = fs.create(TEST_FILE_PATH); + + final byte[] b = new byte[TEST_BUFFER_SIZE]; + new Random().nextBytes(b); + + for (int i = 0; i < 2; i++) { + stream.write(b); + + for (int j = 0; j < FLUSH_TIMES; j++) { + stream.flush(); + Thread.sleep(10); + } + } + + stream.close(); + + final byte[] r = new byte[TEST_BUFFER_SIZE]; + FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB); + + while (inputStream.available() != 0) { + int result = inputStream.read(r); + + assertNotEquals(-1, result); + assertArrayEquals(r, b); + } + + inputStream.close(); + } + + @Test + public void testAbfsOutputStreamSyncFlush() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final FSDataOutputStream stream = fs.create(TEST_FILE_PATH); + + final byte[] b = new byte[TEST_BUFFER_SIZE]; + new Random().nextBytes(b); + stream.write(b); + + for (int i = 0; i < FLUSH_TIMES; i++) { + stream.hsync(); + stream.hflush(); + Thread.sleep(10); + } + stream.close(); + + final byte[] r = new byte[TEST_BUFFER_SIZE]; + FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB); + int result = inputStream.read(r); + + assertNotEquals(-1, result); + assertArrayEquals(r, b); + + inputStream.close(); + } + + + @Test + public void testWriteHeavyBytesToFileSyncFlush() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final FSDataOutputStream stream = fs.create(TEST_FILE_PATH); + final FileSystem.Statistics abfsStatistics = fs.getFsStatistics(); + abfsStatistics.reset(); + + ExecutorService es = Executors.newFixedThreadPool(10); + + final byte[] b = new byte[TEST_BUFFER_SIZE]; + new Random().nextBytes(b); + + List> tasks = new ArrayList<>(); + for (int i = 0; i < FLUSH_TIMES; i++) { + Callable callable = new Callable() { + @Override + public Void call() throws Exception { + stream.write(b); + return null; + } + }; + + tasks.add(es.submit(callable)); + } + + boolean shouldStop = false; + while (!shouldStop) { + shouldStop = true; + for (Future task : tasks) { + if (!task.isDone()) { + stream.hsync(); + shouldStop = false; + Thread.sleep(THREAD_SLEEP_TIME); + } + } + } + + tasks.clear(); + stream.close(); + + es.shutdownNow(); + FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH); + assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, fileStatus.getLen()); + assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, abfsStatistics.getBytesWritten()); + } + + @Test + public void testWriteHeavyBytesToFileAsyncFlush() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.create(TEST_FILE_PATH); + final FSDataOutputStream stream = fs.create(TEST_FILE_PATH); + ExecutorService es = Executors.newFixedThreadPool(10); + + final byte[] b = new byte[TEST_BUFFER_SIZE]; + new Random().nextBytes(b); + + List> tasks = new ArrayList<>(); + for (int i = 0; i < FLUSH_TIMES; i++) { + Callable callable = new Callable() { + @Override + public Void call() throws Exception { + stream.write(b); + return null; + } + }; + + tasks.add(es.submit(callable)); + } + + boolean shouldStop = false; + while (!shouldStop) { + shouldStop = true; + for (Future task : tasks) { + if (!task.isDone()) { + stream.flush(); + shouldStop = false; + } + } + } + Thread.sleep(THREAD_SLEEP_TIME); + tasks.clear(); + stream.close(); + + es.shutdownNow(); + FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH); + assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, fileStatus.getLen()); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java new file mode 100644 index 00000000000..d2ed4008aa2 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java @@ -0,0 +1,50 @@ +/** + * 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.FileNotFoundException; + +import org.junit.Test; + +import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; + +/** + * Test filesystem initialization and creation. + */ +public class ITestAzureBlobFileSystemInitAndCreate extends DependencyInjectedTest { + public ITestAzureBlobFileSystemInitAndCreate() { + super(); + + this.getConfiguration().unset(ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION); + } + + @Override + public void initialize() { + } + + @Override + public void testCleanup() { + } + + @Test (expected = FileNotFoundException.class) + public void ensureFilesystemWillNotBeCreatedIfCreationConfigIsNotSet() throws Exception { + super.initialize(); + this.getFileSystem(); + } +} \ No newline at end of file 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 new file mode 100644 index 00000000000..6059766c2ad --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemListStatus.java @@ -0,0 +1,132 @@ +/** + * 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.FileNotFoundException; +import java.util.ArrayList; +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 org.junit.Assert; +import org.junit.Test; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; + +/** + * Test listStatus operation. + */ +public class ITestAzureBlobFileSystemListStatus extends DependencyInjectedTest { + private static final int TEST_FILES_NUMBER = 6000; + public ITestAzureBlobFileSystemListStatus() { + super(); + } + + @Test + public void testListPath() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final List tasks = new ArrayList<>(); + + ExecutorService es = Executors.newFixedThreadPool(10); + for (int i = 0; i < TEST_FILES_NUMBER; i++) { + final Path fileName = new Path("/test" + i); + Callable callable = new Callable() { + @Override + public Void call() throws Exception { + fs.create(fileName); + return null; + } + }; + + tasks.add(es.submit(callable)); + } + + for (Future task : tasks) { + task.get(); + } + + es.shutdownNow(); + FileStatus[] files = fs.listStatus(new Path("/")); + Assert.assertEquals(files.length, TEST_FILES_NUMBER + 1 /* user directory */); + } + + @Test + public void testListFileVsListDir() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.create(new Path("/testFile")); + + FileStatus[] testFiles = fs.listStatus(new Path("/testFile")); + Assert.assertEquals(testFiles.length, 1); + Assert.assertFalse(testFiles[0].isDirectory()); + } + + @Test + public void testListFileVsListDir2() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.mkdirs(new Path("/testFolder")); + fs.mkdirs(new Path("/testFolder/testFolder2")); + fs.mkdirs(new Path("/testFolder/testFolder2/testFolder3")); + fs.create(new Path("/testFolder/testFolder2/testFolder3/testFile")); + + FileStatus[] testFiles = fs.listStatus(new Path("/testFolder/testFolder2/testFolder3/testFile")); + Assert.assertEquals(testFiles.length, 1); + Assert.assertEquals(testFiles[0].getPath(), new Path(this.getTestUrl(), + "/testFolder/testFolder2/testFolder3/testFile")); + Assert.assertFalse(testFiles[0].isDirectory()); + } + + @Test(expected = FileNotFoundException.class) + public void testListNonExistentDir() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.listStatus(new Path("/testFile/")); + } + + @Test + public void testListFiles() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.mkdirs(new Path("/test")); + + FileStatus[] fileStatuses = fs.listStatus(new Path("/")); + assertEquals(fileStatuses.length, 2); + + fs.mkdirs(new Path("/test/sub")); + fileStatuses = fs.listStatus(new Path("/test")); + assertEquals(fileStatuses.length, 1); + assertEquals(fileStatuses[0].getPath().getName(), "sub"); + assertTrue(fileStatuses[0].isDirectory()); + assertEquals(fileStatuses[0].getLen(), 0); + + fs.create(new Path("/test/f")); + fileStatuses = fs.listStatus(new Path("/test")); + assertEquals(fileStatuses.length, 2); + assertEquals(fileStatuses[0].getPath().getName(), "f"); + assertFalse(fileStatuses[0].isDirectory()); + assertEquals(fileStatuses[0].getLen(), 0); + assertEquals(fileStatuses[1].getPath().getName(), "sub"); + assertTrue(fileStatuses[1].isDirectory()); + assertEquals(fileStatuses[1].getLen(), 0); + } +} 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 new file mode 100644 index 00000000000..b61908c13ea --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemMkDir.java @@ -0,0 +1,88 @@ +/** + * 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.concurrent.Callable; + +import org.junit.Test; + +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.Path; + +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.junit.Assert.assertTrue; + +/** + * Test mkdir operation. + */ +public class ITestAzureBlobFileSystemMkDir extends DependencyInjectedTest { + public ITestAzureBlobFileSystemMkDir() { + super(); + } + + @Test + public void testCreateDirWithExistingDir() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + assertTrue(fs.mkdirs(new Path("testFolder"))); + assertTrue(fs.mkdirs(new Path("testFolder"))); + } + + @Test(expected = FileAlreadyExistsException.class) + public void createDirectoryUnderFile() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.create(new Path("testFile")); + fs.mkdirs(new Path("testFile/TestDirectory")); + } + + @Test + public void testCreateDirectoryOverExistingFiles() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.create(new Path("/testPath")); + FileAlreadyExistsException ex = intercept( + FileAlreadyExistsException.class, + new Callable() { + @Override + public Void call() throws Exception { + fs.mkdirs(new Path("/testPath")); + return null; + } + }); + + assertTrue(ex instanceof FileAlreadyExistsException); + + fs.create(new Path("/testPath1/file1")); + ex = intercept( + FileAlreadyExistsException.class, + new Callable() { + @Override + public Void call() throws Exception { + fs.mkdirs(new Path("/testPath1/file1")); + return null; + } + }); + + assertTrue(ex instanceof FileAlreadyExistsException); + } + + @Test + public void testCreateRoot() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + assertTrue(fs.mkdirs(new Path("/"))); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOpen.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOpen.java new file mode 100644 index 00000000000..fef7f47f720 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOpen.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; + +import java.io.FileNotFoundException; + +import org.junit.Test; + +import org.apache.hadoop.fs.Path; + +/** + * Test open operation. + */ +public class ITestAzureBlobFileSystemOpen extends DependencyInjectedTest { + public ITestAzureBlobFileSystemOpen() throws Exception { + super(); + } + + @Test(expected = FileNotFoundException.class) + public void testOpenDirectory() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.mkdirs(new Path("testFolder")); + fs.open(new Path("testFolder")); + } +} 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 new file mode 100644 index 00000000000..94775875684 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java @@ -0,0 +1,582 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.fs.azurebfs; + + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.azure.NativeAzureFileSystem; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.junit.Test; + +import java.io.EOFException; +import java.io.IOException; +import java.util.Random; +import java.util.concurrent.Callable; + +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertArrayEquals; + + +/** + * Test random read operation. + */ +public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest { + private static final int KILOBYTE = 1024; + private static final int MEGABYTE = KILOBYTE * KILOBYTE; + private static final long TEST_FILE_SIZE = 8 * MEGABYTE; + private static final int MAX_ELAPSEDTIMEMS = 20; + private static final int SEQUENTIAL_READ_BUFFER_SIZE = 16 * KILOBYTE; + private static final int CREATE_BUFFER_SIZE = 26 * KILOBYTE; + + private static final int SEEK_POSITION_ONE = 2* KILOBYTE; + private static final int SEEK_POSITION_TWO = 5 * KILOBYTE; + private static final int SEEK_POSITION_THREE = 10 * KILOBYTE; + private static final int SEEK_POSITION_FOUR = 4100 * KILOBYTE; + + private static final Path TEST_FILE_PATH = new Path( + "/TestRandomRead.txt"); + private static final String WASB = "WASB"; + private static final String ABFS = "ABFS"; + private static long testFileLength = 0; + + public ITestAzureBlobFileSystemRandomRead() throws Exception { + super(); + } + + @Test + public void testBasicRead() throws Exception { + assumeHugeFileExists(); + + try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) { + byte[] buffer = new byte[3 * MEGABYTE]; + + // forward seek and read a kilobyte into first kilobyte of bufferV2 + inputStream.seek(5 * MEGABYTE); + int numBytesRead = inputStream.read(buffer, 0, KILOBYTE); + assertEquals(KILOBYTE, numBytesRead); + + int len = MEGABYTE; + int offset = buffer.length - len; + + // reverse seek and read a megabyte into last megabyte of bufferV1 + inputStream.seek(3 * MEGABYTE); + numBytesRead = inputStream.read(buffer, offset, len); + assertEquals(len, numBytesRead); + } + } + + /** + * Validates the implementation of random read in ABFS + * @throws IOException + */ + @Test + public void testRandomRead() throws Exception { + assumeHugeFileExists(); + try ( + FSDataInputStream inputStreamV1 + = this.getFileSystem().open(TEST_FILE_PATH); + FSDataInputStream inputStreamV2 + = this.getWasbFileSystem().open(TEST_FILE_PATH); + ) { + final int bufferSize = 4 * KILOBYTE; + byte[] bufferV1 = new byte[bufferSize]; + byte[] bufferV2 = new byte[bufferV1.length]; + + verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2); + + inputStreamV1.seek(0); + inputStreamV2.seek(0); + + verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2); + + verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2); + + inputStreamV1.seek(SEEK_POSITION_ONE); + inputStreamV2.seek(SEEK_POSITION_ONE); + + inputStreamV1.seek(0); + inputStreamV2.seek(0); + + verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2); + + verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2); + + verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2); + + inputStreamV1.seek(SEEK_POSITION_TWO); + inputStreamV2.seek(SEEK_POSITION_TWO); + + verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2); + + inputStreamV1.seek(SEEK_POSITION_THREE); + inputStreamV2.seek(SEEK_POSITION_THREE); + + verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2); + + verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2); + + inputStreamV1.seek(SEEK_POSITION_FOUR); + inputStreamV2.seek(SEEK_POSITION_FOUR); + + verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2); + } + } + + /** + * Validates the implementation of Seekable.seekToNewSource + * @throws IOException + */ + @Test + public void testSeekToNewSource() throws Exception { + assumeHugeFileExists(); + try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) { + assertFalse(inputStream.seekToNewSource(0)); + } + } + + /** + * Validates the implementation of InputStream.skip and ensures there is no + * network I/O for AbfsInputStream + * @throws Exception + */ + @Test + public void testSkipBounds() throws Exception { + assumeHugeFileExists(); + try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) { + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); + + long skipped = inputStream.skip(-1); + assertEquals(0, skipped); + + skipped = inputStream.skip(0); + assertEquals(0, skipped); + + assertTrue(testFileLength > 0); + + skipped = inputStream.skip(testFileLength); + assertEquals(testFileLength, skipped); + + intercept(EOFException.class, + new Callable() { + @Override + public Long call() throws Exception { + return inputStream.skip(1); + } + } + ); + long elapsedTimeMs = timer.elapsedTimeMs(); + assertTrue( + String.format( + "There should not be any network I/O (elapsedTimeMs=%1$d).", + elapsedTimeMs), + elapsedTimeMs < MAX_ELAPSEDTIMEMS); + } + } + + /** + * Validates the implementation of Seekable.seek and ensures there is no + * network I/O for forward seek. + * @throws Exception + */ + @Test + public void testValidateSeekBounds() throws Exception { + assumeHugeFileExists(); + try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) { + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); + + inputStream.seek(0); + assertEquals(0, inputStream.getPos()); + + intercept(EOFException.class, + FSExceptionMessages.NEGATIVE_SEEK, + new Callable() { + @Override + public FSDataInputStream call() throws Exception { + inputStream.seek(-1); + return inputStream; + } + } + ); + + assertTrue("Test file length only " + testFileLength, testFileLength > 0); + inputStream.seek(testFileLength); + assertEquals(testFileLength, inputStream.getPos()); + + intercept(EOFException.class, + FSExceptionMessages.CANNOT_SEEK_PAST_EOF, + new Callable() { + @Override + public FSDataInputStream call() throws Exception { + inputStream.seek(testFileLength + 1); + return inputStream; + } + } + ); + + long elapsedTimeMs = timer.elapsedTimeMs(); + assertTrue( + String.format( + "There should not be any network I/O (elapsedTimeMs=%1$d).", + elapsedTimeMs), + elapsedTimeMs < MAX_ELAPSEDTIMEMS); + } + } + + /** + * Validates the implementation of Seekable.seek, Seekable.getPos, + * and InputStream.available. + * @throws Exception + */ + @Test + public void testSeekAndAvailableAndPosition() throws Exception { + assumeHugeFileExists(); + try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) { + byte[] expected1 = {(byte) 'a', (byte) 'b', (byte) 'c'}; + byte[] expected2 = {(byte) 'd', (byte) 'e', (byte) 'f'}; + byte[] expected3 = {(byte) 'b', (byte) 'c', (byte) 'd'}; + byte[] expected4 = {(byte) 'g', (byte) 'h', (byte) 'i'}; + byte[] buffer = new byte[3]; + + int bytesRead = inputStream.read(buffer); + assertEquals(buffer.length, bytesRead); + assertArrayEquals(expected1, buffer); + assertEquals(buffer.length, inputStream.getPos()); + assertEquals(testFileLength - inputStream.getPos(), + inputStream.available()); + + bytesRead = inputStream.read(buffer); + assertEquals(buffer.length, bytesRead); + assertArrayEquals(expected2, buffer); + assertEquals(2 * buffer.length, inputStream.getPos()); + assertEquals(testFileLength - inputStream.getPos(), + inputStream.available()); + + // reverse seek + int seekPos = 0; + inputStream.seek(seekPos); + + bytesRead = inputStream.read(buffer); + assertEquals(buffer.length, bytesRead); + assertArrayEquals(expected1, buffer); + assertEquals(buffer.length + seekPos, inputStream.getPos()); + assertEquals(testFileLength - inputStream.getPos(), + inputStream.available()); + + // reverse seek + seekPos = 1; + inputStream.seek(seekPos); + + bytesRead = inputStream.read(buffer); + assertEquals(buffer.length, bytesRead); + assertArrayEquals(expected3, buffer); + assertEquals(buffer.length + seekPos, inputStream.getPos()); + assertEquals(testFileLength - inputStream.getPos(), + inputStream.available()); + + // forward seek + seekPos = 6; + inputStream.seek(seekPos); + + bytesRead = inputStream.read(buffer); + assertEquals(buffer.length, bytesRead); + assertArrayEquals(expected4, buffer); + assertEquals(buffer.length + seekPos, inputStream.getPos()); + assertEquals(testFileLength - inputStream.getPos(), + inputStream.available()); + } + } + + /** + * Validates the implementation of InputStream.skip, Seekable.getPos, + * and InputStream.available. + * @throws IOException + */ + @Test + public void testSkipAndAvailableAndPosition() throws Exception { + assumeHugeFileExists(); + try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) { + byte[] expected1 = {(byte) 'a', (byte) 'b', (byte) 'c'}; + byte[] expected2 = {(byte) 'd', (byte) 'e', (byte) 'f'}; + byte[] expected3 = {(byte) 'b', (byte) 'c', (byte) 'd'}; + byte[] expected4 = {(byte) 'g', (byte) 'h', (byte) 'i'}; + + assertEquals(testFileLength, inputStream.available()); + assertEquals(0, inputStream.getPos()); + + int n = 3; + long skipped = inputStream.skip(n); + + assertEquals(skipped, inputStream.getPos()); + assertEquals(testFileLength - inputStream.getPos(), + inputStream.available()); + assertEquals(skipped, n); + + byte[] buffer = new byte[3]; + int bytesRead = inputStream.read(buffer); + assertEquals(buffer.length, bytesRead); + assertArrayEquals(expected2, buffer); + assertEquals(buffer.length + skipped, inputStream.getPos()); + assertEquals(testFileLength - inputStream.getPos(), + inputStream.available()); + + // does skip still work after seek? + int seekPos = 1; + inputStream.seek(seekPos); + + bytesRead = inputStream.read(buffer); + assertEquals(buffer.length, bytesRead); + assertArrayEquals(expected3, buffer); + assertEquals(buffer.length + seekPos, inputStream.getPos()); + assertEquals(testFileLength - inputStream.getPos(), + inputStream.available()); + + long currentPosition = inputStream.getPos(); + n = 2; + skipped = inputStream.skip(n); + + assertEquals(currentPosition + skipped, inputStream.getPos()); + assertEquals(testFileLength - inputStream.getPos(), + inputStream.available()); + assertEquals(skipped, n); + + bytesRead = inputStream.read(buffer); + assertEquals(buffer.length, bytesRead); + assertArrayEquals(expected4, buffer); + assertEquals(buffer.length + skipped + currentPosition, + inputStream.getPos()); + assertEquals(testFileLength - inputStream.getPos(), + inputStream.available()); + } + } + + /** + * Ensures parity in the performance of sequential read after reverse seek for + * abfs of the AbfsInputStream. + * @throws IOException + */ + @Test + public void testSequentialReadAfterReverseSeekPerformance() + throws Exception { + assumeHugeFileExists(); + final int maxAttempts = 10; + final double maxAcceptableRatio = 1.01; + double beforeSeekElapsedMs = 0, afterSeekElapsedMs = 0; + double ratio = Double.MAX_VALUE; + for (int i = 0; i < maxAttempts && ratio >= maxAcceptableRatio; i++) { + beforeSeekElapsedMs = sequentialRead(ABFS, + this.getFileSystem(), false); + afterSeekElapsedMs = sequentialRead(ABFS, + this.getFileSystem(), true); + ratio = afterSeekElapsedMs / beforeSeekElapsedMs; + System.out.println((String.format( + "beforeSeekElapsedMs=%1$d, afterSeekElapsedMs=%2$d, ratio=%3$.2f", + (long) beforeSeekElapsedMs, + (long) afterSeekElapsedMs, + ratio))); + } + assertTrue(String.format( + "Performance of ABFS stream after reverse seek is not acceptable:" + + " beforeSeekElapsedMs=%1$d, afterSeekElapsedMs=%2$d," + + " ratio=%3$.2f", + (long) beforeSeekElapsedMs, + (long) afterSeekElapsedMs, + ratio), + ratio < maxAcceptableRatio); + } + + @Test + public void testRandomReadPerformance() throws Exception { + createTestFile(); + assumeHugeFileExists(); + + final AzureBlobFileSystem abFs = this.getFileSystem(); + final NativeAzureFileSystem wasbFs = this.getWasbFileSystem(); + + final int maxAttempts = 10; + final double maxAcceptableRatio = 1.025; + double v1ElapsedMs = 0, v2ElapsedMs = 0; + double ratio = Double.MAX_VALUE; + for (int i = 0; i < maxAttempts && ratio >= maxAcceptableRatio; i++) { + v1ElapsedMs = randomRead(1, wasbFs); + v2ElapsedMs = randomRead(2, abFs); + + ratio = v2ElapsedMs / v1ElapsedMs; + + System.out.println(String.format( + "v1ElapsedMs=%1$d, v2ElapsedMs=%2$d, ratio=%3$.2f", + (long) v1ElapsedMs, + (long) v2ElapsedMs, + ratio)); + } + assertTrue(String.format( + "Performance of version 2 is not acceptable: v1ElapsedMs=%1$d," + + " v2ElapsedMs=%2$d, ratio=%3$.2f", + (long) v1ElapsedMs, + (long) v2ElapsedMs, + ratio), + ratio < maxAcceptableRatio); + } + + + private long sequentialRead(String version, + FileSystem fs, + boolean afterReverseSeek) throws IOException { + byte[] buffer = new byte[SEQUENTIAL_READ_BUFFER_SIZE]; + long totalBytesRead = 0; + long bytesRead = 0; + + try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) { + if (afterReverseSeek) { + while (bytesRead > 0 && totalBytesRead < 4 * MEGABYTE) { + bytesRead = inputStream.read(buffer); + totalBytesRead += bytesRead; + } + totalBytesRead = 0; + inputStream.seek(0); + } + + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); + while ((bytesRead = inputStream.read(buffer)) > 0) { + totalBytesRead += bytesRead; + } + long elapsedTimeMs = timer.elapsedTimeMs(); + + System.out.println(String.format( + "v%1$s: bytesRead=%2$d, elapsedMs=%3$d, Mbps=%4$.2f," + + " afterReverseSeek=%5$s", + version, + totalBytesRead, + elapsedTimeMs, + toMbps(totalBytesRead, elapsedTimeMs), + afterReverseSeek)); + + assertEquals(testFileLength, totalBytesRead); + inputStream.close(); + return elapsedTimeMs; + } + } + + private long randomRead(int version, FileSystem fs) throws Exception { + assumeHugeFileExists(); + final long minBytesToRead = 2 * MEGABYTE; + Random random = new Random(); + byte[] buffer = new byte[8 * KILOBYTE]; + long totalBytesRead = 0; + long bytesRead = 0; + try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) { + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); + do { + bytesRead = inputStream.read(buffer); + totalBytesRead += bytesRead; + inputStream.seek(random.nextInt( + (int) (TEST_FILE_SIZE - buffer.length))); + } while (bytesRead > 0 && totalBytesRead < minBytesToRead); + long elapsedTimeMs = timer.elapsedTimeMs(); + inputStream.close(); + System.out.println(String.format( + "v%1$d: totalBytesRead=%2$d, elapsedTimeMs=%3$d, Mbps=%4$.2f", + version, + totalBytesRead, + elapsedTimeMs, + toMbps(totalBytesRead, elapsedTimeMs))); + assertTrue(minBytesToRead <= totalBytesRead); + return elapsedTimeMs; + } + } + + /** + * Calculate megabits per second from the specified values for bytes and + * milliseconds. + * @param bytes The number of bytes. + * @param milliseconds The number of milliseconds. + * @return The number of megabits per second. + */ + private static double toMbps(long bytes, long milliseconds) { + return bytes / 1000.0 * 8 / milliseconds; + } + + private void createTestFile() throws Exception { + FileSystem fs = this.getWasbFileSystem(); + + if (fs.exists(TEST_FILE_PATH)) { + FileStatus status = fs.getFileStatus(TEST_FILE_PATH); + if (status.getLen() >= TEST_FILE_SIZE) { + return; + } + } + + byte[] buffer = new byte[CREATE_BUFFER_SIZE]; + char character = 'a'; + for (int i = 0; i < buffer.length; i++) { + buffer[i] = (byte) character; + character = (character == 'z') ? 'a' : (char) ((int) character + 1); + } + + System.out.println(("Creating test file {} of size: {} " + TEST_FILE_PATH + + TEST_FILE_SIZE)); + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); + + try(FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) { + int bytesWritten = 0; + while (bytesWritten < TEST_FILE_SIZE) { + outputStream.write(buffer); + bytesWritten += buffer.length; + } + System.out.println("Closing stream {}" + outputStream); + ContractTestUtils.NanoTimer closeTimer + = new ContractTestUtils.NanoTimer(); + outputStream.close(); + closeTimer.end("time to close() output stream"); + } + timer.end("time to write %d KB", TEST_FILE_SIZE / 1024); + testFileLength = fs.getFileStatus(TEST_FILE_PATH).getLen(); + + } + + private void assumeHugeFileExists() throws Exception{ + createTestFile(); + FileSystem fs = this.getFileSystem(); + ContractTestUtils.assertPathExists(this.getFileSystem(), "huge file not created", TEST_FILE_PATH); + FileStatus status = fs.getFileStatus(TEST_FILE_PATH); + ContractTestUtils.assertIsFile(TEST_FILE_PATH, status); + assertTrue("File " + TEST_FILE_PATH + " is empty", status.getLen() > 0); + } + + private void verifyConsistentReads(FSDataInputStream inputStreamV1, + FSDataInputStream inputStreamV2, + byte[] bufferV1, + byte[] bufferV2) throws IOException { + int size = bufferV1.length; + final int numBytesReadV1 = inputStreamV1.read(bufferV1, 0, size); + assertEquals("Bytes read from wasb stream", size, numBytesReadV1); + + final int numBytesReadV2 = inputStreamV2.read(bufferV2, 0, size); + assertEquals("Bytes read from abfs stream", size, numBytesReadV2); + + assertArrayEquals("Mismatch in read data", bufferV1, bufferV2); + } + +} \ No newline at end of file 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 new file mode 100644 index 00000000000..a0e648ca191 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java @@ -0,0 +1,152 @@ +/** + * 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.FileNotFoundException; +import java.util.ArrayList; +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 org.junit.Assert; +import org.junit.Test; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +/** + * Test rename operation. + */ +public class ITestAzureBlobFileSystemRename extends DependencyInjectedTest { + public ITestAzureBlobFileSystemRename() { + super(); + } + + @Test(expected = FileNotFoundException.class) + public void testEnsureFileIsRenamed() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.create(new Path("testfile")); + fs.rename(new Path("testfile"), new Path("testfile2")); + + FileStatus fileStatus = fs.getFileStatus(new Path("testfile2")); + assertNotNull(fileStatus); + + fs.getFileStatus(new Path("testfile")); + } + + @Test + public void testRenameFile() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.mkdirs(new Path("/testSrc")); + fs.create(new Path("/testSrc/file1")); + + fs.rename(new Path("/testSrc"), new Path("/testDst")); + FileStatus[] fileStatus = fs.listStatus(new Path("/testDst")); + assertNotNull(fileStatus); + } + + @Test + public void testRenameFileUsingUnicode() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + //known issue: ListStatus operation to folders/files whose name contains '?' will fail + //This is because Auto rest client didn't encode '?' in the uri query parameters + String[] folders1 = new String[]{"/%2c%26", "/ÖáΠ⇒", "/A +B", "/A~`!@#$%^&*()-_+={};:'>,,, tasks = new ArrayList<>(); + + ExecutorService es = Executors.newFixedThreadPool(10); + for (int i = 0; i < 1000; i++) { + final Path fileName = new Path("/test/" + i); + Callable callable = new Callable() { + @Override + public Void call() throws Exception { + fs.create(fileName); + return null; + } + }; + + tasks.add(es.submit(callable)); + } + + for (Future task : tasks) { + task.get(); + } + + es.shutdownNow(); + fs.rename(new Path("/test"), new Path("/renamedDir")); + + FileStatus[] files = fs.listStatus(new Path("/renamedDir")); + Assert.assertEquals(files.length, 1000); + fs.getFileStatus(new Path("/test")); + } + + @Test + public void testRenameRoot() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + boolean renamed = fs.rename(new Path("/"), new Path("/ddd")); + assertFalse(renamed); + + renamed = fs.rename(new Path(fs.getUri().toString() + "/"), new Path(fs.getUri().toString() + "/s")); + assertFalse(renamed); + } +} 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 new file mode 100644 index 00000000000..aa30a85918c --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java @@ -0,0 +1,78 @@ +/** + * 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.net.URI; + +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; + +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FileStatus; +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.services.AbfsHttpService; +import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl; + +import static org.mockito.Matchers.anyObject; +import static org.mockito.Mockito.doReturn; + +/** + * Test AzureBlobFileSystem initialization. + */ +public class ITestFileSystemInitialization extends DependencyInjectedTest { + public ITestFileSystemInitialization() { + super(); + + this.getMockServiceInjector().removeProvider(AbfsHttpService.class); + this.getMockServiceInjector().replaceInstance(AbfsHttpService.class, Mockito.mock(AbfsHttpService.class)); + } + + @Test + public void ensureAzureBlobFileSystemIsInitialized() throws Exception { + doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah"))) + .when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class)) + .getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject()); + + final FileSystem fs = FileSystem.get(this.getConfiguration()); + final String accountName = this.getAccountName(); + final String filesystem = this.getFileSystemName(); + + Assert.assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SCHEME, filesystem + "@" + accountName, null, null, null)); + Assert.assertNotNull(fs.getWorkingDirectory()); + } + + @Test + public void ensureSecureAzureBlobFileSystemIsInitialized() throws Exception { + doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah"))) + .when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class)) + .getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject()); + + final String accountName = this.getAccountName(); + final String filesystem = this.getFileSystemName(); + final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null); + this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); + + final FileSystem fs = FileSystem.get(this.getConfiguration()); + Assert.assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null)); + Assert.assertNotNull(fs.getWorkingDirectory()); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java new file mode 100644 index 00000000000..a55599b5006 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java @@ -0,0 +1,82 @@ +/** + * 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.net.URI; + +import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; + +import org.apache.hadoop.fs.AbstractFileSystem; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.FileStatus; +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.services.AbfsHttpService; + +import static org.mockito.Matchers.anyObject; +import static org.mockito.Mockito.doReturn; + +/** + * Test AzureBlobFileSystem registration. + */ +public class ITestFileSystemRegistration extends DependencyInjectedTest { + public ITestFileSystemRegistration() throws Exception { + super(); + + this.getMockServiceInjector().removeProvider(AbfsHttpService.class); + this.getMockServiceInjector().replaceInstance(AbfsHttpService.class, Mockito.mock(AbfsHttpService.class)); + } + + @Test + public void ensureAzureBlobFileSystemIsDefaultFileSystem() throws Exception { + doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah"))) + .when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class)) + .getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject()); + + FileSystem fs = FileSystem.get(this.getConfiguration()); + Assert.assertTrue(fs instanceof AzureBlobFileSystem); + + AbstractFileSystem afs = FileContext.getFileContext(this.getConfiguration()).getDefaultFileSystem(); + Assert.assertTrue(afs instanceof Abfs); + } + + @Test + public void ensureSecureAzureBlobFileSystemIsDefaultFileSystem() throws Exception { + doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah"))) + .when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class)) + .getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject()); + + final String accountName = this.getAccountName(); + final String filesystem = this.getFileSystemName(); + + final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null); + this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); + + FileSystem fs = FileSystem.get(this.getConfiguration()); + Assert.assertTrue(fs instanceof SecureAzureBlobFileSystem); + + AbstractFileSystem afs = FileContext.getFileContext(this.getConfiguration()).getDefaultFileSystem(); + Assert.assertTrue(afs instanceof Abfss); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java new file mode 100644 index 00000000000..7010e74cb74 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java @@ -0,0 +1,202 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.fs.azurebfs; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azure.NativeAzureFileSystem; + +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Test; + +import java.io.BufferedReader; +import java.io.InputStreamReader; + +import static junit.framework.TestCase.assertEquals; +import static junit.framework.TestCase.assertFalse; +import static junit.framework.TestCase.assertTrue; + +/** + * Test compatibility between ABFS client and WASB client. + */ +public class ITestWasbAbfsCompatibility extends DependencyInjectedTest { + private static final String WASB_TEST_CONTEXT = "wasb test file"; + private static final String ABFS_TEST_CONTEXT = "abfs test file"; + private static final String TEST_CONTEXT = "THIS IS FOR TEST"; + + public ITestWasbAbfsCompatibility() throws Exception { + super(); + + Assume.assumeFalse(this.isEmulator()); + } + + @Test + public void testListFileStatus() throws Exception { + // crate file using abfs + AzureBlobFileSystem fs = this.getFileSystem(); + NativeAzureFileSystem wasb = this.getWasbFileSystem(); + + Path path1 = new Path("/testfiles/~12/!008/3/abFsTestfile"); + FSDataOutputStream abfsStream = fs.create(path1, true); + abfsStream.write(ABFS_TEST_CONTEXT.getBytes()); + abfsStream.flush(); + abfsStream.hsync(); + abfsStream.close(); + + // create file using wasb + Path path2 = new Path("/testfiles/~12/!008/3/nativeFsTestfile"); + System.out.println(wasb.getUri()); + FSDataOutputStream nativeFsStream = wasb.create(path2, true); + nativeFsStream.write(WASB_TEST_CONTEXT.getBytes()); + nativeFsStream.flush(); + nativeFsStream.hsync(); + nativeFsStream.close(); + // list file using abfs and wasb + FileStatus[] abfsFileStatus = fs.listStatus(new Path("/testfiles/~12/!008/3/")); + FileStatus[] nativeFsFileStatus = wasb.listStatus(new Path("/testfiles/~12/!008/3/")); + + assertEquals(2, abfsFileStatus.length); + assertEquals(2, nativeFsFileStatus.length); + } + + @Test + public void testReadFile() throws Exception { + boolean[] createFileWithAbfs = new boolean[]{false, true, false, true}; + boolean[] readFileWithAbfs = new boolean[]{false, true, true, false}; + + AzureBlobFileSystem abfs = this.getFileSystem(); + NativeAzureFileSystem wasb = this.getWasbFileSystem(); + + FileSystem fs; + BufferedReader br = null; + for (int i = 0; i< 4; i++) { + try { + Path path = new Path("/testfiles/~12/!008/testfile" + i); + if (createFileWithAbfs[i]) { + fs = abfs; + } else { + fs = wasb; + } + + // Write + FSDataOutputStream nativeFsStream = fs.create(path, true); + nativeFsStream.write(TEST_CONTEXT.getBytes()); + nativeFsStream.flush(); + nativeFsStream.hsync(); + nativeFsStream.close(); + + // Check file status + assertEquals(true, fs.exists(path)); + assertEquals(false, fs.getFileStatus(path).isDirectory()); + + // Read + if (readFileWithAbfs[i]) { + fs = abfs; + } else { + fs = wasb; + } + FSDataInputStream inputStream = fs.open(path); + br = new BufferedReader(new InputStreamReader(fs.open(path))); + String line = br.readLine(); + assertEquals(TEST_CONTEXT, line); + + // Remove file + fs.delete(path, true); + assertFalse(fs.exists(path)); + } catch (Exception e) { + e.printStackTrace(); + } finally { + if (br != null) { + br.close(); + } + } + } + } + + @Test + public void testDir() throws Exception { + boolean[] createDirWithAbfs = new boolean[]{false, true, false, true}; + boolean[] readDirWithAbfs = new boolean[]{false, true, true, false}; + + AzureBlobFileSystem abfs = this.getFileSystem(); + NativeAzureFileSystem wasb = this.getWasbFileSystem(); + + FileSystem fs; + for (int i = 0; i < 4; i++) { + Path path = new Path("/testDir/t" + i); + //create + if (createDirWithAbfs[i]) { + fs = abfs; + } else { + fs = wasb; + } + assertTrue(fs.mkdirs(path)); + //check + assertTrue(fs.exists(path)); + //read + if (readDirWithAbfs[i]) { + fs = abfs; + } else { + fs = wasb; + } + assertTrue(fs.exists(path)); + FileStatus dirStatus = fs.getFileStatus(path); + assertTrue(dirStatus.isDirectory()); + fs.delete(path, true); + assertFalse(fs.exists(path)); + } + } + + + @Test + public void testUrlConversion(){ + String abfsUrl = "abfs://abcde-1111-1111-1111-1111@xxxx.dfs.xxx.xxx.xxxx.xxxx"; + String wabsUrl = "wasb://abcde-1111-1111-1111-1111@xxxx.blob.xxx.xxx.xxxx.xxxx"; + Assert.assertEquals(abfsUrl, wasbUrlToAbfsUrl(wabsUrl)); + Assert.assertEquals(wabsUrl, abfsUrlToWasbUrl(abfsUrl)); + } + + @Test + public void testSetWorkingDirectory() throws Exception { + //create folders + AzureBlobFileSystem abfs = this.getFileSystem(); + NativeAzureFileSystem wasb = this.getWasbFileSystem(); + + assertTrue(abfs.mkdirs(new Path("/d1/d2/d3/d4"))); + + //set working directory to path1 + Path path1 = new Path("/d1/d2"); + wasb.setWorkingDirectory(path1); + abfs.setWorkingDirectory(path1); + assertEquals(path1, wasb.getWorkingDirectory()); + assertEquals(path1, abfs.getWorkingDirectory()); + + //set working directory to path2 + Path path2 = new Path("d3/d4"); + wasb.setWorkingDirectory(path2); + abfs.setWorkingDirectory(path2); + + Path path3 = new Path("/d1/d2/d3/d4"); + assertEquals(path3, wasb.getWorkingDirectory()); + assertEquals(path3, abfs.getWorkingDirectory()); + } +} \ No newline at end of file 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 new file mode 100644 index 00000000000..4b447653ab7 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.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.constants; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Responsible to keep all the Azure Blob File System configurations keys in Hadoop configuration file. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class TestConfigurationKeys { + public static final String FS_AZURE_TEST_ACCOUNT_NAME = "fs.azure.test.account.name"; + public static final String FS_AZURE_TEST_ACCOUNT_KEY_PREFIX = "fs.azure.test.account.key."; + public static final String FS_AZURE_TEST_HOST_NAME = "fs.azure.test.host.name"; + public static final String FS_AZURE_TEST_HOST_PORT = "fs.azure.test.host.port"; + public static final String FS_AZURE_CONTRACT_TEST_URI = "fs.contract.test.fs.abfs"; + + private TestConfigurationKeys() {} +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java new file mode 100644 index 00000000000..109f887e29a --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +package org.apache.hadoop.fs.azurebfs.constants; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java new file mode 100644 index 00000000000..5fc81cec2f8 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java @@ -0,0 +1,63 @@ +/** + * 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.contract; + +import java.net.URI; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest; +import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; +import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; + +/** + * Dependency inject for ABFS contract tests. + */ +public class DependencyInjectedContractTest extends DependencyInjectedTest { + private final URI testUri; + + public DependencyInjectedContractTest(final boolean secure) throws Exception { + this(secure, true); + } + + public DependencyInjectedContractTest(final boolean secure, final boolean useExistedFileSystem) throws Exception{ + super(secure); + if (useExistedFileSystem) { + Configuration configuration = getConfiguration(); + String testUrl = configuration.get(TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI); + + if (secure) { + testUrl = testUrl.replaceFirst(FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME); + } + updateTestUrl(testUrl); + + this.testUri = new URI(testUrl); + //Get container for contract tests + configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, this.testUri.toString()); + String[] splitAuthority = this.testUri.getAuthority().split("\\@"); + updateFileSystemName(splitAuthority[0]); + } else { + this.testUri = new URI(super.getTestUrl()); + } + } + + public Configuration getConfiguration() { + return super.getConfiguration(); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java new file mode 100644 index 00000000000..7f7a09ab821 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java @@ -0,0 +1,54 @@ +/** + * 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.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; +import org.apache.hadoop.fs.azurebfs.utils.UriUtils; +import org.apache.hadoop.fs.contract.AbstractBondedFSContract; + +/** + * Azure BlobFileSystem Contract. Test paths are created using any maven fork + * identifier, if defined. This guarantees paths unique to tests + * running in parallel. + */ +public class ITestAbfsFileSystemContract extends AbstractBondedFSContract { + + public static final String CONTRACT_XML = "abfs.xml"; + private final boolean isSecure; + + protected ITestAbfsFileSystemContract(final Configuration conf, boolean secure) { + super(conf); + //insert the base features + addConfResource(CONTRACT_XML); + this.isSecure = secure; + } + + @Override + public String getScheme() { + return isSecure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME; + } + + @Override + public Path getTestPath() { + Path path = new Path(UriUtils.generateUniqueTestPath()); + return path; + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java new file mode 100644 index 00000000000..d4cca146a6a --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java @@ -0,0 +1,70 @@ +/** + * 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.contract; + +import java.util.Arrays; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractAppendTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; + +/** + * Contract test for open operation. + */ +@RunWith(Parameterized.class) +public class ITestAbfsFileSystemContractAppend extends AbstractContractAppendTest { + @Parameterized.Parameters(name = "SecureMode={0}") + public static Iterable secure() { + return Arrays.asList(new Object[][] { {true}, {false} }); + } + + private final boolean isSecure; + private final DependencyInjectedContractTest dependencyInjectedContractTest; + + public ITestAbfsFileSystemContractAppend(final boolean secure) throws Exception { + this.isSecure = secure; + dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); + } + + @Override + public void setup() throws Exception { + dependencyInjectedContractTest.initialize(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return this.dependencyInjectedContractTest.getConfiguration(); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new ITestAbfsFileSystemContract(conf, this.isSecure); + } + + @Override + public void testRenameFileBeingAppended() throws Throwable { + skip("Skipping as renaming an opened file is not supported"); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java new file mode 100644 index 00000000000..4f724e2faf1 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.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.contract; + +import java.util.Arrays; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractConcatTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +/** + * Contract test for concat operation. + */ +@RunWith(Parameterized.class) +public class ITestAbfsFileSystemContractConcat extends AbstractContractConcatTest{ + @Parameterized.Parameters(name = "SecureMode={0}") + public static Iterable secure() { + return Arrays.asList(new Object[][] { {true}, {false} }); + } + + private final boolean isSecure; + private final DependencyInjectedContractTest dependencyInjectedContractTest; + + public ITestAbfsFileSystemContractConcat(final boolean secure) throws Exception { + this.isSecure = secure; + dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure); + } + + @Override + public void setup() throws Exception { + dependencyInjectedContractTest.initialize(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return this.dependencyInjectedContractTest.getConfiguration(); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new ITestAbfsFileSystemContract(conf, this.isSecure); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java new file mode 100644 index 00000000000..16b959f00f7 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java @@ -0,0 +1,63 @@ +/** + * 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.contract; + +import java.util.Arrays; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractCreateTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +/** + * Contract test for create operation. + */ +@RunWith(Parameterized.class) +public class ITestAbfsFileSystemContractCreate extends AbstractContractCreateTest{ + @Parameterized.Parameters(name = "SecureMode={0}") + public static Iterable secure() { + return Arrays.asList(new Object[][] { {true}, {false} }); + } + + private final boolean isSecure; + private final DependencyInjectedContractTest dependencyInjectedContractTest; + + public ITestAbfsFileSystemContractCreate(final boolean secure) throws Exception { + this.isSecure = secure; + dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); + } + + @Override + public void setup() throws Exception { + dependencyInjectedContractTest.initialize(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return this.dependencyInjectedContractTest.getConfiguration(); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new ITestAbfsFileSystemContract(conf, this.isSecure); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java new file mode 100644 index 00000000000..fabd3273eac --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java @@ -0,0 +1,63 @@ +/** + * 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.contract; + +import java.util.Arrays; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractDeleteTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +/** + * Contract test for delete operation. + */ +@RunWith(Parameterized.class) +public class ITestAbfsFileSystemContractDelete extends AbstractContractDeleteTest { + @Parameterized.Parameters(name = "SecureMode={0}") + public static Iterable secure() { + return Arrays.asList(new Object[][] { {true}, {false} }); + } + + private final boolean isSecure; + private final DependencyInjectedContractTest dependencyInjectedContractTest; + + public ITestAbfsFileSystemContractDelete(final boolean secure) throws Exception { + this.isSecure = secure; + dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure); + } + + @Override + public void setup() throws Exception { + dependencyInjectedContractTest.initialize(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return this.dependencyInjectedContractTest.getConfiguration(); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new ITestAbfsFileSystemContract(conf, this.isSecure); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java new file mode 100644 index 00000000000..a1360e49dbf --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java @@ -0,0 +1,44 @@ +/** + * 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.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.tools.contract.AbstractContractDistCpTest; + +/** + * Contract test for distCp operation. + */ +public class ITestAbfsFileSystemContractDistCp extends AbstractContractDistCpTest { + private final DependencyInjectedContractTest dependencyInjectedContractTest; + + public ITestAbfsFileSystemContractDistCp() throws Exception { + dependencyInjectedContractTest = new DependencyInjectedContractTest(false); + } + + @Override + public void setup() throws Exception { + dependencyInjectedContractTest.initialize(); + super.setup(); + } + + @Override + protected ITestAbfsFileSystemContract createContract(Configuration conf) { + return new ITestAbfsFileSystemContract(conf, false); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java new file mode 100644 index 00000000000..5bb41ad9f3a --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.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.contract; + +import java.util.Arrays; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +/** + * Contract test for getFileStatus operation. + */ +@RunWith(Parameterized.class) +public class ITestAbfsFileSystemContractGetFileStatus extends AbstractContractGetFileStatusTest { + @Parameterized.Parameters(name = "SecureMode={0}") + public static Iterable secure() { + return Arrays.asList(new Object[][] { {true}, {false} }); + } + + private final boolean isSecure; + private final DependencyInjectedContractTest dependencyInjectedContractTest; + + public ITestAbfsFileSystemContractGetFileStatus(final boolean secure) throws Exception { + this.isSecure = secure; + dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure); + } + + @Override + public void setup() throws Exception { + dependencyInjectedContractTest.initialize(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return this.dependencyInjectedContractTest.getConfiguration(); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new ITestAbfsFileSystemContract(conf, this.isSecure); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java new file mode 100644 index 00000000000..9d732d5f2f8 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java @@ -0,0 +1,63 @@ +/** + * 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.contract; + +import java.util.Arrays; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractMkdirTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +/** + * Contract test for mkdir operation. + */ +@RunWith(Parameterized.class) +public class ITestAbfsFileSystemContractMkdir extends AbstractContractMkdirTest { + @Parameterized.Parameters(name = "SecureMode={0}") + public static Iterable secure() { + return Arrays.asList(new Object[][] { {true}, {false} }); + } + + private final boolean isSecure; + private final DependencyInjectedContractTest dependencyInjectedContractTest; + + public ITestAbfsFileSystemContractMkdir(final boolean secure) throws Exception { + this.isSecure = secure; + dependencyInjectedContractTest = new DependencyInjectedContractTest(secure); + } + + @Override + public void setup() throws Exception { + dependencyInjectedContractTest.initialize(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return this.dependencyInjectedContractTest.getConfiguration(); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new ITestAbfsFileSystemContract(conf, this.isSecure); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java new file mode 100644 index 00000000000..a71149b8958 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java @@ -0,0 +1,63 @@ +/** + * 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.contract; + +import java.util.Arrays; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractOpenTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +/** + * Contract test for open operation. + */ +@RunWith(Parameterized.class) +public class ITestAbfsFileSystemContractOpen extends AbstractContractOpenTest { + @Parameterized.Parameters(name = "SecureMode={0}") + public static Iterable secure() { + return Arrays.asList(new Object[][] { {true}, {false} }); + } + + private final boolean isSecure; + private final DependencyInjectedContractTest dependencyInjectedContractTest; + + public ITestAbfsFileSystemContractOpen(final boolean secure) throws Exception { + this.isSecure = secure; + dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); + } + + @Override + public void setup() throws Exception { + dependencyInjectedContractTest.initialize(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return this.dependencyInjectedContractTest.getConfiguration(); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new ITestAbfsFileSystemContract(conf, this.isSecure); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java new file mode 100644 index 00000000000..6d1c4ae8a79 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java @@ -0,0 +1,63 @@ +/** + * 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.contract; + +import java.util.Arrays; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractRenameTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +/** + * Contract test for rename operation. + */ +@RunWith(Parameterized.class) +public class ITestAbfsFileSystemContractRename extends AbstractContractRenameTest { + @Parameterized.Parameters(name = "SecureMode={0}") + public static Iterable secure() { + return Arrays.asList(new Object[][] { {true}, {false} }); + } + + private final boolean isSecure; + private final DependencyInjectedContractTest dependencyInjectedContractTest; + + public ITestAbfsFileSystemContractRename(final boolean secure) throws Exception { + this.isSecure = secure; + dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); + } + + @Override + public void setup() throws Exception { + dependencyInjectedContractTest.initialize(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return this.dependencyInjectedContractTest.getConfiguration(); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new ITestAbfsFileSystemContract(conf, this.isSecure); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java new file mode 100644 index 00000000000..46072adac3e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.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.contract; + +import java.util.Arrays; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.junit.Ignore; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** + * Contract test for root directory operation. + */ +@RunWith(Parameterized.class) +public class ITestAbfsFileSystemContractRootDirectory extends AbstractContractRootDirectoryTest { + @Parameterized.Parameters(name = "SecureMode={0}") + public static Iterable secure() { + return Arrays.asList(new Object[][] { {true}, {false} }); + } + + private final boolean isSecure; + private final DependencyInjectedContractTest dependencyInjectedContractTest; + + public ITestAbfsFileSystemContractRootDirectory(final boolean secure) throws Exception { + this.isSecure = secure; + dependencyInjectedContractTest = new DependencyInjectedContractTest(secure); + } + + @Override + public void setup() throws Exception { + dependencyInjectedContractTest.initialize(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return this.dependencyInjectedContractTest.getConfiguration(); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new ITestAbfsFileSystemContract(conf, this.isSecure); + } + + @Override + @Ignore("ABFS always return false when non-recursively remove root dir") + public void testRmNonEmptyRootDirNonRecursive() throws Throwable { + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java new file mode 100644 index 00000000000..1780f6f87b9 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java @@ -0,0 +1,44 @@ +/** + * 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.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.tools.contract.AbstractContractDistCpTest; + +/** + * Contract test for secure distCP operation. + */ +public class ITestAbfsFileSystemContractSecureDistCp extends AbstractContractDistCpTest { + private final DependencyInjectedContractTest dependencyInjectedContractTest; + + public ITestAbfsFileSystemContractSecureDistCp() throws Exception { + dependencyInjectedContractTest = new DependencyInjectedContractTest(true); + } + + @Override + public void setup() throws Exception { + dependencyInjectedContractTest.initialize(); + super.setup(); + } + + @Override + protected ITestAbfsFileSystemContract createContract(Configuration conf) { + return new ITestAbfsFileSystemContract(conf, true); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java new file mode 100644 index 00000000000..aeeb0425189 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java @@ -0,0 +1,63 @@ +/** + * 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.contract; + +import java.util.Arrays; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractSeekTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +/** + * Contract test for seek operation. + */ +@RunWith(Parameterized.class) +public class ITestAbfsFileSystemContractSeek extends AbstractContractSeekTest{ + @Parameterized.Parameters(name = "SecureMode={0}") + public static Iterable secure() { + return Arrays.asList(new Object[][] { {true}, {false} }); + } + + private final boolean isSecure; + private final DependencyInjectedContractTest dependencyInjectedContractTest; + + public ITestAbfsFileSystemContractSeek(final boolean secure) throws Exception { + this.isSecure = secure; + dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); + } + + @Override + public void setup() throws Exception { + dependencyInjectedContractTest.initialize(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return this.dependencyInjectedContractTest.getConfiguration(); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new ITestAbfsFileSystemContract(conf, this.isSecure); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java new file mode 100644 index 00000000000..ea9392d3158 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.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.contract; + +import java.util.Arrays; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractSetTimesTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +/** + * Contract test for setTimes operation. + */ +@RunWith(Parameterized.class) +public class ITestAbfsFileSystemContractSetTimes extends AbstractContractSetTimesTest { + @Parameterized.Parameters(name = "SecureMode={0}") + public static Iterable secure() { + return Arrays.asList(new Object[][] { {true}, {false} }); + } + + private final boolean isSecure; + private final DependencyInjectedContractTest dependencyInjectedContractTest; + + public ITestAbfsFileSystemContractSetTimes(final boolean secure) throws Exception { + this.isSecure = secure; + dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); + } + + @Override + public void setup() throws Exception { + dependencyInjectedContractTest.initialize(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return this.dependencyInjectedContractTest.getConfiguration(); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new ITestAbfsFileSystemContract(conf, this.isSecure); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java new file mode 100644 index 00000000000..e148a055caa --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java @@ -0,0 +1,91 @@ +/** + * 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.contract; + +import java.io.IOException; + +import org.apache.hadoop.fs.FileSystemContractBaseTest; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Basic Contract test for Azure BlobFileSystem. + */ +public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest { + private final DependencyInjectedContractTest dependencyInjectedContractTest; + + public ITestAzureBlobFileSystemBasics() throws Exception { + // If contract tests are running in parallel, some root level tests in this file will fail + // due to the race condition. Hence for this contract test it should be tested in different container + dependencyInjectedContractTest = new DependencyInjectedContractTest(false, false); + } + + @Before + public void setUp() throws Exception { + this.dependencyInjectedContractTest.initialize(); + fs = this.dependencyInjectedContractTest.getFileSystem(); + } + + @Test + public void testListOnFolderWithNoChildren() throws IOException { + assertTrue(fs.mkdirs(path("testListStatus/c/1"))); + + FileStatus[] paths; + paths = fs.listStatus(path("testListStatus")); + assertEquals(1, paths.length); + + // ListStatus on folder with child + paths = fs.listStatus(path("testListStatus/c")); + assertEquals(1, paths.length); + + // Remove the child and listStatus + fs.delete(path("testListStatus/c/1"), true); + paths = fs.listStatus(path("testListStatus/c")); + assertEquals(0, paths.length); + assertTrue(fs.delete(path("testListStatus"), true)); + } + + @Test + public void testListOnfileAndFolder() throws IOException { + Path folderPath = path("testListStatus/folder"); + Path filePath = path("testListStatus/file"); + + assertTrue(fs.mkdirs(folderPath)); + fs.create(filePath); + + FileStatus[] listFolderStatus; + listFolderStatus = fs.listStatus(path("testListStatus")); + assertEquals(filePath, listFolderStatus[0].getPath()); + + //List on file should return absolute path + FileStatus[] listFileStatus = fs.listStatus(filePath); + assertEquals(filePath, listFileStatus[0].getPath()); + } + + @Override + @Ignore("Not implemented in ABFS yet") + public void testMkdirsWithUmask() throws Exception { + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/package-info.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/package-info.java new file mode 100644 index 00000000000..f3ff4834c0f --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +package org.apache.hadoop.fs.azurebfs.contract; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java new file mode 100644 index 00000000000..03320d66414 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java @@ -0,0 +1,124 @@ +/** + * 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.diagnostics; + +import org.apache.commons.codec.Charsets; +import org.junit.Test; + + +import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; +import org.apache.commons.codec.binary.Base64; + +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_BUFFER_SIZE; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE; + +import static org.junit.Assert.assertEquals; + +/** + * Test configuration validators. + */ +public class TestConfigurationValidators extends DependencyInjectedTest { + + private static final String FAKE_KEY = "FakeKey"; + + public TestConfigurationValidators() throws Exception { + super(); + } + + @Test + public void testIntegerConfigValidator() throws Exception { + IntegerConfigurationBasicValidator integerConfigurationValidator = new IntegerConfigurationBasicValidator( + MIN_BUFFER_SIZE, MAX_BUFFER_SIZE, DEFAULT_READ_BUFFER_SIZE, FAKE_KEY, false); + + assertEquals(MIN_BUFFER_SIZE, (int) integerConfigurationValidator.validate("3072")); + assertEquals(DEFAULT_READ_BUFFER_SIZE, (int) integerConfigurationValidator.validate(null)); + assertEquals(MAX_BUFFER_SIZE, (int) integerConfigurationValidator.validate("104857600")); + } + + @Test(expected = InvalidConfigurationValueException.class) + public void testIntegerConfigValidatorThrowsIfMissingValidValue() throws Exception { + IntegerConfigurationBasicValidator integerConfigurationValidator = new IntegerConfigurationBasicValidator( + MIN_BUFFER_SIZE, MAX_BUFFER_SIZE, DEFAULT_READ_BUFFER_SIZE, FAKE_KEY, true); + integerConfigurationValidator.validate("3072"); + } + + @Test + public void testLongConfigValidator() throws Exception { + LongConfigurationBasicValidator longConfigurationValidator = new LongConfigurationBasicValidator( + MIN_BUFFER_SIZE, MAX_BUFFER_SIZE, DEFAULT_WRITE_BUFFER_SIZE, FAKE_KEY, false); + + assertEquals(DEFAULT_WRITE_BUFFER_SIZE, (long) longConfigurationValidator.validate(null)); + assertEquals(MIN_BUFFER_SIZE, (long) longConfigurationValidator.validate("3072")); + assertEquals(MAX_BUFFER_SIZE, (long) longConfigurationValidator.validate("104857600")); + } + + @Test(expected = InvalidConfigurationValueException.class) + public void testLongConfigValidatorThrowsIfMissingValidValue() throws Exception { + LongConfigurationBasicValidator longConfigurationValidator = new LongConfigurationBasicValidator( + MIN_BUFFER_SIZE, MAX_BUFFER_SIZE, DEFAULT_READ_BUFFER_SIZE, FAKE_KEY, true); + longConfigurationValidator.validate(null); + } + + @Test + public void testBooleanConfigValidator() throws Exception { + BooleanConfigurationBasicValidator booleanConfigurationValidator = new BooleanConfigurationBasicValidator(FAKE_KEY, false, false); + + assertEquals(true, booleanConfigurationValidator.validate("true")); + assertEquals(false, booleanConfigurationValidator.validate("False")); + assertEquals(false, booleanConfigurationValidator.validate(null)); + } + + @Test(expected = InvalidConfigurationValueException.class) + public void testBooleanConfigValidatorThrowsIfMissingValidValue() throws Exception { + BooleanConfigurationBasicValidator booleanConfigurationValidator = new BooleanConfigurationBasicValidator(FAKE_KEY, false, true); + booleanConfigurationValidator.validate("almostTrue"); + } + + @Test + public void testStringConfigValidator() throws Exception { + StringConfigurationBasicValidator stringConfigurationValidator = new StringConfigurationBasicValidator(FAKE_KEY, "value", false); + + assertEquals("value", stringConfigurationValidator.validate(null)); + assertEquals("someValue", stringConfigurationValidator.validate("someValue")); + } + + @Test(expected = InvalidConfigurationValueException.class) + public void testStringConfigValidatorThrowsIfMissingValidValue() throws Exception { + StringConfigurationBasicValidator stringConfigurationValidator = new StringConfigurationBasicValidator(FAKE_KEY, "value", true); + stringConfigurationValidator.validate(null); + } + + @Test + public void testBase64StringConfigValidator() throws Exception { + String encodedVal = new String(new Base64().encode("someValue".getBytes()), Charsets.UTF_8); + Base64StringConfigurationBasicValidator base64StringConfigurationValidator = new Base64StringConfigurationBasicValidator(FAKE_KEY, "", false); + + assertEquals("", base64StringConfigurationValidator.validate(null)); + assertEquals(encodedVal, base64StringConfigurationValidator.validate(encodedVal)); + } + + @Test(expected = InvalidConfigurationValueException.class) + public void testBase64StringConfigValidatorThrowsIfMissingValidValue() throws Exception { + Base64StringConfigurationBasicValidator base64StringConfigurationValidator = new Base64StringConfigurationBasicValidator(FAKE_KEY, "value", true); + base64StringConfigurationValidator.validate("some&%Value"); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java new file mode 100644 index 00000000000..c3434acfc2c --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +package org.apache.hadoop.fs.azurebfs.diagnostics; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/package-info.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/package-info.java new file mode 100644 index 00000000000..811fdcb9f37 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +package org.apache.hadoop.fs.azurebfs; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java new file mode 100644 index 00000000000..588df2095e4 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java @@ -0,0 +1,122 @@ +/** + * 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.Hashtable; + +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest; +import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService; + +import static org.junit.Assert.assertEquals; + +/** + * Test AbfsHttpServiceImpl. + */ +public class ITestAbfsHttpServiceImpl extends DependencyInjectedTest { + private static final int TEST_DATA = 100; + private static final Path TEST_PATH = new Path("/testfile"); + public ITestAbfsHttpServiceImpl() { + super(); + } + + @Test + public void testReadWriteBytesToFileAndEnsureThreadPoolCleanup() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + testWriteOneByteToFileAndEnsureThreadPoolCleanup(); + + FSDataInputStream inputStream = fs.open(TEST_PATH, 4 * 1024 * 1024); + int i = inputStream.read(); + + assertEquals(TEST_DATA, i); + } + + @Test + public void testWriteOneByteToFileAndEnsureThreadPoolCleanup() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + FSDataOutputStream stream = fs.create(TEST_PATH); + + stream.write(TEST_DATA); + stream.close(); + + FileStatus fileStatus = fs.getFileStatus(TEST_PATH); + assertEquals(1, fileStatus.getLen()); + } + + @Test + @Ignore("JDK7 doesn't support PATCH, so PUT is used. Fix is applied in latest test tenant") + public void testBase64FileSystemProperties() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final Hashtable properties = new Hashtable<>(); + properties.put("key", "{ value: value }"); + AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setFilesystemProperties( + fs, properties); + Hashtable fetchedProperties = AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getFilesystemProperties(fs); + + Assert.assertEquals(properties, fetchedProperties); + } + + @Test + public void testBase64PathProperties() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final Hashtable properties = new Hashtable<>(); + properties.put("key", "{ value: valueTest }"); + fs.create(TEST_PATH); + AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setPathProperties( + fs, TEST_PATH, properties); + Hashtable fetchedProperties = + AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getPathProperties(fs, TEST_PATH); + + Assert.assertEquals(properties, fetchedProperties); + } + + @Test (expected = Exception.class) + public void testBase64InvalidFileSystemProperties() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final Hashtable properties = new Hashtable<>(); + properties.put("key", "{ value: valueæ­² }"); + AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setFilesystemProperties( + fs, properties); + Hashtable fetchedProperties = AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getFilesystemProperties(fs); + + Assert.assertEquals(properties, fetchedProperties); + } + + @Test (expected = Exception.class) + public void testBase64InvalidPathProperties() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final Hashtable properties = new Hashtable<>(); + properties.put("key", "{ value: valueTestå…© }"); + fs.create(TEST_PATH); + AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setPathProperties( + fs, TEST_PATH, properties); + Hashtable fetchedProperties = + AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getPathProperties(fs, TEST_PATH); + + Assert.assertEquals(properties, fetchedProperties); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java new file mode 100644 index 00000000000..1db93cb0f29 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java @@ -0,0 +1,80 @@ +/** + * 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.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.apache.hadoop.fs.azurebfs.DependencyInjectedTest; +import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; + +import org.junit.Test; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_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.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertNotEquals; + +/** + * Test read, write and seek. + */ +public class ITestReadWriteAndSeek extends DependencyInjectedTest { + private static final Path TEST_PATH = new Path("/testfile"); + public ITestReadWriteAndSeek() { + super(); + } + + @Test + public void testReadAndWriteWithDifferentBufferSizesAndSeek() throws Exception { + testReadWriteAndSeek(MIN_BUFFER_SIZE); + testReadWriteAndSeek(DEFAULT_READ_BUFFER_SIZE); + testReadWriteAndSeek(MAX_BUFFER_SIZE); + } + + private void testReadWriteAndSeek(int bufferSize) throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final ConfigurationServiceImpl configurationservice = (ConfigurationServiceImpl) AbfsServiceProviderImpl.instance().get(ConfigurationService.class); + + fs.create(TEST_PATH); + + configurationservice.setWriteBufferSize(bufferSize); + configurationservice.setReadBufferSize(bufferSize); + + final FSDataOutputStream stream = fs.create(TEST_PATH); + + final byte[] b = new byte[2 * bufferSize]; + new Random().nextBytes(b); + stream.write(b); + stream.close(); + + final byte[] r = new byte[2 * bufferSize]; + final FSDataInputStream inputStream = fs.open(TEST_PATH); + inputStream.seek(bufferSize); + int result = inputStream.read(r, bufferSize, bufferSize); + assertNotEquals(-1, result); + + inputStream.seek(0); + result = inputStream.read(r, 0, bufferSize); + assertNotEquals(-1, result); + assertArrayEquals(r, b); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java new file mode 100644 index 00000000000..e90eb224d05 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java @@ -0,0 +1,79 @@ +/** + * 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.log4j.AppenderSkeleton; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; +import org.apache.log4j.spi.LoggingEvent; +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest; +import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService; +import org.apache.htrace.core.MilliSpan; +import org.apache.htrace.core.TraceScope; + +/** + * Test tracing service. + */ +public class ITestTracingServiceImpl extends DependencyInjectedTest { + private final List messageStorage; + + public ITestTracingServiceImpl() { + super(); + this.messageStorage = new ArrayList<>(); + } + + @Test + public void traceSerializationTest() throws Exception { + Logger rootLogger = Logger.getRootLogger(); + rootLogger.setLevel(Level.TRACE); + rootLogger.addAppender(new AppenderSkeleton() { + @Override + protected void append(LoggingEvent event) { + if (event.getLogger().getName().indexOf("AzureBlobFileSystem") != -1) { + messageStorage.add(event.getRenderedMessage()); + } + } + + @Override + public void close() { + + } + + @Override + public boolean requiresLayout() { + return false; + } + }); + + TracingService tracingService = new TracingServiceImpl(new Configuration()); + TraceScope traceScope = tracingService.traceBegin("Test Scope"); + traceScope.addTimelineAnnotation("Timeline Annotations"); + traceScope.addKVAnnotation("key", "value"); + traceScope.close(); + + // Should not throw exception. + MilliSpan.fromJson(messageStorage.get(0)); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java new file mode 100644 index 00000000000..2f27b168b1f --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.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 com.google.common.annotations.VisibleForTesting; +import com.google.inject.Inject; +import com.google.inject.Singleton; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; +import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; +import org.apache.hadoop.fs.azurebfs.utils.UriUtils; +import org.apache.http.client.utils.URIBuilder; + +/** + * Mock AbfsHttpClientFactoryImpl. + */ +@Singleton +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class MockAbfsHttpClientFactoryImpl extends AbfsHttpClientFactoryImpl { + private final ConfigurationService configurationService; + + @Inject + MockAbfsHttpClientFactoryImpl( + final ConfigurationService configurationService) { + super(configurationService); + + this.configurationService = configurationService; + } + + @VisibleForTesting + URIBuilder getURIBuilder(final String hostName, final FileSystem fs) { + final URIBuilder uriBuilder = new URIBuilder(); + + final String testHost = this.configurationService.getConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_HOST_NAME); + final Integer testHostPort = this.configurationService.getConfiguration().getInt(TestConfigurationKeys.FS_AZURE_TEST_HOST_PORT, 80); + final String testAccount = this.configurationService.getConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME); + + String scheme = FileSystemUriSchemes.HTTP_SCHEME; + + uriBuilder.setScheme(scheme); + uriBuilder.setHost(testHost); + uriBuilder.setPort(testHostPort); + + uriBuilder.setPath("/" + UriUtils.extractRawAccountFromAccountName(testAccount) + "/"); + + return uriBuilder; + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java new file mode 100644 index 00000000000..ff2fb2ad93c --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java @@ -0,0 +1,50 @@ +/** + * 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.conf.Configuration; + +/** + * Mock AbfsServiceInjectorImpl. + */ +public class MockAbfsServiceInjectorImpl extends AbfsServiceInjectorImpl { + public MockAbfsServiceInjectorImpl(Configuration configuration) { + super(configuration); + } + + public void replaceInstance(Class tInterface, Object object) { + this.removeInstance(tInterface); + this.removeProvider(tInterface); + this.getInstances().put(tInterface, object); + } + + public void removeInstance(Class tInterface) { + this.getInstances().remove(tInterface); + } + + public void replaceProvider(Class tInterface, Class tClazz) { + this.removeInstance(tInterface); + this.removeProvider(tInterface); + this.getProviders().put(tInterface, tClazz); + } + + public void removeProvider(Class tInterface) { + this.getProviders().remove(tInterface); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java new file mode 100644 index 00000000000..5992f75666e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java @@ -0,0 +1,36 @@ +/** + * 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 com.google.inject.Guice; +import com.google.inject.Injector; + +/** + * Mock ABFS ServiceProviderImpl. + */ +public final class MockServiceProviderImpl { + public static void create(MockAbfsServiceInjectorImpl abfsServiceInjector) { + Injector injector = Guice.createInjector(abfsServiceInjector); + AbfsServiceProviderImpl.create(injector); + } + + private MockServiceProviderImpl() { + // no-op + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java new file mode 100644 index 00000000000..688c5220ee8 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java @@ -0,0 +1,149 @@ +/** + * 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.lang.reflect.Field; + +import org.apache.commons.codec.Charsets; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation; +import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation; +import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation; +import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation; +import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException; + +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_BACKOFF_INTERVAL; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_BACKOFF_INTERVAL; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MIN_BACKOFF_INTERVAL; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT; + +import org.apache.commons.codec.binary.Base64; +import static org.junit.Assert.assertEquals; + +import org.junit.Test; + +/** + * Test ConfigurationServiceFieldsValidation. + */ +public class TestConfigurationServiceFieldsValidation { + private ConfigurationServiceImpl configService; + + private static final String INT_KEY= "intKey"; + private static final String LONG_KEY= "longKey"; + private static final String STRING_KEY= "stringKey"; + private static final String BASE64_KEY= "base64Key"; + private static final String BOOLEAN_KEY= "booleanKey"; + private static final int DEFAULT_INT = 4194304; + private static final int DEFAULT_LONG = 4194304; + + private static final int TEST_INT = 1234565; + private static final int TEST_LONG = 4194304; + + private final String encodedString; + private final String encodedAccountKey; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = INT_KEY, + MinValue = Integer.MIN_VALUE, + MaxValue = Integer.MAX_VALUE, + DefaultValue = DEFAULT_INT) + private int intField; + + @LongConfigurationValidatorAnnotation(ConfigurationKey = LONG_KEY, + MinValue = Long.MIN_VALUE, + MaxValue = Long.MAX_VALUE, + DefaultValue = DEFAULT_LONG) + private int longField; + + @StringConfigurationValidatorAnnotation(ConfigurationKey = STRING_KEY, + DefaultValue = "default") + private String stringField; + + @Base64StringConfigurationValidatorAnnotation(ConfigurationKey = BASE64_KEY, + DefaultValue = "base64") + private String base64Field; + + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = BOOLEAN_KEY, + DefaultValue = false) + private boolean boolField; + + public TestConfigurationServiceFieldsValidation() throws Exception { + super(); + Base64 base64 = new Base64(); + this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8); + this.encodedAccountKey = new String(base64.encode("someAccountKey".getBytes(Charsets.UTF_8)), Charsets.UTF_8); + Configuration configuration = new Configuration(); + configuration.addResource("azure-bfs-test.xml"); + configuration.set(INT_KEY, "1234565"); + configuration.set(LONG_KEY, "4194304"); + configuration.set(STRING_KEY, "stringValue"); + configuration.set(BASE64_KEY, encodedString); + configuration.set(BOOLEAN_KEY, "true"); + configuration.set(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + "testaccount1.blob.core.windows.net", this.encodedAccountKey); + configService = new ConfigurationServiceImpl(configuration); + } + + @Test + public void testValidateFunctionsInConfigServiceImpl() throws Exception { + Field[] fields = this.getClass().getDeclaredFields(); + for (Field field : fields) { + field.setAccessible(true); + if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) { + assertEquals(TEST_INT, configService.validateInt(field)); + } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) { + assertEquals(DEFAULT_LONG, configService.validateLong(field)); + } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) { + assertEquals("stringValue", configService.validateString(field)); + } else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) { + assertEquals(this.encodedString, configService.validateBase64String(field)); + } else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) { + assertEquals(true, configService.validateBoolean(field)); + } + } + } + + @Test + public void testConfigServiceImplAnnotatedFieldsInitialized() throws Exception { + // test that all the ConfigurationServiceImpl annotated fields have been initialized in the constructor + assertEquals(DEFAULT_WRITE_BUFFER_SIZE, configService.getWriteBufferSize()); + assertEquals(DEFAULT_READ_BUFFER_SIZE, configService.getReadBufferSize()); + assertEquals(DEFAULT_MIN_BACKOFF_INTERVAL, configService.getMinBackoffIntervalMilliseconds()); + assertEquals(DEFAULT_MAX_BACKOFF_INTERVAL, configService.getMaxBackoffIntervalMilliseconds()); + assertEquals(DEFAULT_BACKOFF_INTERVAL, configService.getBackoffIntervalMilliseconds()); + assertEquals(DEFAULT_MAX_RETRY_ATTEMPTS, configService.getMaxIoRetries()); + assertEquals(MAX_AZURE_BLOCK_SIZE, configService.getAzureBlockSize()); + assertEquals(AZURE_BLOCK_LOCATION_HOST_DEFAULT, configService.getAzureBlockLocationHost()); + } + + @Test + public void testGetAccountKey() throws Exception { + String accountKey = configService.getStorageAccountKey("testaccount1.blob.core.windows.net"); + assertEquals(this.encodedAccountKey, accountKey); + } + + @Test (expected = ConfigurationPropertyNotFoundException.class) + public void testGetAccountKeyWithNonExistingAccountName() throws Exception { + configService.getStorageAccountKey("bogusAccountName"); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/package-info.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/package-info.java new file mode 100644 index 00000000000..97c1d71251f --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +package org.apache.hadoop.fs.azurebfs.services; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TestUriUtils.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TestUriUtils.java new file mode 100644 index 00000000000..e71e31de69e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TestUriUtils.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.utils; + +import org.junit.Assert; +import org.junit.Test; + +/** + * Test ABFS UriUtils. + */ +public final class TestUriUtils { + @Test + public void testIfUriContainsAbfs() throws Exception { + Assert.assertTrue(UriUtils.containsAbfsUrl("abfs.dfs.core.windows.net")); + Assert.assertTrue(UriUtils.containsAbfsUrl("abfs.dfs.preprod.core.windows.net")); + Assert.assertFalse(UriUtils.containsAbfsUrl("abfs.dfs.cores.windows.net")); + Assert.assertFalse(UriUtils.containsAbfsUrl("")); + Assert.assertFalse(UriUtils.containsAbfsUrl(null)); + Assert.assertFalse(UriUtils.containsAbfsUrl("abfs.dfs.cores.windows.net")); + Assert.assertFalse(UriUtils.containsAbfsUrl("xhdfs.blob.core.windows.net")); + } + + @Test + public void testExtractRawAccountName() throws Exception { + Assert.assertEquals("abfs", UriUtils.extractRawAccountFromAccountName("abfs.dfs.core.windows.net")); + Assert.assertEquals("abfs", UriUtils.extractRawAccountFromAccountName("abfs.dfs.preprod.core.windows.net")); + Assert.assertEquals(null, UriUtils.extractRawAccountFromAccountName("abfs.dfs.cores.windows.net")); + Assert.assertEquals(null, UriUtils.extractRawAccountFromAccountName("")); + Assert.assertEquals(null, UriUtils.extractRawAccountFromAccountName(null)); + Assert.assertEquals(null, UriUtils.extractRawAccountFromAccountName("abfs.dfs.cores.windows.net")); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/package-info.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/package-info.java new file mode 100644 index 00000000000..d8cc940da1b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +package org.apache.hadoop.fs.azurebfs.utils; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml b/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml new file mode 100644 index 00000000000..caf8677cbec --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml @@ -0,0 +1,64 @@ + + + + + fs.contract.test.root-tests-enabled + false + + + + fs.contract.supports-append + true + + + + fs.contract.supports-seek + true + + + + fs.contract.rename-overwrites-dest + false + + + + fs.contract.rename-returns-false-if-source-missing + true + + + + fs.contract.rename-creates-dest-dirs + false + + + + fs.contract.supports-settimes + false + + + + fs.contract.supports-concat + false + + + + fs.contract.supports-getfilestatus + true + + \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml b/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml new file mode 100644 index 00000000000..508e5f69418 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml @@ -0,0 +1,76 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + fs.AbstractFileSystem.abfs.impl + org.apache.hadoop.fs.azurebfs.Abfs + + + + fs.AbstractFileSystem.abfss.impl + org.apache.hadoop.fs.azurebfs.Abfss + + + \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties index a5e0c4f94e8..fd0cd9de0d0 100644 --- a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties @@ -24,3 +24,5 @@ log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t]: %c{2} (%F: 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 From a271fd0eca75cef8b8ba940cdac8ad4fd21b4462 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 3 Jul 2018 18:55:10 +0200 Subject: [PATCH 02/29] HADOOP-15560. ABFS: removed dependency injection and unnecessary dependencies. Contributed by Da Zhou. --- hadoop-tools/hadoop-azure/pom.xml | 18 - .../src/config/checkstyle-suppressions.xml | 2 +- .../fs/azurebfs/AzureBlobFileSystem.java | 88 ++- ...mpl.java => AzureBlobFileSystemStore.java} | 506 +++++++++--------- .../ServiceResolutionException.java | 36 -- .../services/AbfsHttpClientFactory.java | 39 -- .../contracts/services/AbfsHttpService.java | 162 ------ .../services/AbfsServiceProvider.java | 40 -- .../services/ConfigurationService.java | 143 ----- .../contracts/services/InjectableService.java | 30 -- .../contracts/services/TracingService.java | 66 --- .../fs/azurebfs/services/AbfsClient.java | 7 +- ...erviceImpl.java => AbfsConfiguration.java} | 30 +- .../services/AbfsHttpClientFactoryImpl.java | 116 ---- .../services/AbfsServiceInjectorImpl.java | 81 --- .../services/AbfsServiceProviderImpl.java | 96 ---- .../services/ExponentialRetryPolicy.java | 9 +- .../azurebfs/services/LoggerSpanReceiver.java | 74 --- .../azurebfs/services/TracingServiceImpl.java | 134 ----- .../fs/azurebfs/DependencyInjectedTest.java | 55 +- .../azurebfs/ITestAzureBlobFileSystemE2E.java | 7 +- .../ITestAzureBlobFileSystemRandomRead.java | 7 +- .../ITestFileSystemInitialization.java | 23 +- ...pl.java => ITestFileSystemProperties.java} | 44 +- .../azurebfs/ITestFileSystemRegistration.java | 23 +- .../ITestAzureBlobFileSystemBasics.java | 11 +- .../services/ITestReadWriteAndSeek.java | 8 +- .../services/ITestTracingServiceImpl.java | 79 --- .../MockAbfsHttpClientFactoryImpl.java | 69 --- .../services/MockAbfsServiceInjectorImpl.java | 50 -- .../services/MockServiceProviderImpl.java | 36 -- ...estAbfsConfigurationFieldsValidation.java} | 38 +- .../utils/CleanUpAbfsTestContainer.java | 68 +++ 33 files changed, 462 insertions(+), 1733 deletions(-) rename hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/{services/AbfsHttpServiceImpl.java => AzureBlobFileSystemStore.java} (57%) delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.java delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.java delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.java delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.java rename hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/{ConfigurationServiceImpl.java => AbfsConfiguration.java} (96%) delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.java delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.java delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.java delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.java rename hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/{services/ITestAbfsHttpServiceImpl.java => ITestFileSystemProperties.java} (71%) delete mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java delete mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java delete mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java delete mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java rename hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/{TestConfigurationServiceFieldsValidation.java => TestAbfsConfigurationFieldsValidation.java} (80%) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml index d4046ef2dd7..cbd4dfb5b4b 100644 --- a/hadoop-tools/hadoop-azure/pom.xml +++ b/hadoop-tools/hadoop-azure/pom.xml @@ -149,12 +149,6 @@ provided - - org.threadly - threadly - compile - - com.fasterxml.jackson.core jackson-core @@ -185,18 +179,6 @@ - - org.apache.htrace - htrace-core - compile - - - - org.apache.htrace - htrace-core4 - compile - - com.google.inject guice diff --git a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml index 02043552992..751a22773d7 100644 --- a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml +++ b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml @@ -43,5 +43,5 @@ + files="org[\\/]apache[\\/]hadoop[\\/]fs[\\/]azurebfs[\\/]AzureBlobFileSystemStore.java"/> diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 707c81ec768..cf5acbb0fd3 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 @@ -39,10 +39,8 @@ import com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.threadly.util.ExceptionUtils; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; -import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl; import org.apache.commons.lang.ArrayUtils; import org.apache.hadoop.classification.InterfaceAudience; @@ -58,10 +56,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; -import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService; -import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; -import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService; -import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsServiceProvider; import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.FileSystemOperationUnhandledException; @@ -70,7 +64,6 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Progressable; -import org.apache.htrace.core.TraceScope; /** * A {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on callableFileOperation, T defaultResultValue) throws IOException { - final TraceScope traceScope = tracingService.traceBegin(scopeDescription); try { final T executionResult = callableFileOperation.call(); return new FileSystemOperation(executionResult, null); } catch (AbfsRestOperationException abfsRestOperationException) { return new FileSystemOperation(defaultResultValue, abfsRestOperationException); } catch (AzureBlobFileSystemException azureBlobFileSystemException) { - tracingService.traceException(traceScope, azureBlobFileSystemException); throw new IOException(azureBlobFileSystemException); } catch (Exception exception) { if (exception instanceof ExecutionException) { - exception = (Exception) ExceptionUtils.getRootCause(exception); + exception = (Exception) getRootCause(exception); } - final FileSystemOperationUnhandledException fileSystemOperationUnhandledException = new FileSystemOperationUnhandledException(exception); - tracingService.traceException(traceScope, fileSystemOperationUnhandledException); throw new IOException(fileSystemOperationUnhandledException); - } finally { - tracingService.traceEnd(traceScope); } } @@ -590,6 +557,26 @@ public class AzureBlobFileSystem extends FileSystem { } } + /** + * Gets the root cause of a provided {@link Throwable}. If there is no cause for the + * {@link Throwable} provided into this function, the original {@link Throwable} is returned. + * + * @param throwable starting {@link Throwable} + * @return root cause {@link Throwable} + */ + private Throwable getRootCause(Throwable throwable) { + if (throwable == null) { + throw new IllegalArgumentException("throwable can not be null"); + } + + Throwable result = throwable; + while (result.getCause() != null) { + result = result.getCause(); + } + + return result; + } + @VisibleForTesting FileSystem.Statistics getFsStatistics() { return this.statistics; @@ -609,4 +596,9 @@ public class AzureBlobFileSystem extends FileSystem { return this.exception != null; } } + + @VisibleForTesting + AzureBlobFileSystemStore getAbfsStore() { + return this.abfsStore; + } } \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpServiceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java similarity index 57% rename from hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpServiceImpl.java rename to hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 06e1a8a5cad..134277fd969 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpServiceImpl.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -15,15 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.hadoop.fs.azurebfs; -package org.apache.hadoop.fs.azurebfs.services; - -import javax.xml.bind.DatatypeConverter; import java.io.File; +import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.net.MalformedURLException; import java.net.URI; import java.net.URISyntaxException; +import java.net.URL; import java.nio.ByteBuffer; import java.nio.CharBuffer; import java.nio.charset.CharacterCodingException; @@ -32,89 +33,110 @@ import java.nio.charset.CharsetDecoder; import java.nio.charset.CharsetEncoder; import java.util.ArrayList; import java.util.Arrays; -import java.util.Set; import java.util.HashSet; import java.util.Hashtable; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; +import java.util.Set; +import javax.xml.bind.DatatypeConverter; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.inject.Inject; -import com.google.inject.Singleton; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException; -import org.joda.time.DateTime; -import org.joda.time.format.DateTimeFormat; - -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; - -import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; -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.exceptions.InvalidFileSystemPropertyException; -import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService; -import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory; -import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; -import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService; -import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema; -import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; -import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; 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.exceptions.FileSystemOperationUnhandledException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidFileSystemPropertyException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TimeoutException; +import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; +import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema; +import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsConfiguration; +import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; +import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; +import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; +import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy; +import org.apache.hadoop.fs.azurebfs.services.SharedKeyCredentials; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; - +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.http.client.utils.URIBuilder; +import org.joda.time.DateTime; +import org.joda.time.format.DateTimeFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.apache.hadoop.util.Time.now; -@Singleton -@InterfaceAudience.Private +/** + * Provides the bridging logic between Hadoop's abstract filesystem and Azure Storage + */ +@InterfaceAudience.Public @InterfaceStability.Evolving -final class AbfsHttpServiceImpl implements AbfsHttpService { - public static final Logger LOG = LoggerFactory.getLogger(AbfsHttpService.class); +public class AzureBlobFileSystemStore { + private static final Logger LOG = LoggerFactory.getLogger(AzureBlobFileSystemStore.class); + + private AbfsClient client; + private URI uri; + private final UserGroupInformation userGroupInformation; private static final String DATE_TIME_PATTERN = "E, dd MMM yyyy HH:mm:ss 'GMT'"; private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1"; private static final int LIST_MAX_RESULTS = 5000; private static final int DELETE_DIRECTORY_TIMEOUT_MILISECONDS = 180000; private static final int RENAME_TIMEOUT_MILISECONDS = 180000; - private final AbfsHttpClientFactory abfsHttpClientFactory; - private final ConcurrentHashMap clientCache; - private final ConfigurationService configurationService; + private final AbfsConfiguration abfsConfiguration; private final Set azureAtomicRenameDirSet; - @Inject - AbfsHttpServiceImpl( - final ConfigurationService configurationService, - final AbfsHttpClientFactory abfsHttpClientFactory, - final TracingService tracingService) { - Preconditions.checkNotNull(abfsHttpClientFactory, "abfsHttpClientFactory"); - Preconditions.checkNotNull(configurationService, "configurationService"); - Preconditions.checkNotNull(tracingService, "tracingService"); - this.configurationService = configurationService; - this.clientCache = new ConcurrentHashMap<>(); - this.abfsHttpClientFactory = abfsHttpClientFactory; - this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(configurationService.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA))); + public AzureBlobFileSystemStore(URI uri, boolean isSeure, Configuration configuration, UserGroupInformation userGroupInformation) + throws AzureBlobFileSystemException { + this.uri = uri; + try { + this.abfsConfiguration = new AbfsConfiguration(configuration); + } catch (IllegalAccessException exception) { + throw new FileSystemOperationUnhandledException(exception); + } + + this.userGroupInformation = userGroupInformation; + this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA))); + + initializeClient(uri, isSeure); } - @Override - public Hashtable getFilesystemProperties(final AzureBlobFileSystem azureBlobFileSystem) - throws AzureBlobFileSystemException{ - final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); + @VisibleForTesting + URIBuilder getURIBuilder(final String hostName, boolean isSecure) { + String scheme = isSecure ? FileSystemUriSchemes.HTTPS_SCHEME : FileSystemUriSchemes.HTTP_SCHEME; + final URIBuilder uriBuilder = new URIBuilder(); + uriBuilder.setScheme(scheme); + uriBuilder.setHost(hostName); + + return uriBuilder; + } + + public AbfsConfiguration getAbfsConfiguration() { + return this.abfsConfiguration; + } + + public Hashtable getFilesystemProperties() throws AzureBlobFileSystemException { this.LOG.debug( - "getFilesystemProperties for filesystem: {}", - client.getFileSystem()); + "getFilesystemProperties for filesystem: {}", + client.getFileSystem()); final Hashtable parsedXmsProperties; @@ -126,19 +148,15 @@ final class AbfsHttpServiceImpl implements AbfsHttpService { return parsedXmsProperties; } - @Override - public void setFilesystemProperties(final AzureBlobFileSystem azureBlobFileSystem, final Hashtable properties) throws - AzureBlobFileSystemException { + public void setFilesystemProperties(final Hashtable properties) throws AzureBlobFileSystemException { if (properties == null || properties.size() == 0) { return; } - final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); - this.LOG.debug( - "setFilesystemProperties for filesystem: {} with properties: {}", - client.getFileSystem(), - properties); + "setFilesystemProperties for filesystem: {} with properties: {}", + client.getFileSystem(), + properties); final String commaSeparatedProperties; try { @@ -146,18 +164,15 @@ final class AbfsHttpServiceImpl implements AbfsHttpService { } catch (CharacterCodingException ex) { throw new InvalidAbfsRestOperationException(ex); } + client.setFilesystemProperties(commaSeparatedProperties); } - @Override - public Hashtable getPathProperties(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws - AzureBlobFileSystemException { - final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); - + public Hashtable getPathProperties(final Path path) throws AzureBlobFileSystemException { this.LOG.debug( - "getPathProperties for filesystem: {} path: {}", - client.getFileSystem(), - path.toString()); + "getPathProperties for filesystem: {} path: {}", + client.getFileSystem(), + path.toString()); final Hashtable parsedXmsProperties; final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); @@ -169,17 +184,12 @@ final class AbfsHttpServiceImpl implements AbfsHttpService { return parsedXmsProperties; } - @Override - public void setPathProperties(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final Hashtable properties) throws - AzureBlobFileSystemException { - final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); - + public void setPathProperties(final Path path, final Hashtable properties) throws AzureBlobFileSystemException { this.LOG.debug( - "setFilesystemProperties for filesystem: {} path: {} with properties: {}", - client.getFileSystem(), - path.toString(), - properties); + "setFilesystemProperties for filesystem: {} path: {} with properties: {}", + client.getFileSystem(), + path.toString(), + properties); final String commaSeparatedProperties; try { @@ -190,71 +200,55 @@ final class AbfsHttpServiceImpl implements AbfsHttpService { client.setPathProperties("/" + getRelativePath(path), commaSeparatedProperties); } - @Override - public void createFilesystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException { - final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); - + public void createFilesystem() throws AzureBlobFileSystemException { this.LOG.debug( - "createFilesystem for filesystem: {}", - client.getFileSystem()); + "createFilesystem for filesystem: {}", + client.getFileSystem()); client.createFilesystem(); } - @Override - public void deleteFilesystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException { - final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); - + public void deleteFilesystem() throws AzureBlobFileSystemException { this.LOG.debug( - "deleteFilesystem for filesystem: {}", - client.getFileSystem()); + "deleteFilesystem for filesystem: {}", + client.getFileSystem()); client.deleteFilesystem(); } - @Override - public OutputStream createFile(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean overwrite) throws - AzureBlobFileSystemException { - final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); - + public OutputStream createFile(final Path path, final boolean overwrite) throws AzureBlobFileSystemException { this.LOG.debug( - "createFile filesystem: {} path: {} overwrite: {}", - client.getFileSystem(), - path.toString(), - overwrite); + "createFile filesystem: {} path: {} overwrite: {}", + client.getFileSystem(), + path.toString(), + overwrite); client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), true, overwrite); final OutputStream outputStream; outputStream = new FSDataOutputStream( - new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), 0, - configurationService.getWriteBufferSize()), null); + new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), 0, + abfsConfiguration.getWriteBufferSize()), null); return outputStream; } - @Override - public Void createDirectory(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException { - final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); - + public Void createDirectory(final Path path) throws AzureBlobFileSystemException { this.LOG.debug( - "createDirectory filesystem: {} path: {} overwrite: {}", - client.getFileSystem(), - path.toString()); + "createDirectory filesystem: {} path: {} overwrite: {}", + client.getFileSystem(), + path.toString()); client.createPath("/" + getRelativePath(path), false, true); return null; } - @Override - public InputStream openFileForRead(final AzureBlobFileSystem azureBlobFileSystem, final Path path, - final FileSystem.Statistics statistics) throws AzureBlobFileSystemException { - final AbfsClient client = getOrCreateClient(azureBlobFileSystem); + public InputStream openFileForRead(final Path path, final FileSystem.Statistics statistics) throws AzureBlobFileSystemException { this.LOG.debug( - "openFileForRead filesystem: {} path: {}", - client.getFileSystem(), - path.toString()); + "openFileForRead filesystem: {} path: {}", + client.getFileSystem(), + path.toString()); final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); @@ -264,28 +258,25 @@ final class AbfsHttpServiceImpl implements AbfsHttpService { if (parseIsDirectory(resourceType)) { throw new AbfsRestOperationException( - AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), - AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), - "openFileForRead must be used with files and not directories", - null); + AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), + AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), + "openFileForRead must be used with files and not directories", + null); } // Add statistics for InputStream return new FSDataInputStream( - new AbfsInputStream(client, statistics, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength, - configurationService.getReadBufferSize(), configurationService.getReadAheadQueueDepth(), eTag)); + new AbfsInputStream(client, statistics, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength, + abfsConfiguration.getReadBufferSize(), abfsConfiguration.getReadAheadQueueDepth(), eTag)); } - @Override - public OutputStream openFileForWrite(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean overwrite) throws - AzureBlobFileSystemException { - final AbfsClient client = getOrCreateClient(azureBlobFileSystem); - + public OutputStream openFileForWrite(final Path path, final boolean overwrite) throws + AzureBlobFileSystemException { this.LOG.debug( - "openFileForWrite filesystem: {} path: {} overwrite: {}", - client.getFileSystem(), - path.toString(), - overwrite); + "openFileForWrite filesystem: {} path: {} overwrite: {}", + client.getFileSystem(), + path.toString(), + overwrite); final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); @@ -294,37 +285,34 @@ final class AbfsHttpServiceImpl implements AbfsHttpService { if (parseIsDirectory(resourceType)) { throw new AbfsRestOperationException( - AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), - AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), - "openFileForRead must be used with files and not directories", - null); + AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), + AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), + "openFileForRead must be used with files and not directories", + null); } final long offset = overwrite ? 0 : contentLength; final OutputStream outputStream; outputStream = new FSDataOutputStream( - new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), - offset, configurationService.getWriteBufferSize()), null); + new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), + offset, abfsConfiguration.getWriteBufferSize()), null); return outputStream; } - @Override - public void rename(final AzureBlobFileSystem azureBlobFileSystem, final Path source, final Path destination) throws - AzureBlobFileSystemException { + public void rename(final Path source, final Path destination) throws + AzureBlobFileSystemException { if (isAtomicRenameKey(source.getName())) { this.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."); + +" create and delete operations are atomic if Namespace is enabled for your Azure Storage account."); } - final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); - this.LOG.debug( - "renameAsync filesystem: {} source: {} destination: {}", - client.getFileSystem(), - source.toString(), - destination.toString()); + "renameAsync filesystem: {} source: {} destination: {}", + client.getFileSystem(), + source.toString(), + destination.toString()); String continuation = null; long deadline = now() + RENAME_TIMEOUT_MILISECONDS; @@ -332,30 +320,28 @@ final class AbfsHttpServiceImpl implements AbfsHttpService { do { if (now() > deadline) { LOG.debug( - "Rename {} to {} timed out.", - source, - destination); + "Rename {} to {} timed out.", + source, + destination); throw new TimeoutException("Rename timed out."); } AbfsRestOperation op = client.renamePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(source), - AbfsHttpConstants.FORWARD_SLASH + getRelativePath(destination), continuation); + AbfsHttpConstants.FORWARD_SLASH + getRelativePath(destination), continuation); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); } while (continuation != null && !continuation.isEmpty()); } - @Override - public void delete(final AzureBlobFileSystem azureBlobFileSystem, final Path path, final boolean recursive) throws - AzureBlobFileSystemException { - final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); + public void delete(final Path path, final boolean recursive) throws + AzureBlobFileSystemException { this.LOG.debug( - "delete filesystem: {} path: {} recursive: {}", - client.getFileSystem(), - path.toString(), - String.valueOf(recursive)); + "delete filesystem: {} path: {} recursive: {}", + client.getFileSystem(), + path.toString(), + String.valueOf(recursive)); String continuation = null; long deadline = now() + DELETE_DIRECTORY_TIMEOUT_MILISECONDS; @@ -363,7 +349,7 @@ final class AbfsHttpServiceImpl implements AbfsHttpService { do { if (now() > deadline) { this.LOG.debug( - "Delete directory {} timed out.", path); + "Delete directory {} timed out.", path); throw new TimeoutException("Delete directory timed out."); } @@ -374,60 +360,55 @@ final class AbfsHttpServiceImpl implements AbfsHttpService { } while (continuation != null && !continuation.isEmpty()); } - @Override - public FileStatus getFileStatus(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException { - final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); + public FileStatus getFileStatus(final Path path) throws IOException { this.LOG.debug( - "getFileStatus filesystem: {} path: {}", - client.getFileSystem(), - path.toString()); + "getFileStatus filesystem: {} path: {}", + client.getFileSystem(), + path.toString()); if (path.isRoot()) { AbfsRestOperation op = client.getFilesystemProperties(); - final long blockSize = configurationService.getAzureBlockSize(); + final long blockSize = abfsConfiguration.getAzureBlockSize(); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); final String lastModified = op.getResult().getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED); return new VersionedFileStatus( - azureBlobFileSystem.getOwnerUser(), - azureBlobFileSystem.getOwnerUserPrimaryGroup(), - 0, - true, - 1, - blockSize, - parseLastModifiedTime(lastModified).getMillis(), - path, - eTag); + userGroupInformation.getUserName(), + userGroupInformation.getPrimaryGroupName(), + 0, + true, + 1, + blockSize, + parseLastModifiedTime(lastModified).getMillis(), + path, + eTag); } else { AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); - final long blockSize = configurationService.getAzureBlockSize(); + final long blockSize = abfsConfiguration.getAzureBlockSize(); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); final String lastModified = op.getResult().getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED); final String contentLength = op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH); final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); return new VersionedFileStatus( - azureBlobFileSystem.getOwnerUser(), - azureBlobFileSystem.getOwnerUserPrimaryGroup(), - parseContentLength(contentLength), - parseIsDirectory(resourceType), - 1, - blockSize, - parseLastModifiedTime(lastModified).getMillis(), - path, - eTag); + userGroupInformation.getUserName(), + userGroupInformation.getPrimaryGroupName(), + parseContentLength(contentLength), + parseIsDirectory(resourceType), + 1, + blockSize, + parseLastModifiedTime(lastModified).getMillis(), + path, + eTag); } } - @Override - public FileStatus[] listStatus(final AzureBlobFileSystem azureBlobFileSystem, final Path path) throws AzureBlobFileSystemException { - final AbfsClient client = this.getOrCreateClient(azureBlobFileSystem); - + public FileStatus[] listStatus(final Path path) throws IOException { this.LOG.debug( - "listStatus filesystem: {} path: {}", - client.getFileSystem(), - path.toString()); + "listStatus filesystem: {} path: {}", + client.getFileSystem(), + path.toString()); String relativePath = path.isRoot() ? AbfsHttpConstants.EMPTY_STRING : getRelativePath(path); String continuation = null; @@ -439,13 +420,13 @@ final class AbfsHttpServiceImpl implements AbfsHttpService { ListResultSchema retrievedSchema = op.getResult().getListResultSchema(); if (retrievedSchema == null) { throw new AbfsRestOperationException( - AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), - AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), - "listStatusAsync path not found", - null, op.getResult()); + AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), + AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), + "listStatusAsync path not found", + null, op.getResult()); } - long blockSize = configurationService.getAzureBlockSize(); + long blockSize = abfsConfiguration.getAzureBlockSize(); for (ListResultEntrySchema entry : retrievedSchema.paths()) { long lastModifiedMillis = 0; @@ -453,22 +434,25 @@ final class AbfsHttpServiceImpl implements AbfsHttpService { boolean isDirectory = entry.isDirectory() == null ? false : entry.isDirectory(); if (entry.lastModified() != null && !entry.lastModified().isEmpty()) { final DateTime dateTime = DateTime.parse( - entry.lastModified(), - DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC()); + entry.lastModified(), + DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC()); lastModifiedMillis = dateTime.getMillis(); } + Path entryPath = new Path(File.separator + entry.name()); + entryPath = entryPath.makeQualified(this.uri, entryPath); + fileStatuses.add( - new VersionedFileStatus( - azureBlobFileSystem.getOwnerUser(), - azureBlobFileSystem.getOwnerUserPrimaryGroup(), - contentLength, - isDirectory, - 1, - blockSize, - lastModifiedMillis, - azureBlobFileSystem.makeQualified(new Path(File.separator + entry.name())), - entry.eTag())); + new VersionedFileStatus( + userGroupInformation.getUserName(), + userGroupInformation.getPrimaryGroupName(), + contentLength, + isDirectory, + 1, + blockSize, + lastModifiedMillis, + entryPath, + entry.eTag())); } } while (continuation != null && !continuation.isEmpty()); @@ -476,16 +460,55 @@ final class AbfsHttpServiceImpl implements AbfsHttpService { return fileStatuses.toArray(new FileStatus[0]); } - @Override - public synchronized void closeFileSystem(final AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException { - this.clientCache.remove(azureBlobFileSystem); - } - - @Override public boolean isAtomicRenameKey(String key) { return isKeyForDirectorySet(key, azureAtomicRenameDirSet); } + private void initializeClient(URI uri, boolean isSeure) throws AzureBlobFileSystemException { + if (this.client != null) { + return; + } + + final String authority = uri.getRawAuthority(); + if (null == authority) { + throw new InvalidUriAuthorityException(uri.toString()); + } + + if (!authority.contains(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER)) { + throw new InvalidUriAuthorityException(uri.toString()); + } + + final String[] authorityParts = authority.split(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2); + + if (authorityParts.length < 2 || "".equals(authorityParts[0])) { + final String errMsg = String + .format("URI '%s' has a malformed authority, expected container name. " + + "Authority takes the form "+ FileSystemUriSchemes.ABFS_SCHEME + "://[@]", + uri.toString()); + throw new InvalidUriException(errMsg); + } + + final String fileSystemName = authorityParts[0]; + final String accountName = authorityParts[1]; + + final URIBuilder uriBuilder = getURIBuilder(accountName, isSeure); + + final String url = uriBuilder.toString() + AbfsHttpConstants.FORWARD_SLASH + fileSystemName; + + URL baseUrl; + try { + baseUrl = new URL(url); + } catch (MalformedURLException e) { + throw new InvalidUriException(String.format("URI '%s' is malformed", uri.toString())); + } + + SharedKeyCredentials creds = + new SharedKeyCredentials(accountName.substring(0, accountName.indexOf(AbfsHttpConstants.DOT)), + this.abfsConfiguration.getStorageAccountKey(accountName)); + + this.client = new AbfsClient(baseUrl, creds, abfsConfiguration, new ExponentialRetryPolicy()); + } + private String getRelativePath(final Path path) { Preconditions.checkNotNull(path, "path"); final String relativePath = path.toUri().getPath(); @@ -505,23 +528,6 @@ final class AbfsHttpServiceImpl implements AbfsHttpService { return relativePath; } - private synchronized AbfsClient getOrCreateClient(final AzureBlobFileSystem azureBlobFileSystem) throws - AzureBlobFileSystemException { - Preconditions.checkNotNull(azureBlobFileSystem, "azureBlobFileSystem"); - - AbfsClient client = this.clientCache.get(azureBlobFileSystem); - - if (client != null) { - return client; - } - - client = abfsHttpClientFactory.create(azureBlobFileSystem); - this.clientCache.put( - azureBlobFileSystem, - client); - return client; - } - private long parseContentLength(final String contentLength) { if (contentLength == null) { return -1; @@ -536,12 +542,12 @@ final class AbfsHttpServiceImpl implements AbfsHttpService { private DateTime parseLastModifiedTime(final String lastModifiedTime) { return DateTime.parse( - lastModifiedTime, - DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC()); + lastModifiedTime, + DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC()); } private String convertXmsPropertiesToCommaSeparatedString(final Hashtable properties) throws - CharacterCodingException { + CharacterCodingException { StringBuilder commaSeparatedProperties = new StringBuilder(); final CharsetEncoder encoder = Charset.forName(XMS_PROPERTIES_ENCODING).newEncoder(); @@ -571,7 +577,7 @@ final class AbfsHttpServiceImpl implements AbfsHttpService { } private Hashtable parseCommaSeparatedXmsProperties(String xMsProperties) throws - InvalidFileSystemPropertyException, InvalidAbfsRestOperationException { + InvalidFileSystemPropertyException, InvalidAbfsRestOperationException { Hashtable properties = new Hashtable<>(); final CharsetDecoder decoder = Charset.forName(XMS_PROPERTIES_ENCODING).newDecoder(); @@ -633,15 +639,15 @@ final class AbfsHttpServiceImpl implements AbfsHttpService { private final String version; VersionedFileStatus( - final String owner, final String group, - final long length, final boolean isdir, final int blockReplication, - final long blocksize, final long modificationTime, final Path path, - String version) { + final String owner, final String group, + final long length, final boolean isdir, final int blockReplication, + final long blocksize, final long modificationTime, final Path path, + String version) { super(length, isdir, blockReplication, blocksize, modificationTime, 0, - new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL), - owner, - group, - path); + new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL), + owner, + group, + path); this.version = version; } @@ -690,4 +696,6 @@ final class AbfsHttpServiceImpl implements AbfsHttpService { return this.version; } } -} \ No newline at end of file + + +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.java deleted file mode 100644 index 694d9023c59..00000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ServiceResolutionException.java +++ /dev/null @@ -1,36 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.contracts.exceptions; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl; - -/** - * Thrown a service is either not configured to be injected or the service is not existing. - * For service registration - * @see AbfsServiceProviderImpl - */ -@InterfaceAudience.Public -@InterfaceStability.Evolving -public final class ServiceResolutionException extends AzureBlobFileSystemException { - public ServiceResolutionException(String serviceName, Exception innerException) { - super(String.format("%s cannot be resolved.", serviceName), innerException); - } -} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.java deleted file mode 100644 index c433f9a216f..00000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpClientFactory.java +++ /dev/null @@ -1,39 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.hadoop.fs.azurebfs.contracts.services; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; -import org.apache.hadoop.fs.azurebfs.services.AbfsClient; - -/** - * AbfsClient factory. - */ -@InterfaceAudience.Public -@InterfaceStability.Evolving -public interface AbfsHttpClientFactory extends InjectableService { - /** - * Creates and configures an instance of new AbfsClient - * @return AbfsClient instance - */ - AbfsClient create(AzureBlobFileSystem fs) throws AzureBlobFileSystemException; -} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java deleted file mode 100644 index 3107fa3b01f..00000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsHttpService.java +++ /dev/null @@ -1,162 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.contracts.services; - -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Hashtable; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.FileStatus; -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.AzureBlobFileSystemException; - -/** - * File System http service to provide network calls for file system operations. - */ -@InterfaceAudience.Public -@InterfaceStability.Evolving -public interface AbfsHttpService extends InjectableService { - /** - * Gets filesystem properties on the Azure service. - * @param azureBlobFileSystem filesystem to get the properties. - * @return Hashtable hash table containing all the filesystem properties. - */ - Hashtable getFilesystemProperties(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException; - - - /** - * Sets filesystem properties on the Azure service. - * @param azureBlobFileSystem filesystem to get the properties. - * @param properties file system properties to set. - */ - void setFilesystemProperties(AzureBlobFileSystem azureBlobFileSystem, Hashtable properties) throws - AzureBlobFileSystemException; - - /** - * Gets path properties on the Azure service. - * @param azureBlobFileSystem filesystem to get the properties of the path. - * @param path path to get properties. - * @return Hashtable hash table containing all the path properties. - */ - Hashtable getPathProperties(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException; - - /** - * Sets path properties on the Azure service. - * @param azureBlobFileSystem filesystem to get the properties of the path. - * @param path path to set properties. - * @param properties hash table containing all the path properties. - */ - void setPathProperties(AzureBlobFileSystem azureBlobFileSystem, Path path, Hashtable properties) throws - AzureBlobFileSystemException; - - /** - * Creates filesystem on the Azure service. - * @param azureBlobFileSystem filesystem to be created. - */ - void createFilesystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException; - - /** - * Deletes filesystem on the Azure service. - * @param azureBlobFileSystem filesystem to be deleted. - */ - void deleteFilesystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException; - - /** - * Creates a file on the Azure service. - * @param azureBlobFileSystem filesystem to create file or directory. - * @param path path of the file to be created. - * @param overwrite should overwrite. - * @return OutputStream stream to the file. - */ - OutputStream createFile(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean overwrite) throws AzureBlobFileSystemException; - - /** - * Creates a directory on the Azure service. - * @param azureBlobFileSystem filesystem to create file or directory. - * @param path path of the directory to be created. - * @return OutputStream stream to the file. - */ - Void createDirectory(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException; - - /** - * Opens a file to read and returns the stream. - * @param azureBlobFileSystem filesystem to read a file from. - * @param path file path to read. - * @return InputStream a stream to the file to read. - */ - InputStream openFileForRead(AzureBlobFileSystem azureBlobFileSystem, Path path, FileSystem.Statistics statistics) throws AzureBlobFileSystemException; - - /** - * Opens a file to write and returns the stream. - * @param azureBlobFileSystem filesystem to write a file to. - * @param path file path to write. - * @param overwrite should overwrite. - * @return OutputStream a stream to the file to write. - */ - OutputStream openFileForWrite(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean overwrite) throws AzureBlobFileSystemException; - - /** - * Renames a file or directory from source to destination. - * @param azureBlobFileSystem filesystem to rename a path. - * @param source source path. - * @param destination destination path. - */ - void rename(AzureBlobFileSystem azureBlobFileSystem, Path source, Path destination) throws AzureBlobFileSystemException; - - /** - * Deletes a file or directory. - * @param azureBlobFileSystem filesystem to delete the path. - * @param path file path to be deleted. - * @param recursive true if path is a directory and recursive deletion is desired. - */ - void delete(AzureBlobFileSystem azureBlobFileSystem, Path path, boolean recursive) throws AzureBlobFileSystemException; - - /** - * Gets path's status under the provided path on the Azure service. - * @param azureBlobFileSystem filesystem to perform the get file status operation. - * @param path path delimiter. - * @return FileStatus FileStatus of the path in the file system. - */ - FileStatus getFileStatus(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException; - - /** - * Lists all the paths under the provided path on the Azure service. - * @param azureBlobFileSystem filesystem to perform the list operation. - * @param path path delimiter. - * @return FileStatus[] list of all paths in the file system. - */ - FileStatus[] listStatus(AzureBlobFileSystem azureBlobFileSystem, Path path) throws AzureBlobFileSystemException; - - /** - * Closes the client to filesystem to Azure service. - * @param azureBlobFileSystem filesystem to perform the list operation. - */ - void closeFileSystem(AzureBlobFileSystem azureBlobFileSystem) throws AzureBlobFileSystemException; - - /** - * Checks for the given path if it is marked as atomic rename directory or not. - * @param key - * @return True if the given path is listed under atomic rename property otherwise False. - */ - boolean isAtomicRenameKey(String key); -} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java deleted file mode 100644 index bd98baec4e7..00000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AbfsServiceProvider.java +++ /dev/null @@ -1,40 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.contracts.services; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ServiceResolutionException; - -/** - * Dependency injected Azure Storage services provider interface. - */ -@InterfaceAudience.Public -@InterfaceStability.Evolving -public interface AbfsServiceProvider { - /** - * Returns an instance of resolved injectable service by class name. - * The injectable service must be configured first to be resolvable. - * @param clazz the injectable service which is expected to be returned. - * @param The type of injectable service. - * @return T instance - * @throws ServiceResolutionException if the service is not resolvable. - */ - T get(Class clazz) throws ServiceResolutionException; -} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java deleted file mode 100644 index ee40c9d4951..00000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ConfigurationService.java +++ /dev/null @@ -1,143 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.contracts.services; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException; - -/** - * Configuration service collects required Azure Hadoop configurations and provides it to the consumers. - */ -@InterfaceAudience.Public -@InterfaceStability.Evolving -public interface ConfigurationService extends InjectableService { - /** - * Checks if ABFS is running from Emulator; - * @return is emulator mode. - */ - boolean isEmulator(); - - /** - * Retrieves storage secure mode from Hadoop configuration; - * @return storage secure mode; - */ - boolean isSecureMode(); - - /** - * Retrieves storage account key for provided account name from Hadoop configuration. - * @param accountName the account name to retrieve the key. - * @return storage account key; - */ - String getStorageAccountKey(String accountName) throws ConfigurationPropertyNotFoundException; - - /** - * Returns Hadoop configuration. - * @return Hadoop configuration. - */ - Configuration getConfiguration(); - - /** - * Retrieves configured write buffer size - * @return the size of the write buffer - */ - int getWriteBufferSize(); - - /** - * Retrieves configured read buffer size - * @return the size of the read buffer - */ - int getReadBufferSize(); - - /** - * Retrieves configured min backoff interval - * @return min backoff interval - */ - int getMinBackoffIntervalMilliseconds(); - - /** - * Retrieves configured max backoff interval - * @return max backoff interval - */ - int getMaxBackoffIntervalMilliseconds(); - - /** - * Retrieves configured backoff interval - * @return backoff interval - */ - int getBackoffIntervalMilliseconds(); - - /** - * Retrieves configured num of retries - * @return num of retries - */ - int getMaxIoRetries(); - - /** - * Retrieves configured azure block size - * @return azure block size - */ - long getAzureBlockSize(); - - /** - * Retrieves configured azure block location host - * @return azure block location host - */ - String getAzureBlockLocationHost(); - - /** - * Retrieves configured number of concurrent threads - * @return number of concurrent write threads - */ - int getMaxConcurrentWriteThreads(); - - /** - * Retrieves configured number of concurrent threads - * @return number of concurrent read threads - */ - int getMaxConcurrentReadThreads(); - - /** - * Retrieves configured boolean for tolerating out of band writes to files - * @return configured boolean for tolerating out of band writes to files - */ - boolean getTolerateOobAppends(); - - /** - * Retrieves the comma-separated list of directories to receive special treatment so that folder - * rename is made atomic. The default value for this setting is just '/hbase'. - * Example directories list : /hbase,/data - * @see AtomicRenameProperty - * @return atomic rename directories - */ - String getAzureAtomicRenameDirs(); - - /** - * Retrieves configured boolean for creating remote file system during initialization - * @return configured boolean for creating remote file system during initialization - */ - boolean getCreateRemoteFileSystemDuringInitialization(); - - /** - * Retrieves configured value of read ahead queue - * @return depth of read ahead - */ - int getReadAheadQueueDepth(); -} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.java deleted file mode 100644 index 8b3801f8e41..00000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/InjectableService.java +++ /dev/null @@ -1,30 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.contracts.services; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; - -/** - * Marker interface for all the injectable services. - */ -@InterfaceAudience.Public -@InterfaceStability.Evolving -public interface InjectableService { -} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.java deleted file mode 100644 index 267d11f1144..00000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/TracingService.java +++ /dev/null @@ -1,66 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.contracts.services; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; -import org.apache.htrace.core.SpanId; -import org.apache.htrace.core.TraceScope; - -/** - * Azure Blob File System tracing service. - */ -@InterfaceAudience.Private -@InterfaceStability.Evolving -public interface TracingService extends InjectableService { - /** - * Creates a {@link TraceScope} object with the provided description. - * @param description the trace description. - * @return created traceScope. - */ - TraceScope traceBegin(String description); - - /** - * Creates a {@link TraceScope} object with the provided description. - * @param description the trace description. - * @param parentSpanId the span id of the parent trace scope. - * @return create traceScope - */ - TraceScope traceBegin(String description, SpanId parentSpanId); - - /** - * Gets current thread latest generated traceScope id. - * @return current thread latest generated traceScope id. - */ - SpanId getCurrentTraceScopeSpanId(); - - /** - * Appends the provided exception to the trace scope. - * @param traceScope the scope which exception needs to be attached to. - * @param azureBlobFileSystemException the exception to be attached to the scope. - */ - void traceException(TraceScope traceScope, AzureBlobFileSystemException azureBlobFileSystemException); - - /** - * Ends the provided traceScope. - * @param traceScope the scope that needs to be ended. - */ - void traceEnd(TraceScope traceScope); -} \ No newline at end of file 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 c17a5c18dfc..a78e7af4660 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 @@ -28,7 +28,6 @@ import java.util.Locale; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; -import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams; @@ -45,17 +44,17 @@ public class AbfsClient { private final String xMsVersion = "2018-03-28"; private final ExponentialRetryPolicy retryPolicy; private final String filesystem; - private final ConfigurationService configurationService; + private final AbfsConfiguration abfsConfiguration; private final String userAgent; public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, - final ConfigurationService configurationService, + final AbfsConfiguration abfsConfiguration, final ExponentialRetryPolicy exponentialRetryPolicy) { this.baseUrl = baseUrl; this.sharedKeyCredentials = sharedKeyCredentials; String baseUrlString = baseUrl.toString(); this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(AbfsHttpConstants.FORWARD_SLASH) + 1); - this.configurationService = configurationService; + this.abfsConfiguration = abfsConfiguration; this.retryPolicy = exponentialRetryPolicy; this.userAgent = initializeUserAgent(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ConfigurationServiceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConfiguration.java similarity index 96% rename from hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ConfigurationServiceImpl.java rename to hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConfiguration.java index 568ee5da63a..8def1bba20a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ConfigurationServiceImpl.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConfiguration.java @@ -22,8 +22,6 @@ import java.lang.reflect.Field; import java.util.Map; import com.google.common.annotations.VisibleForTesting; -import com.google.inject.Inject; -import com.google.inject.Singleton; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -37,17 +35,18 @@ import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidati import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; -import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; import org.apache.hadoop.fs.azurebfs.diagnostics.Base64StringConfigurationBasicValidator; import org.apache.hadoop.fs.azurebfs.diagnostics.BooleanConfigurationBasicValidator; import org.apache.hadoop.fs.azurebfs.diagnostics.IntegerConfigurationBasicValidator; import org.apache.hadoop.fs.azurebfs.diagnostics.LongConfigurationBasicValidator; import org.apache.hadoop.fs.azurebfs.diagnostics.StringConfigurationBasicValidator; -@Singleton +/** + * Configuration for Azure Blob FileSystem. + */ @InterfaceAudience.Private @InterfaceStability.Evolving -class ConfigurationServiceImpl implements ConfigurationService { +public class AbfsConfiguration{ private final Configuration configuration; private final boolean isSecure; @@ -118,8 +117,7 @@ class ConfigurationServiceImpl implements ConfigurationService { private Map storageAccountKeys; - @Inject - ConfigurationServiceImpl(final Configuration configuration) throws IllegalAccessException, InvalidConfigurationValueException { + public AbfsConfiguration(final Configuration configuration) throws IllegalAccessException, InvalidConfigurationValueException { this.configuration = configuration; this.isSecure = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_SECURE_MODE, false); @@ -141,17 +139,14 @@ class ConfigurationServiceImpl implements ConfigurationService { } } - @Override public boolean isEmulator() { return this.getConfiguration().getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false); } - @Override public boolean isSecureMode() { return this.isSecure; } - @Override public String getStorageAccountKey(final String accountName) throws ConfigurationPropertyNotFoundException { String accountKey = this.storageAccountKeys.get(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + accountName); if (accountKey == null) { @@ -161,77 +156,62 @@ class ConfigurationServiceImpl implements ConfigurationService { return accountKey; } - @Override public Configuration getConfiguration() { return this.configuration; } - @Override public int getWriteBufferSize() { return this.writeBufferSize; } - @Override public int getReadBufferSize() { return this.readBufferSize; } - @Override public int getMinBackoffIntervalMilliseconds() { return this.minBackoffInterval; } - @Override public int getMaxBackoffIntervalMilliseconds() { return this.maxBackoffInterval; } - @Override public int getBackoffIntervalMilliseconds() { return this.backoffInterval; } - @Override public int getMaxIoRetries() { return this.maxIoRetries; } - @Override public long getAzureBlockSize() { return this.azureBlockSize; } - @Override public String getAzureBlockLocationHost() { return this.azureBlockLocationHost; } - @Override public int getMaxConcurrentWriteThreads() { return this.maxConcurrentWriteThreads; } - @Override public int getMaxConcurrentReadThreads() { return this.maxConcurrentReadThreads; } - @Override public boolean getTolerateOobAppends() { return this.tolerateOobAppends; } - @Override public String getAzureAtomicRenameDirs() { return this.azureAtomicDirs; } - @Override public boolean getCreateRemoteFileSystemDuringInitialization() { return this.createRemoteFileSystemDuringInitialization; } - @Override public int getReadAheadQueueDepth() { return this.readAheadQueueDepth; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.java deleted file mode 100644 index 9e4c27bdcfc..00000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientFactoryImpl.java +++ /dev/null @@ -1,116 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.services; - -import java.net.MalformedURLException; -import java.net.URI; -import java.net.URL; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.inject.Inject; -import com.google.inject.Singleton; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; -import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; -import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; -import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory; -import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; -import org.apache.http.client.utils.URIBuilder; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; - -@Singleton -@InterfaceAudience.Private -@InterfaceStability.Evolving -class AbfsHttpClientFactoryImpl implements AbfsHttpClientFactory { - private final ConfigurationService configurationService; - - @Inject - AbfsHttpClientFactoryImpl( - final ConfigurationService configurationService) { - - Preconditions.checkNotNull(configurationService, "configurationService"); - - this.configurationService = configurationService; - } - - @VisibleForTesting - URIBuilder getURIBuilder(final String hostName, final FileSystem fs) { - final AzureBlobFileSystem abfs = (AzureBlobFileSystem) fs; - - String scheme = FileSystemUriSchemes.HTTP_SCHEME; - - if (abfs.isSecure()) { - scheme = FileSystemUriSchemes.HTTPS_SCHEME; - } - - final URIBuilder uriBuilder = new URIBuilder(); - uriBuilder.setScheme(scheme); - uriBuilder.setHost(hostName); - - return uriBuilder; - } - - public AbfsClient create(final AzureBlobFileSystem fs) throws AzureBlobFileSystemException { - final URI uri = fs.getUri(); - final String authority = uri.getRawAuthority(); - if (null == authority) { - throw new InvalidUriAuthorityException(uri.toString()); - } - - if (!authority.contains(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER)) { - throw new InvalidUriAuthorityException(uri.toString()); - } - - final String[] authorityParts = authority.split(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2); - - if (authorityParts.length < 2 || "".equals(authorityParts[0])) { - final String errMsg = String - .format("URI '%s' has a malformed authority, expected container name. " - + "Authority takes the form "+ FileSystemUriSchemes.ABFS_SCHEME + "://[@]", - uri.toString()); - throw new InvalidUriException(errMsg); - } - - final String fileSystemName = authorityParts[0]; - final String accountName = authorityParts[1]; - - final URIBuilder uriBuilder = getURIBuilder(accountName, fs); - - final String url = uriBuilder.toString() + AbfsHttpConstants.FORWARD_SLASH + fileSystemName; - - URL baseUrl; - try { - baseUrl = new URL(url); - } catch (MalformedURLException e) { - throw new InvalidUriException(String.format("URI '%s' is malformed", uri.toString())); - } - - SharedKeyCredentials creds = - new SharedKeyCredentials(accountName.substring(0, accountName.indexOf(AbfsHttpConstants.DOT)), - this.configurationService.getStorageAccountKey(accountName)); - - return new AbfsClient(baseUrl, creds, configurationService, new ExponentialRetryPolicy()); - } -} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java deleted file mode 100644 index 1cbf6b5c3a2..00000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceInjectorImpl.java +++ /dev/null @@ -1,81 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.services; - -import java.util.HashMap; -import java.util.Map; - -import com.google.inject.AbstractModule; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory; -import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService; -import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; -import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService; - -/** - * This class is responsible to configure all the services used by Azure Blob File System. - */ -@InterfaceAudience.Private -@InterfaceStability.Evolving -class AbfsServiceInjectorImpl extends AbstractModule { - private final Configuration configuration; - private final Map providers; - private final Map instances; - - AbfsServiceInjectorImpl(Configuration configuration) { - this.providers = new HashMap<>(); - this.instances = new HashMap<>(); - this.configuration = configuration; - - this.instances.put(Configuration.class, this.configuration); - - this.providers.put(ConfigurationService.class, ConfigurationServiceImpl.class); - - this.providers.put(AbfsHttpService.class, AbfsHttpServiceImpl.class); - this.providers.put(AbfsHttpClientFactory.class, AbfsHttpClientFactoryImpl.class); - - this.providers.put(TracingService.class, TracingServiceImpl.class); - } - - @Override - protected void configure() { - for (Map.Entry entrySet : this.instances.entrySet()) { - bind(entrySet.getKey()).toInstance(entrySet.getValue()); - } - - for (Map.Entry entrySet : this.providers.entrySet()) { - bind(entrySet.getKey()).to(entrySet.getValue()); - } - } - - protected Configuration getConfiguration() { - return this.configuration; - } - - protected Map getProviders() { - return this.providers; - } - - protected Map getInstances() { - return this.instances; - } -} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.java deleted file mode 100644 index 8560620a69b..00000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsServiceProviderImpl.java +++ /dev/null @@ -1,96 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.services; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.inject.Guice; -import com.google.inject.Injector; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ServiceResolutionException; -import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsServiceProvider; -import org.apache.hadoop.fs.azurebfs.contracts.services.InjectableService; - -/** - * Dependency injected Azure Storage services provider. - */ -@InterfaceAudience.Public -@InterfaceStability.Evolving -public final class AbfsServiceProviderImpl implements AbfsServiceProvider { - private static AbfsServiceProviderImpl abfsServiceProvider; - private final Injector abfsServiceInjector; - - private AbfsServiceProviderImpl(final Configuration configuration) { - this.abfsServiceInjector = Guice.createInjector(new AbfsServiceInjectorImpl(Preconditions.checkNotNull(configuration, "configuration"))); - } - - @VisibleForTesting - private AbfsServiceProviderImpl(final Injector abfsServiceInjector) { - Preconditions.checkNotNull(abfsServiceInjector, "abfsServiceInjector"); - this.abfsServiceInjector = abfsServiceInjector; - } - - /** - * Create an instance or returns existing instance of service provider. - * This method must be marked as synchronized to ensure thread-safety. - * @param configuration hadoop configuration. - * @return AbfsServiceProvider the service provider instance. - */ - public static synchronized AbfsServiceProvider create(final Configuration configuration) { - if (abfsServiceProvider == null) { - abfsServiceProvider = new AbfsServiceProviderImpl(configuration); - } - - return abfsServiceProvider; - } - - /** - * Returns current instance of service provider. - * @return AbfsServiceProvider the service provider instance. - */ - public static AbfsServiceProvider instance() { - return abfsServiceProvider; - } - - @VisibleForTesting - static synchronized AbfsServiceProvider create(Injector serviceInjector) { - abfsServiceProvider = new AbfsServiceProviderImpl(serviceInjector); - return abfsServiceProvider; - } - - /** - * Returns an instance of resolved injectable service by class name. - * The injectable service must be configured first to be resolvable. - * @param clazz the injectable service which is expected to be returned. - * @param The type of injectable service. - * @return T instance - * @throws ServiceResolutionException if the service is not resolvable. - */ - @Override - public T get(final Class clazz) throws ServiceResolutionException { - try { - return this.abfsServiceInjector.getInstance(clazz); - } catch (Exception ex) { - throw new ServiceResolutionException(clazz.getSimpleName(), ex); - } - } -} \ No newline at end of file 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 0c9261216ac..54aa1abd698 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 @@ -21,7 +21,10 @@ package org.apache.hadoop.fs.azurebfs.services; import java.util.Random; import java.net.HttpURLConnection; -class ExponentialRetryPolicy { +/** + * Retry policy used by AbfsClient. + * */ +public class ExponentialRetryPolicy { /** * Represents the default number of retry attempts. */ @@ -83,7 +86,7 @@ class ExponentialRetryPolicy { /** * Initializes a new instance of the {@link ExponentialRetryPolicy} class. */ - ExponentialRetryPolicy() { + public ExponentialRetryPolicy() { this(DEFAULT_CLIENT_RETRY_COUNT, DEFAULT_MIN_BACKOFF, DEFAULT_MAX_BACKOFF, DEFAULT_CLIENT_BACKOFF); } @@ -96,7 +99,7 @@ class ExponentialRetryPolicy { * @param deltaBackoff The value that will be used to calculate a random delta in the exponential delay * between retries. */ - ExponentialRetryPolicy(final int retryCount, final int minBackoff, final int maxBackoff, final int deltaBackoff) { + public ExponentialRetryPolicy(final int retryCount, final int minBackoff, final int maxBackoff, final int deltaBackoff) { this.retryCount = retryCount; this.minBackoff = minBackoff; this.maxBackoff = maxBackoff; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.java deleted file mode 100644 index 99190e6d4a7..00000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/LoggerSpanReceiver.java +++ /dev/null @@ -1,74 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.services; - -import java.io.IOException; - -import com.google.common.base.Preconditions; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; -import org.apache.htrace.core.HTraceConfiguration; -import org.apache.htrace.core.Span; -import org.apache.htrace.core.SpanReceiver; -import org.apache.htrace.fasterxml.jackson.core.JsonProcessingException; -import org.apache.htrace.fasterxml.jackson.databind.ObjectMapper; -import org.apache.htrace.fasterxml.jackson.databind.ObjectWriter; -import org.apache.htrace.fasterxml.jackson.databind.SerializationFeature; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * LoggerSpanReceiver is a layer between HTrace and log4j only used for {@link org.apache.hadoop.fs.azurebfs.contracts.services.TracingService} - */ -@InterfaceAudience.Public -@InterfaceStability.Evolving -public class LoggerSpanReceiver extends SpanReceiver { - private static final ObjectWriter JSON_WRITER = - new ObjectMapper() - .configure(SerializationFeature.INDENT_OUTPUT, true) - .configure(SerializationFeature.WRITE_BIGDECIMAL_AS_PLAIN, true) - .configure(SerializationFeature.WRITE_EMPTY_JSON_ARRAYS, false) - .configure(SerializationFeature.USE_EQUALITY_FOR_OBJECT_ID, false) - .writer(); - - public LoggerSpanReceiver(HTraceConfiguration hTraceConfiguration) { - Preconditions.checkNotNull(hTraceConfiguration, "hTraceConfiguration"); - } - - @Override - public void receiveSpan(final Span span) { - String jsonValue; - - Logger logger = LoggerFactory.getLogger(AzureBlobFileSystem.class); - - try { - jsonValue = JSON_WRITER.writeValueAsString(span); - logger.trace(jsonValue); - } catch (JsonProcessingException e) { - logger.error("Json processing error: " + e.getMessage()); - } - } - - @Override - public void close() throws IOException { - // No-Op - } -} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.java deleted file mode 100644 index 57b6463322b..00000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TracingServiceImpl.java +++ /dev/null @@ -1,134 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.services; - -import java.io.PrintWriter; -import java.io.StringWriter; -import java.util.Objects; - -import com.google.common.base.Preconditions; -import com.google.inject.Inject; -import com.google.inject.Singleton; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; -import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService; -import org.apache.htrace.core.HTraceConfiguration; -import org.apache.htrace.core.Sampler; -import org.apache.htrace.core.SpanId; -import org.apache.htrace.core.TraceScope; -import org.apache.htrace.core.Tracer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Singleton -@InterfaceAudience.Private -@InterfaceStability.Evolving -final class TracingServiceImpl implements TracingService { - private static final Logger LOG = LoggerFactory.getLogger(TracingService.class); - - private final Tracer tracer; - private final ThreadLocal currentScopeId; - - @Inject - TracingServiceImpl( - final Configuration configuration) { - Preconditions.checkNotNull(configuration, "configuration"); - - this.currentScopeId = new ThreadLocal<>(); - - this.tracer = new Tracer.Builder(TracingService.class.getSimpleName()). - conf(new HTraceConfiguration() { - @Override - public String get(String key) { - if (Objects.equals(key, Tracer.SPAN_RECEIVER_CLASSES_KEY)) { - return LoggerSpanReceiver.class.getName(); - } - return null; - } - - @Override - public String get(String key, String defaultValue) { - String value = get(key); - if (value != null) { - return value; - } - return defaultValue; - } - }). - build(); - - this.tracer.addSampler(Sampler.ALWAYS); - } - - @Override - public TraceScope traceBegin(String description) { - if (this.LOG.isTraceEnabled()) { - TraceScope traceScope = this.tracer.newScope(description); - this.currentScopeId.set(traceScope.getSpanId()); - return traceScope; - } - - return null; - } - - @Override - public TraceScope traceBegin(String description, SpanId parentSpanId) { - if (this.LOG.isTraceEnabled()) { - TraceScope traceScope = this.tracer.newScope(description, parentSpanId); - this.currentScopeId.set(traceScope.getSpanId()); - return traceScope; - } - - return null; - } - - @Override - public void traceException(TraceScope traceScope, AzureBlobFileSystemException azureBlobFileSystemException) { - if (this.LOG.isTraceEnabled()) { - Preconditions.checkNotNull(traceScope, "traceScope"); - Preconditions.checkNotNull(azureBlobFileSystemException, "azureBlobFileSystemException"); - - StringWriter stringWriter = new StringWriter(); - PrintWriter printWriter = new PrintWriter(stringWriter); - azureBlobFileSystemException.printStackTrace(printWriter); - printWriter.flush(); - - traceScope.addKVAnnotation("Exception", stringWriter.toString()); - } - } - - @Override - public SpanId getCurrentTraceScopeSpanId() { - return this.currentScopeId.get(); - } - - @Override - public void traceEnd(TraceScope traceScope) { - if (this.LOG.isTraceEnabled()) { - Preconditions.checkNotNull(traceScope, "traceScope"); - - SpanId[] parents = traceScope.getSpan().getParents(); - this.currentScopeId.set(parents != null && parents.length > 0 ? parents[parents.length - 1] : null); - traceScope.close(); - } - } -} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java index 5ec1e2ed424..74a530c9cce 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java @@ -23,11 +23,9 @@ import java.util.UUID; import java.util.concurrent.Callable; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; -import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl; import org.junit.After; import org.junit.Assert; import org.junit.Before; -import org.mockito.internal.util.MockUtil; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; @@ -38,12 +36,6 @@ import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation; import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; -import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpClientFactory; -import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService; -import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; -import org.apache.hadoop.fs.azurebfs.services.MockAbfsHttpClientFactoryImpl; -import org.apache.hadoop.fs.azurebfs.services.MockAbfsServiceInjectorImpl; -import org.apache.hadoop.fs.azurebfs.services.MockServiceProviderImpl; import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -54,7 +46,6 @@ import static org.junit.Assume.assumeNotNull; * Provide dependencies for AzureBlobFileSystem tests. */ public abstract class DependencyInjectedTest { - private final MockAbfsServiceInjectorImpl mockServiceInjector; private final boolean isEmulator; private NativeAzureFileSystem wasb; private String abfsScheme; @@ -64,21 +55,19 @@ public abstract class DependencyInjectedTest { private String accountName; private String testUrl; + public static final String TEST_CONTAINER_PREFIX = "abfs-testcontainer-"; + public DependencyInjectedTest(final boolean secure) { this(secure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME); } - public MockAbfsServiceInjectorImpl getMockServiceInjector() { - return this.mockServiceInjector; - } - protected DependencyInjectedTest() { this(FileSystemUriSchemes.ABFS_SCHEME); } private DependencyInjectedTest(final String scheme) { abfsScheme = scheme; - fileSystemName = UUID.randomUUID().toString(); + fileSystemName = TEST_CONTAINER_PREFIX + UUID.randomUUID().toString(); configuration = new Configuration(); configuration.addResource("azure-bfs-test.xml"); @@ -98,18 +87,14 @@ public abstract class DependencyInjectedTest { this.testUrl = defaultUri.toString(); configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); configuration.setBoolean(ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true); - this.mockServiceInjector = new MockAbfsServiceInjectorImpl(configuration); this.isEmulator = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false); this.accountName = this.configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME); } @Before public void initialize() throws Exception { - if (this.isEmulator) { - this.mockServiceInjector.replaceProvider(AbfsHttpClientFactory.class, MockAbfsHttpClientFactoryImpl.class); - } - - MockServiceProviderImpl.create(this.mockServiceInjector); + //Create filesystem first to make sure getWasbFileSystem() can return an existed filesystem. + this.getFileSystem(); if (!this.isEmulator) { final URI wasbUri = new URI(abfsUrlToWasbUrl(this.getTestUrl())); @@ -133,28 +118,24 @@ public abstract class DependencyInjectedTest { FileSystem.closeAll(); final AzureBlobFileSystem fs = this.getFileSystem(); - final AbfsHttpService abfsHttpService = AbfsServiceProviderImpl.instance().get(AbfsHttpService.class); - abfsHttpService.deleteFilesystem(fs); + final AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); + abfsStore.deleteFilesystem(); - if (!(new MockUtil().isMock(abfsHttpService))) { - AbfsRestOperationException ex = intercept( - AbfsRestOperationException.class, - new Callable() { - @Override - public Void call() throws Exception { - abfsHttpService.getFilesystemProperties(fs); - return null; - } - }); + AbfsRestOperationException ex = intercept( + AbfsRestOperationException.class, + new Callable() { + @Override + public Void call() throws Exception { + fs.getAbfsStore().getFilesystemProperties(); + return null; + } + }); - assertEquals(FILE_SYSTEM_NOT_FOUND.getStatusCode(), ex.getStatusCode()); - } + assertEquals(FILE_SYSTEM_NOT_FOUND.getStatusCode(), ex.getStatusCode()); } public AzureBlobFileSystem getFileSystem() throws Exception { - final Configuration configuration = AbfsServiceProviderImpl.instance().get(ConfigurationService.class).getConfiguration(); - final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(configuration); - return fs; + return (AzureBlobFileSystem) FileSystem.get(this.configuration); } protected NativeAzureFileSystem getWasbFileSystem() { 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 4985f58fdbf..ad22f999fe6 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 @@ -22,15 +22,12 @@ import java.io.IOException; import java.util.Arrays; import java.util.Random; - -import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl; import org.junit.Test; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; @@ -52,8 +49,6 @@ public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest { super(); Configuration configuration = this.getConfiguration(); configuration.set(ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH, "0"); - this.getMockServiceInjector().replaceInstance(Configuration.class, configuration); - } @Test @@ -82,7 +77,7 @@ public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest { @Test (expected = IOException.class) public void testOOBWrites() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - int readBufferSize = AbfsServiceProviderImpl.instance().get(ConfigurationService.class).getReadBufferSize(); + int readBufferSize = fs.getAbfsStore().getAbfsConfiguration().getReadBufferSize(); fs.create(TEST_FILE); FSDataOutputStream writeStream = fs.create(TEST_FILE); 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 94775875684..8b96c69c8fc 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 @@ -535,17 +535,16 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest { character = (character == 'z') ? 'a' : (char) ((int) character + 1); } - System.out.println(("Creating test file {} of size: {} " + TEST_FILE_PATH - + TEST_FILE_SIZE)); + System.out.println(String.format("Creating test file %s of size: %d ", TEST_FILE_PATH, TEST_FILE_SIZE)); ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); - try(FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) { + try (FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) { int bytesWritten = 0; while (bytesWritten < TEST_FILE_SIZE) { outputStream.write(buffer); bytesWritten += buffer.length; } - System.out.println("Closing stream {}" + outputStream); + System.out.println(String.format("Closing stream %s", outputStream)); ContractTestUtils.NanoTimer closeTimer = new ContractTestUtils.NanoTimer(); outputStream.close(); 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 aa30a85918c..29af1b8c8d9 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 @@ -22,18 +22,10 @@ import java.net.URI; import org.junit.Assert; import org.junit.Test; -import org.mockito.Mockito; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; -import org.apache.hadoop.fs.FileStatus; 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.services.AbfsHttpService; -import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl; - -import static org.mockito.Matchers.anyObject; -import static org.mockito.Mockito.doReturn; /** * Test AzureBlobFileSystem initialization. @@ -41,18 +33,11 @@ import static org.mockito.Mockito.doReturn; public class ITestFileSystemInitialization extends DependencyInjectedTest { public ITestFileSystemInitialization() { super(); - - this.getMockServiceInjector().removeProvider(AbfsHttpService.class); - this.getMockServiceInjector().replaceInstance(AbfsHttpService.class, Mockito.mock(AbfsHttpService.class)); } @Test public void ensureAzureBlobFileSystemIsInitialized() throws Exception { - doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah"))) - .when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class)) - .getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject()); - - final FileSystem fs = FileSystem.get(this.getConfiguration()); + final FileSystem fs = this.getFileSystem(); final String accountName = this.getAccountName(); final String filesystem = this.getFileSystemName(); @@ -62,16 +47,12 @@ public class ITestFileSystemInitialization extends DependencyInjectedTest { @Test public void ensureSecureAzureBlobFileSystemIsInitialized() throws Exception { - doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah"))) - .when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class)) - .getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject()); - final String accountName = this.getAccountName(); final String filesystem = this.getFileSystemName(); final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null); this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); - final FileSystem fs = FileSystem.get(this.getConfiguration()); + final FileSystem fs = this.getFileSystem(); Assert.assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null)); Assert.assertNotNull(fs.getWorkingDirectory()); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java similarity index 71% rename from hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java rename to hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java index 588df2095e4..62d967e2099 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpServiceImpl.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.azurebfs.services; +package org.apache.hadoop.fs.azurebfs; import java.util.Hashtable; @@ -28,19 +28,16 @@ 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.AzureBlobFileSystem; -import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest; -import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsHttpService; import static org.junit.Assert.assertEquals; /** - * Test AbfsHttpServiceImpl. + * Test FileSystemProperties. */ -public class ITestAbfsHttpServiceImpl extends DependencyInjectedTest { +public class ITestFileSystemProperties extends DependencyInjectedTest { private static final int TEST_DATA = 100; private static final Path TEST_PATH = new Path("/testfile"); - public ITestAbfsHttpServiceImpl() { + public ITestFileSystemProperties() { super(); } @@ -71,11 +68,11 @@ public class ITestAbfsHttpServiceImpl extends DependencyInjectedTest { @Ignore("JDK7 doesn't support PATCH, so PUT is used. Fix is applied in latest test tenant") public void testBase64FileSystemProperties() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: value }"); - AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setFilesystemProperties( - fs, properties); - Hashtable fetchedProperties = AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getFilesystemProperties(fs); + fs.getAbfsStore().setFilesystemProperties(properties); + Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties(); Assert.assertEquals(properties, fetchedProperties); } @@ -86,10 +83,9 @@ public class ITestAbfsHttpServiceImpl extends DependencyInjectedTest { final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueTest }"); fs.create(TEST_PATH); - AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setPathProperties( - fs, TEST_PATH, properties); + fs.getAbfsStore().setPathProperties(TEST_PATH, properties); Hashtable fetchedProperties = - AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getPathProperties(fs, TEST_PATH); + fs.getAbfsStore().getPathProperties(TEST_PATH); Assert.assertEquals(properties, fetchedProperties); } @@ -99,9 +95,8 @@ public class ITestAbfsHttpServiceImpl extends DependencyInjectedTest { final AzureBlobFileSystem fs = this.getFileSystem(); final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueæ­² }"); - AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setFilesystemProperties( - fs, properties); - Hashtable fetchedProperties = AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getFilesystemProperties(fs); + fs.getAbfsStore().setFilesystemProperties(properties); + Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties(); Assert.assertEquals(properties, fetchedProperties); } @@ -112,10 +107,19 @@ public class ITestAbfsHttpServiceImpl extends DependencyInjectedTest { final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueTestå…© }"); fs.create(TEST_PATH); - AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).setPathProperties( - fs, TEST_PATH, properties); - Hashtable fetchedProperties = - AbfsServiceProviderImpl.instance().get(AbfsHttpService.class).getPathProperties(fs, TEST_PATH); + fs.getAbfsStore().setPathProperties(TEST_PATH, properties); + Hashtable fetchedProperties = fs.getAbfsStore().getPathProperties(TEST_PATH); + + Assert.assertEquals(properties, fetchedProperties); + } + + @Test + public void testSetFileSystemProperties() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + final Hashtable properties = new Hashtable<>(); + properties.put("containerForDevTest", "true"); + fs.getAbfsStore().setFilesystemProperties(properties); + Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties(); Assert.assertEquals(properties, fetchedProperties); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java index a55599b5006..ef61e5263aa 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java @@ -20,22 +20,14 @@ package org.apache.hadoop.fs.azurebfs; import java.net.URI; -import org.apache.hadoop.fs.azurebfs.services.AbfsServiceProviderImpl; import org.junit.Assert; import org.junit.Test; -import org.mockito.Mockito; import org.apache.hadoop.fs.AbstractFileSystem; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FileContext; -import org.apache.hadoop.fs.FileStatus; 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.services.AbfsHttpService; - -import static org.mockito.Matchers.anyObject; -import static org.mockito.Mockito.doReturn; /** * Test AzureBlobFileSystem registration. @@ -43,17 +35,10 @@ import static org.mockito.Mockito.doReturn; public class ITestFileSystemRegistration extends DependencyInjectedTest { public ITestFileSystemRegistration() throws Exception { super(); - - this.getMockServiceInjector().removeProvider(AbfsHttpService.class); - this.getMockServiceInjector().replaceInstance(AbfsHttpService.class, Mockito.mock(AbfsHttpService.class)); } @Test public void ensureAzureBlobFileSystemIsDefaultFileSystem() throws Exception { - doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah"))) - .when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class)) - .getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject()); - FileSystem fs = FileSystem.get(this.getConfiguration()); Assert.assertTrue(fs instanceof AzureBlobFileSystem); @@ -63,14 +48,10 @@ public class ITestFileSystemRegistration extends DependencyInjectedTest { @Test public void ensureSecureAzureBlobFileSystemIsDefaultFileSystem() throws Exception { - doReturn(new FileStatus(0, true, 0, 0, 0, new Path("/blah"))) - .when(AbfsServiceProviderImpl.instance().get(AbfsHttpService.class)) - .getFileStatus((AzureBlobFileSystem) anyObject(), (Path) anyObject()); - final String accountName = this.getAccountName(); - final String filesystem = this.getFileSystemName(); + final String fileSystemName = this.getFileSystemName(); - final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null); + final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, fileSystemName + "@" + accountName, null, null, null); this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); FileSystem fs = FileSystem.get(this.getConfiguration()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java index e148a055caa..9f3b4a71603 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.FileSystemContractBaseTest; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.junit.After; import org.junit.Before; import org.junit.Ignore; import org.junit.Test; @@ -37,7 +38,7 @@ public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest { private final DependencyInjectedContractTest dependencyInjectedContractTest; public ITestAzureBlobFileSystemBasics() throws Exception { - // If contract tests are running in parallel, some root level tests in this file will fail + // If all contract tests are running in parallel, some root level tests in FileSystemContractBaseTest will fail // due to the race condition. Hence for this contract test it should be tested in different container dependencyInjectedContractTest = new DependencyInjectedContractTest(false, false); } @@ -48,6 +49,14 @@ public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest { fs = this.dependencyInjectedContractTest.getFileSystem(); } + @After + public void testCleanup() throws Exception { + // This contract test is not using existing container for test, + // instead it creates its own temp container for test, hence we need to destroy + // it after the test. + this.dependencyInjectedContractTest.testCleanup(); + } + @Test public void testListOnFolderWithNoChildren() throws IOException { assertTrue(fs.mkdirs(path("testListStatus/c/1"))); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java index 1db93cb0f29..cabe0494df8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java @@ -25,7 +25,6 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest; -import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; import org.junit.Test; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE; @@ -52,12 +51,11 @@ public class ITestReadWriteAndSeek extends DependencyInjectedTest { private void testReadWriteAndSeek(int bufferSize) throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - final ConfigurationServiceImpl configurationservice = (ConfigurationServiceImpl) AbfsServiceProviderImpl.instance().get(ConfigurationService.class); + final AbfsConfiguration abfsConfiguration = new AbfsConfiguration(this.getConfiguration()); fs.create(TEST_PATH); - - configurationservice.setWriteBufferSize(bufferSize); - configurationservice.setReadBufferSize(bufferSize); + abfsConfiguration.setWriteBufferSize(bufferSize); + abfsConfiguration.setReadBufferSize(bufferSize); final FSDataOutputStream stream = fs.create(TEST_PATH); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java deleted file mode 100644 index e90eb224d05..00000000000 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestTracingServiceImpl.java +++ /dev/null @@ -1,79 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.services; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.log4j.AppenderSkeleton; -import org.apache.log4j.Level; -import org.apache.log4j.Logger; -import org.apache.log4j.spi.LoggingEvent; -import org.junit.Test; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest; -import org.apache.hadoop.fs.azurebfs.contracts.services.TracingService; -import org.apache.htrace.core.MilliSpan; -import org.apache.htrace.core.TraceScope; - -/** - * Test tracing service. - */ -public class ITestTracingServiceImpl extends DependencyInjectedTest { - private final List messageStorage; - - public ITestTracingServiceImpl() { - super(); - this.messageStorage = new ArrayList<>(); - } - - @Test - public void traceSerializationTest() throws Exception { - Logger rootLogger = Logger.getRootLogger(); - rootLogger.setLevel(Level.TRACE); - rootLogger.addAppender(new AppenderSkeleton() { - @Override - protected void append(LoggingEvent event) { - if (event.getLogger().getName().indexOf("AzureBlobFileSystem") != -1) { - messageStorage.add(event.getRenderedMessage()); - } - } - - @Override - public void close() { - - } - - @Override - public boolean requiresLayout() { - return false; - } - }); - - TracingService tracingService = new TracingServiceImpl(new Configuration()); - TraceScope traceScope = tracingService.traceBegin("Test Scope"); - traceScope.addTimelineAnnotation("Timeline Annotations"); - traceScope.addKVAnnotation("key", "value"); - traceScope.close(); - - // Should not throw exception. - MilliSpan.fromJson(messageStorage.get(0)); - } -} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java deleted file mode 100644 index 2f27b168b1f..00000000000 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsHttpClientFactoryImpl.java +++ /dev/null @@ -1,69 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.services; - -import com.google.common.annotations.VisibleForTesting; -import com.google.inject.Inject; -import com.google.inject.Singleton; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; -import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; -import org.apache.hadoop.fs.azurebfs.contracts.services.ConfigurationService; -import org.apache.hadoop.fs.azurebfs.utils.UriUtils; -import org.apache.http.client.utils.URIBuilder; - -/** - * Mock AbfsHttpClientFactoryImpl. - */ -@Singleton -@InterfaceAudience.Private -@InterfaceStability.Evolving -public class MockAbfsHttpClientFactoryImpl extends AbfsHttpClientFactoryImpl { - private final ConfigurationService configurationService; - - @Inject - MockAbfsHttpClientFactoryImpl( - final ConfigurationService configurationService) { - super(configurationService); - - this.configurationService = configurationService; - } - - @VisibleForTesting - URIBuilder getURIBuilder(final String hostName, final FileSystem fs) { - final URIBuilder uriBuilder = new URIBuilder(); - - final String testHost = this.configurationService.getConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_HOST_NAME); - final Integer testHostPort = this.configurationService.getConfiguration().getInt(TestConfigurationKeys.FS_AZURE_TEST_HOST_PORT, 80); - final String testAccount = this.configurationService.getConfiguration().get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME); - - String scheme = FileSystemUriSchemes.HTTP_SCHEME; - - uriBuilder.setScheme(scheme); - uriBuilder.setHost(testHost); - uriBuilder.setPort(testHostPort); - - uriBuilder.setPath("/" + UriUtils.extractRawAccountFromAccountName(testAccount) + "/"); - - return uriBuilder; - } -} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java deleted file mode 100644 index ff2fb2ad93c..00000000000 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockAbfsServiceInjectorImpl.java +++ /dev/null @@ -1,50 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.services; - -import org.apache.hadoop.conf.Configuration; - -/** - * Mock AbfsServiceInjectorImpl. - */ -public class MockAbfsServiceInjectorImpl extends AbfsServiceInjectorImpl { - public MockAbfsServiceInjectorImpl(Configuration configuration) { - super(configuration); - } - - public void replaceInstance(Class tInterface, Object object) { - this.removeInstance(tInterface); - this.removeProvider(tInterface); - this.getInstances().put(tInterface, object); - } - - public void removeInstance(Class tInterface) { - this.getInstances().remove(tInterface); - } - - public void replaceProvider(Class tInterface, Class tClazz) { - this.removeInstance(tInterface); - this.removeProvider(tInterface); - this.getProviders().put(tInterface, tClazz); - } - - public void removeProvider(Class tInterface) { - this.getProviders().remove(tInterface); - } -} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java deleted file mode 100644 index 5992f75666e..00000000000 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/MockServiceProviderImpl.java +++ /dev/null @@ -1,36 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.services; - -import com.google.inject.Guice; -import com.google.inject.Injector; - -/** - * Mock ABFS ServiceProviderImpl. - */ -public final class MockServiceProviderImpl { - public static void create(MockAbfsServiceInjectorImpl abfsServiceInjector) { - Injector injector = Guice.createInjector(abfsServiceInjector); - AbfsServiceProviderImpl.create(injector); - } - - private MockServiceProviderImpl() { - // no-op - } -} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java similarity index 80% rename from hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java rename to hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java index 688c5220ee8..441a35a10eb 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestConfigurationServiceFieldsValidation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java @@ -47,8 +47,8 @@ import org.junit.Test; /** * Test ConfigurationServiceFieldsValidation. */ -public class TestConfigurationServiceFieldsValidation { - private ConfigurationServiceImpl configService; +public class TestAbfsConfigurationFieldsValidation { + private AbfsConfiguration abfsConfiguration; private static final String INT_KEY= "intKey"; private static final String LONG_KEY= "longKey"; @@ -88,7 +88,7 @@ public class TestConfigurationServiceFieldsValidation { DefaultValue = false) private boolean boolField; - public TestConfigurationServiceFieldsValidation() throws Exception { + public TestAbfsConfigurationFieldsValidation() throws Exception { super(); Base64 base64 = new Base64(); this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8); @@ -101,7 +101,7 @@ public class TestConfigurationServiceFieldsValidation { configuration.set(BASE64_KEY, encodedString); configuration.set(BOOLEAN_KEY, "true"); configuration.set(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + "testaccount1.blob.core.windows.net", this.encodedAccountKey); - configService = new ConfigurationServiceImpl(configuration); + abfsConfiguration = new AbfsConfiguration(configuration); } @Test @@ -110,15 +110,15 @@ public class TestConfigurationServiceFieldsValidation { for (Field field : fields) { field.setAccessible(true); if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) { - assertEquals(TEST_INT, configService.validateInt(field)); + assertEquals(TEST_INT, abfsConfiguration.validateInt(field)); } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) { - assertEquals(DEFAULT_LONG, configService.validateLong(field)); + assertEquals(DEFAULT_LONG, abfsConfiguration.validateLong(field)); } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) { - assertEquals("stringValue", configService.validateString(field)); + assertEquals("stringValue", abfsConfiguration.validateString(field)); } else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) { - assertEquals(this.encodedString, configService.validateBase64String(field)); + assertEquals(this.encodedString, abfsConfiguration.validateBase64String(field)); } else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) { - assertEquals(true, configService.validateBoolean(field)); + assertEquals(true, abfsConfiguration.validateBoolean(field)); } } } @@ -126,24 +126,24 @@ public class TestConfigurationServiceFieldsValidation { @Test public void testConfigServiceImplAnnotatedFieldsInitialized() throws Exception { // test that all the ConfigurationServiceImpl annotated fields have been initialized in the constructor - assertEquals(DEFAULT_WRITE_BUFFER_SIZE, configService.getWriteBufferSize()); - assertEquals(DEFAULT_READ_BUFFER_SIZE, configService.getReadBufferSize()); - assertEquals(DEFAULT_MIN_BACKOFF_INTERVAL, configService.getMinBackoffIntervalMilliseconds()); - assertEquals(DEFAULT_MAX_BACKOFF_INTERVAL, configService.getMaxBackoffIntervalMilliseconds()); - assertEquals(DEFAULT_BACKOFF_INTERVAL, configService.getBackoffIntervalMilliseconds()); - assertEquals(DEFAULT_MAX_RETRY_ATTEMPTS, configService.getMaxIoRetries()); - assertEquals(MAX_AZURE_BLOCK_SIZE, configService.getAzureBlockSize()); - assertEquals(AZURE_BLOCK_LOCATION_HOST_DEFAULT, configService.getAzureBlockLocationHost()); + assertEquals(DEFAULT_WRITE_BUFFER_SIZE, abfsConfiguration.getWriteBufferSize()); + assertEquals(DEFAULT_READ_BUFFER_SIZE, abfsConfiguration.getReadBufferSize()); + assertEquals(DEFAULT_MIN_BACKOFF_INTERVAL, abfsConfiguration.getMinBackoffIntervalMilliseconds()); + assertEquals(DEFAULT_MAX_BACKOFF_INTERVAL, abfsConfiguration.getMaxBackoffIntervalMilliseconds()); + assertEquals(DEFAULT_BACKOFF_INTERVAL, abfsConfiguration.getBackoffIntervalMilliseconds()); + assertEquals(DEFAULT_MAX_RETRY_ATTEMPTS, abfsConfiguration.getMaxIoRetries()); + assertEquals(MAX_AZURE_BLOCK_SIZE, abfsConfiguration.getAzureBlockSize()); + assertEquals(AZURE_BLOCK_LOCATION_HOST_DEFAULT, abfsConfiguration.getAzureBlockLocationHost()); } @Test public void testGetAccountKey() throws Exception { - String accountKey = configService.getStorageAccountKey("testaccount1.blob.core.windows.net"); + String accountKey = abfsConfiguration.getStorageAccountKey("testaccount1.blob.core.windows.net"); assertEquals(this.encodedAccountKey, accountKey); } @Test (expected = ConfigurationPropertyNotFoundException.class) public void testGetAccountKeyWithNonExistingAccountName() throws Exception { - configService.getStorageAccountKey("bogusAccountName"); + abfsConfiguration.getStorageAccountKey("bogusAccountName"); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java new file mode 100644 index 00000000000..2716bffe1b4 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.fs.azurebfs.utils; + +import com.microsoft.azure.storage.CloudStorageAccount; +import com.microsoft.azure.storage.blob.CloudBlobClient; +import com.microsoft.azure.storage.blob.CloudBlobContainer; + +import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount; +import org.junit.Test; + +import static org.apache.hadoop.fs.azurebfs.DependencyInjectedTest.TEST_CONTAINER_PREFIX; + +/** + * If unit tests were interrupted and crushed accidentally, the test containers won't be deleted. + * In that case, dev can use this tool to list and delete all test containers. + * By default, all test container used in E2E tests sharing same prefix: "abfs-testcontainer-" + */ +public class CleanUpAbfsTestContainer { + @Test + public void testEnumContainers() throws Throwable { + int count = 0; + CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount(); + CloudBlobClient blobClient = storageAccount.createCloudBlobClient(); + Iterable containers + = blobClient.listContainers(TEST_CONTAINER_PREFIX); + for (CloudBlobContainer container : containers) { + count++; + System.out.println(String.format("Container %s URI %s", + container.getName(), + container.getUri())); + } + System.out.println(String.format("Found %d test containers", count)); + } + + @Test + public void testDeleteContainers() throws Throwable { + int count = 0; + CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount(); + CloudBlobClient blobClient = storageAccount.createCloudBlobClient(); + Iterable containers + = blobClient.listContainers(TEST_CONTAINER_PREFIX); + for (CloudBlobContainer container : containers) { + System.out.println(String.format("Container %s URI %s", + container.getName(), + container.getUri())); + if (container.deleteIfExists()) { + count++; + } + } + System.out.println(String.format("Deleted %s test containers", count)); + } +} From ce03a93f78c4d97ccb48a3906fcd77ad0ac756be Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Wed, 8 Aug 2018 18:52:12 +0000 Subject: [PATCH 03/29] HADOOP-15446. ABFS: tune imports & javadocs; stabilise tests. Contributed by Steve Loughran and Da Zhou. --- .../apache/hadoop/fs/RawLocalFileSystem.java | 2 +- .../src/main/resources/core-default.xml | 15 + .../site/markdown/filesystem/filesystem.md | 11 +- .../contract/AbstractContractAppendTest.java | 57 ++-- .../contract/AbstractContractConcatTest.java | 34 +- .../AbstractContractGetFileStatusTest.java | 26 +- .../contract/AbstractContractMkdirTest.java | 8 + .../fs/contract/AbstractFSContract.java | 2 - .../hadoop/fs/contract/ContractTestUtils.java | 19 +- .../org/apache/hadoop/fs/azurebfs/Abfs.java | 4 +- .../org/apache/hadoop/fs/azurebfs/Abfss.java | 4 +- .../fs/azurebfs/AzureBlobFileSystem.java | 98 +++--- .../fs/azurebfs/AzureBlobFileSystemStore.java | 151 ++++----- .../azurebfs/SecureAzureBlobFileSystem.java | 4 +- .../azurebfs/constants/AbfsHttpConstants.java | 2 +- .../constants/HttpHeaderConfigurations.java | 2 +- .../azurebfs/constants/HttpQueryParams.java | 2 +- .../ConfigurationValidationAnnotations.java | 14 +- .../diagnostics/ConfigurationValidator.java | 6 +- .../AzureBlobFileSystemException.java | 4 +- .../exceptions/InvalidUriException.java | 4 +- ...se64StringConfigurationBasicValidator.java | 2 +- .../BooleanConfigurationBasicValidator.java | 4 +- .../ConfigurationBasicValidator.java | 2 +- .../IntegerConfigurationBasicValidator.java | 2 +- .../LongConfigurationBasicValidator.java | 4 +- .../StringConfigurationBasicValidator.java | 4 +- .../fs/azurebfs/services/AbfsClient.java | 157 ++++----- .../azurebfs/services/AbfsHttpOperation.java | 6 +- .../fs/azurebfs/services/AbfsInputStream.java | 5 +- .../azurebfs/services/AbfsOutputStream.java | 125 +++---- .../azurebfs/services/AbfsRestOperation.java | 24 +- .../services/AbfsUriQueryBuilder.java | 6 +- .../services/ExponentialRetryPolicy.java | 2 +- .../fs/azurebfs/services/ReadBuffer.java | 4 +- .../azurebfs/services/ReadBufferManager.java | 56 ++-- .../azurebfs/services/ReadBufferWorker.java | 4 +- .../services/SharedKeyCredentials.java | 32 +- .../hadoop-azure/src/site/markdown/abfs.md | 72 +++++ .../src/site/markdown/testing_azure.md | 76 +++++ .../ITestAzureNativeContractAppend.java | 23 ++ .../azurebfs/AbstractAbfsIntegrationTest.java | 304 ++++++++++++++++++ .../fs/azurebfs/AbstractAbfsScaleTest.java | 53 +++ .../fs/azurebfs/DependencyInjectedTest.java | 206 ------------ .../ITestAzureBlobFileSystemAppend.java | 28 +- .../ITestAzureBlobFileSystemBackCompat.java | 16 +- .../ITestAzureBlobFileSystemCopy.java | 45 ++- .../ITestAzureBlobFileSystemCreate.java | 54 ++-- .../ITestAzureBlobFileSystemDelete.java | 79 +++-- .../azurebfs/ITestAzureBlobFileSystemE2E.java | 66 ++-- .../ITestAzureBlobFileSystemE2EScale.java | 80 ++--- .../ITestAzureBlobFileSystemFileStatus.java | 45 ++- .../ITestAzureBlobFileSystemFlush.java | 201 ++++++------ ...ITestAzureBlobFileSystemInitAndCreate.java | 17 +- .../ITestAzureBlobFileSystemListStatus.java | 119 ++++--- .../ITestAzureBlobFileSystemMkDir.java | 55 +--- .../ITestAzureBlobFileSystemOpen.java | 41 --- .../ITestAzureBlobFileSystemRandomRead.java | 52 +-- .../ITestAzureBlobFileSystemRename.java | 125 ++++--- ...ITestAzureBlobFileSystemRenameUnicode.java | 98 ++++++ .../ITestFileSystemInitialization.java | 45 ++- .../azurebfs/ITestFileSystemProperties.java | 47 ++- .../azurebfs/ITestFileSystemRegistration.java | 78 +++-- .../azurebfs/ITestWasbAbfsCompatibility.java | 170 +++++----- .../constants/TestConfigurationKeys.java | 11 +- ...Test.java => ABFSContractTestBinding.java} | 19 +- ...tract.java => AbfsFileSystemContract.java} | 23 +- .../ITestAbfsFileSystemContractAppend.java | 14 +- .../ITestAbfsFileSystemContractConcat.java | 14 +- .../ITestAbfsFileSystemContractCreate.java | 10 +- .../ITestAbfsFileSystemContractDelete.java | 12 +- .../ITestAbfsFileSystemContractDistCp.java | 10 +- ...stAbfsFileSystemContractGetFileStatus.java | 12 +- .../ITestAbfsFileSystemContractMkdir.java | 12 +- .../ITestAbfsFileSystemContractOpen.java | 12 +- .../ITestAbfsFileSystemContractRename.java | 12 +- ...stAbfsFileSystemContractRootDirectory.java | 12 +- ...estAbfsFileSystemContractSecureDistCp.java | 10 +- .../ITestAbfsFileSystemContractSeek.java | 12 +- .../ITestAbfsFileSystemContractSetTimes.java | 12 +- .../ITestAzureBlobFileSystemBasics.java | 25 +- .../TestConfigurationValidators.java | 5 +- ...ek.java => ITestAbfsReadWriteAndSeek.java} | 67 ++-- ...TestAbfsConfigurationFieldsValidation.java | 6 +- .../utils/CleanUpAbfsTestContainer.java | 6 +- .../hadoop-azure/src/test/resources/abfs.xml | 4 +- .../src/test/resources/azure-bfs-test.xml | 14 +- .../src/test/resources/log4j.properties | 32 ++ 88 files changed, 1984 insertions(+), 1485 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/site/markdown/abfs.md create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsScaleTest.java delete mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java delete mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOpen.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java rename hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/{DependencyInjectedContractTest.java => ABFSContractTestBinding.java} (79%) rename hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/{ITestAbfsFileSystemContract.java => AbfsFileSystemContract.java} (73%) rename hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/{ITestReadWriteAndSeek.java => ITestAbfsReadWriteAndSeek.java} (58%) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java index bd003ae90ab..6e9d433a8e3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java @@ -291,7 +291,7 @@ public class RawLocalFileSystem extends FileSystem { Progressable progress) throws IOException { FileStatus status = getFileStatus(f); if (status.isDirectory()) { - throw new IOException("Cannot append to a diretory (=" + f + " )"); + throw new FileAlreadyExistsException("Cannot append to a directory: " + f); } return new FSDataOutputStream(new BufferedOutputStream( createOutputStreamWithMode(f, true, null), bufferSize), statistics, 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 e15fab8ab16..471dacc9221 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 @@ -1639,16 +1639,31 @@ SAS keys to communicate with Azure storage. + fs.abfs.impl org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem The implementation class of the Azure Blob Filesystem + fs.abfss.impl org.apache.hadoop.fs.azurebfs.SecureAzureBlobFileSystem The implementation class of the Secure Azure Blob Filesystem + + + fs.AbstractFileSystem.abfs.impl + org.apache.hadoop.fs.azurebfs.Abfs + AbstractFileSystem implementation class of abfs:// + + + + fs.AbstractFileSystem.abfss.impl + org.apache.hadoop.fs.azurebfs.Abfss + AbstractFileSystem implementation class of abfss:// + + fs.azure.local.sas.key.mode false diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md index 2637f5442d3..28c6fbe240e 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md @@ -544,15 +544,6 @@ atomic. The combined operation, including `mkdirs(parent(F))` MAY be atomic. The return value is always true—even if a new directory is not created (this is defined in HDFS). -#### Implementation Notes: Local FileSystem - -The local FileSystem does not raise an exception if `mkdirs(p)` is invoked -on a path that exists and is a file. Instead the operation returns false. - - if isFile(FS, p): - FS' = FS - result = False - ### `FSDataOutputStream create(Path, ...)` @@ -641,7 +632,7 @@ Implementations without a compliant call SHOULD throw `UnsupportedOperationExcep if not exists(FS, p) : raise FileNotFoundException - if not isFile(FS, p) : raise [FileNotFoundException, IOException] + if not isFile(FS, p) : raise [FileAlreadyExistsException, FileNotFoundException, IOException] #### Postconditions diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractAppendTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractAppendTest.java index d61b6354498..0be220e6511 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractAppendTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractAppendTest.java @@ -18,7 +18,12 @@ package org.apache.hadoop.fs.contract; +import java.io.FileNotFoundException; +import java.io.IOException; + import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.junit.Test; import org.slf4j.Logger; @@ -27,6 +32,7 @@ import org.slf4j.LoggerFactory; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test append -if supported @@ -75,15 +81,10 @@ public abstract class AbstractContractAppendTest extends AbstractFSContractTestB @Test public void testAppendNonexistentFile() throws Throwable { - try { - FSDataOutputStream out = getFileSystem().append(target); - //got here: trouble - out.close(); - fail("expected a failure"); - } catch (Exception e) { - //expected - handleExpectedException(e); - } + //expected + handleExpectedException( + intercept(Exception.class, + () -> getFileSystem().append(target).close())); } @Test @@ -116,15 +117,9 @@ public abstract class AbstractContractAppendTest extends AbstractFSContractTestB @Test public void testAppendMissingTarget() throws Throwable { - try { - FSDataOutputStream out = getFileSystem().append(target); - //got here: trouble - out.close(); - fail("expected a failure"); - } catch (Exception e) { - //expected - handleExpectedException(e); - } + handleExpectedException( + intercept(Exception.class, + () -> getFileSystem().append(target).close())); } @Test @@ -149,4 +144,30 @@ public abstract class AbstractContractAppendTest extends AbstractFSContractTestB dataset.length); ContractTestUtils.compareByteArrays(dataset, bytes, dataset.length); } + + @Test + public void testAppendFileAfterDelete() throws Exception { + final FileSystem fs = getFileSystem(); + final Path filePath = target; + fs.create(filePath); + fs.delete(filePath, false); + intercept(FileNotFoundException.class, + () -> fs.append(filePath)); + } + + @Test + public void testAppendDirectory() throws Exception { + final FileSystem fs = getFileSystem(); + + final Path folderPath = target; + fs.mkdirs(folderPath); + IOException ex = intercept(IOException.class, + () -> fs.append(folderPath)); + if (ex instanceof FileAlreadyExistsException) { + handleExpectedException(ex); + } else { + handleRelaxedException("Append to a directory", + "FileAlreadyExistsException", ex); + } + } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractConcatTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractConcatTest.java index 7b120861edc..d30e0d66eff 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractConcatTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractConcatTest.java @@ -19,15 +19,16 @@ package org.apache.hadoop.fs.contract; import org.apache.hadoop.fs.Path; + import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertFileHasLength; -import static org.apache.hadoop.fs.contract.ContractTestUtils.cleanup; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test concat -if supported @@ -60,25 +61,15 @@ public abstract class AbstractContractConcatTest extends AbstractFSContractTestB @Test public void testConcatEmptyFiles() throws Throwable { touch(getFileSystem(), target); - try { - getFileSystem().concat(target, new Path[0]); - fail("expected a failure"); - } catch (Exception e) { - //expected - handleExpectedException(e); - } + handleExpectedException(intercept(Exception.class, + () -> getFileSystem().concat(target, new Path[0]))); } @Test public void testConcatMissingTarget() throws Throwable { - try { - getFileSystem().concat(target, - new Path[] { zeroByteFile}); - fail("expected a failure"); - } catch (Exception e) { - //expected - handleExpectedException(e); - } + handleExpectedException( + intercept(Exception.class, + () -> getFileSystem().concat(target, new Path[]{zeroByteFile}))); } @Test @@ -98,15 +89,8 @@ public abstract class AbstractContractConcatTest extends AbstractFSContractTestB public void testConcatOnSelf() throws Throwable { byte[] block = dataset(TEST_FILE_LEN, 0, 255); createFile(getFileSystem(), target, false, block); - try { - getFileSystem().concat(target, - new Path[]{target}); - } catch (Exception e) { - //expected - handleExpectedException(e); - } + handleExpectedException(intercept(Exception.class, + () -> getFileSystem().concat(target, new Path[]{target}))); } - - } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java index 269e35ea669..cb706ede917 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java @@ -32,6 +32,7 @@ import org.apache.hadoop.fs.RemoteIterator; import org.junit.Test; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test getFileStatus and related listing operations. @@ -275,35 +276,22 @@ public abstract class AbstractContractGetFileStatusTest extends @Test public void testLocatedStatusNoDir() throws Throwable { describe("test the LocatedStatus call on a path which is not present"); - try { - RemoteIterator iterator - = getFileSystem().listLocatedStatus(path("missing")); - fail("Expected an exception, got an iterator: " + iterator); - } catch (FileNotFoundException expected) { - // expected - } + intercept(FileNotFoundException.class, + () -> getFileSystem().listLocatedStatus(path("missing"))); } @Test public void testListStatusNoDir() throws Throwable { describe("test the listStatus(path) call on a path which is not present"); - try { - getFileSystem().listStatus(path("missing")); - fail("Expected an exception"); - } catch (FileNotFoundException expected) { - // expected - } + intercept(FileNotFoundException.class, + () -> getFileSystem().listStatus(path("missing"))); } @Test public void testListStatusFilteredNoDir() throws Throwable { describe("test the listStatus(path, filter) call on a missing path"); - try { - getFileSystem().listStatus(path("missing"), ALL_PATHS); - fail("Expected an exception"); - } catch (FileNotFoundException expected) { - // expected - } + intercept(FileNotFoundException.class, + () -> getFileSystem().listStatus(path("missing"), ALL_PATHS)); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMkdirTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMkdirTest.java index c5a546dccdd..de44bc232e7 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMkdirTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMkdirTest.java @@ -26,6 +26,7 @@ import org.junit.Test; import java.io.IOException; +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.dataset; @@ -175,4 +176,11 @@ public abstract class AbstractContractMkdirTest extends AbstractFSContractTestBa } } + @Test + public void testCreateDirWithExistingDir() throws Exception { + Path path = path("testCreateDirWithExistingDir"); + final FileSystem fs = getFileSystem(); + assertMkdirs(fs, path); + assertMkdirs(fs, path); + } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContract.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContract.java index d3dafe974a5..f09496a6082 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContract.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContract.java @@ -148,7 +148,6 @@ public abstract class AbstractFSContract extends Configured { * @param feature feature to query * @param defval default value * @return true if the feature is supported - * @throws IOException IO problems */ public boolean isSupported(String feature, boolean defval) { return getConf().getBoolean(getConfKey(feature), defval); @@ -160,7 +159,6 @@ public abstract class AbstractFSContract extends Configured { * @param feature feature to query * @param defval default value * @return true if the feature is supported - * @throws IOException IO problems */ public int getLimit(String feature, int defval) { return getConf().getInt(getConfKey(feature), defval); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java index 38a6fb10138..ba1204848ad 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java @@ -187,8 +187,11 @@ public class ContractTestUtils extends Assert { (short) 1, buffersize); } - out.write(src, 0, len); - out.close(); + try { + out.write(src, 0, len); + } finally { + out.close(); + } assertFileHasLength(fs, path, len); } @@ -1021,6 +1024,18 @@ public class ContractTestUtils extends Assert { found); } + /** + * Execute {@link FileSystem#mkdirs(Path)}; expect {@code true} back. + * (Note: does not work for localFS if the directory already exists) + * Does not perform any validation of the created directory. + * @param fs filesystem + * @param dir directory to create + * @throws IOException IO Problem + */ + public static void assertMkdirs(FileSystem fs, Path dir) throws IOException { + assertTrue("mkdirs(" + dir + ") returned false", fs.mkdirs(dir)); + } + /** * Test for the host being an OSX machine. * @return true if the JVM thinks that is running on OSX diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java index 707e264a365..32df9422386 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; -import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.DelegateToFileSystem; @@ -32,7 +31,6 @@ import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; * Azure Blob File System implementation of AbstractFileSystem. * This impl delegates to the old FileSystem */ -@InterfaceAudience.Public @InterfaceStability.Evolving public class Abfs extends DelegateToFileSystem { @@ -45,4 +43,4 @@ public class Abfs extends DelegateToFileSystem { public int getUriDefaultPort() { return -1; } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java index 19c0f7a7d62..c33265ce324 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; -import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.DelegateToFileSystem; @@ -32,7 +31,6 @@ import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; * Azure Blob File System implementation of AbstractFileSystem. * This impl delegates to the old FileSystem */ -@InterfaceAudience.Public @InterfaceStability.Evolving public class Abfss extends DelegateToFileSystem { @@ -45,4 +43,4 @@ public class Abfss extends DelegateToFileSystem { public int getUriDefaultPort() { return -1; } -} \ No newline at end of file +} 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 cf5acbb0fd3..9f58f6b040a 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 @@ -36,14 +36,10 @@ import java.util.concurrent.Future; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.fs.PathIOException; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.commons.lang.ArrayUtils; -import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; @@ -54,13 +50,15 @@ import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; -import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.FileSystemOperationUnhandledException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; +import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Progressable; @@ -69,8 +67,7 @@ import org.apache.hadoop.util.Progressable; * A {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on Windows Azure */ -@InterfaceAudience.Public -@InterfaceStability.Stable +@InterfaceStability.Evolving public class AzureBlobFileSystem extends FileSystem { public static final Logger LOG = LoggerFactory.getLogger(AzureBlobFileSystem.class); private URI uri; @@ -88,8 +85,7 @@ public class AzureBlobFileSystem extends FileSystem { super.initialize(uri, configuration); setConf(configuration); - this.LOG.debug( - "Initializing AzureBlobFileSystem for {}", uri); + LOG.debug("Initializing AzureBlobFileSystem for {}", uri); this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority()); this.userGroupInformation = UserGroupInformation.getCurrentUser(); @@ -97,16 +93,24 @@ public class AzureBlobFileSystem extends FileSystem { this.primaryUserGroup = userGroupInformation.getPrimaryGroupName(); this.abfsStore = new AzureBlobFileSystemStore(uri, this.isSecure(), configuration, userGroupInformation); - this.LOG.debug( - "Initializing NativeAzureFileSystem for {}", uri); + LOG.debug("Initializing NativeAzureFileSystem for {}", uri); this.setWorkingDirectory(this.getHomeDirectory()); if (abfsStore.getAbfsConfiguration().getCreateRemoteFileSystemDuringInitialization()) { this.createFileSystem(); } + } - this.mkdirs(this.workingDir); + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "AzureBlobFileSystem{"); + sb.append("uri=").append(uri); + sb.append(", user='").append(user).append('\''); + sb.append(", primaryUserGroup='").append(primaryUserGroup).append('\''); + sb.append('}'); + return sb.toString(); } public boolean isSecure() { @@ -120,8 +124,7 @@ public class AzureBlobFileSystem extends FileSystem { @Override public FSDataInputStream open(final Path path, final int bufferSize) throws IOException { - this.LOG.debug( - "AzureBlobFileSystem.open path: {} bufferSize: {}", path.toString(), bufferSize); + LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize); try { InputStream inputStream = abfsStore.openFileForRead(makeQualified(path), statistics); @@ -135,9 +138,8 @@ public class AzureBlobFileSystem extends FileSystem { @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 { - this.LOG.debug( - "AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}", - f.toString(), + LOG.debug("AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}", + f, permission, overwrite, blockSize); @@ -196,7 +198,7 @@ public class AzureBlobFileSystem extends FileSystem { @Override public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress) throws IOException { - this.LOG.debug( + LOG.debug( "AzureBlobFileSystem.append path: {} bufferSize: {}", f.toString(), bufferSize); @@ -211,7 +213,7 @@ public class AzureBlobFileSystem extends FileSystem { } public boolean rename(final Path src, final Path dst) throws IOException { - this.LOG.debug( + LOG.debug( "AzureBlobFileSystem.rename src: {} dst: {}", src.toString(), dst.toString()); Path parentFolder = src.getParent(); @@ -250,7 +252,7 @@ public class AzureBlobFileSystem extends FileSystem { @Override public boolean delete(final Path f, final boolean recursive) throws IOException { - this.LOG.debug( + LOG.debug( "AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), recursive); if (f.isRoot()) { @@ -273,7 +275,7 @@ public class AzureBlobFileSystem extends FileSystem { @Override public FileStatus[] listStatus(final Path f) throws IOException { - this.LOG.debug( + LOG.debug( "AzureBlobFileSystem.listStatus path: {}", f.toString()); try { @@ -287,8 +289,8 @@ public class AzureBlobFileSystem extends FileSystem { @Override public boolean mkdirs(final Path f, final FsPermission permission) throws IOException { - this.LOG.debug( - "AzureBlobFileSystem.mkdirs path: {} permissions: {}", f.toString(), permission); + LOG.debug( + "AzureBlobFileSystem.mkdirs path: {} permissions: {}", f, permission); final Path parentFolder = f.getParent(); if (parentFolder == null) { @@ -312,13 +314,13 @@ public class AzureBlobFileSystem extends FileSystem { } super.close(); - this.LOG.debug("AzureBlobFileSystem.close"); + LOG.debug("AzureBlobFileSystem.close"); this.isClosed = true; } @Override public FileStatus getFileStatus(final Path f) throws IOException { - this.LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f.toString()); + LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f); try { return abfsStore.getFileStatus(makeQualified(f)); @@ -350,7 +352,8 @@ public class AzureBlobFileSystem extends FileSystem { @Override public Path getHomeDirectory() { return makeQualified(new Path( - FileSystemConfigurations.USER_HOME_DIRECTORY_PREFIX + "/" + this.userGroupInformation.getShortUserName())); + FileSystemConfigurations.USER_HOME_DIRECTORY_PREFIX + + "/" + this.userGroupInformation.getShortUserName())); } /** @@ -360,7 +363,7 @@ public class AzureBlobFileSystem extends FileSystem { */ @Override public BlockLocation[] getFileBlockLocations(FileStatus file, - long start, long len) throws IOException { + long start, long len) { if (file == null) { return null; } @@ -403,7 +406,7 @@ public class AzureBlobFileSystem extends FileSystem { } private boolean deleteRoot() throws IOException { - this.LOG.debug("Deleting root content"); + LOG.debug("Deleting root content"); final ExecutorService executorService = Executors.newFixedThreadPool(10); @@ -441,15 +444,14 @@ public class AzureBlobFileSystem extends FileSystem { private FileStatus tryGetFileStatus(final Path f) { try { return getFileStatus(f); - } - catch (IOException ex) { - this.LOG.debug("File not found {}", f.toString()); + } catch (IOException ex) { + LOG.debug("File not found {}", f); return null; } } private void createFileSystem() throws IOException { - this.LOG.debug( + LOG.debug( "AzureBlobFileSystem.createFileSystem uri: {}", uri); try { this.abfsStore.createFilesystem(); @@ -493,7 +495,8 @@ public class AzureBlobFileSystem extends FileSystem { return false; } - if (scheme.equals(FileSystemUriSchemes.ABFS_SCHEME) || scheme.equals(FileSystemUriSchemes.ABFS_SECURE_SCHEME)) { + if (scheme.equals(FileSystemUriSchemes.ABFS_SCHEME) + || scheme.equals(FileSystemUriSchemes.ABFS_SECURE_SCHEME)) { return true; } @@ -501,34 +504,45 @@ public class AzureBlobFileSystem extends FileSystem { } @VisibleForTesting - FileSystemOperation execute( + FileSystemOperation execute( final String scopeDescription, final Callable callableFileOperation) throws IOException { return execute(scopeDescription, callableFileOperation, null); } @VisibleForTesting - FileSystemOperation execute( + FileSystemOperation execute( final String scopeDescription, final Callable callableFileOperation, T defaultResultValue) throws IOException { try { final T executionResult = callableFileOperation.call(); - return new FileSystemOperation(executionResult, null); + return new FileSystemOperation<>(executionResult, null); } catch (AbfsRestOperationException abfsRestOperationException) { - return new FileSystemOperation(defaultResultValue, abfsRestOperationException); + return new FileSystemOperation<>(defaultResultValue, abfsRestOperationException); } catch (AzureBlobFileSystemException azureBlobFileSystemException) { throw new IOException(azureBlobFileSystemException); } catch (Exception exception) { if (exception instanceof ExecutionException) { exception = (Exception) getRootCause(exception); } - final FileSystemOperationUnhandledException fileSystemOperationUnhandledException = new FileSystemOperationUnhandledException(exception); + final FileSystemOperationUnhandledException fileSystemOperationUnhandledException + = new FileSystemOperationUnhandledException(exception); throw new IOException(fileSystemOperationUnhandledException); } } + /** + * Given a path and exception, choose which IOException subclass + * to create. + * Will return if and only iff the error code is in the list of allowed + * error codes. + * @param path path of operation triggering exception; may be null + * @param exception the exception caught + * @param allowedErrorCodesList varargs list of error codes. + * @throws IOException if the exception error code is not on the allowed list. + */ private void checkException(final Path path, final AzureBlobFileSystemException exception, final AzureServiceErrorCode... allowedErrorCodesList) throws IOException { @@ -542,9 +556,11 @@ public class AzureBlobFileSystem extends FileSystem { //AbfsRestOperationException.getMessage() contains full error info including path/uri. if (statusCode == HttpURLConnection.HTTP_NOT_FOUND) { - throw new FileNotFoundException(ere.getMessage()); + throw (IOException)new FileNotFoundException(ere.getMessage()) + .initCause(exception); } else if (statusCode == HttpURLConnection.HTTP_CONFLICT) { - throw new FileAlreadyExistsException(ere.getMessage()); + throw (IOException)new FileAlreadyExistsException(ere.getMessage()) + .initCause(exception); } else { throw ere; } @@ -601,4 +617,4 @@ public class AzureBlobFileSystem extends FileSystem { AzureBlobFileSystemStore getAbfsStore() { return this.abfsStore; } -} \ No newline at end of file +} 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 134277fd969..8ac31ce0372 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 @@ -103,7 +103,7 @@ public class AzureBlobFileSystemStore { private final Set azureAtomicRenameDirSet; - public AzureBlobFileSystemStore(URI uri, boolean isSeure, Configuration configuration, UserGroupInformation userGroupInformation) + public AzureBlobFileSystemStore(URI uri, boolean isSecure, Configuration configuration, UserGroupInformation userGroupInformation) throws AzureBlobFileSystemException { this.uri = uri; try { @@ -113,9 +113,10 @@ public class AzureBlobFileSystemStore { } this.userGroupInformation = userGroupInformation; - this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA))); + this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList( + abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA))); - initializeClient(uri, isSeure); + initializeClient(uri, isSecure); } @VisibleForTesting @@ -134,8 +135,7 @@ public class AzureBlobFileSystemStore { } public Hashtable getFilesystemProperties() throws AzureBlobFileSystemException { - this.LOG.debug( - "getFilesystemProperties for filesystem: {}", + LOG.debug("getFilesystemProperties for filesystem: {}", client.getFileSystem()); final Hashtable parsedXmsProperties; @@ -148,13 +148,13 @@ public class AzureBlobFileSystemStore { return parsedXmsProperties; } - public void setFilesystemProperties(final Hashtable properties) throws AzureBlobFileSystemException { - if (properties == null || properties.size() == 0) { + public void setFilesystemProperties(final Hashtable properties) + throws AzureBlobFileSystemException { + if (properties == null || properties.isEmpty()) { return; } - this.LOG.debug( - "setFilesystemProperties for filesystem: {} with properties: {}", + LOG.debug("setFilesystemProperties for filesystem: {} with properties: {}", client.getFileSystem(), properties); @@ -169,10 +169,9 @@ public class AzureBlobFileSystemStore { } public Hashtable getPathProperties(final Path path) throws AzureBlobFileSystemException { - this.LOG.debug( - "getPathProperties for filesystem: {} path: {}", + LOG.debug("getPathProperties for filesystem: {} path: {}", client.getFileSystem(), - path.toString()); + path); final Hashtable parsedXmsProperties; final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); @@ -185,10 +184,9 @@ public class AzureBlobFileSystemStore { } public void setPathProperties(final Path path, final Hashtable properties) throws AzureBlobFileSystemException { - this.LOG.debug( - "setFilesystemProperties for filesystem: {} path: {} with properties: {}", + LOG.debug("setFilesystemProperties for filesystem: {} path: {} with properties: {}", client.getFileSystem(), - path.toString(), + path, properties); final String commaSeparatedProperties; @@ -201,26 +199,23 @@ public class AzureBlobFileSystemStore { } public void createFilesystem() throws AzureBlobFileSystemException { - this.LOG.debug( - "createFilesystem for filesystem: {}", + LOG.debug("createFilesystem for filesystem: {}", client.getFileSystem()); client.createFilesystem(); } public void deleteFilesystem() throws AzureBlobFileSystemException { - this.LOG.debug( - "deleteFilesystem for filesystem: {}", + LOG.debug("deleteFilesystem for filesystem: {}", client.getFileSystem()); client.deleteFilesystem(); } public OutputStream createFile(final Path path, final boolean overwrite) throws AzureBlobFileSystemException { - this.LOG.debug( - "createFile filesystem: {} path: {} overwrite: {}", + LOG.debug("createFile filesystem: {} path: {} overwrite: {}", client.getFileSystem(), - path.toString(), + path, overwrite); client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), true, overwrite); @@ -232,23 +227,19 @@ public class AzureBlobFileSystemStore { return outputStream; } - public Void createDirectory(final Path path) throws AzureBlobFileSystemException { - this.LOG.debug( - "createDirectory filesystem: {} path: {} overwrite: {}", + public void createDirectory(final Path path) throws AzureBlobFileSystemException { + LOG.debug("createDirectory filesystem: {} path: {}", client.getFileSystem(), - path.toString()); + path); client.createPath("/" + getRelativePath(path), false, true); - - return null; } public InputStream openFileForRead(final Path path, final FileSystem.Statistics statistics) throws AzureBlobFileSystemException { - this.LOG.debug( - "openFileForRead filesystem: {} path: {}", + LOG.debug("openFileForRead filesystem: {} path: {}", client.getFileSystem(), - path.toString()); + path); final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); @@ -266,16 +257,16 @@ public class AzureBlobFileSystemStore { // Add statistics for InputStream return new FSDataInputStream( - new AbfsInputStream(client, statistics, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength, + new AbfsInputStream(client, statistics, + AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength, abfsConfiguration.getReadBufferSize(), abfsConfiguration.getReadAheadQueueDepth(), eTag)); } public OutputStream openFileForWrite(final Path path, final boolean overwrite) throws AzureBlobFileSystemException { - this.LOG.debug( - "openFileForWrite filesystem: {} path: {} overwrite: {}", + LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {}", client.getFileSystem(), - path.toString(), + path, overwrite); final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); @@ -304,23 +295,21 @@ public class AzureBlobFileSystemStore { AzureBlobFileSystemException { if (isAtomicRenameKey(source.getName())) { - this.LOG.warn("The atomic rename feature is not supported by the ABFS scheme; however rename," + 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."); } - this.LOG.debug( - "renameAsync filesystem: {} source: {} destination: {}", + LOG.debug("renameAsync filesystem: {} source: {} destination: {}", client.getFileSystem(), - source.toString(), - destination.toString()); + source, + destination); String continuation = null; long deadline = now() + RENAME_TIMEOUT_MILISECONDS; do { if (now() > deadline) { - LOG.debug( - "Rename {} to {} timed out.", + LOG.debug("Rename {} to {} timed out.", source, destination); @@ -334,13 +323,12 @@ public class AzureBlobFileSystemStore { } while (continuation != null && !continuation.isEmpty()); } - public void delete(final Path path, final boolean recursive) throws - AzureBlobFileSystemException { + public void delete(final Path path, final boolean recursive) + throws AzureBlobFileSystemException { - this.LOG.debug( - "delete filesystem: {} path: {} recursive: {}", + LOG.debug("delete filesystem: {} path: {} recursive: {}", client.getFileSystem(), - path.toString(), + path, String.valueOf(recursive)); String continuation = null; @@ -348,13 +336,13 @@ public class AzureBlobFileSystemStore { do { if (now() > deadline) { - this.LOG.debug( - "Delete directory {} timed out.", path); + LOG.debug("Delete directory {} timed out.", path); throw new TimeoutException("Delete directory timed out."); } - AbfsRestOperation op = client.deletePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), recursive, continuation); + AbfsRestOperation op = client.deletePath( + AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), recursive, continuation); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); } while (continuation != null && !continuation.isEmpty()); @@ -362,10 +350,9 @@ public class AzureBlobFileSystemStore { public FileStatus getFileStatus(final Path path) throws IOException { - this.LOG.debug( - "getFileStatus filesystem: {} path: {}", + LOG.debug("getFileStatus filesystem: {} path: {}", client.getFileSystem(), - path.toString()); + path); if (path.isRoot()) { AbfsRestOperation op = client.getFilesystemProperties(); @@ -405,10 +392,9 @@ public class AzureBlobFileSystemStore { } public FileStatus[] listStatus(final Path path) throws IOException { - this.LOG.debug( - "listStatus filesystem: {} path: {}", + LOG.debug("listStatus filesystem: {} path: {}", client.getFileSystem(), - path.toString()); + path); String relativePath = path.isRoot() ? AbfsHttpConstants.EMPTY_STRING : getRelativePath(path); String continuation = null; @@ -480,10 +466,12 @@ public class AzureBlobFileSystemStore { final String[] authorityParts = authority.split(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2); - if (authorityParts.length < 2 || "".equals(authorityParts[0])) { + if (authorityParts.length < 2 || authorityParts[0] != null + && authorityParts[0].isEmpty()) { final String errMsg = String - .format("URI '%s' has a malformed authority, expected container name. " - + "Authority takes the form "+ FileSystemUriSchemes.ABFS_SCHEME + "://[@]", + .format("'%s' has a malformed authority, expected container name. " + + "Authority takes the form " + + FileSystemUriSchemes.ABFS_SCHEME + "://[@]", uri.toString()); throw new InvalidUriException(errMsg); } @@ -499,11 +487,16 @@ public class AzureBlobFileSystemStore { try { baseUrl = new URL(url); } catch (MalformedURLException e) { - throw new InvalidUriException(String.format("URI '%s' is malformed", uri.toString())); + throw new InvalidUriException(uri.toString()); } + int dotIndex = accountName.indexOf(AbfsHttpConstants.DOT); + if (dotIndex <= 0) { + throw new InvalidUriException( + uri.toString() + " - account name is not fully qualified."); + } SharedKeyCredentials creds = - new SharedKeyCredentials(accountName.substring(0, accountName.indexOf(AbfsHttpConstants.DOT)), + new SharedKeyCredentials(accountName.substring(0, dotIndex), this.abfsConfiguration.getStorageAccountKey(accountName)); this.client = new AbfsClient(baseUrl, creds, abfsConfiguration, new ExponentialRetryPolicy()); @@ -513,7 +506,7 @@ public class AzureBlobFileSystemStore { Preconditions.checkNotNull(path, "path"); final String relativePath = path.toUri().getPath(); - if (relativePath.length() == 0) { + if (relativePath.isEmpty()) { return relativePath; } @@ -537,7 +530,8 @@ public class AzureBlobFileSystemStore { } private boolean parseIsDirectory(final String resourceType) { - return resourceType == null ? false : resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY); + return resourceType != null + && resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY); } private DateTime parseLastModifiedTime(final String lastModifiedTime) { @@ -628,7 +622,7 @@ public class AzureBlobFileSystemStore { } } } catch (URISyntaxException e) { - this.LOG.info("URI syntax error creating URI for {}", dir); + LOG.info("URI syntax error creating URI for {}", dir); } } @@ -658,20 +652,21 @@ public class AzureBlobFileSystemStore { */ @Override public boolean equals(Object obj) { - if (obj == this) { - return true; - } - - if (obj == null) { + if (!(obj instanceof FileStatus)) { return false; } - if (this.getClass() == obj.getClass()) { - VersionedFileStatus other = (VersionedFileStatus) obj; - return this.getPath().equals(other.getPath()) && this.version.equals(other.version); + FileStatus other = (FileStatus) obj; + + if (!other.equals(this)) {// compare the path + return false; } - return false; + if (other instanceof VersionedFileStatus) { + return this.version.equals(((VersionedFileStatus)other).version); + } + + return true; } /** @@ -695,6 +690,16 @@ public class AzureBlobFileSystemStore { public String getVersion() { return this.version; } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "VersionedFileStatus{"); + sb.append(super.toString()); + sb.append("; version='").append(version).append('\''); + sb.append('}'); + return sb.toString(); + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.java index a6ad8299231..15fe5427252 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.java @@ -18,7 +18,6 @@ package org.apache.hadoop.fs.azurebfs; -import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; @@ -26,7 +25,6 @@ import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; * A secure {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on Windows Azure */ -@InterfaceAudience.Public @InterfaceStability.Evolving public class SecureAzureBlobFileSystem extends AzureBlobFileSystem { @Override @@ -38,4 +36,4 @@ public class SecureAzureBlobFileSystem extends AzureBlobFileSystem { public String getScheme() { return FileSystemUriSchemes.ABFS_SECURE_SCHEME; } -} \ No newline at end of file +} 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 2ec4db0197c..f80bc605cb6 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 @@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; /** - * Responsible to keep all constant keys used in abfs rest client here + * Responsible to keep all constant keys used in abfs rest client here. */ @InterfaceAudience.Public @InterfaceStability.Evolving 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 9b7f9bc6ce7..4603b5fd03e 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 @@ -21,7 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; /** - * Responsible to keep all abfs http headers here + * Responsible to keep all abfs http headers here. */ @InterfaceAudience.Public @InterfaceStability.Evolving 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 a9f7d3350b7..f58d33a1302 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 @@ -21,7 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; /** - * Responsible to keep all Http Query params here + * Responsible to keep all Http Query params here. */ @InterfaceAudience.Public @InterfaceStability.Evolving diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java index 462ebbc88f6..82c571a3b03 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java @@ -25,12 +25,12 @@ import java.lang.annotation.Target; import org.apache.hadoop.classification.InterfaceStability; /** - * Definitions of Annotations for all types of the validators + * Definitions of Annotations for all types of the validators. */ @InterfaceStability.Evolving public class ConfigurationValidationAnnotations { /** - * Describes the requirements when validating the annotated int field + * Describes the requirements when validating the annotated int field. */ @Target({ ElementType.FIELD }) @Retention(RetentionPolicy.RUNTIME) @@ -47,7 +47,7 @@ public class ConfigurationValidationAnnotations { } /** - * Describes the requirements when validating the annotated long field + * Describes the requirements when validating the annotated long field. */ @Target({ ElementType.FIELD }) @Retention(RetentionPolicy.RUNTIME) @@ -64,7 +64,7 @@ public class ConfigurationValidationAnnotations { } /** - * Describes the requirements when validating the annotated String field + * Describes the requirements when validating the annotated String field. */ @Target({ ElementType.FIELD }) @Retention(RetentionPolicy.RUNTIME) @@ -77,7 +77,7 @@ public class ConfigurationValidationAnnotations { } /** - * Describes the requirements when validating the annotated String field + * Describes the requirements when validating the annotated String field. */ @Target({ ElementType.FIELD }) @Retention(RetentionPolicy.RUNTIME) @@ -90,7 +90,7 @@ public class ConfigurationValidationAnnotations { } /** - * Describes the requirements when validating the annotated boolean field + * Describes the requirements when validating the annotated boolean field. */ @Target({ ElementType.FIELD }) @Retention(RetentionPolicy.RUNTIME) @@ -101,4 +101,4 @@ public class ConfigurationValidationAnnotations { boolean ThrowIfInvalid() default false; } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java index 796f7859567..d61229ee803 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java @@ -18,19 +18,17 @@ package org.apache.hadoop.fs.azurebfs.contracts.diagnostics; -import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; /** * ConfigurationValidator to validate the value of a configuration key - * @param the type of the validator and the validated value + * @param the type of the validator and the validated value. */ -@InterfaceAudience.Public @InterfaceStability.Evolving public interface ConfigurationValidator { /** - * Validates the configValue + * Validates the configValue. * @return validated value of type T */ T validate(String configValue) throws InvalidConfigurationValueException; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java index f31c680628b..9b1bead886e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java @@ -40,7 +40,7 @@ public abstract class AzureBlobFileSystemException extends IOException { @Override public String toString() { if (this.getMessage() == null && this.getCause() == null) { - return ""; + return "AzureBlobFileSystemException"; } if (this.getCause() == null) { @@ -53,4 +53,4 @@ public abstract class AzureBlobFileSystemException extends IOException { return this.getMessage() + this.getCause().toString(); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.java index a84495afc61..4fa01509779 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.java @@ -28,6 +28,6 @@ import org.apache.hadoop.classification.InterfaceStability; @InterfaceStability.Evolving public final class InvalidUriException extends AzureBlobFileSystemException { public InvalidUriException(String url) { - super(String.format("%s is invalid.", url)); + super(String.format("Invalid URI %s", url)); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java index 69288c5198f..6bb997bbc79 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java @@ -26,7 +26,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationVa import org.apache.commons.codec.binary.Base64; /** -* String Base64 configuration value Validator +* String Base64 configuration value Validator. */ @InterfaceAudience.Public @InterfaceStability.Evolving diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java index c9927ff0856..b16abdd09b5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java @@ -23,7 +23,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; /** - * Boolean configuration value validator + * Boolean configuration value validator. */ @InterfaceAudience.Public @InterfaceStability.Evolving @@ -47,4 +47,4 @@ public class BooleanConfigurationBasicValidator extends ConfigurationBasicValida throw new InvalidConfigurationValueException(getConfigKey()); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java index 7da809cdcad..8555a29805a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java @@ -38,7 +38,7 @@ abstract class ConfigurationBasicValidator implements ConfigurationValidator /** * This method handles the base case where the configValue is null, based on the throwIfInvalid it either throws or returns the defaultVal, - * otherwise it returns null indicating that the configValue needs to be validated further + * otherwise it returns null indicating that the configValue needs to be validated further. * @param configValue the configuration value set by the user * @return the defaultVal in case the configValue is null and not required to be set, null in case the configValue not null * @throws InvalidConfigurationValueException in case the configValue is null and required to be set diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java index ec38cd8ea47..26c7d2f0ac1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java @@ -24,7 +24,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidato import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; /** - * Integer configuration value Validator + * Integer configuration value Validator. */ @InterfaceAudience.Public @InterfaceStability.Evolving diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java index 559dbc0c49b..32ac14cea61 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java @@ -17,15 +17,13 @@ */ package org.apache.hadoop.fs.azurebfs.diagnostics; -import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; /** - * Long configuration value Validator + * Long configuration value Validator. */ -@InterfaceAudience.Public @InterfaceStability.Evolving public class LongConfigurationBasicValidator extends ConfigurationBasicValidator implements ConfigurationValidator { private final long min; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java index d6f9c59e5d2..0d344d13434 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java @@ -18,15 +18,13 @@ package org.apache.hadoop.fs.azurebfs.diagnostics; -import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; /** - * String configuration value Validator + * String configuration value Validator. */ -@InterfaceAudience.Public @InterfaceStability.Evolving public class StringConfigurationBasicValidator extends ConfigurationBasicValidator implements ConfigurationValidator{ 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 a78e7af4660..2b3ccc0472d 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 @@ -26,14 +26,17 @@ import java.util.ArrayList; import java.util.List; import java.util.Locale; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; -import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; -import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; -import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; + + +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.*; + /** * AbfsClient */ @@ -53,7 +56,7 @@ public class AbfsClient { this.baseUrl = baseUrl; this.sharedKeyCredentials = sharedKeyCredentials; String baseUrlString = baseUrl.toString(); - this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(AbfsHttpConstants.FORWARD_SLASH) + 1); + this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(FORWARD_SLASH) + 1); this.abfsConfiguration = abfsConfiguration; this.retryPolicy = exponentialRetryPolicy; this.userAgent = initializeUserAgent(); @@ -73,19 +76,19 @@ public class AbfsClient { List createDefaultHeaders() { final List requestHeaders = new ArrayList(); - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_VERSION, xMsVersion)); - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.ACCEPT, AbfsHttpConstants.APPLICATION_JSON - + AbfsHttpConstants.COMMA + AbfsHttpConstants.SINGLE_WHITE_SPACE + AbfsHttpConstants.APPLICATION_OCTET_STREAM)); - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.ACCEPT_CHARSET, - AbfsHttpConstants.UTF_8)); - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.CONTENT_TYPE, AbfsHttpConstants.EMPTY_STRING)); - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.USER_AGENT, userAgent)); + requestHeaders.add(new AbfsHttpHeader(X_MS_VERSION, xMsVersion)); + requestHeaders.add(new AbfsHttpHeader(ACCEPT, APPLICATION_JSON + + COMMA + SINGLE_WHITE_SPACE + APPLICATION_OCTET_STREAM)); + requestHeaders.add(new AbfsHttpHeader(ACCEPT_CHARSET, + UTF_8)); + requestHeaders.add(new AbfsHttpHeader(CONTENT_TYPE, EMPTY_STRING)); + requestHeaders.add(new AbfsHttpHeader(USER_AGENT, userAgent)); return requestHeaders; } AbfsUriQueryBuilder createDefaultUriQueryBuilder() { final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_TIMEOUT, AbfsHttpConstants.DEFAULT_TIMEOUT); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_TIMEOUT, DEFAULT_TIMEOUT); return abfsUriQueryBuilder; } @@ -93,12 +96,12 @@ public class AbfsClient { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( this, - AbfsHttpConstants.HTTP_METHOD_PUT, + HTTP_METHOD_PUT, url, requestHeaders); op.execute(); @@ -109,19 +112,19 @@ public class AbfsClient { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE, - AbfsHttpConstants.HTTP_METHOD_PATCH)); + requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, + HTTP_METHOD_PATCH)); - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PROPERTIES, + requestHeaders.add(new AbfsHttpHeader(X_MS_PROPERTIES, properties)); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( this, - AbfsHttpConstants.HTTP_METHOD_PUT, + HTTP_METHOD_PUT, url, requestHeaders); op.execute(); @@ -133,16 +136,16 @@ public class AbfsClient { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_DIRECTORY, relativePath == null ? "" : urlEncode(relativePath)); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RECURSIVE, String.valueOf(recursive)); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_MAXRESULTS, String.valueOf(listMaxResults)); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_DIRECTORY, relativePath == null ? "" : urlEncode(relativePath)); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RECURSIVE, String.valueOf(recursive)); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_MAXRESULTS, String.valueOf(listMaxResults)); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( this, - AbfsHttpConstants.HTTP_METHOD_GET, + HTTP_METHOD_GET, url, requestHeaders); op.execute(); @@ -153,12 +156,12 @@ public class AbfsClient { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( this, - AbfsHttpConstants.HTTP_METHOD_HEAD, + HTTP_METHOD_HEAD, url, requestHeaders); op.execute(); @@ -169,12 +172,12 @@ public class AbfsClient { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, AbfsHttpConstants.FILESYSTEM); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( this, - AbfsHttpConstants.HTTP_METHOD_DELETE, + HTTP_METHOD_DELETE, url, requestHeaders); op.execute(); @@ -185,16 +188,16 @@ public class AbfsClient { throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); if (!overwrite) { - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_NONE_MATCH, "*")); + requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, "*")); } final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RESOURCE, isFile ? AbfsHttpConstants.FILE : AbfsHttpConstants.DIRECTORY); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, isFile ? FILE : DIRECTORY); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( this, - AbfsHttpConstants.HTTP_METHOD_PUT, + HTTP_METHOD_PUT, url, requestHeaders); op.execute(); @@ -205,17 +208,17 @@ public class AbfsClient { throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); - final String encodedRenameSource = urlEncode(AbfsHttpConstants.FORWARD_SLASH + this.getFileSystem() + source); - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_RENAME_SOURCE, encodedRenameSource)); - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_NONE_MATCH, AbfsHttpConstants.STAR)); + final String encodedRenameSource = urlEncode(FORWARD_SLASH + this.getFileSystem() + source); + requestHeaders.add(new AbfsHttpHeader(X_MS_RENAME_SOURCE, encodedRenameSource)); + requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, STAR)); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation); final URL url = createRequestUrl(destination, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( this, - AbfsHttpConstants.HTTP_METHOD_PUT, + HTTP_METHOD_PUT, url, requestHeaders); op.execute(); @@ -227,17 +230,17 @@ public class AbfsClient { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE, - AbfsHttpConstants.HTTP_METHOD_PATCH)); + requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, + HTTP_METHOD_PATCH)); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.APPEND_ACTION); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_POSITION, Long.toString(position)); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, APPEND_ACTION); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(position)); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( this, - AbfsHttpConstants.HTTP_METHOD_PUT, + HTTP_METHOD_PUT, url, requestHeaders, buffer, offset, length); op.execute(); @@ -245,44 +248,46 @@ public class AbfsClient { } - public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData) throws AzureBlobFileSystemException { + public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData) + throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE, - AbfsHttpConstants.HTTP_METHOD_PATCH)); + requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, + HTTP_METHOD_PATCH)); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.FLUSH_ACTION); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_POSITION, Long.toString(position)); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RETAIN_UNCOMMITTED_DATA, String.valueOf(retainUncommittedData)); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, FLUSH_ACTION); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(position)); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RETAIN_UNCOMMITTED_DATA, String.valueOf(retainUncommittedData)); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( this, - AbfsHttpConstants.HTTP_METHOD_PUT, + HTTP_METHOD_PUT, url, requestHeaders); op.execute(); return op; } - public AbfsRestOperation setPathProperties(final String path, final String properties) throws AzureBlobFileSystemException { + public AbfsRestOperation setPathProperties(final String path, final String properties) + throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE, - AbfsHttpConstants.HTTP_METHOD_PATCH)); + requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, + HTTP_METHOD_PATCH)); - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PROPERTIES, properties)); + requestHeaders.add(new AbfsHttpHeader(X_MS_PROPERTIES, properties)); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_PROPERTIES_ACTION); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, SET_PROPERTIES_ACTION); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( this, - AbfsHttpConstants.HTTP_METHOD_PUT, + HTTP_METHOD_PUT, url, requestHeaders); op.execute(); @@ -297,7 +302,7 @@ public class AbfsClient { final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( this, - AbfsHttpConstants.HTTP_METHOD_HEAD, + HTTP_METHOD_HEAD, url, requestHeaders); op.execute(); @@ -307,9 +312,9 @@ public class AbfsClient { public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset, final int bufferLength, final String eTag) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.RANGE, + requestHeaders.add(new AbfsHttpHeader(RANGE, String.format("bytes=%d-%d", position, position + bufferLength - 1))); - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_MATCH, eTag)); + requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag)); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); @@ -317,7 +322,7 @@ public class AbfsClient { final AbfsRestOperation op = new AbfsRestOperation( this, - AbfsHttpConstants.HTTP_METHOD_GET, + HTTP_METHOD_GET, url, requestHeaders, buffer, @@ -333,13 +338,13 @@ public class AbfsClient { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_RECURSIVE, String.valueOf(recursive)); - abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_CONTINUATION, continuation); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_RECURSIVE, String.valueOf(recursive)); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation); final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( this, - AbfsHttpConstants.HTTP_METHOD_DELETE, + HTTP_METHOD_DELETE, url, requestHeaders); op.execute(); @@ -347,7 +352,7 @@ public class AbfsClient { } private URL createRequestUrl(final String query) throws AzureBlobFileSystemException { - return createRequestUrl(AbfsHttpConstants.EMPTY_STRING, query); + return createRequestUrl(EMPTY_STRING, query); } private URL createRequestUrl(final String path, final String query) @@ -357,8 +362,8 @@ public class AbfsClient { try { encodedPath = urlEncode(path); } catch (AzureBlobFileSystemException ex) { - this.LOG.debug( - "Unexpected error.", ex); + LOG.debug("Unexpected error.", ex); + throw new InvalidUriException(path); } final StringBuilder sb = new StringBuilder(); @@ -378,9 +383,9 @@ public class AbfsClient { private static String urlEncode(final String value) throws AzureBlobFileSystemException { String encodedString = null; try { - encodedString = URLEncoder.encode(value, AbfsHttpConstants.UTF_8) - .replace(AbfsHttpConstants.PLUS, AbfsHttpConstants.PLUS_ENCODE) - .replace(AbfsHttpConstants.FORWARD_SLASH_ENCODE, AbfsHttpConstants.FORWARD_SLASH); + encodedString = URLEncoder.encode(value, UTF_8) + .replace(PLUS, PLUS_ENCODE) + .replace(FORWARD_SLASH_ENCODE, FORWARD_SLASH); } catch (UnsupportedEncodingException ex) { throw new InvalidUriException(value); } @@ -391,11 +396,11 @@ public class AbfsClient { private String initializeUserAgent() { final String userAgentComment = String.format(Locale.ROOT, "(JavaJRE %s; %s %s)", - System.getProperty(AbfsHttpConstants.JAVA_VERSION), - System.getProperty(AbfsHttpConstants.OS_NAME) - .replaceAll(AbfsHttpConstants.SINGLE_WHITE_SPACE, AbfsHttpConstants.EMPTY_STRING), - System.getProperty(AbfsHttpConstants.OS_VERSION)); + System.getProperty(JAVA_VERSION), + System.getProperty(OS_NAME) + .replaceAll(SINGLE_WHITE_SPACE, EMPTY_STRING), + System.getProperty(OS_VERSION)); - return String.format(AbfsHttpConstants.CLIENT_VERSION + " %s", userAgentComment); + return String.format(CLIENT_VERSION + " %s", userAgentComment); } -} \ No newline at end of file +} 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 0ea936569b9..53f69004d8c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -30,12 +30,12 @@ import com.fasterxml.jackson.core.JsonFactory; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonToken; import com.fasterxml.jackson.databind.ObjectMapper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Represents an HTTP operation. @@ -427,4 +427,4 @@ public class AbfsHttpOperation { private boolean isNullInputStream(InputStream stream) { return stream == null ? true : false; } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 6554380ebd0..848ce8ac953 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 @@ -29,7 +29,7 @@ import org.apache.hadoop.fs.FileSystem.Statistics; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; /** - * The AbfsInputStream for AbfsClient + * The AbfsInputStream for AbfsClient. */ public class AbfsInputStream extends FSInputStream { private final AbfsClient client; @@ -59,7 +59,6 @@ public class AbfsInputStream extends FSInputStream { final int bufferSize, final int readAheadQueueDepth, final String eTag) { - super(); this.client = client; this.statistics = statistics; this.path = path; @@ -379,4 +378,4 @@ public class AbfsInputStream extends FSInputStream { public boolean markSupported() { return false; } -} \ No newline at end of file +} 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 de5c934d64e..2dbcee57f59 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 @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.azurebfs.services; import java.io.IOException; +import java.io.InterruptedIOException; import java.io.OutputStream; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.LinkedBlockingQueue; @@ -35,7 +36,7 @@ import org.apache.hadoop.fs.Syncable; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; /** - * The BlobFsOutputStream for Rest AbfsClient + * The BlobFsOutputStream for Rest AbfsClient. */ public class AbfsOutputStream extends OutputStream implements Syncable { private final AbfsClient client; @@ -79,8 +80,8 @@ public class AbfsOutputStream extends OutputStream implements Syncable { maxConcurrentRequestCount, 10L, TimeUnit.SECONDS, - new LinkedBlockingQueue()); - this.completionService = new ExecutorCompletionService(this.threadExecutor); + new LinkedBlockingQueue<>()); + this.completionService = new ExecutorCompletionService<>(this.threadExecutor); } /** @@ -111,9 +112,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable { @Override public synchronized void write(final byte[] data, final int off, final int length) throws IOException { - if (this.lastError != null) { - throw this.lastError; - } + maybeThrowLastError(); Preconditions.checkArgument(data != null, "null data"); @@ -142,6 +141,19 @@ public class AbfsOutputStream extends OutputStream implements Syncable { } } + /** + * Throw the last error recorded if not null. + * After the stream is closed, this is always set to + * an exception, so acts as a guard against method invocation once + * closed. + * @throws IOException if lastError is set + */ + private void maybeThrowLastError() throws IOException { + if (lastError != null) { + throw lastError; + } + } + /** * Flushes this output stream and forces any buffered output bytes to be * written out. If any data remains in the payload it is committed to the @@ -150,7 +162,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable { */ @Override public void flush() throws IOException { - this.flushInternalAsync(); + flushInternalAsync(); } /** Similar to posix fsync, flush out the data in client's user buffer @@ -159,7 +171,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable { */ @Override public void hsync() throws IOException { - this.flushInternal(); + flushInternal(); } /** Flush out the data in client's user buffer. After the return of @@ -168,7 +180,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable { */ @Override public void hflush() throws IOException { - this.flushInternal(); + flushInternal(); } /** @@ -186,34 +198,30 @@ public class AbfsOutputStream extends OutputStream implements Syncable { } try { - this.flushInternal(); - this.threadExecutor.shutdown(); + flushInternal(); + threadExecutor.shutdown(); } finally { - this.lastError = new IOException(FSExceptionMessages.STREAM_IS_CLOSED); - this.buffer = null; - this.bufferIndex = 0; - this.closed = true; - this.writeOperations.clear(); - if (!this.threadExecutor.isShutdown()) { - this.threadExecutor.shutdownNow(); + lastError = new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + buffer = null; + bufferIndex = 0; + closed = true; + writeOperations.clear(); + if (!threadExecutor.isShutdown()) { + threadExecutor.shutdownNow(); } } } private synchronized void flushInternal() throws IOException { - if (this.lastError != null) { - throw this.lastError; - } - this.writeCurrentBufferToService(); - this.flushWrittenBytesToService(); + maybeThrowLastError(); + writeCurrentBufferToService(); + flushWrittenBytesToService(); } private synchronized void flushInternalAsync() throws IOException { - if (this.lastError != null) { - throw this.lastError; - } - this.writeCurrentBufferToService(); - this.flushWrittenBytesToServiceAsync(); + maybeThrowLastError(); + writeCurrentBufferToService(); + flushWrittenBytesToServiceAsync(); } private synchronized void writeCurrentBufferToService() throws IOException { @@ -221,19 +229,19 @@ public class AbfsOutputStream extends OutputStream implements Syncable { return; } - final byte[] bytes = this.buffer; + final byte[] bytes = buffer; final int bytesLength = bufferIndex; - this.buffer = new byte[bufferSize]; - this.bufferIndex = 0; - final long offset = this.position; - this.position += bytesLength; + buffer = new byte[bufferSize]; + bufferIndex = 0; + final long offset = position; + position += bytesLength; - if (this.threadExecutor.getQueue().size() >= maxConcurrentRequestCount * 2) { - this.waitForTaskToComplete(); + if (threadExecutor.getQueue().size() >= maxConcurrentRequestCount * 2) { + waitForTaskToComplete(); } - final Future job = this.completionService.submit(new Callable() { + final Future job = completionService.submit(new Callable() { @Override public Void call() throws Exception { client.append(path, offset, bytes, 0, @@ -242,25 +250,25 @@ public class AbfsOutputStream extends OutputStream implements Syncable { } }); - this.writeOperations.add(new WriteOperation(job, offset, bytesLength)); + writeOperations.add(new WriteOperation(job, offset, bytesLength)); // Try to shrink the queue shrinkWriteOperationQueue(); } private synchronized void flushWrittenBytesToService() throws IOException { - for (WriteOperation writeOperation : this.writeOperations) { + for (WriteOperation writeOperation : writeOperations) { try { writeOperation.task.get(); } catch (Exception ex) { - if (AzureBlobFileSystemException.class.isInstance(ex.getCause())) { - ex = AzureBlobFileSystemException.class.cast(ex.getCause()); + if (ex.getCause() instanceof AzureBlobFileSystemException) { + ex = (AzureBlobFileSystemException)ex.getCause(); } - this.lastError = new IOException(ex); - throw this.lastError; + lastError = new IOException(ex); + throw lastError; } } - flushWrittenBytesToServiceInternal(this.position, false); + flushWrittenBytesToServiceInternal(position, false); } private synchronized void flushWrittenBytesToServiceAsync() throws IOException { @@ -273,7 +281,8 @@ public class AbfsOutputStream extends OutputStream implements Syncable { this.lastTotalAppendOffset = 0; } - private synchronized void flushWrittenBytesToServiceInternal(final long offset, final boolean retainUncommitedData) throws IOException { + private synchronized void flushWrittenBytesToServiceInternal(final long offset, + final boolean retainUncommitedData) throws IOException { try { client.flush(path, offset, retainUncommitedData); } catch (AzureBlobFileSystemException ex) { @@ -288,31 +297,33 @@ public class AbfsOutputStream extends OutputStream implements Syncable { */ private synchronized void shrinkWriteOperationQueue() throws IOException { try { - while (this.writeOperations.peek() != null && this.writeOperations.peek().task.isDone()) { - this.writeOperations.peek().task.get(); - this.lastTotalAppendOffset += this.writeOperations.peek().length; - this.writeOperations.remove(); + while (writeOperations.peek() != null && writeOperations.peek().task.isDone()) { + writeOperations.peek().task.get(); + lastTotalAppendOffset += writeOperations.peek().length; + writeOperations.remove(); } } catch (Exception e) { - if (AzureBlobFileSystemException.class.isInstance(e.getCause())) { - this.lastError = IOException.class.cast(e.getCause()); + if (e.getCause() instanceof AzureBlobFileSystemException) { + lastError = (AzureBlobFileSystemException)e.getCause(); } else { - this.lastError = new IOException(e); + lastError = new IOException(e); } - throw this.lastError; + throw lastError; } } private void waitForTaskToComplete() throws IOException { boolean completed; - for (completed = false; this.completionService.poll() != null; completed = true) {} + for (completed = false; completionService.poll() != null; completed = true) { + // keep polling until there is no data + } if (!completed) { try { - this.completionService.take(); + completionService.take(); } catch (InterruptedException e) { - this.lastError = new IOException(e); - throw this.lastError; + lastError = (IOException)new InterruptedIOException(e.toString()).initCause(e); + throw lastError; } } } @@ -332,4 +343,4 @@ public class AbfsOutputStream extends OutputStream implements Syncable { this.length = length; } } -} \ No newline at end of file +} 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 17fc35afcb6..61263985002 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java @@ -23,15 +23,16 @@ import java.net.HttpURLConnection; import java.net.URL; import java.util.List; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException; -import org.slf4j.Logger; - /** - * The AbfsRestOperation for Rest AbfsClient + * The AbfsRestOperation for Rest AbfsClient. */ public class AbfsRestOperation { // Blob FS client, which has the credentials, retry policy, and logs. @@ -47,7 +48,7 @@ public class AbfsRestOperation { // request body and all the download methods have a response body. private final boolean hasRequestBody; - private final Logger logger; + private final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); // For uploads, this is the request entity body. For downloads, // this will hold the response entity body. @@ -79,7 +80,6 @@ public class AbfsRestOperation { this.requestHeaders = requestHeaders; this.hasRequestBody = (AbfsHttpConstants.HTTP_METHOD_PUT.equals(method) || AbfsHttpConstants.HTTP_METHOD_PATCH.equals(method)); - this.logger = client.LOG; } /** @@ -150,11 +150,11 @@ public class AbfsRestOperation { httpOperation.processResponse(buffer, bufferOffset, bufferLength); } catch (IOException ex) { - if (logger.isDebugEnabled()) { + if (LOG.isDebugEnabled()) { if (httpOperation != null) { - logger.debug("HttpRequestFailure: " + httpOperation.toString(), ex); + LOG.debug("HttpRequestFailure: " + httpOperation.toString(), ex); } else { - logger.debug("HttpRequestFailure: " + method + "," + url, ex); + LOG.debug("HttpRequestFailure: " + method + "," + url, ex); } } if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) { @@ -163,8 +163,8 @@ public class AbfsRestOperation { return false; } - if (logger.isDebugEnabled()) { - logger.debug("HttpRequest: " + httpOperation.toString()); + if (LOG.isDebugEnabled()) { + LOG.debug("HttpRequest: " + httpOperation.toString()); } if (client.getRetryPolicy().shouldRetry(retryCount, httpOperation.getStatusCode())) { @@ -175,4 +175,4 @@ public class AbfsRestOperation { return true; } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java index bac66af8824..36248533125 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java @@ -18,13 +18,13 @@ package org.apache.hadoop.fs.azurebfs.services; -import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; - import java.util.HashMap; import java.util.Map; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; + /** - * The UrlQueryBuilder for Rest AbfsClient + * The UrlQueryBuilder for Rest AbfsClient. */ public class AbfsUriQueryBuilder { private Map parameters; 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 54aa1abd698..5eb7a6639a6 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 @@ -43,7 +43,7 @@ public class ExponentialRetryPolicy { private static final int DEFAULT_MAX_BACKOFF = 1000 * 30; /** - *Represents the default minimum amount of time used when calculating the exponential + * Represents the default minimum amount of time used when calculating the exponential * delay between retries. */ private static final int DEFAULT_MIN_BACKOFF = 1000 * 3; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java index 1fac13dcaa8..00e4f008ad0 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java @@ -18,10 +18,10 @@ package org.apache.hadoop.fs.azurebfs.services; -import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; - import java.util.concurrent.CountDownLatch; +import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; + class ReadBuffer { private AbfsInputStream stream; 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 164e54992ae..5b71cf05225 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 @@ -28,7 +28,7 @@ import java.util.Stack; import java.util.concurrent.CountDownLatch; /** - * The Read Buffer Manager for Rest AbfsClient + * The Read Buffer Manager for Rest AbfsClient. */ final class ReadBufferManager { private static final Logger LOGGER = LoggerFactory.getLogger(ReadBufferManager.class); @@ -40,11 +40,11 @@ final class ReadBufferManager { private Thread[] threads = new Thread[NUM_THREADS]; private byte[][] buffers; // array of byte[] buffers, to hold the data that is read - private Stack freeList = new Stack(); // indices in buffers[] array that are available + private Stack freeList = new Stack<>(); // indices in buffers[] array that are available - private Queue readAheadQueue = new LinkedList(); // queue of requests that are not picked up by any worker thread yet - private LinkedList inProgressList = new LinkedList(); // requests being processed by worker threads - private LinkedList completedReadList = new LinkedList(); // buffers available for reading + private Queue readAheadQueue = new LinkedList<>(); // queue of requests that are not picked up by any worker thread yet + private LinkedList inProgressList = new LinkedList<>(); // requests being processed by worker threads + private LinkedList completedReadList = new LinkedList<>(); // buffers available for reading private static final ReadBufferManager BUFFER_MANAGER; // singleton, initialized in static initialization block static { @@ -85,7 +85,7 @@ final class ReadBufferManager { /** - * {@link AbfsInputStream} calls this method to queue read-aheads + * {@link AbfsInputStream} calls this method to queue read-aheads. * * @param stream The {@link AbfsInputStream} for which to do the read-ahead * @param requestedOffset The offset in the file which shoukd be read @@ -93,15 +93,15 @@ final class ReadBufferManager { */ void queueReadAhead(final AbfsInputStream stream, final long requestedOffset, final int requestedLength) { if (LOGGER.isTraceEnabled()) { - LOGGER.trace("Start Queueing readAhead for " + stream.getPath() + " offset " + requestedOffset - + " length " + requestedLength); + LOGGER.trace("Start Queueing readAhead for {} offset {} length {}", + stream.getPath(), requestedOffset, requestedLength); } ReadBuffer buffer; synchronized (this) { if (isAlreadyQueued(stream, requestedOffset)) { return; // already queued, do not queue again } - if (freeList.size() == 0 && !tryEvict()) { + if (freeList.isEmpty() && !tryEvict()) { return; // no buffers available, cannot queue anything } @@ -121,8 +121,8 @@ final class ReadBufferManager { notifyAll(); } if (LOGGER.isTraceEnabled()) { - LOGGER.trace("Done q-ing readAhead for file " + stream.getPath() + " offset " + requestedOffset - + " buffer idx " + buffer.getBufferindex()); + LOGGER.trace("Done q-ing readAhead for file {} offset {} buffer idx {}", + stream.getPath(), requestedOffset, buffer.getBufferindex()); } } @@ -144,7 +144,8 @@ final class ReadBufferManager { int getBlock(final AbfsInputStream stream, final long position, final int length, final byte[] buffer) { // not synchronized, so have to be careful with locking if (LOGGER.isTraceEnabled()) { - LOGGER.trace("getBlock for file " + stream.getPath() + " position " + position + " thread " + Thread.currentThread().getName()); + LOGGER.trace("getBlock for file {} position {} thread {}", + stream.getPath(), position, Thread.currentThread().getName()); } waitForProcess(stream, position); @@ -155,12 +156,13 @@ final class ReadBufferManager { } if (bytesRead > 0) { if (LOGGER.isTraceEnabled()) { - LOGGER.trace("Done read from Cache for " + stream.getPath() + " position " + position + " length " + bytesRead); + LOGGER.trace("Done read from Cache for {} position {} length {}", + stream.getPath(), position, bytesRead); } return bytesRead; } - // otherwise, just say we got nothing - calling thread can do it's own read + // otherwise, just say we got nothing - calling thread can do its own read return 0; } @@ -179,8 +181,8 @@ final class ReadBufferManager { if (readBuf != null) { // if in in-progress queue, then block for it try { if (LOGGER.isTraceEnabled()) { - LOGGER.trace("got a relevant read buffer for file " + stream.getPath() + " offset " + readBuf.getOffset() - + " buffer idx " + readBuf.getBufferindex()); + LOGGER.trace("got a relevant read buffer for file {} offset {} buffer idx {}", + stream.getPath(), readBuf.getOffset(), readBuf.getBufferindex()); } readBuf.getLatch().await(); // blocking wait on the caller stream's thread // Note on correctness: readBuf gets out of inProgressList only in 1 place: after worker thread @@ -193,8 +195,8 @@ final class ReadBufferManager { Thread.currentThread().interrupt(); } if (LOGGER.isTraceEnabled()) { - LOGGER.trace("latch done for file " + stream.getPath() + " buffer idx " + readBuf.getBufferindex() - + " length " + readBuf.getLength()); + LOGGER.trace("latch done for file {} buffer idx {} length {}", + stream.getPath(), readBuf.getBufferindex(), readBuf.getLength()); } } } @@ -254,8 +256,8 @@ final class ReadBufferManager { freeList.push(buf.getBufferindex()); completedReadList.remove(buf); if (LOGGER.isTraceEnabled()) { - LOGGER.trace("Evicting buffer idx " + buf.getBufferindex() + "; was used for file " + buf.getStream().getPath() - + " offset " + buf.getOffset() + " length " + buf.getLength()); + LOGGER.trace("Evicting buffer idx {}; was used for file {} offset {} length {}", + buf.getBufferindex(), buf.getStream().getPath(), buf.getOffset(), buf.getLength()); } return true; } @@ -344,13 +346,14 @@ final class ReadBufferManager { inProgressList.add(buffer); } if (LOGGER.isTraceEnabled()) { - LOGGER.trace("ReadBufferWorker picked file " + buffer.getStream().getPath() + " for offset " + buffer.getOffset()); + LOGGER.trace("ReadBufferWorker picked file {} for offset {}", + buffer.getStream().getPath(), buffer.getOffset()); } return buffer; } /** - * ReadBufferWorker thread calls this method to post completion + * ReadBufferWorker thread calls this method to post completion. * * @param buffer the buffer whose read was completed * @param result the {@link ReadBufferStatus} after the read operation in the worker thread @@ -358,8 +361,8 @@ final class ReadBufferManager { */ void doneReading(final ReadBuffer buffer, final ReadBufferStatus result, final int bytesActuallyRead) { if (LOGGER.isTraceEnabled()) { - LOGGER.trace("ReadBufferWorker completed file " + buffer.getStream().getPath() + " for offset " + buffer.getOffset() - + " bytes " + bytesActuallyRead); + LOGGER.trace("ReadBufferWorker completed file {} for offset {} bytes {}", + buffer.getStream().getPath(), buffer.getOffset(), bytesActuallyRead); } synchronized (this) { inProgressList.remove(buffer); @@ -380,8 +383,9 @@ final class ReadBufferManager { /** * Similar to System.currentTimeMillis, except implemented with System.nanoTime(). * System.currentTimeMillis can go backwards when system clock is changed (e.g., with NTP time synchronization), - * making it unsuitable for measuring time intervals. nanotime is strictly monotonically increasing, - * so it is much more suitable to measuring intervals. + * making it unsuitable for measuring time intervals. nanotime is strictly monotonically increasing per CPU core. + * Note: it is not monotonic across Sockets, and even within a CPU, its only the + * more recent parts which share a clock across all cores. * * @return current time in milliseconds */ diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java index 2d0c96e15c0..af69de0f089 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java @@ -18,10 +18,10 @@ package org.apache.hadoop.fs.azurebfs.services; -import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; - import java.util.concurrent.CountDownLatch; +import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; + class ReadBufferWorker implements Runnable { protected static final CountDownLatch UNLEASH_WORKERS = new CountDownLatch(1); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java index dd598921915..105a1a2dbf6 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java @@ -22,6 +22,7 @@ import javax.crypto.Mac; import javax.crypto.spec.SecretKeySpec; import java.io.UnsupportedEncodingException; import java.net.HttpURLConnection; +import java.net.URL; import java.net.URLDecoder; import java.text.DateFormat; import java.text.SimpleDateFormat; @@ -38,11 +39,11 @@ import java.util.TimeZone; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.commons.codec.Charsets; +import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; -import org.apache.commons.codec.binary.Base64; -import org.apache.commons.codec.Charsets; /** * Represents the shared key credentials used to access an Azure Storage * account. @@ -89,7 +90,7 @@ public class SharedKeyCredentials { } private String computeHmac256(final String stringToSign) { - byte[] utf8Bytes = null; + byte[] utf8Bytes; try { utf8Bytes = stringToSign.getBytes(AbfsHttpConstants.UTF_8); } catch (final UnsupportedEncodingException e) { @@ -158,7 +159,7 @@ public class SharedKeyCredentials { } /** - * Initialie the HmacSha256 associated with the account key. + * Initialize the HmacSha256 associated with the account key. */ private void initializeMac() { // Initializes the HMAC-SHA256 Mac and SecretKey. @@ -171,7 +172,7 @@ public class SharedKeyCredentials { } /** - * Append a string to a string builder with a newline constant + * Append a string to a string builder with a newline constant. * * @param builder the StringBuilder object * @param element the string to append. @@ -194,9 +195,10 @@ public class SharedKeyCredentials { * @param conn the HttpURLConnection for the operation. * @return A canonicalized string. */ - private static String canonicalizeHttpRequest(final java.net.URL address, final String accountName, - final String method, final String contentType, final long contentLength, final String date, - final HttpURLConnection conn) throws UnsupportedEncodingException { + private static String canonicalizeHttpRequest(final URL address, + final String accountName, final String method, final String contentType, + final long contentLength, final String date, final HttpURLConnection conn) + throws UnsupportedEncodingException { // The first element should be the Method of the request. // I.e. GET, POST, PUT, or HEAD. @@ -246,7 +248,8 @@ public class SharedKeyCredentials { * @param accountName the account name for the request. * @return the canonicalized resource string. */ - private static String getCanonicalizedResource(final java.net.URL address, final String accountName) throws UnsupportedEncodingException { + private static String getCanonicalizedResource(final URL address, + final String accountName) throws UnsupportedEncodingException { // Resource path final StringBuilder resourcepath = new StringBuilder(AbfsHttpConstants.FORWARD_SLASH); resourcepath.append(accountName); @@ -263,7 +266,7 @@ public class SharedKeyCredentials { final Map queryVariables = parseQueryString(address.getQuery()); - final Map lowercasedKeyNameValue = new HashMap(); + final Map lowercasedKeyNameValue = new HashMap<>(); for (final Entry entry : queryVariables.entrySet()) { // sort the value and organize it as comma separated values @@ -303,14 +306,17 @@ public class SharedKeyCredentials { } /** - * Gets all the values for the given header in the one to many map, performs a trimStart() on each return value + * Gets all the values for the given header in the one to many map, + * performs a trimStart() on each return value. * * @param headers a one to many map of key / values representing the header values for the connection. * @param headerName the name of the header to lookup * @return an ArrayList of all trimmed values corresponding to the requested headerName. This may be empty * if the header is not found. */ - private static ArrayList getHeaderValues(final Map> headers, final String headerName) { + private static ArrayList getHeaderValues( + final Map> headers, + final String headerName) { final ArrayList arrayOfValues = new ArrayList(); List values = null; @@ -338,7 +344,7 @@ public class SharedKeyCredentials { * @return a HashMap of the key values. */ private static HashMap parseQueryString(String parseString) throws UnsupportedEncodingException { - final HashMap retVals = new HashMap(); + final HashMap retVals = new HashMap<>(); if (parseString == null || parseString.isEmpty()) { return retVals; } diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md new file mode 100644 index 00000000000..a4b3483de70 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md @@ -0,0 +1,72 @@ + + +# Hadoop Azure Support: ABFS — Azure Data Lake Storage Gen2 + + + +## Introduction + +The `hadoop-azure` module provides support for the Azure Data Lake Storage Gen2 +storage layer through the "abfs" connector + +To make it part of Apache Hadoop's default classpath, simply make sure that +`HADOOP_OPTIONAL_TOOLS` in `hadoop-env.sh` has `hadoop-azure` in the list. + +## Features + +* Read and write data stored in an Azure Blob Storage account. +* *Fully Consistent* view of the storage across all clients. +* Can read data written through the wasb: connector. +* Present a hierarchical file system view by implementing the standard Hadoop + [`FileSystem`](../api/org/apache/hadoop/fs/FileSystem.html) interface. +* Supports configuration of multiple Azure Blob Storage accounts. +* Can act as a source or destination of data in Hadoop MapReduce, Apache Hive, Apache Spark +* Tested at scale on both Linux and Windows. +* Can be used as a replacement for HDFS on Hadoop clusters deployed in Azure infrastructure. + + + +## Limitations + +* File last access time is not tracked. + + +## Technical notes + +### Security + +### Consistency and Concurrency + +*TODO*: complete/review + +The abfs client has a fully consistent view of the store, which has complete Create Read Update and Delete consistency for data and metadata. +(Compare and contrast with S3 which only offers Create consistency; S3Guard adds CRUD to metadata, but not the underlying data). + +### Performance + +*TODO*: check these. + +* File Rename: `O(1)`. +* Directory Rename: `O(files)`. +* Directory Delete: `O(files)`. + +## Testing ABFS + +See the relevant section in [Testing Azure](testing_azure.html). + +## References + +* [A closer look at Azure Data Lake Storage Gen2](https://azure.microsoft.com/en-gb/blog/a-closer-look-at-azure-data-lake-storage-gen2/); +MSDN Article from June 28, 2018. 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 b58e68be5f3..c148807aaa0 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md @@ -574,3 +574,79 @@ mvn test -Dtest=CleanupTestContainers This will delete the containers; the output log of the test run will provide the details and summary of the operation. + + +## Testing ABFS + +The ABFS Connector tests share the same account as the wasb tests; this is +needed for cross-connector compatibility tests. + +This makes for a somewhat complex set of configuration options. + +Here are the settings for an account `ACCOUNTNAME` + +```xml + + abfs.account.name + ACCOUNTNAME + + + + abfs.account.full.name + ${abfs.account.name}.dfs.core.windows.net + + + + abfs.account.key + SECRETKEY== + + + + fs.azure.account.key.ACCOUNTNAME.dfs.core.windows.net + ${abfs.account.key} + + + + fs.azure.account.key.ACCOUNTNAME.blob.core.windows.net + ${abfs.account.key} + + + + fs.azure.test.account.key.ACCOUNTNAME.dfs.core.windows.net + ${abfs.account.key} + + + + fs.azure.test.account.key.ACCOUNTNAME.blob.core.windows.net + ${abfs.account.key} + + + + fs.azure.account.key.ACCOUNTNAME + ${abfs.account.key} + + + + fs.azure.test.account.key.ACCOUNTNAME + ${abfs.account.key} + + + + fs.azure.test.account.name + ${abfs.account.full.name} + + + + fs.contract.test.fs.abfs + abfs://TESTCONTAINER@ACCOUNTNAME.dfs.core.windows.net + Container for contract tests + + + + fs.contract.test.fs.abfss + abfss://TESTCONTAINER@ACCOUNTNAME.dfs.core.windows.net + Container for contract tests + + + +``` diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractAppend.java index fd21bd20b2e..db4a843287a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractAppend.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractAppend.java @@ -18,10 +18,19 @@ package org.apache.hadoop.fs.azure.contract; +import java.io.FileNotFoundException; +import java.io.IOException; + +import org.junit.Test; + import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.AbstractContractAppendTest; import org.apache.hadoop.fs.contract.AbstractFSContract; import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Append test, skipping one of them. @@ -38,4 +47,18 @@ public class ITestAzureNativeContractAppend extends AbstractContractAppendTest { public void testRenameFileBeingAppended() throws Throwable { skip("Skipping as renaming an opened file is not supported"); } + + /** + * Wasb returns a different exception, so change the intercept logic here. + */ + @Override + @Test + public void testAppendDirectory() throws Exception { + final FileSystem fs = getFileSystem(); + + final Path folderPath = path("testAppendDirectory"); + fs.mkdirs(folderPath); + intercept(FileNotFoundException.class, + () -> fs.append(folderPath)); + } } 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 new file mode 100644 index 00000000000..106fa09e438 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -0,0 +1,304 @@ +/** + * 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.URI; +import java.util.Hashtable; +import java.util.UUID; +import java.util.concurrent.Callable; + +import com.google.common.base.Preconditions; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.rules.TestName; +import org.junit.rules.Timeout; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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.azure.AbstractWasbTestWithTimeout; +import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore; +import org.apache.hadoop.fs.azure.NativeAzureFileSystem; +import org.apache.hadoop.fs.azure.integration.AzureTestConstants; +import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation; +import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.utils.UriUtils; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.io.IOUtils; + +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.*; +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; +import static org.junit.Assume.assumeTrue; +import static org.hamcrest.CoreMatchers.*; + +/** + * Base for AzureBlobFileSystem Integration tests. + * + * Important: This is for integration tests only. + */ +public abstract class AbstractAbfsIntegrationTest extends + AbstractWasbTestWithTimeout { + + private static final Logger LOG = + LoggerFactory.getLogger(AbstractAbfsIntegrationTest.class); + + private final boolean isEmulator; + private NativeAzureFileSystem wasb; + private AzureBlobFileSystem abfs; + private String abfsScheme; + + private Configuration configuration; + private String fileSystemName; + private String accountName; + private String testUrl; + + protected AbstractAbfsIntegrationTest(final boolean secure) { + this(secure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME); + } + + protected AbstractAbfsIntegrationTest() { + this(FileSystemUriSchemes.ABFS_SCHEME); + } + + private AbstractAbfsIntegrationTest(final String scheme) { + abfsScheme = scheme; + fileSystemName = ABFS_TEST_CONTAINER_PREFIX + UUID.randomUUID().toString(); + configuration = new Configuration(); + configuration.addResource(ABFS_TEST_RESOURCE_XML); + + String accountName = configuration.get(FS_AZURE_TEST_ACCOUNT_NAME, ""); + assumeTrue("Not set: " + FS_AZURE_TEST_ACCOUNT_NAME, + !accountName.isEmpty()); + assertThat("The key in " + FS_AZURE_TEST_ACCOUNT_KEY_PREFIX + + " is not bound to an ABFS account", + accountName, containsString("dfs.core.windows.net")); + String fullKey = FS_AZURE_TEST_ACCOUNT_KEY_PREFIX + + accountName; + assumeTrue("Not set: " + fullKey, + configuration.get(fullKey) != null); + + final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName(); + URI defaultUri = null; + + try { + defaultUri = new URI(abfsScheme, abfsUrl, null, null, null); + } catch (Exception ex) { + throw new AssertionError(ex); + } + + this.testUrl = defaultUri.toString(); + configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); + configuration.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true); + this.isEmulator = this.configuration.getBoolean(FS_AZURE_EMULATOR_ENABLED, false); + this.accountName = this.configuration.get(FS_AZURE_TEST_ACCOUNT_NAME); + } + + + @Before + public void setup() throws Exception { + //Create filesystem first to make sure getWasbFileSystem() can return an existing filesystem. + createFileSystem(); + + if (!isEmulator) { + final URI wasbUri = new URI(abfsUrlToWasbUrl(getTestUrl())); + final AzureNativeFileSystemStore azureNativeFileSystemStore = + new AzureNativeFileSystemStore(); + azureNativeFileSystemStore.initialize( + wasbUri, + getConfiguration(), + new AzureFileSystemInstrumentation(getConfiguration())); + + wasb = new NativeAzureFileSystem(azureNativeFileSystemStore); + wasb.initialize(wasbUri, configuration); + } + } + + @After + public void teardown() throws Exception { + try { + IOUtils.closeStream(wasb); + wasb = null; + + if (abfs == null) { + return; + } + + final AzureBlobFileSystemStore abfsStore = abfs.getAbfsStore(); + abfsStore.deleteFilesystem(); + + AbfsRestOperationException ex = intercept( + AbfsRestOperationException.class, + new Callable>() { + @Override + public Hashtable call() throws Exception { + return abfsStore.getFilesystemProperties(); + } + }); + if (FILE_SYSTEM_NOT_FOUND.getStatusCode() != ex.getStatusCode()) { + LOG.warn("Deleted test filesystem may still exist: {}", abfs, ex); + } + } catch (Exception e) { + LOG.warn("During cleanup: {}", e, e); + } finally { + IOUtils.closeStream(abfs); + abfs = null; + } + } + + public AzureBlobFileSystem getFileSystem() throws IOException { + return abfs; + } + + /** + * Creates the filesystem; updates the {@link #abfs} field. + * @return the created filesystem. + * @throws IOException failure during create/init. + */ + public AzureBlobFileSystem createFileSystem() throws IOException { + Preconditions.checkState(abfs == null, + "existing ABFS instance exists: %s", abfs); + abfs = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + return abfs; + } + + + protected NativeAzureFileSystem getWasbFileSystem() { + return wasb; + } + + protected String getHostName() { + return configuration.get(FS_AZURE_TEST_HOST_NAME); + } + + protected void setTestUrl(String testUrl) { + this.testUrl = testUrl; + } + + protected String getTestUrl() { + return testUrl; + } + + protected void setFileSystemName(String fileSystemName) { + this.fileSystemName = fileSystemName; + } + protected String getFileSystemName() { + return fileSystemName; + } + + protected String getAccountName() { + return configuration.get(FS_AZURE_TEST_ACCOUNT_NAME); + } + + protected String getAccountKey() { + return configuration.get( + FS_AZURE_TEST_ACCOUNT_KEY_PREFIX + + getAccountName()); + } + + protected Configuration getConfiguration() { + return configuration; + } + + protected boolean isEmulator() { + return isEmulator; + } + + /** + * Write a buffer to a file. + * @param path path + * @param buffer buffer + * @throws IOException failure + */ + protected void write(Path path, byte[] buffer) throws IOException { + ContractTestUtils.writeDataset(getFileSystem(), path, buffer, buffer.length, + CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT, false); + } + + /** + * Touch a file in the test store. Will overwrite any existing file. + * @param path path + * @throws IOException failure. + */ + protected void touch(Path path) throws IOException { + ContractTestUtils.touch(getFileSystem(), path); + } + + protected static String wasbUrlToAbfsUrl(final String wasbUrl) { + return convertTestUrls( + wasbUrl, FileSystemUriSchemes.WASB_SCHEME, FileSystemUriSchemes.WASB_SECURE_SCHEME, FileSystemUriSchemes.WASB_DNS_PREFIX, + FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME, FileSystemUriSchemes.ABFS_DNS_PREFIX); + } + + protected static String abfsUrlToWasbUrl(final String abfsUrl) { + 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); + } + + 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) { + String data = null; + 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; + } + + public Path getTestPath() { + Path path = new Path(UriUtils.generateUniqueTestPath()); + return path; + } + + /** + * Create a path under the test path provided by + * {@link #getTestPath()}. + * @param filepath path string in + * @return a path qualified by the test filesystem + * @throws IOException IO problems + */ + protected Path path(String filepath) throws IOException { + return getFileSystem().makeQualified( + new Path(getTestPath(), filepath)); + } + +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsScaleTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsScaleTest.java new file mode 100644 index 00000000000..cfda7a7eef9 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsScaleTest.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.azure.integration.AzureTestConstants; + +import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.assumeScaleTestsEnabled; + +/** + * Integration tests at bigger scale; configurable as to + * size, off by default. + */ +public class AbstractAbfsScaleTest extends AbstractAbfsIntegrationTest { + + protected static final Logger LOG = + LoggerFactory.getLogger(AbstractAbfsScaleTest.class); + + @Override + protected int getTestTimeoutMillis() { + return AzureTestConstants.SCALE_TEST_TIMEOUT_MILLIS; + } + + @Override + public void setup() throws Exception { + super.setup(); + LOG.debug("Scale test operation count = {}", getOperationCount()); + assumeScaleTestsEnabled(getConfiguration()); + } + + protected long getOperationCount() { + return getConfiguration().getLong(AzureTestConstants.KEY_OPERATION_COUNT, + AzureTestConstants.DEFAULT_OPERATION_COUNT); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java deleted file mode 100644 index 74a530c9cce..00000000000 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/DependencyInjectedTest.java +++ /dev/null @@ -1,206 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs; - -import java.net.URI; -import java.util.UUID; -import java.util.concurrent.Callable; - -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore; -import org.apache.hadoop.fs.azure.NativeAzureFileSystem; -import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation; -import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; -import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; -import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; - -import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; -import static org.junit.Assert.assertEquals; -import static org.junit.Assume.assumeNotNull; - -/** - * Provide dependencies for AzureBlobFileSystem tests. - */ -public abstract class DependencyInjectedTest { - private final boolean isEmulator; - private NativeAzureFileSystem wasb; - private String abfsScheme; - - private Configuration configuration; - private String fileSystemName; - private String accountName; - private String testUrl; - - public static final String TEST_CONTAINER_PREFIX = "abfs-testcontainer-"; - - public DependencyInjectedTest(final boolean secure) { - this(secure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME); - } - - protected DependencyInjectedTest() { - this(FileSystemUriSchemes.ABFS_SCHEME); - } - - private DependencyInjectedTest(final String scheme) { - abfsScheme = scheme; - fileSystemName = TEST_CONTAINER_PREFIX + UUID.randomUUID().toString(); - configuration = new Configuration(); - configuration.addResource("azure-bfs-test.xml"); - - assumeNotNull(configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME)); - assumeNotNull(configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_KEY_PREFIX + configuration.get(TestConfigurationKeys - .FS_AZURE_TEST_ACCOUNT_NAME))); - - final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName(); - URI defaultUri = null; - - try { - defaultUri = new URI(abfsScheme, abfsUrl, null, null, null); - } catch (Exception ex) { - Assert.fail(ex.getMessage()); - } - - this.testUrl = defaultUri.toString(); - configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); - configuration.setBoolean(ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true); - this.isEmulator = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false); - this.accountName = this.configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME); - } - - @Before - public void initialize() throws Exception { - //Create filesystem first to make sure getWasbFileSystem() can return an existed filesystem. - this.getFileSystem(); - - if (!this.isEmulator) { - final URI wasbUri = new URI(abfsUrlToWasbUrl(this.getTestUrl())); - final AzureNativeFileSystemStore azureNativeFileSystemStore = new AzureNativeFileSystemStore(); - azureNativeFileSystemStore.initialize( - wasbUri, - this.getConfiguration(), - new AzureFileSystemInstrumentation(this.getConfiguration())); - - this.wasb = new NativeAzureFileSystem(azureNativeFileSystemStore); - this.wasb.initialize(wasbUri, configuration); - } - } - - @After - public void testCleanup() throws Exception { - if (this.wasb != null) { - this.wasb.close(); - } - - FileSystem.closeAll(); - - final AzureBlobFileSystem fs = this.getFileSystem(); - final AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); - abfsStore.deleteFilesystem(); - - AbfsRestOperationException ex = intercept( - AbfsRestOperationException.class, - new Callable() { - @Override - public Void call() throws Exception { - fs.getAbfsStore().getFilesystemProperties(); - return null; - } - }); - - assertEquals(FILE_SYSTEM_NOT_FOUND.getStatusCode(), ex.getStatusCode()); - } - - public AzureBlobFileSystem getFileSystem() throws Exception { - return (AzureBlobFileSystem) FileSystem.get(this.configuration); - } - - protected NativeAzureFileSystem getWasbFileSystem() { - return this.wasb; - } - - protected String getHostName() { - return configuration.get(TestConfigurationKeys.FS_AZURE_TEST_HOST_NAME); - } - - protected void updateTestUrl(String testUrl) { - this.testUrl = testUrl; - } - protected String getTestUrl() { - return testUrl; - } - - protected void updateFileSystemName(String fileSystemName) { - this.fileSystemName = fileSystemName; - } - protected String getFileSystemName() { - return fileSystemName; - } - - protected String getAccountName() { - return configuration.get(TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_NAME); - } - - protected String getAccountKey() { - return configuration.get( - TestConfigurationKeys.FS_AZURE_TEST_ACCOUNT_KEY_PREFIX - + getAccountName()); - } - - protected Configuration getConfiguration() { - return this.configuration; - } - - protected boolean isEmulator() { - return isEmulator; - } - - protected static String wasbUrlToAbfsUrl(final String wasbUrl) { - return convertTestUrls( - wasbUrl, FileSystemUriSchemes.WASB_SCHEME, FileSystemUriSchemes.WASB_SECURE_SCHEME, FileSystemUriSchemes.WASB_DNS_PREFIX, - FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME, FileSystemUriSchemes.ABFS_DNS_PREFIX); - } - - protected static String abfsUrlToWasbUrl(final String abfsUrl) { - 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); - } - - 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) { - String data = null; - if (url.startsWith(fromNonSecureScheme + "://")) { - data = url.replace(fromNonSecureScheme + "://", toNonSecureScheme + "://"); - } else if (url.startsWith(fromSecureScheme + "://")) { - data = url.replace(fromSecureScheme + "://", toSecureScheme + "://"); - } - - data = data.replace("." + fromDnsPrefix + ".", "." + toDnsPrefix + "."); - return data; - } -} 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 10d42d1399d..f2e26ec19f2 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 @@ -25,13 +25,13 @@ import org.junit.Test; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; - -import static org.junit.Assert.assertEquals; +import org.apache.hadoop.fs.contract.ContractTestUtils; /** * Test append operations. */ -public class ITestAzureBlobFileSystemAppend extends DependencyInjectedTest { +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"); public ITestAzureBlobFileSystemAppend() { @@ -40,7 +40,7 @@ public class ITestAzureBlobFileSystemAppend extends DependencyInjectedTest { @Test(expected = FileNotFoundException.class) public void testAppendDirShouldFail() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); final Path filePath = TEST_FILE_PATH; fs.mkdirs(filePath); fs.append(filePath, 0); @@ -48,21 +48,21 @@ public class ITestAzureBlobFileSystemAppend extends DependencyInjectedTest { @Test public void testAppendWithLength0() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - FSDataOutputStream stream = fs.create(TEST_FILE_PATH); - final byte[] b = new byte[1024]; - new Random().nextBytes(b); - stream.write(b, 1000, 0); - - assertEquals(0, stream.getPos()); + final AzureBlobFileSystem fs = getFileSystem(); + try(FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { + final byte[] b = new byte[1024]; + new Random().nextBytes(b); + stream.write(b, 1000, 0); + assertEquals(0, stream.getPos()); + } } @Test(expected = FileNotFoundException.class) public void testAppendFileAfterDelete() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); final Path filePath = TEST_FILE_PATH; - fs.create(filePath); + ContractTestUtils.touch(fs, filePath); fs.delete(filePath, false); fs.append(filePath); @@ -70,7 +70,7 @@ public class ITestAzureBlobFileSystemAppend extends DependencyInjectedTest { @Test(expected = FileNotFoundException.class) public void testAppendDirectory() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); final Path folderPath = TEST_FOLDER_PATH; fs.mkdirs(folderPath); fs.append(folderPath); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java index d107c9d008b..d6964814cc6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java @@ -27,13 +27,11 @@ import org.junit.Test; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - /** * Test AzureBlobFileSystem back compatibility with WASB. */ -public class ITestAzureBlobFileSystemBackCompat extends DependencyInjectedTest { +public class ITestAzureBlobFileSystemBackCompat extends + AbstractAbfsIntegrationTest { public ITestAzureBlobFileSystemBackCompat() { super(); } @@ -54,13 +52,13 @@ public class ITestAzureBlobFileSystemBackCompat extends DependencyInjectedTest { blockBlob.uploadText(""); FileStatus[] fileStatuses = fs.listStatus(new Path("/test/10/")); - assertEquals(fileStatuses.length, 2); - assertEquals(fileStatuses[0].getPath().getName(), "10"); + assertEquals(2, fileStatuses.length); + assertEquals("10", fileStatuses[0].getPath().getName()); assertTrue(fileStatuses[0].isDirectory()); - assertEquals(fileStatuses[0].getLen(), 0); - assertEquals(fileStatuses[1].getPath().getName(), "123"); + assertEquals(0, fileStatuses[0].getLen()); + assertEquals("123", fileStatuses[1].getPath().getName()); assertTrue(fileStatuses[1].isDirectory()); - assertEquals(fileStatuses[1].getLen(), 0); + assertEquals(0, fileStatuses[1].getLen()); } private String getBlobConnectionString() { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java index c158e03e622..90eff97854c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java @@ -33,30 +33,29 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile; /** * Test copy operation. */ -public class ITestAzureBlobFileSystemCopy extends DependencyInjectedTest { +public class ITestAzureBlobFileSystemCopy extends AbstractAbfsIntegrationTest { public ITestAzureBlobFileSystemCopy() { super(); } @Test public void testCopyFromLocalFileSystem() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); Path localFilePath = new Path(System.getProperty("test.build.data", "azure_test")); - FileSystem localFs = FileSystem.get(new Configuration()); + FileSystem localFs = FileSystem.getLocal(new Configuration()); localFs.delete(localFilePath, true); try { writeString(localFs, localFilePath, "Testing"); Path dstPath = new Path("copiedFromLocal"); assertTrue(FileUtil.copy(localFs, localFilePath, fs, dstPath, false, fs.getConf())); - assertTrue(fs.exists(dstPath)); + assertIsFile(fs, dstPath); assertEquals("Testing", readString(fs, dstPath)); fs.delete(dstPath, true); } finally { @@ -65,36 +64,32 @@ public class ITestAzureBlobFileSystemCopy extends DependencyInjectedTest { } private String readString(FileSystem fs, Path testFile) throws IOException { - FSDataInputStream inputStream = fs.open(testFile); - String ret = readString(inputStream); - inputStream.close(); - return ret; + return readString(fs.open(testFile)); } private String readString(FSDataInputStream inputStream) throws IOException { - BufferedReader reader = new BufferedReader(new InputStreamReader( - inputStream)); - final int bufferSize = 1024; - char[] buffer = new char[bufferSize]; - int count = reader.read(buffer, 0, bufferSize); - if (count > bufferSize) { - throw new IOException("Exceeded buffer size"); + try (BufferedReader reader = new BufferedReader(new InputStreamReader( + inputStream))) { + final int bufferSize = 1024; + char[] buffer = new char[bufferSize]; + int count = reader.read(buffer, 0, bufferSize); + if (count > bufferSize) { + throw new IOException("Exceeded buffer size"); + } + return new String(buffer, 0, count); } - inputStream.close(); - return new String(buffer, 0, count); } private void writeString(FileSystem fs, Path path, String value) throws IOException { - FSDataOutputStream outputStream = fs.create(path, true); - writeString(outputStream, value); + writeString(fs.create(path, true), value); } private void writeString(FSDataOutputStream outputStream, String value) throws IOException { - BufferedWriter writer = new BufferedWriter(new OutputStreamWriter( - outputStream)); - writer.write(value); - writer.close(); + try(BufferedWriter writer = new BufferedWriter( + new OutputStreamWriter(outputStream))) { + writer.write(value); + } } } 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 c9b99e6d8a2..1e43f9a3601 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 @@ -24,18 +24,17 @@ import java.util.EnumSet; import org.junit.Test; import org.apache.hadoop.fs.CreateFlag; -import org.apache.hadoop.fs.FileAlreadyExistsException; -import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile; /** * Test create operation. */ -public class ITestAzureBlobFileSystemCreate extends DependencyInjectedTest { +public class ITestAzureBlobFileSystemCreate 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_CHILD_FILE = "childFile"; @@ -43,68 +42,65 @@ public class ITestAzureBlobFileSystemCreate extends DependencyInjectedTest { super(); } - @Test(expected = FileAlreadyExistsException.class) - public void testCreateFileWithExistingDir() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - fs.mkdirs(TEST_FOLDER_PATH); - fs.create(TEST_FOLDER_PATH); - } - @Test - public void testEnsureFileCreated() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - fs.create(TEST_FILE_PATH); - - FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH); - assertNotNull(fileStatus); + public void testEnsureFileCreatedImmediately() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + FSDataOutputStream out = fs.create(TEST_FILE_PATH); + try { + assertIsFile(fs, TEST_FILE_PATH); + } finally { + out.close(); + } + assertIsFile(fs, TEST_FILE_PATH); } @Test @SuppressWarnings("deprecation") public void testCreateNonRecursive() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE); try { fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null); - assertTrue("Should've thrown", false); - } catch (FileNotFoundException e) { + fail("Should've thrown"); + } catch (FileNotFoundException expected) { } fs.mkdirs(TEST_FOLDER_PATH); fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null) .close(); - assertTrue(fs.exists(testFile)); + assertIsFile(fs, testFile); } @Test @SuppressWarnings("deprecation") public void testCreateNonRecursive1() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE); try { fs.createNonRecursive(testFile, FsPermission.getDefault(), EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE), 1024, (short) 1, 1024, null); - assertTrue("Should've thrown", false); - } catch (FileNotFoundException e) { + fail("Should've thrown"); + } catch (FileNotFoundException expected) { } fs.mkdirs(TEST_FOLDER_PATH); fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null) .close(); - assertTrue(fs.exists(testFile)); + assertIsFile(fs, testFile); + } @Test @SuppressWarnings("deprecation") public void testCreateNonRecursive2() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); Path testFile = new Path(TEST_FOLDER_PATH, TEST_CHILD_FILE); try { fs.createNonRecursive(testFile, FsPermission.getDefault(), false, 1024, (short) 1, 1024, null); - assertTrue("Should've thrown", false); + fail("Should've thrown"); } catch (FileNotFoundException e) { } fs.mkdirs(TEST_FOLDER_PATH); fs.createNonRecursive(testFile, true, 1024, (short) 1, 1024, null) .close(); - assertTrue(fs.exists(testFile)); + assertIsFile(fs, testFile); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index 372a0876b11..91d1723a752 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 @@ -28,71 +28,79 @@ import java.util.concurrent.Future; import org.junit.Test; +import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import static org.junit.Assert.assertEquals; +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted; +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test delete operation. */ -public class ITestAzureBlobFileSystemDelete extends DependencyInjectedTest { +public class ITestAzureBlobFileSystemDelete extends + AbstractAbfsIntegrationTest { public ITestAzureBlobFileSystemDelete() { super(); } @Test public void testDeleteRoot() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); fs.mkdirs(new Path("/testFolder0")); fs.mkdirs(new Path("/testFolder1")); fs.mkdirs(new Path("/testFolder2")); - fs.create(new Path("/testFolder1/testfile")); - fs.create(new Path("/testFolder1/testfile2")); - fs.create(new Path("/testFolder1/testfile3")); + touch(new Path("/testFolder1/testfile")); + touch(new Path("/testFolder1/testfile2")); + touch(new Path("/testFolder1/testfile3")); - FileStatus[] ls = fs.listStatus(new Path("/")); - assertEquals(4, ls.length); // and user dir + Path root = new Path("/"); + FileStatus[] ls = fs.listStatus(root); + assertEquals(3, ls.length); - fs.delete(new Path("/"), true); - ls = fs.listStatus(new Path("/")); - assertEquals(0, ls.length); + fs.delete(root, true); + ls = fs.listStatus(root); + assertEquals("listing size", 0, ls.length); } - @Test(expected = FileNotFoundException.class) + @Test() public void testOpenFileAfterDelete() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - fs.create(new Path("/testFile")); - fs.delete(new Path("/testFile"), false); + final AzureBlobFileSystem fs = getFileSystem(); + Path testfile = new Path("/testFile"); + touch(testfile); + assertDeleted(fs, testfile, false); - fs.open(new Path("/testFile")); + intercept(FileNotFoundException.class, + () -> fs.open(testfile)); } - @Test(expected = FileNotFoundException.class) + @Test public void testEnsureFileIsDeleted() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - fs.create(new Path("testfile")); - fs.delete(new Path("testfile"), false); - - fs.getFileStatus(new Path("testfile")); + final AzureBlobFileSystem fs = getFileSystem(); + Path testfile = new Path("testfile"); + touch(testfile); + assertDeleted(fs, testfile, false); + assertPathDoesNotExist(fs, "deleted", testfile); } - @Test(expected = FileNotFoundException.class) + @Test public void testDeleteDirectory() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - fs.mkdirs(new Path("testfile")); + final AzureBlobFileSystem fs = getFileSystem(); + Path dir = new Path("testfile"); + fs.mkdirs(dir); fs.mkdirs(new Path("testfile/test1")); fs.mkdirs(new Path("testfile/test1/test2")); - fs.delete(new Path("testfile"), true); - fs.getFileStatus(new Path("testfile")); + assertDeleted(fs, dir, true); + assertPathDoesNotExist(fs, "deleted", dir); } - @Test(expected = FileNotFoundException.class) + @Test public void testDeleteFirstLevelDirectory() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - final List tasks = new ArrayList<>(); + final AzureBlobFileSystem fs = getFileSystem(); + final List> tasks = new ArrayList<>(); ExecutorService es = Executors.newFixedThreadPool(10); for (int i = 0; i < 1000; i++) { @@ -100,7 +108,7 @@ public class ITestAzureBlobFileSystemDelete extends DependencyInjectedTest { Callable callable = new Callable() { @Override public Void call() throws Exception { - fs.create(fileName); + touch(fileName); return null; } }; @@ -113,7 +121,12 @@ public class ITestAzureBlobFileSystemDelete extends DependencyInjectedTest { } es.shutdownNow(); - fs.delete(new Path("/test"), true); - fs.getFileStatus(new Path("/test")); + Path dir = new Path("/test"); + // first try a non-recursive delete, expect failure + intercept(FileAlreadyExistsException.class, + () -> fs.delete(dir, false)); + assertDeleted(fs, dir, true); + assertPathDoesNotExist(fs, "deleted", dir); + } } 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 ad22f999fe6..057dfa03115 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 @@ -38,7 +38,7 @@ import static org.junit.Assert.assertArrayEquals; /** * Test end to end between ABFS client and ABFS server. */ -public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest { +public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest { private static final Path TEST_FILE = new Path("testfile"); private static final int TEST_BYTE = 100; private static final int TEST_OFFSET = 100; @@ -53,11 +53,11 @@ public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest { @Test public void testWriteOneByteToFile() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - FSDataOutputStream stream = fs.create(TEST_FILE); + final AzureBlobFileSystem fs = getFileSystem(); - stream.write(TEST_BYTE); - stream.close(); + try(FSDataOutputStream stream = fs.create(TEST_FILE)) { + stream.write(TEST_BYTE); + } FileStatus fileStatus = fs.getFileStatus(TEST_FILE); assertEquals(1, fileStatus.getLen()); @@ -65,52 +65,52 @@ public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest { @Test public void testReadWriteBytesToFile() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); testWriteOneByteToFile(); - FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE); - int i = inputStream.read(); - inputStream.close(); - - assertEquals(TEST_BYTE, i); + try(FSDataInputStream inputStream = fs.open(TEST_FILE, + TEST_DEFAULT_BUFFER_SIZE)) { + assertEquals(TEST_BYTE, inputStream.read()); + } } @Test (expected = IOException.class) public void testOOBWrites() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); int readBufferSize = fs.getAbfsStore().getAbfsConfiguration().getReadBufferSize(); - fs.create(TEST_FILE); - FSDataOutputStream writeStream = fs.create(TEST_FILE); - byte[] bytesToRead = new byte[readBufferSize]; final byte[] b = new byte[2 * readBufferSize]; new Random().nextBytes(b); - writeStream.write(b); - writeStream.flush(); - writeStream.close(); - FSDataInputStream readStream = fs.open(TEST_FILE); - readStream.read(bytesToRead, 0, readBufferSize); + try(FSDataOutputStream writeStream = fs.create(TEST_FILE)) { + writeStream.write(b); + writeStream.flush(); + } - writeStream = fs.create(TEST_FILE); - writeStream.write(b); - writeStream.flush(); - writeStream.close(); + try (FSDataInputStream readStream = fs.open(TEST_FILE)) { + assertEquals(readBufferSize, + readStream.read(bytesToRead, 0, readBufferSize)); - readStream.read(bytesToRead, 0, readBufferSize); - readStream.close(); + try (FSDataOutputStream writeStream = fs.create(TEST_FILE)) { + writeStream.write(b); + writeStream.flush(); + } + + assertEquals(readBufferSize, + readStream.read(bytesToRead, 0, readBufferSize)); + } } @Test public void testWriteWithBufferOffset() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - final FSDataOutputStream stream = fs.create(TEST_FILE); + final AzureBlobFileSystem fs = getFileSystem(); final byte[] b = new byte[1024 * 1000]; new Random().nextBytes(b); - stream.write(b, TEST_OFFSET, b.length - TEST_OFFSET); - stream.close(); + try(final FSDataOutputStream stream = fs.create(TEST_FILE)) { + stream.write(b, TEST_OFFSET, b.length - TEST_OFFSET); + } final byte[] r = new byte[TEST_DEFAULT_READ_BUFFER_SIZE]; FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE); @@ -124,13 +124,11 @@ public class ITestAzureBlobFileSystemE2E extends DependencyInjectedTest { @Test public void testReadWriteHeavyBytesToFileWithSmallerChunks() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - final FSDataOutputStream stream = fs.create(TEST_FILE); + final AzureBlobFileSystem fs = getFileSystem(); final byte[] writeBuffer = new byte[5 * 1000 * 1024]; new Random().nextBytes(writeBuffer); - stream.write(writeBuffer); - stream.close(); + write(TEST_FILE, writeBuffer); final byte[] readBuffer = new byte[5 * 1000 * 1024]; FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java index 616253bca35..04690de2403 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java @@ -26,7 +26,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.junit.Assert; import org.junit.Test; import org.apache.hadoop.fs.FSDataInputStream; @@ -35,28 +34,24 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; - /** * Test end to end between ABFS client and ABFS server with heavy traffic. */ -public class ITestAzureBlobFileSystemE2EScale extends DependencyInjectedTest { +public class ITestAzureBlobFileSystemE2EScale extends + AbstractAbfsScaleTest { private static final int TEN = 10; private static final int ONE_THOUSAND = 1000; private static final int BASE_SIZE = 1024; private static final int ONE_MB = 1024 * 1024; private static final int DEFAULT_WRITE_TIMES = 100; - private static final Path TEST_FILE = new Path("testfile"); + private static final Path TEST_FILE = new Path("ITestAzureBlobFileSystemE2EScale"); public ITestAzureBlobFileSystemE2EScale() { - super(); } @Test - public void testWriteHeavyBytesToFile() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + public void testWriteHeavyBytesToFileAcrossThreads() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); final FSDataOutputStream stream = fs.create(TEST_FILE); ExecutorService es = Executors.newFixedThreadPool(TEN); @@ -65,7 +60,8 @@ public class ITestAzureBlobFileSystemE2EScale extends DependencyInjectedTest { new Random().nextBytes(b); List> tasks = new ArrayList<>(); - for (int i = 0; i < DEFAULT_WRITE_TIMES; i++) { + int operationCount = DEFAULT_WRITE_TIMES; + for (int i = 0; i < operationCount; i++) { Callable callable = new Callable() { @Override public Void call() throws Exception { @@ -86,48 +82,38 @@ public class ITestAzureBlobFileSystemE2EScale extends DependencyInjectedTest { es.shutdownNow(); FileStatus fileStatus = fs.getFileStatus(TEST_FILE); - assertEquals(testWriteBufferSize * DEFAULT_WRITE_TIMES, fileStatus.getLen()); - } - - @Test - public void testReadWriteHeavyBytesToFile() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - final FSDataOutputStream stream = fs.create(TEST_FILE); - - int testBufferSize = 5 * TEN * ONE_THOUSAND * BASE_SIZE; - final byte[] b = new byte[testBufferSize]; - new Random().nextBytes(b); - stream.write(b); - stream.close(); - - final byte[] r = new byte[testBufferSize]; - FSDataInputStream inputStream = fs.open(TEST_FILE, 4 * ONE_MB); - int result = inputStream.read(r); - inputStream.close(); - - assertNotEquals(-1, result); - assertArrayEquals(r, b); + assertEquals(testWriteBufferSize * operationCount, fileStatus.getLen()); } @Test public void testReadWriteHeavyBytesToFileWithStatistics() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - final FSDataOutputStream stream = fs.create(TEST_FILE); - final FileSystem.Statistics abfsStatistics = fs.getFsStatistics(); - abfsStatistics.reset(); + final AzureBlobFileSystem fs = getFileSystem(); + final FileSystem.Statistics abfsStatistics; + int testBufferSize; + final byte[] sourceData; + try(final FSDataOutputStream stream = fs.create(TEST_FILE)) { + abfsStatistics = fs.getFsStatistics(); + abfsStatistics.reset(); - int testBufferSize = 5 * TEN * ONE_THOUSAND * BASE_SIZE; - final byte[] b = new byte[testBufferSize]; - new Random().nextBytes(b); - stream.write(b); - stream.close(); + testBufferSize = 5 * TEN * ONE_THOUSAND * BASE_SIZE; + sourceData = new byte[testBufferSize]; + new Random().nextBytes(sourceData); + stream.write(sourceData); + } - final byte[] r = new byte[testBufferSize]; - FSDataInputStream inputStream = fs.open(TEST_FILE, 4 * ONE_MB); - inputStream.read(r); - inputStream.close(); + final byte[] remoteData = new byte[testBufferSize]; + int bytesRead; + try (FSDataInputStream inputStream = fs.open(TEST_FILE, 4 * ONE_MB)) { + bytesRead = inputStream.read(remoteData); + } + + String stats = abfsStatistics.toString(); + assertEquals("Bytes read in " + stats, + remoteData.length, abfsStatistics.getBytesRead()); + assertEquals("bytes written in " + stats, + sourceData.length, abfsStatistics.getBytesWritten()); + assertEquals("bytesRead from read() call", testBufferSize, bytesRead ); + assertArrayEquals("round tripped data", sourceData, remoteData); - Assert.assertEquals(r.length, abfsStatistics.getBytesRead()); - Assert.assertEquals(b.length, abfsStatistics.getBytesWritten()); } } 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 bfa662d455b..791694bf0f1 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 @@ -18,6 +18,8 @@ package org.apache.hadoop.fs.azurebfs; +import java.io.IOException; + import org.junit.Test; import org.apache.hadoop.fs.FileStatus; @@ -25,12 +27,11 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; -import static org.junit.Assert.assertEquals; - /** * Test FileStatus. */ -public class ITestAzureBlobFileSystemFileStatus extends DependencyInjectedTest { +public class ITestAzureBlobFileSystemFileStatus extends + AbstractAbfsIntegrationTest { private static final Path TEST_FILE = new Path("testFile"); private static final Path TEST_FOLDER = new Path("testDir"); public ITestAzureBlobFileSystemFileStatus() { @@ -41,24 +42,38 @@ public class ITestAzureBlobFileSystemFileStatus extends DependencyInjectedTest { public void testEnsureStatusWorksForRoot() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - fs.getFileStatus(new Path("/")); - fs.listStatus(new Path("/")); + Path root = new Path("/"); + FileStatus[] rootls = fs.listStatus(root); + assertEquals("root listing", 0, rootls.length); } @Test public void testFileStatusPermissionsAndOwnerAndGroup() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - fs.create(TEST_FILE); + touch(TEST_FILE); + validateStatus(fs, TEST_FILE); + } + + private FileStatus validateStatus(final AzureBlobFileSystem fs, final Path name) + throws IOException { + FileStatus fileStatus = fs.getFileStatus(name); + String errorInStatus = "error in " + fileStatus + " from " + fs; + assertEquals(errorInStatus + ": permission", + new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL), + fileStatus.getPermission()); + assertEquals(errorInStatus + ": owner", + fs.getOwnerUser(), fileStatus.getOwner()); + assertEquals(errorInStatus + ": group", + fs.getOwnerUserPrimaryGroup(), fileStatus.getGroup()); + return fileStatus; + } + + @Test + public void testFolderStatusPermissionsAndOwnerAndGroup() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); fs.mkdirs(TEST_FOLDER); - FileStatus fileStatus = fs.getFileStatus(TEST_FILE); - assertEquals(new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL), fileStatus.getPermission()); - assertEquals(fs.getOwnerUser(), fileStatus.getGroup()); - assertEquals(fs.getOwnerUserPrimaryGroup(), fileStatus.getOwner()); - - fileStatus = fs.getFileStatus(TEST_FOLDER); - assertEquals(new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL), fileStatus.getPermission()); - assertEquals(fs.getOwnerUser(), fileStatus.getGroup()); - assertEquals(fs.getOwnerUserPrimaryGroup(), fileStatus.getOwner()); + validateStatus(fs, TEST_FOLDER); } + } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index 8c2e8ce32dd..d90f0186da1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -34,14 +34,10 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; - /** * Test flush operation. */ -public class ITestAzureBlobFileSystemFlush extends DependencyInjectedTest { +public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { private static final int BASE_SIZE = 1024; private static final int ONE_THOUSAND = 1000; private static final int TEST_BUFFER_SIZE = 5 * ONE_THOUSAND * BASE_SIZE; @@ -56,146 +52,145 @@ public class ITestAzureBlobFileSystemFlush extends DependencyInjectedTest { } @Test - public void testAbfsOutputStreamAsyncFlushWithRetainUncommitedData() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - final FSDataOutputStream stream = fs.create(TEST_FILE_PATH); + public void testAbfsOutputStreamAsyncFlushWithRetainUncommittedData() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final byte[] b; + try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { + b = new byte[TEST_BUFFER_SIZE]; + new Random().nextBytes(b); - final byte[] b = new byte[TEST_BUFFER_SIZE]; - new Random().nextBytes(b); + for (int i = 0; i < 2; i++) { + stream.write(b); - for (int i = 0; i < 2; i++) { + for (int j = 0; j < FLUSH_TIMES; j++) { + stream.flush(); + Thread.sleep(10); + } + } + } + + final byte[] r = new byte[TEST_BUFFER_SIZE]; + try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB)) { + while (inputStream.available() != 0) { + int result = inputStream.read(r); + + assertNotEquals("read returned -1", -1, result); + assertArrayEquals("buffer read from stream", r, b); + } + } + } + + @Test + public void testAbfsOutputStreamSyncFlush() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final byte[] b; + try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { + b = new byte[TEST_BUFFER_SIZE]; + new Random().nextBytes(b); stream.write(b); - for (int j = 0; j < FLUSH_TIMES; j++) { - stream.flush(); + for (int i = 0; i < FLUSH_TIMES; i++) { + stream.hsync(); + stream.hflush(); Thread.sleep(10); } } - stream.close(); - final byte[] r = new byte[TEST_BUFFER_SIZE]; - FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB); - - while (inputStream.available() != 0) { + try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB)) { int result = inputStream.read(r); assertNotEquals(-1, result); assertArrayEquals(r, b); } - - inputStream.close(); - } - - @Test - public void testAbfsOutputStreamSyncFlush() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - final FSDataOutputStream stream = fs.create(TEST_FILE_PATH); - - final byte[] b = new byte[TEST_BUFFER_SIZE]; - new Random().nextBytes(b); - stream.write(b); - - for (int i = 0; i < FLUSH_TIMES; i++) { - stream.hsync(); - stream.hflush(); - Thread.sleep(10); - } - stream.close(); - - final byte[] r = new byte[TEST_BUFFER_SIZE]; - FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB); - int result = inputStream.read(r); - - assertNotEquals(-1, result); - assertArrayEquals(r, b); - - inputStream.close(); } @Test public void testWriteHeavyBytesToFileSyncFlush() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - final FSDataOutputStream stream = fs.create(TEST_FILE_PATH); - final FileSystem.Statistics abfsStatistics = fs.getFsStatistics(); - abfsStatistics.reset(); + final AzureBlobFileSystem fs = getFileSystem(); + final FileSystem.Statistics abfsStatistics; + ExecutorService es; + try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { + abfsStatistics = fs.getFsStatistics(); + abfsStatistics.reset(); - ExecutorService es = Executors.newFixedThreadPool(10); + es = Executors.newFixedThreadPool(10); - final byte[] b = new byte[TEST_BUFFER_SIZE]; - new Random().nextBytes(b); + final byte[] b = new byte[TEST_BUFFER_SIZE]; + new Random().nextBytes(b); - List> tasks = new ArrayList<>(); - for (int i = 0; i < FLUSH_TIMES; i++) { - Callable callable = new Callable() { - @Override - public Void call() throws Exception { - stream.write(b); - return null; - } - }; + List> tasks = new ArrayList<>(); + for (int i = 0; i < FLUSH_TIMES; i++) { + Callable callable = new Callable() { + @Override + public Void call() throws Exception { + stream.write(b); + return null; + } + }; - tasks.add(es.submit(callable)); - } + tasks.add(es.submit(callable)); + } - boolean shouldStop = false; - while (!shouldStop) { - shouldStop = true; - for (Future task : tasks) { - if (!task.isDone()) { - stream.hsync(); - shouldStop = false; - Thread.sleep(THREAD_SLEEP_TIME); + boolean shouldStop = false; + while (!shouldStop) { + shouldStop = true; + for (Future task : tasks) { + if (!task.isDone()) { + stream.hsync(); + shouldStop = false; + Thread.sleep(THREAD_SLEEP_TIME); + } } } - } - tasks.clear(); - stream.close(); + tasks.clear(); + } es.shutdownNow(); FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH); - assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, fileStatus.getLen()); - assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, abfsStatistics.getBytesWritten()); + long expectedWrites = (long) TEST_BUFFER_SIZE * FLUSH_TIMES; + assertEquals("Wrong file length in " + fileStatus, expectedWrites, fileStatus.getLen()); + assertEquals("wrong bytes Written count in " + abfsStatistics, + expectedWrites, abfsStatistics.getBytesWritten()); } @Test public void testWriteHeavyBytesToFileAsyncFlush() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - fs.create(TEST_FILE_PATH); - final FSDataOutputStream stream = fs.create(TEST_FILE_PATH); + final AzureBlobFileSystem fs = getFileSystem(); ExecutorService es = Executors.newFixedThreadPool(10); + try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { - final byte[] b = new byte[TEST_BUFFER_SIZE]; - new Random().nextBytes(b); + final byte[] b = new byte[TEST_BUFFER_SIZE]; + new Random().nextBytes(b); - List> tasks = new ArrayList<>(); - for (int i = 0; i < FLUSH_TIMES; i++) { - Callable callable = new Callable() { - @Override - public Void call() throws Exception { - stream.write(b); - return null; - } - }; + List> tasks = new ArrayList<>(); + for (int i = 0; i < FLUSH_TIMES; i++) { + Callable callable = new Callable() { + @Override + public Void call() throws Exception { + stream.write(b); + return null; + } + }; - tasks.add(es.submit(callable)); - } + tasks.add(es.submit(callable)); + } - boolean shouldStop = false; - while (!shouldStop) { - shouldStop = true; - for (Future task : tasks) { - if (!task.isDone()) { - stream.flush(); - shouldStop = false; + boolean shouldStop = false; + while (!shouldStop) { + shouldStop = true; + for (Future task : tasks) { + if (!task.isDone()) { + stream.flush(); + shouldStop = false; + } } } + Thread.sleep(THREAD_SLEEP_TIME); + tasks.clear(); } - Thread.sleep(THREAD_SLEEP_TIME); - tasks.clear(); - stream.close(); es.shutdownNow(); FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java index d2ed4008aa2..5a6e46db016 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java @@ -22,29 +22,32 @@ import java.io.FileNotFoundException; import org.junit.Test; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; /** * Test filesystem initialization and creation. */ -public class ITestAzureBlobFileSystemInitAndCreate extends DependencyInjectedTest { +public class ITestAzureBlobFileSystemInitAndCreate extends + AbstractAbfsIntegrationTest { public ITestAzureBlobFileSystemInitAndCreate() { - super(); this.getConfiguration().unset(ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION); } @Override - public void initialize() { + public void setup() { } @Override - public void testCleanup() { + public void teardown() { } @Test (expected = FileNotFoundException.class) public void ensureFilesystemWillNotBeCreatedIfCreationConfigIsNotSet() throws Exception { - super.initialize(); - this.getFileSystem(); + super.setup(); + final AzureBlobFileSystem fs = this.getFileSystem(); + FileStatus[] fileStatuses = fs.listStatus(new Path("/")); } -} \ No newline at end of file +} 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 6059766c2ad..b87abe68355 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 @@ -26,20 +26,21 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.junit.Assert; import org.junit.Test; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertFalse; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test listStatus operation. */ -public class ITestAzureBlobFileSystemListStatus extends DependencyInjectedTest { +public class ITestAzureBlobFileSystemListStatus extends + AbstractAbfsIntegrationTest { private static final int TEST_FILES_NUMBER = 6000; public ITestAzureBlobFileSystemListStatus() { super(); @@ -47,8 +48,8 @@ public class ITestAzureBlobFileSystemListStatus extends DependencyInjectedTest { @Test public void testListPath() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - final List tasks = new ArrayList<>(); + final AzureBlobFileSystem fs = getFileSystem(); + final List> tasks = new ArrayList<>(); ExecutorService es = Executors.newFixedThreadPool(10); for (int i = 0; i < TEST_FILES_NUMBER; i++) { @@ -56,7 +57,7 @@ public class ITestAzureBlobFileSystemListStatus extends DependencyInjectedTest { Callable callable = new Callable() { @Override public Void call() throws Exception { - fs.create(fileName); + touch(fileName); return null; } }; @@ -70,63 +71,101 @@ public class ITestAzureBlobFileSystemListStatus extends DependencyInjectedTest { es.shutdownNow(); FileStatus[] files = fs.listStatus(new Path("/")); - Assert.assertEquals(files.length, TEST_FILES_NUMBER + 1 /* user directory */); + assertEquals(TEST_FILES_NUMBER, files.length /* user directory */); } + /** + * Creates a file, verifies that listStatus returns it, + * even while the file is still open for writing. + */ @Test public void testListFileVsListDir() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - fs.create(new Path("/testFile")); - - FileStatus[] testFiles = fs.listStatus(new Path("/testFile")); - Assert.assertEquals(testFiles.length, 1); - Assert.assertFalse(testFiles[0].isDirectory()); + final AzureBlobFileSystem fs = getFileSystem(); + Path path = new Path("/testFile"); + try(FSDataOutputStream ignored = fs.create(path)) { + FileStatus[] testFiles = fs.listStatus(path); + assertEquals("length of test files", 1, testFiles.length); + FileStatus status = testFiles[0]; + assertIsFileReference(status); + } } @Test public void testListFileVsListDir2() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); fs.mkdirs(new Path("/testFolder")); fs.mkdirs(new Path("/testFolder/testFolder2")); fs.mkdirs(new Path("/testFolder/testFolder2/testFolder3")); - fs.create(new Path("/testFolder/testFolder2/testFolder3/testFile")); + Path testFile0Path = new Path("/testFolder/testFolder2/testFolder3/testFile"); + ContractTestUtils.touch(fs, testFile0Path); - FileStatus[] testFiles = fs.listStatus(new Path("/testFolder/testFolder2/testFolder3/testFile")); - Assert.assertEquals(testFiles.length, 1); - Assert.assertEquals(testFiles[0].getPath(), new Path(this.getTestUrl(), - "/testFolder/testFolder2/testFolder3/testFile")); - Assert.assertFalse(testFiles[0].isDirectory()); + FileStatus[] testFiles = fs.listStatus(testFile0Path); + assertEquals("Wrong listing size of file " + testFile0Path, + 1, testFiles.length); + FileStatus file0 = testFiles[0]; + assertEquals("Wrong path for " + file0, + new Path(getTestUrl(), "/testFolder/testFolder2/testFolder3/testFile"), + file0.getPath()); + assertIsFileReference(file0); } @Test(expected = FileNotFoundException.class) public void testListNonExistentDir() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); fs.listStatus(new Path("/testFile/")); } @Test public void testListFiles() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - fs.mkdirs(new Path("/test")); + final AzureBlobFileSystem fs = getFileSystem(); + Path testDir = new Path("/test"); + fs.mkdirs(testDir); FileStatus[] fileStatuses = fs.listStatus(new Path("/")); - assertEquals(fileStatuses.length, 2); + assertEquals(1, fileStatuses.length); fs.mkdirs(new Path("/test/sub")); - fileStatuses = fs.listStatus(new Path("/test")); - assertEquals(fileStatuses.length, 1); - assertEquals(fileStatuses[0].getPath().getName(), "sub"); - assertTrue(fileStatuses[0].isDirectory()); - assertEquals(fileStatuses[0].getLen(), 0); + fileStatuses = fs.listStatus(testDir); + assertEquals(1, fileStatuses.length); + assertEquals("sub", fileStatuses[0].getPath().getName()); + assertIsDirectoryReference(fileStatuses[0]); + Path childF = fs.makeQualified(new Path("/test/f")); + touch(childF); + fileStatuses = fs.listStatus(testDir); + assertEquals(2, fileStatuses.length); + final FileStatus childStatus = fileStatuses[0]; + assertEquals(childF, childStatus.getPath()); + assertEquals("f", childStatus.getPath().getName()); + assertIsFileReference(childStatus); + assertEquals(0, childStatus.getLen()); + final FileStatus status1 = fileStatuses[1]; + assertEquals("sub", status1.getPath().getName()); + assertIsDirectoryReference(status1); + // look at the child through getFileStatus + LocatedFileStatus locatedChildStatus = fs.listFiles(childF, false).next(); + assertIsFileReference(locatedChildStatus); - fs.create(new Path("/test/f")); - fileStatuses = fs.listStatus(new Path("/test")); - assertEquals(fileStatuses.length, 2); - assertEquals(fileStatuses[0].getPath().getName(), "f"); - assertFalse(fileStatuses[0].isDirectory()); - assertEquals(fileStatuses[0].getLen(), 0); - assertEquals(fileStatuses[1].getPath().getName(), "sub"); - assertTrue(fileStatuses[1].isDirectory()); - assertEquals(fileStatuses[1].getLen(), 0); + fs.delete(testDir, true); + intercept(FileNotFoundException.class, + () -> fs.listFiles(childF, false).next()); + + // do some final checks on the status (failing due to version checks) + assertEquals("Path mismatch of " + locatedChildStatus, + childF, locatedChildStatus.getPath()); + assertEquals("locatedstatus.equals(status)", + locatedChildStatus, childStatus); + assertEquals("status.equals(locatedstatus)", + childStatus, locatedChildStatus); + } + + private void assertIsDirectoryReference(FileStatus status) { + assertTrue("Not a directory: " + status, status.isDirectory()); + assertFalse("Not a directory: " + status, status.isFile()); + assertEquals(0, status.getLen()); + } + + private void assertIsFileReference(FileStatus status) { + assertFalse("Not a file: " + status, status.isDirectory()); + assertTrue("Not a file: " + status, status.isFile()); } } 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 b61908c13ea..1bb2c54b376 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 @@ -18,71 +18,30 @@ package org.apache.hadoop.fs.azurebfs; -import java.util.concurrent.Callable; - import org.junit.Test; -import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.Path; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; -import static org.junit.Assert.assertTrue; +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs; /** * Test mkdir operation. */ -public class ITestAzureBlobFileSystemMkDir extends DependencyInjectedTest { +public class ITestAzureBlobFileSystemMkDir extends AbstractAbfsIntegrationTest { public ITestAzureBlobFileSystemMkDir() { super(); } @Test public void testCreateDirWithExistingDir() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - assertTrue(fs.mkdirs(new Path("testFolder"))); - assertTrue(fs.mkdirs(new Path("testFolder"))); - } - - @Test(expected = FileAlreadyExistsException.class) - public void createDirectoryUnderFile() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - fs.create(new Path("testFile")); - fs.mkdirs(new Path("testFile/TestDirectory")); - } - - @Test - public void testCreateDirectoryOverExistingFiles() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - fs.create(new Path("/testPath")); - FileAlreadyExistsException ex = intercept( - FileAlreadyExistsException.class, - new Callable() { - @Override - public Void call() throws Exception { - fs.mkdirs(new Path("/testPath")); - return null; - } - }); - - assertTrue(ex instanceof FileAlreadyExistsException); - - fs.create(new Path("/testPath1/file1")); - ex = intercept( - FileAlreadyExistsException.class, - new Callable() { - @Override - public Void call() throws Exception { - fs.mkdirs(new Path("/testPath1/file1")); - return null; - } - }); - - assertTrue(ex instanceof FileAlreadyExistsException); + final AzureBlobFileSystem fs = getFileSystem(); + Path path = new Path("testFolder"); + assertMkdirs(fs, path); + assertMkdirs(fs, path); } @Test public void testCreateRoot() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - assertTrue(fs.mkdirs(new Path("/"))); + assertMkdirs(getFileSystem(), new Path("/")); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOpen.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOpen.java deleted file mode 100644 index fef7f47f720..00000000000 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOpen.java +++ /dev/null @@ -1,41 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs; - -import java.io.FileNotFoundException; - -import org.junit.Test; - -import org.apache.hadoop.fs.Path; - -/** - * Test open operation. - */ -public class ITestAzureBlobFileSystemOpen extends DependencyInjectedTest { - public ITestAzureBlobFileSystemOpen() throws Exception { - super(); - } - - @Test(expected = FileNotFoundException.class) - public void testOpenDirectory() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - fs.mkdirs(new Path("testFolder")); - fs.open(new Path("testFolder")); - } -} 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 8b96c69c8fc..c61de6764ea 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,32 +18,31 @@ package org.apache.hadoop.fs.azurebfs; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FSExceptionMessages; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.azure.NativeAzureFileSystem; -import org.apache.hadoop.fs.contract.ContractTestUtils; -import org.junit.Test; - import java.io.EOFException; import java.io.IOException; import java.util.Random; import java.util.concurrent.Callable; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertArrayEquals; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azure.NativeAzureFileSystem; +import org.apache.hadoop.fs.contract.ContractTestUtils; + +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test random read operation. */ -public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest { +public class ITestAzureBlobFileSystemRandomRead extends + AbstractAbfsScaleTest { private static final int KILOBYTE = 1024; private static final int MEGABYTE = KILOBYTE * KILOBYTE; private static final long TEST_FILE_SIZE = 8 * MEGABYTE; @@ -62,6 +61,9 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest { private static final String ABFS = "ABFS"; private static long testFileLength = 0; + private static final Logger LOG = + LoggerFactory.getLogger(ITestAzureBlobFileSystemRandomRead.class); + public ITestAzureBlobFileSystemRandomRead() throws Exception { super(); } @@ -76,7 +78,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest { // forward seek and read a kilobyte into first kilobyte of bufferV2 inputStream.seek(5 * MEGABYTE); int numBytesRead = inputStream.read(buffer, 0, KILOBYTE); - assertEquals(KILOBYTE, numBytesRead); + assertEquals("Wrong number of bytes read", KILOBYTE, numBytesRead); int len = MEGABYTE; int offset = buffer.length - len; @@ -84,7 +86,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest { // reverse seek and read a megabyte into last megabyte of bufferV1 inputStream.seek(3 * MEGABYTE); numBytesRead = inputStream.read(buffer, offset, len); - assertEquals(len, numBytesRead); + assertEquals("Wrong number of bytes read after seek", len, numBytesRead); } } @@ -391,7 +393,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest { afterSeekElapsedMs = sequentialRead(ABFS, this.getFileSystem(), true); ratio = afterSeekElapsedMs / beforeSeekElapsedMs; - System.out.println((String.format( + LOG.info((String.format( "beforeSeekElapsedMs=%1$d, afterSeekElapsedMs=%2$d, ratio=%3$.2f", (long) beforeSeekElapsedMs, (long) afterSeekElapsedMs, @@ -425,7 +427,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest { ratio = v2ElapsedMs / v1ElapsedMs; - System.out.println(String.format( + LOG.info(String.format( "v1ElapsedMs=%1$d, v2ElapsedMs=%2$d, ratio=%3$.2f", (long) v1ElapsedMs, (long) v2ElapsedMs, @@ -464,7 +466,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest { } long elapsedTimeMs = timer.elapsedTimeMs(); - System.out.println(String.format( + LOG.info(String.format( "v%1$s: bytesRead=%2$d, elapsedMs=%3$d, Mbps=%4$.2f," + " afterReverseSeek=%5$s", version, @@ -496,7 +498,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest { } while (bytesRead > 0 && totalBytesRead < minBytesToRead); long elapsedTimeMs = timer.elapsedTimeMs(); inputStream.close(); - System.out.println(String.format( + LOG.info(String.format( "v%1$d: totalBytesRead=%2$d, elapsedTimeMs=%3$d, Mbps=%4$.2f", version, totalBytesRead, @@ -535,7 +537,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest { character = (character == 'z') ? 'a' : (char) ((int) character + 1); } - System.out.println(String.format("Creating test file %s of size: %d ", TEST_FILE_PATH, TEST_FILE_SIZE)); + LOG.info(String.format("Creating test file %s of size: %d ", TEST_FILE_PATH, TEST_FILE_SIZE)); ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); try (FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) { @@ -544,7 +546,7 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest { outputStream.write(buffer); bytesWritten += buffer.length; } - System.out.println(String.format("Closing stream %s", outputStream)); + LOG.info("Closing stream {}", outputStream); ContractTestUtils.NanoTimer closeTimer = new ContractTestUtils.NanoTimer(); outputStream.close(); @@ -578,4 +580,4 @@ public class ITestAzureBlobFileSystemRandomRead extends DependencyInjectedTest { assertArrayEquals("Mismatch in read data", bufferV1, bufferV2); } -} \ No newline at end of file +} 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 a0e648ca191..1a0edaf54e8 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,7 +18,6 @@ package org.apache.hadoop.fs.azurebfs; -import java.io.FileNotFoundException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.Callable; @@ -26,93 +25,74 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.junit.Assert; import org.junit.Test; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory; +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.fs.contract.ContractTestUtils.assertIsFile; /** * Test rename operation. */ -public class ITestAzureBlobFileSystemRename extends DependencyInjectedTest { +public class ITestAzureBlobFileSystemRename extends + AbstractAbfsIntegrationTest { public ITestAzureBlobFileSystemRename() { - super(); } - @Test(expected = FileNotFoundException.class) + @Test public void testEnsureFileIsRenamed() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - fs.create(new Path("testfile")); - fs.rename(new Path("testfile"), new Path("testfile2")); + final AzureBlobFileSystem fs = getFileSystem(); + Path src = path("testEnsureFileIsRenamed-src"); + touch(src); + Path dest = path("testEnsureFileIsRenamed-dest"); + fs.delete(dest, true); + assertRenameOutcome(fs, src, dest, true); - FileStatus fileStatus = fs.getFileStatus(new Path("testfile2")); - assertNotNull(fileStatus); - - fs.getFileStatus(new Path("testfile")); + assertIsFile(fs, dest); + assertPathDoesNotExist(fs, "expected renamed", src); } @Test - public void testRenameFile() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - fs.mkdirs(new Path("/testSrc")); - fs.create(new Path("/testSrc/file1")); + public void testRenameFileUnderDir() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + Path sourceDir = new Path("/testSrc"); + assertMkdirs(fs, sourceDir); + String filename = "file1"; + Path file1 = new Path(sourceDir, filename); + touch(file1); - fs.rename(new Path("/testSrc"), new Path("/testDst")); - FileStatus[] fileStatus = fs.listStatus(new Path("/testDst")); - assertNotNull(fileStatus); + Path destDir = new Path("/testDst"); + assertRenameOutcome(fs, sourceDir, destDir, true); + FileStatus[] fileStatus = fs.listStatus(destDir); + assertNotNull("Null file status", fileStatus); + FileStatus status = fileStatus[0]; + assertEquals("Wrong filename in " + status, + filename, status.getPath().getName()); } @Test - public void testRenameFileUsingUnicode() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - //known issue: ListStatus operation to folders/files whose name contains '?' will fail - //This is because Auto rest client didn't encode '?' in the uri query parameters - String[] folders1 = new String[]{"/%2c%26", "/ÖáΠ⇒", "/A +B", "/A~`!@#$%^&*()-_+={};:'>,,, tasks = new ArrayList<>(); + final AzureBlobFileSystem fs = getFileSystem(); + final List> tasks = new ArrayList<>(); ExecutorService es = Executors.newFixedThreadPool(10); for (int i = 0; i < 1000; i++) { @@ -120,7 +100,7 @@ public class ITestAzureBlobFileSystemRename extends DependencyInjectedTest { Callable callable = new Callable() { @Override public Void call() throws Exception { - fs.create(fileName); + touch(fileName); return null; } }; @@ -133,20 +113,25 @@ public class ITestAzureBlobFileSystemRename extends DependencyInjectedTest { } es.shutdownNow(); - fs.rename(new Path("/test"), new Path("/renamedDir")); + Path source = new Path("/test"); + Path dest = new Path("/renamedDir"); + assertRenameOutcome(fs, source, dest, true); - FileStatus[] files = fs.listStatus(new Path("/renamedDir")); - Assert.assertEquals(files.length, 1000); - fs.getFileStatus(new Path("/test")); + FileStatus[] files = fs.listStatus(dest); + assertEquals("Wrong number of files in listing", 1000, files.length); + assertPathDoesNotExist(fs, "rename source dir", source); } @Test public void testRenameRoot() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - boolean renamed = fs.rename(new Path("/"), new Path("/ddd")); - assertFalse(renamed); - - renamed = fs.rename(new Path(fs.getUri().toString() + "/"), new Path(fs.getUri().toString() + "/s")); - assertFalse(renamed); + final AzureBlobFileSystem fs = getFileSystem(); + assertRenameOutcome(fs, + new Path("/"), + new Path("/testRenameRoot"), + false); + assertRenameOutcome(fs, + new Path(fs.getUri().toString() + "/"), + new Path(fs.getUri().toString() + "/s"), + false); } } 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 new file mode 100644 index 00000000000..0ac7fcf08f0 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java @@ -0,0 +1,98 @@ +/* + * 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.Arrays; + +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 static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory; +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.assertPathExists; +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome; + +/** + * Parameterized test of rename operations of unicode paths. + */ +@RunWith(Parameterized.class) +public class ITestAzureBlobFileSystemRenameUnicode extends + AbstractAbfsIntegrationTest { + + @Parameterized.Parameter + public String srcDir; + + @Parameterized.Parameter(1) + public String destDir; + + @Parameterized.Parameter(2) + public String filename; + + @Parameterized.Parameters + public static Iterable params() { + return Arrays.asList( + new Object[][]{ + {"/src", "/dest", "filename"}, + {"/%2c%26", "/abcÖ⇒123", "%2c%27"}, + {"/ÖáΠ⇒", "/abcÖáΠ⇒123", "中文"}, + {"/A +B", "/B+ C", "C +D"}, + { + "/A~`!@#$%^&*()-_+={};:'>,,, 0); + assertEquals(fileStatus[0].getPath().getName(), filename); + } +} 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 29af1b8c8d9..3a44909a314 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,9 +20,9 @@ package org.apache.hadoop.fs.azurebfs; import java.net.URI; -import org.junit.Assert; 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.azurebfs.constants.FileSystemUriSchemes; @@ -30,30 +30,45 @@ import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; /** * Test AzureBlobFileSystem initialization. */ -public class ITestFileSystemInitialization extends DependencyInjectedTest { +public class ITestFileSystemInitialization extends AbstractAbfsIntegrationTest { public ITestFileSystemInitialization() { super(); } @Test public void ensureAzureBlobFileSystemIsInitialized() throws Exception { - final FileSystem fs = this.getFileSystem(); - final String accountName = this.getAccountName(); - final String filesystem = this.getFileSystemName(); + final AzureBlobFileSystem fs = getFileSystem(); + final String accountName = getAccountName(); + final String filesystem = getFileSystemName(); - Assert.assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SCHEME, filesystem + "@" + accountName, null, null, null)); - Assert.assertNotNull(fs.getWorkingDirectory()); + assertEquals(fs.getUri(), + new URI(FileSystemUriSchemes.ABFS_SCHEME, + filesystem + "@" + accountName, + null, + null, + null)); + assertNotNull("working directory", fs.getWorkingDirectory()); } @Test public void ensureSecureAzureBlobFileSystemIsInitialized() throws Exception { - final String accountName = this.getAccountName(); - final String filesystem = this.getFileSystemName(); - final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null); - this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); + final String accountName = getAccountName(); + final String filesystem = getFileSystemName(); + final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, + filesystem + "@" + accountName, + null, + null, + null); + Configuration conf = getConfiguration(); + conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); - final FileSystem fs = this.getFileSystem(); - Assert.assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, null, null)); - Assert.assertNotNull(fs.getWorkingDirectory()); + try(SecureAzureBlobFileSystem fs = (SecureAzureBlobFileSystem) FileSystem.newInstance(conf)) { + assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, + filesystem + "@" + accountName, + null, + null, + null)); + assertNotNull("working directory", fs.getWorkingDirectory()); + } } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java index 62d967e2099..1c71125ffaf 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java @@ -20,7 +20,6 @@ package org.apache.hadoop.fs.azurebfs; import java.util.Hashtable; -import org.junit.Assert; import org.junit.Ignore; import org.junit.Test; @@ -34,31 +33,29 @@ import static org.junit.Assert.assertEquals; /** * Test FileSystemProperties. */ -public class ITestFileSystemProperties extends DependencyInjectedTest { +public class ITestFileSystemProperties extends AbstractAbfsIntegrationTest { private static final int TEST_DATA = 100; private static final Path TEST_PATH = new Path("/testfile"); public ITestFileSystemProperties() { - super(); } @Test public void testReadWriteBytesToFileAndEnsureThreadPoolCleanup() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); testWriteOneByteToFileAndEnsureThreadPoolCleanup(); - FSDataInputStream inputStream = fs.open(TEST_PATH, 4 * 1024 * 1024); - int i = inputStream.read(); - - assertEquals(TEST_DATA, i); + try(FSDataInputStream inputStream = fs.open(TEST_PATH, 4 * 1024 * 1024)) { + int i = inputStream.read(); + assertEquals(TEST_DATA, i); + } } @Test public void testWriteOneByteToFileAndEnsureThreadPoolCleanup() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - FSDataOutputStream stream = fs.create(TEST_PATH); - - stream.write(TEST_DATA); - stream.close(); + final AzureBlobFileSystem fs = getFileSystem(); + try(FSDataOutputStream stream = fs.create(TEST_PATH)) { + stream.write(TEST_DATA); + } FileStatus fileStatus = fs.getFileStatus(TEST_PATH); assertEquals(1, fileStatus.getLen()); @@ -67,60 +64,60 @@ public class ITestFileSystemProperties extends DependencyInjectedTest { @Test @Ignore("JDK7 doesn't support PATCH, so PUT is used. Fix is applied in latest test tenant") public void testBase64FileSystemProperties() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: value }"); fs.getAbfsStore().setFilesystemProperties(properties); Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties(); - Assert.assertEquals(properties, fetchedProperties); + assertEquals(properties, fetchedProperties); } @Test public void testBase64PathProperties() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueTest }"); - fs.create(TEST_PATH); + touch(TEST_PATH); fs.getAbfsStore().setPathProperties(TEST_PATH, properties); Hashtable fetchedProperties = fs.getAbfsStore().getPathProperties(TEST_PATH); - Assert.assertEquals(properties, fetchedProperties); + assertEquals(properties, fetchedProperties); } @Test (expected = Exception.class) public void testBase64InvalidFileSystemProperties() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueæ­² }"); fs.getAbfsStore().setFilesystemProperties(properties); Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties(); - Assert.assertEquals(properties, fetchedProperties); + assertEquals(properties, fetchedProperties); } @Test (expected = Exception.class) public void testBase64InvalidPathProperties() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); final Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueTestå…© }"); - fs.create(TEST_PATH); + touch(TEST_PATH); fs.getAbfsStore().setPathProperties(TEST_PATH, properties); Hashtable fetchedProperties = fs.getAbfsStore().getPathProperties(TEST_PATH); - Assert.assertEquals(properties, fetchedProperties); + assertEquals(properties, fetchedProperties); } @Test public void testSetFileSystemProperties() throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); + final AzureBlobFileSystem fs = getFileSystem(); final Hashtable properties = new Hashtable<>(); properties.put("containerForDevTest", "true"); fs.getAbfsStore().setFilesystemProperties(properties); Hashtable fetchedProperties = fs.getAbfsStore().getFilesystemProperties(); - Assert.assertEquals(properties, fetchedProperties); + assertEquals(properties, fetchedProperties); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java index ef61e5263aa..5d1cf91a9c3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java @@ -20,10 +20,9 @@ package org.apache.hadoop.fs.azurebfs; import java.net.URI; -import org.junit.Assert; import org.junit.Test; -import org.apache.hadoop.fs.AbstractFileSystem; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.FileSystem; @@ -31,33 +30,76 @@ import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; /** * Test AzureBlobFileSystem registration. + * Use casts to have interesting stack traces on failures. */ -public class ITestFileSystemRegistration extends DependencyInjectedTest { +public class ITestFileSystemRegistration extends AbstractAbfsIntegrationTest { + + protected static final String ABFS = "org.apache.hadoop.fs.azurebfs.Abfs"; + protected static final String ABFSS = "org.apache.hadoop.fs.azurebfs.Abfss"; + public ITestFileSystemRegistration() throws Exception { - super(); + } + + private void assertConfigMatches(Configuration conf, String key, String expected) { + String v = conf.get(key); + assertNotNull("No value for key " + key, v); + assertEquals("Wrong value for key " + key, expected, v); + } + + @Test + public void testAbfsFileSystemRegistered() throws Throwable { + assertConfigMatches(new Configuration(true), + "fs.abfs.impl", + "org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem"); + } + + @Test + public void testSecureAbfsFileSystemRegistered() throws Throwable { + assertConfigMatches(new Configuration(true), + "fs.abfss.impl", + "org.apache.hadoop.fs.azurebfs.SecureAzureBlobFileSystem"); + } + + @Test + public void testAbfsFileContextRegistered() throws Throwable { + assertConfigMatches(new Configuration(true), + "fs.AbstractFileSystem.abfs.impl", + ABFS); + } + + @Test + public void testSecureAbfsFileContextRegistered() throws Throwable { + assertConfigMatches(new Configuration(true), + "fs.AbstractFileSystem.abfss.impl", + ABFSS); } @Test public void ensureAzureBlobFileSystemIsDefaultFileSystem() throws Exception { - FileSystem fs = FileSystem.get(this.getConfiguration()); - Assert.assertTrue(fs instanceof AzureBlobFileSystem); + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(getConfiguration()); + assertNotNull("filesystem", fs); - AbstractFileSystem afs = FileContext.getFileContext(this.getConfiguration()).getDefaultFileSystem(); - Assert.assertTrue(afs instanceof Abfs); + Abfs afs = (Abfs) FileContext.getFileContext(getConfiguration()).getDefaultFileSystem(); + assertNotNull("filecontext", afs); } @Test public void ensureSecureAzureBlobFileSystemIsDefaultFileSystem() throws Exception { - final String accountName = this.getAccountName(); - final String fileSystemName = this.getFileSystemName(); + final String accountName = getAccountName(); + final String fileSystemName = getFileSystemName(); - final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, fileSystemName + "@" + accountName, null, null, null); - this.getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); + final URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, + fileSystemName + "@" + accountName, + null, + null, + null); + getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, + defaultUri.toString()); - FileSystem fs = FileSystem.get(this.getConfiguration()); - Assert.assertTrue(fs instanceof SecureAzureBlobFileSystem); - - AbstractFileSystem afs = FileContext.getFileContext(this.getConfiguration()).getDefaultFileSystem(); - Assert.assertTrue(afs instanceof Abfss); + SecureAzureBlobFileSystem fs = (SecureAzureBlobFileSystem) FileSystem.get( + getConfiguration()); + assertNotNull("filesystem", fs); + Abfss afs = (Abfss) FileContext.getFileContext(getConfiguration()).getDefaultFileSystem(); + assertNotNull("filecontext", afs); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java index 7010e74cb74..a89c0443350 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java @@ -17,59 +17,62 @@ */ package org.apache.hadoop.fs.azurebfs; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.azure.NativeAzureFileSystem; - -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; - import java.io.BufferedReader; import java.io.InputStreamReader; -import static junit.framework.TestCase.assertEquals; -import static junit.framework.TestCase.assertFalse; -import static junit.framework.TestCase.assertTrue; +import org.junit.Assume; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azure.NativeAzureFileSystem; +import org.apache.hadoop.fs.contract.ContractTestUtils; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted; +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory; +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs; +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists; /** * Test compatibility between ABFS client and WASB client. */ -public class ITestWasbAbfsCompatibility extends DependencyInjectedTest { +public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest { private static final String WASB_TEST_CONTEXT = "wasb test file"; private static final String ABFS_TEST_CONTEXT = "abfs test file"; private static final String TEST_CONTEXT = "THIS IS FOR TEST"; - public ITestWasbAbfsCompatibility() throws Exception { - super(); + private static final Logger LOG = + LoggerFactory.getLogger(ITestWasbAbfsCompatibility.class); - Assume.assumeFalse(this.isEmulator()); + public ITestWasbAbfsCompatibility() throws Exception { + Assume.assumeFalse("Emulator is not supported", isEmulator()); } @Test public void testListFileStatus() throws Exception { // crate file using abfs - AzureBlobFileSystem fs = this.getFileSystem(); - NativeAzureFileSystem wasb = this.getWasbFileSystem(); + AzureBlobFileSystem fs = getFileSystem(); + NativeAzureFileSystem wasb = getWasbFileSystem(); Path path1 = new Path("/testfiles/~12/!008/3/abFsTestfile"); - FSDataOutputStream abfsStream = fs.create(path1, true); - abfsStream.write(ABFS_TEST_CONTEXT.getBytes()); - abfsStream.flush(); - abfsStream.hsync(); - abfsStream.close(); + try(FSDataOutputStream abfsStream = fs.create(path1, true)) { + abfsStream.write(ABFS_TEST_CONTEXT.getBytes()); + abfsStream.flush(); + abfsStream.hsync(); + } // create file using wasb Path path2 = new Path("/testfiles/~12/!008/3/nativeFsTestfile"); - System.out.println(wasb.getUri()); - FSDataOutputStream nativeFsStream = wasb.create(path2, true); - nativeFsStream.write(WASB_TEST_CONTEXT.getBytes()); - nativeFsStream.flush(); - nativeFsStream.hsync(); - nativeFsStream.close(); + LOG.info("{}", wasb.getUri()); + try(FSDataOutputStream nativeFsStream = wasb.create(path2, true)) { + nativeFsStream.write(WASB_TEST_CONTEXT.getBytes()); + nativeFsStream.flush(); + nativeFsStream.hsync(); + } // list file using abfs and wasb FileStatus[] abfsFileStatus = fs.listStatus(new Path("/testfiles/~12/!008/3/")); FileStatus[] nativeFsFileStatus = wasb.listStatus(new Path("/testfiles/~12/!008/3/")); @@ -83,52 +86,34 @@ public class ITestWasbAbfsCompatibility extends DependencyInjectedTest { boolean[] createFileWithAbfs = new boolean[]{false, true, false, true}; boolean[] readFileWithAbfs = new boolean[]{false, true, true, false}; - AzureBlobFileSystem abfs = this.getFileSystem(); - NativeAzureFileSystem wasb = this.getWasbFileSystem(); + AzureBlobFileSystem abfs = getFileSystem(); + NativeAzureFileSystem wasb = getWasbFileSystem(); - FileSystem fs; - BufferedReader br = null; for (int i = 0; i< 4; i++) { - try { - Path path = new Path("/testfiles/~12/!008/testfile" + i); - if (createFileWithAbfs[i]) { - fs = abfs; - } else { - fs = wasb; - } + Path path = new Path("/testfiles/~12/!008/testfile" + i); + final FileSystem createFs = createFileWithAbfs[i] ? abfs : wasb; - // Write - FSDataOutputStream nativeFsStream = fs.create(path, true); + // Write + try(FSDataOutputStream nativeFsStream = createFs.create(path, true)) { nativeFsStream.write(TEST_CONTEXT.getBytes()); nativeFsStream.flush(); nativeFsStream.hsync(); - nativeFsStream.close(); - - // Check file status - assertEquals(true, fs.exists(path)); - assertEquals(false, fs.getFileStatus(path).isDirectory()); - - // Read - if (readFileWithAbfs[i]) { - fs = abfs; - } else { - fs = wasb; - } - FSDataInputStream inputStream = fs.open(path); - br = new BufferedReader(new InputStreamReader(fs.open(path))); - String line = br.readLine(); - assertEquals(TEST_CONTEXT, line); - - // Remove file - fs.delete(path, true); - assertFalse(fs.exists(path)); - } catch (Exception e) { - e.printStackTrace(); - } finally { - if (br != null) { - br.close(); - } } + + // Check file status + ContractTestUtils.assertIsFile(createFs, path); + + // Read + final FileSystem readFs = readFileWithAbfs[i] ? abfs : wasb; + + try(BufferedReader br =new BufferedReader(new InputStreamReader(readFs.open(path)))) { + String line = br.readLine(); + assertEquals("Wrong text from " + readFs, + TEST_CONTEXT, line); + } + + // Remove file + assertDeleted(readFs, path, true); } } @@ -137,32 +122,22 @@ public class ITestWasbAbfsCompatibility extends DependencyInjectedTest { boolean[] createDirWithAbfs = new boolean[]{false, true, false, true}; boolean[] readDirWithAbfs = new boolean[]{false, true, true, false}; - AzureBlobFileSystem abfs = this.getFileSystem(); - NativeAzureFileSystem wasb = this.getWasbFileSystem(); + AzureBlobFileSystem abfs = getFileSystem(); + NativeAzureFileSystem wasb = getWasbFileSystem(); - FileSystem fs; for (int i = 0; i < 4; i++) { Path path = new Path("/testDir/t" + i); //create - if (createDirWithAbfs[i]) { - fs = abfs; - } else { - fs = wasb; - } - assertTrue(fs.mkdirs(path)); + final FileSystem createFs = createDirWithAbfs[i] ? abfs : wasb; + assertTrue(createFs.mkdirs(path)); //check - assertTrue(fs.exists(path)); + assertPathExists(createFs, "Created dir not found with " + createFs, path); //read - if (readDirWithAbfs[i]) { - fs = abfs; - } else { - fs = wasb; - } - assertTrue(fs.exists(path)); - FileStatus dirStatus = fs.getFileStatus(path); - assertTrue(dirStatus.isDirectory()); - fs.delete(path, true); - assertFalse(fs.exists(path)); + final FileSystem readFs = readDirWithAbfs[i] ? abfs : wasb; + assertPathExists(readFs, "Created dir not found with " + readFs, + path); + assertIsDirectory(readFs, path); + assertDeleted(readFs, path, true); } } @@ -171,17 +146,18 @@ public class ITestWasbAbfsCompatibility extends DependencyInjectedTest { public void testUrlConversion(){ String abfsUrl = "abfs://abcde-1111-1111-1111-1111@xxxx.dfs.xxx.xxx.xxxx.xxxx"; String wabsUrl = "wasb://abcde-1111-1111-1111-1111@xxxx.blob.xxx.xxx.xxxx.xxxx"; - Assert.assertEquals(abfsUrl, wasbUrlToAbfsUrl(wabsUrl)); - Assert.assertEquals(wabsUrl, abfsUrlToWasbUrl(abfsUrl)); + assertEquals(abfsUrl, wasbUrlToAbfsUrl(wabsUrl)); + assertEquals(wabsUrl, abfsUrlToWasbUrl(abfsUrl)); } @Test public void testSetWorkingDirectory() throws Exception { //create folders - AzureBlobFileSystem abfs = this.getFileSystem(); - NativeAzureFileSystem wasb = this.getWasbFileSystem(); + AzureBlobFileSystem abfs = getFileSystem(); + NativeAzureFileSystem wasb = getWasbFileSystem(); - assertTrue(abfs.mkdirs(new Path("/d1/d2/d3/d4"))); + Path d1d4 = new Path("/d1/d2/d3/d4"); + assertMkdirs(abfs, d1d4); //set working directory to path1 Path path1 = new Path("/d1/d2"); @@ -195,8 +171,8 @@ public class ITestWasbAbfsCompatibility extends DependencyInjectedTest { wasb.setWorkingDirectory(path2); abfs.setWorkingDirectory(path2); - Path path3 = new Path("/d1/d2/d3/d4"); + Path path3 = d1d4; assertEquals(path3, wasb.getWorkingDirectory()); assertEquals(path3, abfs.getWorkingDirectory()); } -} \ No newline at end of file +} 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 4b447653ab7..fc7312aadac 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 @@ -18,14 +18,9 @@ package org.apache.hadoop.fs.azurebfs.constants; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; - /** * Responsible to keep all the Azure Blob File System configurations keys in Hadoop configuration file. */ -@InterfaceAudience.Public -@InterfaceStability.Evolving public final class TestConfigurationKeys { public static final String FS_AZURE_TEST_ACCOUNT_NAME = "fs.azure.test.account.name"; public static final String FS_AZURE_TEST_ACCOUNT_KEY_PREFIX = "fs.azure.test.account.key."; @@ -33,5 +28,9 @@ public final class TestConfigurationKeys { public static final String FS_AZURE_TEST_HOST_PORT = "fs.azure.test.host.port"; public static final String FS_AZURE_CONTRACT_TEST_URI = "fs.contract.test.fs.abfs"; + public static final String ABFS_TEST_RESOURCE_XML = "azure-bfs-test.xml"; + + public static final String ABFS_TEST_CONTAINER_PREFIX = "abfs-testcontainer-"; + private TestConfigurationKeys() {} -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java similarity index 79% rename from hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java rename to hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java index 5fc81cec2f8..ffd5babc5bf 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java @@ -22,36 +22,37 @@ import java.net.URI; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; -import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest; +import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; /** - * Dependency inject for ABFS contract tests. + * Bind ABFS contract tests to the Azure test setup/teardown. */ -public class DependencyInjectedContractTest extends DependencyInjectedTest { +public class ABFSContractTestBinding extends AbstractAbfsIntegrationTest { private final URI testUri; - public DependencyInjectedContractTest(final boolean secure) throws Exception { + public ABFSContractTestBinding(final boolean secure) throws Exception { this(secure, true); } - public DependencyInjectedContractTest(final boolean secure, final boolean useExistedFileSystem) throws Exception{ + public ABFSContractTestBinding(final boolean secure, + final boolean useExistingFileSystem) throws Exception{ super(secure); - if (useExistedFileSystem) { + if (useExistingFileSystem) { Configuration configuration = getConfiguration(); String testUrl = configuration.get(TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI); if (secure) { testUrl = testUrl.replaceFirst(FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME); } - updateTestUrl(testUrl); + setTestUrl(testUrl); this.testUri = new URI(testUrl); //Get container for contract tests configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, this.testUri.toString()); String[] splitAuthority = this.testUri.getAuthority().split("\\@"); - updateFileSystemName(splitAuthority[0]); + setFileSystemName(splitAuthority[0]); } else { this.testUri = new URI(super.getTestUrl()); } @@ -60,4 +61,4 @@ public class DependencyInjectedContractTest extends DependencyInjectedTest { public Configuration getConfiguration() { return super.getConfiguration(); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java similarity index 73% rename from hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java rename to hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java index 7f7a09ab821..d365e6e9169 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java @@ -29,12 +29,12 @@ import org.apache.hadoop.fs.contract.AbstractBondedFSContract; * identifier, if defined. This guarantees paths unique to tests * running in parallel. */ -public class ITestAbfsFileSystemContract extends AbstractBondedFSContract { +public class AbfsFileSystemContract extends AbstractBondedFSContract { public static final String CONTRACT_XML = "abfs.xml"; private final boolean isSecure; - protected ITestAbfsFileSystemContract(final Configuration conf, boolean secure) { + protected AbfsFileSystemContract(final Configuration conf, boolean secure) { super(conf); //insert the base features addConfResource(CONTRACT_XML); @@ -43,12 +43,23 @@ public class ITestAbfsFileSystemContract extends AbstractBondedFSContract { @Override public String getScheme() { - return isSecure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME; + return isSecure ? + FileSystemUriSchemes.ABFS_SECURE_SCHEME + : FileSystemUriSchemes.ABFS_SCHEME; } @Override public Path getTestPath() { - Path path = new Path(UriUtils.generateUniqueTestPath()); - return path; + return new Path(UriUtils.generateUniqueTestPath()); } -} \ No newline at end of file + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "AbfsFileSystemContract{"); + sb.append("isSecure=").append(isSecure); + sb.append(super.toString()); + sb.append('}'); + return sb.toString(); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java index d4cca146a6a..a302fccfd9c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java @@ -20,6 +20,7 @@ package org.apache.hadoop.fs.azurebfs.contract; import java.util.Arrays; +import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -40,31 +41,32 @@ public class ITestAbfsFileSystemContractAppend extends AbstractContractAppendTes } private final boolean isSecure; - private final DependencyInjectedContractTest dependencyInjectedContractTest; + private final ABFSContractTestBinding binding; public ITestAbfsFileSystemContractAppend(final boolean secure) throws Exception { this.isSecure = secure; - dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); + binding = new ABFSContractTestBinding(this.isSecure); } @Override public void setup() throws Exception { - dependencyInjectedContractTest.initialize(); + binding.setup(); super.setup(); } @Override protected Configuration createConfiguration() { - return this.dependencyInjectedContractTest.getConfiguration(); + return binding.getConfiguration(); } @Override protected AbstractFSContract createContract(final Configuration conf) { - return new ITestAbfsFileSystemContract(conf, this.isSecure); + return new AbfsFileSystemContract(conf, isSecure); } @Override + @Test public void testRenameFileBeingAppended() throws Throwable { skip("Skipping as renaming an opened file is not supported"); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java index 4f724e2faf1..c31a6d2a5aa 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java @@ -37,26 +37,26 @@ public class ITestAbfsFileSystemContractConcat extends AbstractContractConcatTes } private final boolean isSecure; - private final DependencyInjectedContractTest dependencyInjectedContractTest; + private final ABFSContractTestBinding binding; public ITestAbfsFileSystemContractConcat(final boolean secure) throws Exception { - this.isSecure = secure; - dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure); + isSecure = secure; + binding = new ABFSContractTestBinding(isSecure); } @Override public void setup() throws Exception { - dependencyInjectedContractTest.initialize(); + binding.setup(); super.setup(); } @Override protected Configuration createConfiguration() { - return this.dependencyInjectedContractTest.getConfiguration(); + return binding.getConfiguration(); } @Override protected AbstractFSContract createContract(final Configuration conf) { - return new ITestAbfsFileSystemContract(conf, this.isSecure); + return new AbfsFileSystemContract(conf, isSecure); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java index 16b959f00f7..ce4d22963e1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java @@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractCreate extends AbstractContractCreateTes } private final boolean isSecure; - private final DependencyInjectedContractTest dependencyInjectedContractTest; + private final ABFSContractTestBinding binding; public ITestAbfsFileSystemContractCreate(final boolean secure) throws Exception { this.isSecure = secure; - dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); + binding = new ABFSContractTestBinding(this.isSecure); } @Override public void setup() throws Exception { - dependencyInjectedContractTest.initialize(); + binding.setup(); super.setup(); } @Override protected Configuration createConfiguration() { - return this.dependencyInjectedContractTest.getConfiguration(); + return binding.getConfiguration(); } @Override protected AbstractFSContract createContract(final Configuration conf) { - return new ITestAbfsFileSystemContract(conf, this.isSecure); + return new AbfsFileSystemContract(conf, isSecure); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java index fabd3273eac..310731cc0eb 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java @@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractDelete extends AbstractContractDeleteTes } private final boolean isSecure; - private final DependencyInjectedContractTest dependencyInjectedContractTest; + private final ABFSContractTestBinding binding; public ITestAbfsFileSystemContractDelete(final boolean secure) throws Exception { this.isSecure = secure; - dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure); + binding = new ABFSContractTestBinding(isSecure); } @Override public void setup() throws Exception { - dependencyInjectedContractTest.initialize(); + binding.setup(); super.setup(); } @Override protected Configuration createConfiguration() { - return this.dependencyInjectedContractTest.getConfiguration(); + return binding.getConfiguration(); } @Override protected AbstractFSContract createContract(final Configuration conf) { - return new ITestAbfsFileSystemContract(conf, this.isSecure); + return new AbfsFileSystemContract(conf, isSecure); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java index a1360e49dbf..c2cf25569b2 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java @@ -25,20 +25,20 @@ import org.apache.hadoop.tools.contract.AbstractContractDistCpTest; * Contract test for distCp operation. */ public class ITestAbfsFileSystemContractDistCp extends AbstractContractDistCpTest { - private final DependencyInjectedContractTest dependencyInjectedContractTest; + private final ABFSContractTestBinding binding; public ITestAbfsFileSystemContractDistCp() throws Exception { - dependencyInjectedContractTest = new DependencyInjectedContractTest(false); + binding = new ABFSContractTestBinding(false); } @Override public void setup() throws Exception { - dependencyInjectedContractTest.initialize(); + binding.setup(); super.setup(); } @Override - protected ITestAbfsFileSystemContract createContract(Configuration conf) { - return new ITestAbfsFileSystemContract(conf, false); + protected AbfsFileSystemContract createContract(Configuration conf) { + return new AbfsFileSystemContract(conf, false); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java index 5bb41ad9f3a..9ad3b215e10 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java @@ -37,26 +37,26 @@ public class ITestAbfsFileSystemContractGetFileStatus extends AbstractContractGe } private final boolean isSecure; - private final DependencyInjectedContractTest dependencyInjectedContractTest; + private final ABFSContractTestBinding binding; public ITestAbfsFileSystemContractGetFileStatus(final boolean secure) throws Exception { this.isSecure = secure; - dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure); + binding = new ABFSContractTestBinding(isSecure); } @Override public void setup() throws Exception { - dependencyInjectedContractTest.initialize(); + binding.setup(); super.setup(); } @Override protected Configuration createConfiguration() { - return this.dependencyInjectedContractTest.getConfiguration(); + return this.binding.getConfiguration(); } @Override protected AbstractFSContract createContract(final Configuration conf) { - return new ITestAbfsFileSystemContract(conf, this.isSecure); + return new AbfsFileSystemContract(conf, this.isSecure); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java index 9d732d5f2f8..6265ca1f92c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java @@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractMkdir extends AbstractContractMkdirTest } private final boolean isSecure; - private final DependencyInjectedContractTest dependencyInjectedContractTest; + private final ABFSContractTestBinding binding; public ITestAbfsFileSystemContractMkdir(final boolean secure) throws Exception { this.isSecure = secure; - dependencyInjectedContractTest = new DependencyInjectedContractTest(secure); + binding = new ABFSContractTestBinding(secure); } @Override public void setup() throws Exception { - dependencyInjectedContractTest.initialize(); + binding.setup(); super.setup(); } @Override protected Configuration createConfiguration() { - return this.dependencyInjectedContractTest.getConfiguration(); + return binding.getConfiguration(); } @Override protected AbstractFSContract createContract(final Configuration conf) { - return new ITestAbfsFileSystemContract(conf, this.isSecure); + return new AbfsFileSystemContract(conf, isSecure); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java index a71149b8958..ae4bb2a4fc6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java @@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractOpen extends AbstractContractOpenTest { } private final boolean isSecure; - private final DependencyInjectedContractTest dependencyInjectedContractTest; + private final ABFSContractTestBinding binding; public ITestAbfsFileSystemContractOpen(final boolean secure) throws Exception { this.isSecure = secure; - dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); + binding = new ABFSContractTestBinding(this.isSecure); } @Override public void setup() throws Exception { - dependencyInjectedContractTest.initialize(); + binding.setup(); super.setup(); } @Override protected Configuration createConfiguration() { - return this.dependencyInjectedContractTest.getConfiguration(); + return binding.getConfiguration(); } @Override protected AbstractFSContract createContract(final Configuration conf) { - return new ITestAbfsFileSystemContract(conf, this.isSecure); + return new AbfsFileSystemContract(conf, isSecure); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java index 6d1c4ae8a79..6e6a7280350 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java @@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractRename extends AbstractContractRenameTes } private final boolean isSecure; - private final DependencyInjectedContractTest dependencyInjectedContractTest; + private final ABFSContractTestBinding binding; public ITestAbfsFileSystemContractRename(final boolean secure) throws Exception { this.isSecure = secure; - dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); + binding = new ABFSContractTestBinding(this.isSecure); } @Override public void setup() throws Exception { - dependencyInjectedContractTest.initialize(); + binding.setup(); super.setup(); } @Override protected Configuration createConfiguration() { - return this.dependencyInjectedContractTest.getConfiguration(); + return binding.getConfiguration(); } @Override protected AbstractFSContract createContract(final Configuration conf) { - return new ITestAbfsFileSystemContract(conf, this.isSecure); + return new AbfsFileSystemContract(conf, isSecure); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java index 46072adac3e..01dea2d164d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java @@ -37,31 +37,31 @@ public class ITestAbfsFileSystemContractRootDirectory extends AbstractContractRo } private final boolean isSecure; - private final DependencyInjectedContractTest dependencyInjectedContractTest; + private final ABFSContractTestBinding binding; public ITestAbfsFileSystemContractRootDirectory(final boolean secure) throws Exception { this.isSecure = secure; - dependencyInjectedContractTest = new DependencyInjectedContractTest(secure); + binding = new ABFSContractTestBinding(secure); } @Override public void setup() throws Exception { - dependencyInjectedContractTest.initialize(); + binding.setup(); super.setup(); } @Override protected Configuration createConfiguration() { - return this.dependencyInjectedContractTest.getConfiguration(); + return binding.getConfiguration(); } @Override protected AbstractFSContract createContract(final Configuration conf) { - return new ITestAbfsFileSystemContract(conf, this.isSecure); + return new AbfsFileSystemContract(conf, isSecure); } @Override @Ignore("ABFS always return false when non-recursively remove root dir") public void testRmNonEmptyRootDirNonRecursive() throws Throwable { } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java index 1780f6f87b9..5ed74668408 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java @@ -25,20 +25,20 @@ import org.apache.hadoop.tools.contract.AbstractContractDistCpTest; * Contract test for secure distCP operation. */ public class ITestAbfsFileSystemContractSecureDistCp extends AbstractContractDistCpTest { - private final DependencyInjectedContractTest dependencyInjectedContractTest; + private final ABFSContractTestBinding binding; public ITestAbfsFileSystemContractSecureDistCp() throws Exception { - dependencyInjectedContractTest = new DependencyInjectedContractTest(true); + binding = new ABFSContractTestBinding(true); } @Override public void setup() throws Exception { - dependencyInjectedContractTest.initialize(); + binding.setup(); super.setup(); } @Override - protected ITestAbfsFileSystemContract createContract(Configuration conf) { - return new ITestAbfsFileSystemContract(conf, true); + protected AbfsFileSystemContract createContract(Configuration conf) { + return new AbfsFileSystemContract(conf, true); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java index aeeb0425189..5e0ea0c94ea 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java @@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractSeek extends AbstractContractSeekTest{ } private final boolean isSecure; - private final DependencyInjectedContractTest dependencyInjectedContractTest; + private final ABFSContractTestBinding binding; public ITestAbfsFileSystemContractSeek(final boolean secure) throws Exception { this.isSecure = secure; - dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); + binding = new ABFSContractTestBinding(this.isSecure); } @Override public void setup() throws Exception { - dependencyInjectedContractTest.initialize(); + binding.setup(); super.setup(); } @Override protected Configuration createConfiguration() { - return this.dependencyInjectedContractTest.getConfiguration(); + return binding.getConfiguration(); } @Override protected AbstractFSContract createContract(final Configuration conf) { - return new ITestAbfsFileSystemContract(conf, this.isSecure); + return new AbfsFileSystemContract(conf, isSecure); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java index ea9392d3158..8d23b0bbffd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java @@ -37,26 +37,26 @@ public class ITestAbfsFileSystemContractSetTimes extends AbstractContractSetTime } private final boolean isSecure; - private final DependencyInjectedContractTest dependencyInjectedContractTest; + private final ABFSContractTestBinding binding; public ITestAbfsFileSystemContractSetTimes(final boolean secure) throws Exception { this.isSecure = secure; - dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure); + binding = new ABFSContractTestBinding(this.isSecure); } @Override public void setup() throws Exception { - dependencyInjectedContractTest.initialize(); + binding.setup(); super.setup(); } @Override protected Configuration createConfiguration() { - return this.dependencyInjectedContractTest.getConfiguration(); + return binding.getConfiguration(); } @Override protected AbstractFSContract createContract(final Configuration conf) { - return new ITestAbfsFileSystemContract(conf, this.isSecure); + return new AbfsFileSystemContract(conf, isSecure); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java index 9f3b4a71603..d8854a2b0a0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java @@ -22,8 +22,8 @@ import java.io.IOException; import org.apache.hadoop.fs.FileSystemContractBaseTest; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; -import org.junit.After; import org.junit.Before; import org.junit.Ignore; import org.junit.Test; @@ -35,26 +35,31 @@ import static org.junit.Assert.assertTrue; * Basic Contract test for Azure BlobFileSystem. */ public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest { - private final DependencyInjectedContractTest dependencyInjectedContractTest; + private final ABFSContractTestBinding binding; public ITestAzureBlobFileSystemBasics() throws Exception { // If all contract tests are running in parallel, some root level tests in FileSystemContractBaseTest will fail // due to the race condition. Hence for this contract test it should be tested in different container - dependencyInjectedContractTest = new DependencyInjectedContractTest(false, false); + binding = new ABFSContractTestBinding(false, false); } + @Before public void setUp() throws Exception { - this.dependencyInjectedContractTest.initialize(); - fs = this.dependencyInjectedContractTest.getFileSystem(); + binding.setup(); + fs = binding.getFileSystem(); } - @After - public void testCleanup() throws Exception { + @Override + public void tearDown() throws Exception { // This contract test is not using existing container for test, // instead it creates its own temp container for test, hence we need to destroy // it after the test. - this.dependencyInjectedContractTest.testCleanup(); + try { + super.tearDown(); + } finally { + binding.teardown(); + } } @Test @@ -82,7 +87,7 @@ public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest { Path filePath = path("testListStatus/file"); assertTrue(fs.mkdirs(folderPath)); - fs.create(filePath); + ContractTestUtils.touch(fs, filePath); FileStatus[] listFolderStatus; listFolderStatus = fs.listStatus(path("testListStatus")); @@ -97,4 +102,4 @@ public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest { @Ignore("Not implemented in ABFS yet") public void testMkdirsWithUmask() throws Exception { } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java index 03320d66414..e9e90c993a7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java @@ -19,10 +19,11 @@ package org.apache.hadoop.fs.azurebfs.diagnostics; import org.apache.commons.codec.Charsets; + +import org.junit.Assert; import org.junit.Test; -import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; import org.apache.commons.codec.binary.Base64; @@ -36,7 +37,7 @@ import static org.junit.Assert.assertEquals; /** * Test configuration validators. */ -public class TestConfigurationValidators extends DependencyInjectedTest { +public class TestConfigurationValidators extends Assert { private static final String FAKE_KEY = "FakeKey"; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.java similarity index 58% rename from hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java rename to hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.java index cabe0494df8..dd06fe3edfc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.java @@ -18,61 +18,74 @@ package org.apache.hadoop.fs.azurebfs.services; +import java.util.Arrays; import java.util.Random; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.AbstractAbfsScaleTest; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; -import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest; -import org.junit.Test; -import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_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.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertNotEquals; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE; /** * Test read, write and seek. + * Uses package-private methods in AbfsConfiguration, which is why it is in + * this package. */ -public class ITestReadWriteAndSeek extends DependencyInjectedTest { +@RunWith(Parameterized.class) +public class ITestAbfsReadWriteAndSeek extends AbstractAbfsScaleTest { private static final Path TEST_PATH = new Path("/testfile"); - public ITestReadWriteAndSeek() { - super(); + + @Parameterized.Parameters(name = "Size={0}") + public static Iterable sizes() { + return Arrays.asList(new Object[][]{{MIN_BUFFER_SIZE}, + {DEFAULT_READ_BUFFER_SIZE}, + {MAX_BUFFER_SIZE}}); + } + + private final int size; + + public ITestAbfsReadWriteAndSeek(final int size) { + this.size = size; } @Test public void testReadAndWriteWithDifferentBufferSizesAndSeek() throws Exception { - testReadWriteAndSeek(MIN_BUFFER_SIZE); - testReadWriteAndSeek(DEFAULT_READ_BUFFER_SIZE); - testReadWriteAndSeek(MAX_BUFFER_SIZE); + testReadWriteAndSeek(size); } private void testReadWriteAndSeek(int bufferSize) throws Exception { - final AzureBlobFileSystem fs = this.getFileSystem(); - final AbfsConfiguration abfsConfiguration = new AbfsConfiguration(this.getConfiguration()); + final AzureBlobFileSystem fs = getFileSystem(); + final AbfsConfiguration abfsConfiguration = new AbfsConfiguration(getConfiguration()); - fs.create(TEST_PATH); abfsConfiguration.setWriteBufferSize(bufferSize); abfsConfiguration.setReadBufferSize(bufferSize); - final FSDataOutputStream stream = fs.create(TEST_PATH); final byte[] b = new byte[2 * bufferSize]; new Random().nextBytes(b); - stream.write(b); - stream.close(); + try(final FSDataOutputStream stream = fs.create(TEST_PATH)) { + stream.write(b); + } - final byte[] r = new byte[2 * bufferSize]; - final FSDataInputStream inputStream = fs.open(TEST_PATH); - inputStream.seek(bufferSize); - int result = inputStream.read(r, bufferSize, bufferSize); - assertNotEquals(-1, result); - - inputStream.seek(0); - result = inputStream.read(r, 0, bufferSize); - assertNotEquals(-1, result); - assertArrayEquals(r, b); + final byte[] readBuffer = new byte[2 * bufferSize]; + int result; + try(final FSDataInputStream inputStream = fs.open(TEST_PATH)) { + inputStream.seek(bufferSize); + result = inputStream.read(readBuffer, bufferSize, bufferSize); + assertNotEquals(-1, result); + inputStream.seek(0); + result = inputStream.read(readBuffer, 0, bufferSize); + } + assertNotEquals("data read in final read()", -1, result); + assertArrayEquals(readBuffer, b); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java index 441a35a10eb..ebaafa42e02 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java @@ -48,7 +48,7 @@ import org.junit.Test; * Test ConfigurationServiceFieldsValidation. */ public class TestAbfsConfigurationFieldsValidation { - private AbfsConfiguration abfsConfiguration; + private final AbfsConfiguration abfsConfiguration; private static final String INT_KEY= "intKey"; private static final String LONG_KEY= "longKey"; @@ -89,12 +89,10 @@ public class TestAbfsConfigurationFieldsValidation { private boolean boolField; public TestAbfsConfigurationFieldsValidation() throws Exception { - super(); Base64 base64 = new Base64(); this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8); this.encodedAccountKey = new String(base64.encode("someAccountKey".getBytes(Charsets.UTF_8)), Charsets.UTF_8); - Configuration configuration = new Configuration(); - configuration.addResource("azure-bfs-test.xml"); + Configuration configuration = new Configuration(false); configuration.set(INT_KEY, "1234565"); configuration.set(LONG_KEY, "4194304"); configuration.set(STRING_KEY, "stringValue"); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java index 2716bffe1b4..9051a72e127 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java @@ -24,7 +24,7 @@ import com.microsoft.azure.storage.blob.CloudBlobContainer; import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount; import org.junit.Test; -import static org.apache.hadoop.fs.azurebfs.DependencyInjectedTest.TEST_CONTAINER_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.ABFS_TEST_CONTAINER_PREFIX; /** * If unit tests were interrupted and crushed accidentally, the test containers won't be deleted. @@ -38,7 +38,7 @@ public class CleanUpAbfsTestContainer { CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount(); CloudBlobClient blobClient = storageAccount.createCloudBlobClient(); Iterable containers - = blobClient.listContainers(TEST_CONTAINER_PREFIX); + = blobClient.listContainers(ABFS_TEST_CONTAINER_PREFIX); for (CloudBlobContainer container : containers) { count++; System.out.println(String.format("Container %s URI %s", @@ -54,7 +54,7 @@ public class CleanUpAbfsTestContainer { CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount(); CloudBlobClient blobClient = storageAccount.createCloudBlobClient(); Iterable containers - = blobClient.listContainers(TEST_CONTAINER_PREFIX); + = blobClient.listContainers(ABFS_TEST_CONTAINER_PREFIX); for (CloudBlobContainer container : containers) { System.out.println(String.format("Container %s URI %s", container.getName(), diff --git a/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml b/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml index caf8677cbec..d065ace8b58 100644 --- a/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml +++ b/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml @@ -1,3 +1,4 @@ + - fs.contract.test.root-tests-enabled @@ -61,4 +61,4 @@ fs.contract.supports-getfilestatus true - \ No newline at end of file + diff --git a/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml b/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml index 508e5f69418..464a8e670e9 100644 --- a/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml +++ b/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml @@ -1,4 +1,5 @@ + - - - fs.AbstractFileSystem.abfs.impl - org.apache.hadoop.fs.azurebfs.Abfs - - - - fs.AbstractFileSystem.abfss.impl - org.apache.hadoop.fs.azurebfs.Abfss - - - \ No newline at end of file + diff --git a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties index fd0cd9de0d0..bac431d482d 100644 --- a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties @@ -26,3 +26,35 @@ 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 + +# 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 +log4j.logger.org.apache.hadoop.conf.Configuration.deprecation=WARN +log4j.logger.org.apache.hadoop.util.GSet=WARN +# MiniDFS clusters can be noisy +log4j.logger.org.apache.hadoop.hdfs.server=ERROR +log4j.logger.org.apache.hadoop.metrics2=WARN +log4j.logger.org.apache.hadoop.net.NetworkTopology=WARN +log4j.logger.org.apache.hadoop.util.JvmPauseMonitor=WARN +log4j.logger.org.apache.hadoop.ipc=WARN +log4j.logger.org.apache.hadoop.http=WARN +log4j.logger.org.apache.hadoop.security.authentication.server.AuthenticationFilter=WARN +log4j.logger.org.apache.hadoop.util.HostsFileReader=WARN +log4j.logger.org.apache.commons.beanutils=WARN +log4j.logger.org.apache.hadoop.hdfs.StateChange=WARN +log4j.logger.BlockStateChange=WARN +log4j.logger.org.apache.hadoop.hdfs.DFSUtil=WARN +## YARN can be noisy too +log4j.logger.org.apache.hadoop.yarn.server.resourcemanager.scheduler=WARN +log4j.logger.org.apache.hadoop.yarn.server.nodemanager=WARN +log4j.logger.org.apache.hadoop.yarn.event=WARN +log4j.logger.org.apache.hadoop.yarn.util.ResourceCalculatorPlugin=ERROR +log4j.logger.org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor=WARN +log4j.logger.org.apache.hadoop.mapred.IndexCache=WARN +log4j.logger.org.apache.hadoop.yarn.webapp.WebApps=WARN +log4j.logger.org.apache.hadoop.yarn.server.resourcemanager.security=WARN +log4j.logger.org.apache.hadoop.yarn.util.AbstractLivelinessMonitor=WARN +log4j.logger.org.apache.hadoop.security.token.delegation=WARN +log4j.logger.org.apache.hadoop.mapred.ShuffleHandler=WARN +log4j.logger.org.apache.hadoop.ipc.Server=WARN From b54b0c1b676c616aef9574e4e88ea30c314c79dc Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Sat, 11 Aug 2018 00:10:26 +0000 Subject: [PATCH 04/29] HADOOP-15659. Code changes for bug fix and new tests. Contributed by Da Zhou. --- hadoop-tools/hadoop-azure/pom.xml | 30 ++-- .../{services => }/AbfsConfiguration.java | 69 ++++++++- .../fs/azurebfs/AzureBlobFileSystem.java | 55 +++++-- .../fs/azurebfs/AzureBlobFileSystemStore.java | 39 ++--- .../azurebfs/constants/ConfigurationKeys.java | 6 + .../constants/FileSystemConfigurations.java | 4 +- .../exceptions/KeyProviderException.java | 42 ++++++ .../services/AzureServiceErrorCode.java | 1 + .../services/ListResultEntrySchema.java | 2 +- .../contracts/services/ListResultSchema.java | 2 +- .../fs/azurebfs/services/AbfsClient.java | 26 ++-- .../azurebfs/services/AbfsHttpOperation.java | 19 +-- .../fs/azurebfs/services/AbfsInputStream.java | 2 +- .../azurebfs/services/AbfsOutputStream.java | 25 ++-- .../azurebfs/services/AbfsRestOperation.java | 2 +- .../services/AbfsUriQueryBuilder.java | 8 +- .../fs/azurebfs/services/KeyProvider.java | 42 ++++++ .../services/ShellDecryptionKeyProvider.java | 63 ++++++++ .../azurebfs/services/SimpleKeyProvider.java | 54 +++++++ .../azurebfs/AbstractAbfsIntegrationTest.java | 17 ++- .../hadoop/fs/azurebfs/ITestAbfsClient.java | 45 ++++++ .../ITestAbfsReadWriteAndSeek.java | 8 +- .../azurebfs/ITestAzureBlobFileSystemE2E.java | 2 +- .../ITestAzureBlobFileSystemE2EScale.java | 4 +- .../ITestAzureBlobFileSystemFinalize.java | 60 ++++++++ .../ITestAzureBlobFileSystemFlush.java | 136 +++++++++++++++++- ...ITestAzureBlobFileSystemInitAndCreate.java | 4 +- .../ITestAzureBlobFileSystemRename.java | 3 +- .../azurebfs/ITestFileSystemProperties.java | 4 - ...TestAbfsConfigurationFieldsValidation.java | 8 +- .../contract/AbfsFileSystemContract.java | 5 +- .../fs/azurebfs/services/TestAbfsClient.java | 60 ++++++++ .../TestShellDecryptionKeyProvider.java | 89 ++++++++++++ 33 files changed, 824 insertions(+), 112 deletions(-) rename hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/{services => }/AbfsConfiguration.java (83%) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/KeyProviderException.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/KeyProvider.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ShellDecryptionKeyProvider.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SimpleKeyProvider.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java rename hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/{services => }/ITestAbfsReadWriteAndSeek.java (90%) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java rename hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/{services => }/TestAbfsConfigurationFieldsValidation.java (97%) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml index cbd4dfb5b4b..7d0406c6cf8 100644 --- a/hadoop-tools/hadoop-azure/pom.xml +++ b/hadoop-tools/hadoop-azure/pom.xml @@ -149,17 +149,6 @@ provided - - com.fasterxml.jackson.core - jackson-core - compile - - - - com.fasterxml.jackson.core - jackson-databind - compile - org.apache.httpcomponents @@ -197,18 +186,25 @@ guava - - joda-time - joda-time - compile - - org.eclipse.jetty jetty-util-ajax compile + + org.codehaus.jackson + jackson-mapper-asl + compile + + + org.codehaus.jackson + jackson-core-asl + compile + + + + junit diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java similarity index 83% rename from hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConfiguration.java rename to hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 8def1bba20a..1fb5df9aa39 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.azurebfs.services; +package org.apache.hadoop.fs.azurebfs; import java.lang.reflect.Field; import java.util.Map; @@ -33,13 +33,17 @@ import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidati import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException; import org.apache.hadoop.fs.azurebfs.diagnostics.Base64StringConfigurationBasicValidator; import org.apache.hadoop.fs.azurebfs.diagnostics.BooleanConfigurationBasicValidator; import org.apache.hadoop.fs.azurebfs.diagnostics.IntegerConfigurationBasicValidator; import org.apache.hadoop.fs.azurebfs.diagnostics.LongConfigurationBasicValidator; import org.apache.hadoop.fs.azurebfs.diagnostics.StringConfigurationBasicValidator; +import org.apache.hadoop.fs.azurebfs.services.KeyProvider; +import org.apache.hadoop.fs.azurebfs.services.SimpleKeyProvider; /** * Configuration for Azure Blob FileSystem. @@ -111,10 +115,23 @@ public class AbfsConfiguration{ DefaultValue = FileSystemConfigurations.DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION) private boolean createRemoteFileSystemDuringInitialization; + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION, + DefaultValue = FileSystemConfigurations.DEFAULT_AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION) + private boolean skipUserGroupMetadataDuringInitialization; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH, DefaultValue = FileSystemConfigurations.DEFAULT_READ_AHEAD_QUEUE_DEPTH) private int readAheadQueueDepth; + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_ENABLE_FLUSH, + DefaultValue = FileSystemConfigurations.DEFAULT_ENABLE_FLUSH) + private boolean enableFlush; + + @StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, + DefaultValue = "") + private String userAgentId; + private Map storageAccountKeys; public AbfsConfiguration(final Configuration configuration) throws IllegalAccessException, InvalidConfigurationValueException { @@ -147,13 +164,38 @@ public class AbfsConfiguration{ return this.isSecure; } - public String getStorageAccountKey(final String accountName) throws ConfigurationPropertyNotFoundException { - String accountKey = this.storageAccountKeys.get(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + accountName); - if (accountKey == null) { + public String getStorageAccountKey(final String accountName) throws AzureBlobFileSystemException { + String key; + String keyProviderClass = + configuration.get(ConfigurationKeys.AZURE_KEY_ACCOUNT_KEYPROVIDER_PREFIX + accountName); + KeyProvider keyProvider; + + if (keyProviderClass == null) { + // No key provider was provided so use the provided key as is. + keyProvider = new SimpleKeyProvider(); + } else { + // create an instance of the key provider class and verify it + // implements KeyProvider + Object keyProviderObject; + try { + Class clazz = configuration.getClassByName(keyProviderClass); + keyProviderObject = clazz.newInstance(); + } catch (Exception e) { + throw new KeyProviderException("Unable to load key provider class.", e); + } + if (!(keyProviderObject instanceof KeyProvider)) { + throw new KeyProviderException(keyProviderClass + + " specified in config is not a valid KeyProvider class."); + } + keyProvider = (KeyProvider) keyProviderObject; + } + key = keyProvider.getStorageAccountKey(accountName, configuration); + + if (key == null) { throw new ConfigurationPropertyNotFoundException(accountName); } - return accountKey; + return key; } public Configuration getConfiguration() { @@ -212,10 +254,22 @@ public class AbfsConfiguration{ return this.createRemoteFileSystemDuringInitialization; } + public boolean getSkipUserGroupMetadataDuringInitialization() { + return this.skipUserGroupMetadataDuringInitialization; + } + public int getReadAheadQueueDepth() { return this.readAheadQueueDepth; } + public boolean isFlushEnabled() { + return this.enableFlush; + } + + public String getCustomUserAgentPrefix() { + return this.userAgentId; + } + void validateStorageAccountKeys() throws InvalidConfigurationValueException { Base64StringConfigurationBasicValidator validator = new Base64StringConfigurationBasicValidator( ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME, "", true); @@ -294,4 +348,9 @@ public class AbfsConfiguration{ void setWriteBufferSize(int bufferSize) { this.writeBufferSize = bufferSize; } + + @VisibleForTesting + void setEnableFlush(boolean enableFlush) { + this.enableFlush = enableFlush; + } } \ No newline at end of file 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 9f58f6b040a..b0a30a0b55c 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 @@ -36,6 +36,7 @@ import java.util.concurrent.Future; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -90,7 +91,6 @@ public class AzureBlobFileSystem extends FileSystem { this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority()); this.userGroupInformation = UserGroupInformation.getCurrentUser(); this.user = userGroupInformation.getUserName(); - this.primaryUserGroup = userGroupInformation.getPrimaryGroupName(); this.abfsStore = new AzureBlobFileSystemStore(uri, this.isSecure(), configuration, userGroupInformation); LOG.debug("Initializing NativeAzureFileSystem for {}", uri); @@ -98,7 +98,16 @@ public class AzureBlobFileSystem extends FileSystem { this.setWorkingDirectory(this.getHomeDirectory()); if (abfsStore.getAbfsConfiguration().getCreateRemoteFileSystemDuringInitialization()) { - this.createFileSystem(); + if (!this.fileSystemExists()) { + this.createFileSystem(); + } + } + + if (!abfsStore.getAbfsConfiguration().getSkipUserGroupMetadataDuringInitialization()) { + this.primaryUserGroup = userGroupInformation.getPrimaryGroupName(); + } else { + //Provide a default group name + this.primaryUserGroup = this.user; } } @@ -375,7 +384,7 @@ public class AzureBlobFileSystem extends FileSystem { if (file.getLen() < start) { return new BlockLocation[0]; } - final String blobLocationHost = this.abfsStore.getAbfsConfiguration().getAzureBlockLocationHost(); + final String blobLocationHost = abfsStore.getAbfsConfiguration().getAzureBlockLocationHost(); final String[] name = { blobLocationHost }; final String[] host = { blobLocationHost }; @@ -397,6 +406,13 @@ public class AzureBlobFileSystem extends FileSystem { return locations; } + @Override + protected void finalize() throws Throwable { + LOG.debug("finalize() called."); + close(); + super.finalize(); + } + public String getOwnerUser() { return user; } @@ -450,13 +466,31 @@ public class AzureBlobFileSystem extends FileSystem { } } + private boolean fileSystemExists() throws IOException { + LOG.debug( + "AzureBlobFileSystem.fileSystemExists uri: {}", uri); + try { + abfsStore.getFilesystemProperties(); + } catch (AzureBlobFileSystemException ex) { + try { + checkException(null, ex); + // Because HEAD request won't contain message body, + // there is not way to get the storage error code + // workaround here is to check its status code. + } catch (FileNotFoundException e) { + return false; + } + } + return true; + } + private void createFileSystem() throws IOException { LOG.debug( "AzureBlobFileSystem.createFileSystem uri: {}", uri); try { - this.abfsStore.createFilesystem(); + abfsStore.createFilesystem(); } catch (AzureBlobFileSystemException ex) { - checkException(null, ex, AzureServiceErrorCode.FILE_SYSTEM_ALREADY_EXISTS); + checkException(null, ex); } } @@ -556,10 +590,10 @@ public class AzureBlobFileSystem extends FileSystem { //AbfsRestOperationException.getMessage() contains full error info including path/uri. if (statusCode == HttpURLConnection.HTTP_NOT_FOUND) { - throw (IOException)new FileNotFoundException(ere.getMessage()) + throw (IOException) new FileNotFoundException(ere.getMessage()) .initCause(exception); } else if (statusCode == HttpURLConnection.HTTP_CONFLICT) { - throw (IOException)new FileAlreadyExistsException(ere.getMessage()) + throw (IOException) new FileAlreadyExistsException(ere.getMessage()) .initCause(exception); } else { throw ere; @@ -615,6 +649,11 @@ public class AzureBlobFileSystem extends FileSystem { @VisibleForTesting AzureBlobFileSystemStore getAbfsStore() { - return this.abfsStore; + return abfsStore; + } + + @VisibleForTesting + AbfsClient getAbfsClient() { + return abfsStore.getClient(); } } 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 8ac31ce0372..ba721497d74 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 @@ -31,8 +31,11 @@ import java.nio.charset.CharacterCodingException; import java.nio.charset.Charset; import java.nio.charset.CharsetDecoder; import java.nio.charset.CharsetEncoder; +import java.text.ParseException; +import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; +import java.util.Date; import java.util.HashSet; import java.util.Hashtable; import java.util.Map; @@ -65,7 +68,6 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema; import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; -import org.apache.hadoop.fs.azurebfs.services.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; @@ -75,8 +77,6 @@ import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.security.UserGroupInformation; import org.apache.http.client.utils.URIBuilder; -import org.joda.time.DateTime; -import org.joda.time.format.DateTimeFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -223,7 +223,7 @@ public class AzureBlobFileSystemStore { final OutputStream outputStream; outputStream = new FSDataOutputStream( new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), 0, - abfsConfiguration.getWriteBufferSize()), null); + abfsConfiguration.getWriteBufferSize(), abfsConfiguration.isFlushEnabled()), null); return outputStream; } @@ -287,7 +287,7 @@ public class AzureBlobFileSystemStore { final OutputStream outputStream; outputStream = new FSDataOutputStream( new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), - offset, abfsConfiguration.getWriteBufferSize()), null); + offset, abfsConfiguration.getWriteBufferSize(), abfsConfiguration.isFlushEnabled()), null); return outputStream; } @@ -366,7 +366,7 @@ public class AzureBlobFileSystemStore { true, 1, blockSize, - parseLastModifiedTime(lastModified).getMillis(), + parseLastModifiedTime(lastModified), path, eTag); } else { @@ -385,7 +385,7 @@ public class AzureBlobFileSystemStore { parseIsDirectory(resourceType), 1, blockSize, - parseLastModifiedTime(lastModified).getMillis(), + parseLastModifiedTime(lastModified), path, eTag); } @@ -419,10 +419,7 @@ public class AzureBlobFileSystemStore { long contentLength = entry.contentLength() == null ? 0 : entry.contentLength(); boolean isDirectory = entry.isDirectory() == null ? false : entry.isDirectory(); if (entry.lastModified() != null && !entry.lastModified().isEmpty()) { - final DateTime dateTime = DateTime.parse( - entry.lastModified(), - DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC()); - lastModifiedMillis = dateTime.getMillis(); + lastModifiedMillis = parseLastModifiedTime(entry.lastModified()); } Path entryPath = new Path(File.separator + entry.name()); @@ -534,10 +531,16 @@ public class AzureBlobFileSystemStore { && resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY); } - private DateTime parseLastModifiedTime(final String lastModifiedTime) { - return DateTime.parse( - lastModifiedTime, - DateTimeFormat.forPattern(DATE_TIME_PATTERN).withZoneUTC()); + private long parseLastModifiedTime(final String lastModifiedTime) { + long parsedTime = 0; + try { + Date utcDate = new SimpleDateFormat(DATE_TIME_PATTERN).parse(lastModifiedTime); + parsedTime = utcDate.getTime(); + } catch (ParseException e) { + LOG.error("Failed to parse the date {0}", lastModifiedTime); + } finally { + return parsedTime; + } } private String convertXmsPropertiesToCommaSeparatedString(final Hashtable properties) throws @@ -663,7 +666,7 @@ public class AzureBlobFileSystemStore { } if (other instanceof VersionedFileStatus) { - return this.version.equals(((VersionedFileStatus)other).version); + return this.version.equals(((VersionedFileStatus) other).version); } return true; @@ -702,5 +705,9 @@ public class AzureBlobFileSystemStore { } } + @VisibleForTesting + AbfsClient getClient() { + return this.client; + } } 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 ead1003982b..9c805a2a7c2 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 @@ -49,9 +49,15 @@ public final class ConfigurationKeys { public static final String AZURE_CONCURRENT_CONNECTION_VALUE_IN = "fs.azure.concurrentRequestCount.in"; public static final String AZURE_TOLERATE_CONCURRENT_APPEND = "fs.azure.io.read.tolerate.concurrent.append"; 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_AUTOTHROTTLING_ENABLE = "fs.azure.autothrottling.enable"; public static final String FS_AZURE_ATOMIC_RENAME_KEY = "fs.azure.atomic.rename.key"; public static final String FS_AZURE_READ_AHEAD_QUEUE_DEPTH = "fs.azure.readaheadqueue.depth"; + public static final String FS_AZURE_ENABLE_FLUSH = "fs.azure.enable.flush"; + public static final String FS_AZURE_USER_AGENT_PREFIX_KEY = "fs.azure.user.agent.prefix"; + + public static final String AZURE_KEY_ACCOUNT_KEYPROVIDER_PREFIX = "fs.azure.account.keyprovider."; + public static final String AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT = "fs.azure.shellkeyprovider.script"; 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 482158c1d73..1655d040493 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 @@ -39,7 +39,7 @@ public final class FileSystemConfigurations { private static final int ONE_MB = ONE_KB * ONE_KB; // Default upload and download buffer size - public static final int DEFAULT_WRITE_BUFFER_SIZE = 4 * ONE_MB; // 4 MB + public static final int DEFAULT_WRITE_BUFFER_SIZE = 8 * ONE_MB; // 8 MB public static final int DEFAULT_READ_BUFFER_SIZE = 4 * ONE_MB; // 4 MB public static final int MIN_BUFFER_SIZE = 16 * ONE_KB; // 16 KB public static final int MAX_BUFFER_SIZE = 100 * ONE_MB; // 100 MB @@ -50,10 +50,12 @@ public final class FileSystemConfigurations { public static final int MAX_CONCURRENT_WRITE_THREADS = 8; public static final boolean DEFAULT_READ_TOLERATE_CONCURRENT_APPEND = false; public static final boolean DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION = false; + 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 int DEFAULT_READ_AHEAD_QUEUE_DEPTH = -1; + public static final boolean DEFAULT_ENABLE_FLUSH = true; private FileSystemConfigurations() {} } \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/KeyProviderException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/KeyProviderException.java new file mode 100644 index 00000000000..6723d699f56 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/KeyProviderException.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.contracts.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; + +/** + * Thrown if there is a problem instantiating a KeyProvider or retrieving a key + * using a KeyProvider object. + */ +@InterfaceAudience.Private +public class KeyProviderException extends AzureBlobFileSystemException { + private static final long serialVersionUID = 1L; + + public KeyProviderException(String message) { + super(message); + } + + public KeyProviderException(String message, Throwable cause) { + super(message); + } + + public KeyProviderException(Throwable t) { + super(t.getMessage()); + } +} 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 90e580f9a68..a89f339967c 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 @@ -43,6 +43,7 @@ public enum AzureServiceErrorCode { INVALID_RENAME_SOURCE_PATH("InvalidRenameSourcePath", HttpURLConnection.HTTP_CONFLICT, null), INGRESS_OVER_ACCOUNT_LIMIT(null, HttpURLConnection.HTTP_UNAVAILABLE, "Ingress is over the account limit."), EGRESS_OVER_ACCOUNT_LIMIT(null, HttpURLConnection.HTTP_UNAVAILABLE, "Egress is over the account limit."), + INVALID_QUERY_PARAMETER_VALUE("InvalidQueryParameterValue", HttpURLConnection.HTTP_BAD_REQUEST, null), UNKNOWN(null, -1, null); private final String errorCode; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java index 02a7ac9b01c..903ff69e9e3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java @@ -18,7 +18,7 @@ package org.apache.hadoop.fs.azurebfs.contracts.services; -import com.fasterxml.jackson.annotation.JsonProperty; +import org.codehaus.jackson.annotate.JsonProperty; import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java index baf06dca250..32597423c86 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java @@ -20,7 +20,7 @@ package org.apache.hadoop.fs.azurebfs.contracts.services; import java.util.List; -import com.fasterxml.jackson.annotation.JsonProperty; +import org.codehaus.jackson.annotate.JsonProperty; import org.apache.hadoop.classification.InterfaceStability; 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 2b3ccc0472d..60369be9bc7 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 @@ -26,12 +26,13 @@ import java.util.ArrayList; import java.util.List; import java.util.Locale; +import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; - +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*; @@ -44,7 +45,7 @@ public class AbfsClient { public static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); private final URL baseUrl; private final SharedKeyCredentials sharedKeyCredentials; - private final String xMsVersion = "2018-03-28"; + private final String xMsVersion = "2018-06-17"; private final ExponentialRetryPolicy retryPolicy; private final String filesystem; private final AbfsConfiguration abfsConfiguration; @@ -59,7 +60,7 @@ public class AbfsClient { this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(FORWARD_SLASH) + 1); this.abfsConfiguration = abfsConfiguration; this.retryPolicy = exponentialRetryPolicy; - this.userAgent = initializeUserAgent(); + this.userAgent = initializeUserAgent(abfsConfiguration); } public String getFileSystem() { @@ -137,7 +138,7 @@ public class AbfsClient { final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_DIRECTORY, relativePath == null ? "" : urlEncode(relativePath)); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_DIRECTORY, relativePath == null ? "" : relativePath); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RECURSIVE, String.valueOf(recursive)); abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation); abfsUriQueryBuilder.addQuery(QUERY_PARAM_MAXRESULTS, String.valueOf(listMaxResults)); @@ -380,8 +381,8 @@ public class AbfsClient { return url; } - private static String urlEncode(final String value) throws AzureBlobFileSystemException { - String encodedString = null; + public static String urlEncode(final String value) throws AzureBlobFileSystemException { + String encodedString; try { encodedString = URLEncoder.encode(value, UTF_8) .replace(PLUS, PLUS_ENCODE) @@ -393,14 +394,23 @@ public class AbfsClient { return encodedString; } - private String initializeUserAgent() { + @VisibleForTesting + String initializeUserAgent(final AbfsConfiguration abfsConfiguration) { final String userAgentComment = String.format(Locale.ROOT, "(JavaJRE %s; %s %s)", System.getProperty(JAVA_VERSION), System.getProperty(OS_NAME) .replaceAll(SINGLE_WHITE_SPACE, EMPTY_STRING), System.getProperty(OS_VERSION)); - + String customUserAgentId = abfsConfiguration.getCustomUserAgentPrefix(); + if (customUserAgentId != null && !customUserAgentId.isEmpty()) { + return String.format(Locale.ROOT, CLIENT_VERSION + " %s %s", userAgentComment, customUserAgentId); + } return String.format(CLIENT_VERSION + " %s", userAgentComment); } + + @VisibleForTesting + URL getBaseUrl() { + return baseUrl; + } } 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 53f69004d8c..2bfcff25003 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -26,10 +26,11 @@ import java.net.URL; import java.util.List; import java.util.UUID; -import com.fasterxml.jackson.core.JsonFactory; -import com.fasterxml.jackson.core.JsonParser; -import com.fasterxml.jackson.core.JsonToken; -import com.fasterxml.jackson.databind.ObjectMapper; +import org.codehaus.jackson.JsonFactory; +import org.codehaus.jackson.JsonParser; +import org.codehaus.jackson.JsonToken; +import org.codehaus.jackson.map.ObjectMapper; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -167,7 +168,7 @@ public class AbfsHttpOperation { */ public AbfsHttpOperation(final URL url, final String method, final List requestHeaders) throws IOException { - this.isTraceEnabled = this.LOG.isTraceEnabled(); + this.isTraceEnabled = LOG.isTraceEnabled(); this.url = url; this.method = method; this.clientRequestId = UUID.randomUUID().toString(); @@ -303,7 +304,7 @@ public class AbfsHttpOperation { } } } catch (IOException ex) { - this.LOG.error("UnexpectedError: ", ex); + LOG.error("UnexpectedError: ", ex); throw ex; } finally { if (this.isTraceEnabled) { @@ -355,7 +356,7 @@ public class AbfsHttpOperation { return; } JsonFactory jf = new JsonFactory(); - try (JsonParser jp = jf.createParser(stream)) { + try (JsonParser jp = jf.createJsonParser(stream)) { String fieldName, fieldValue; jp.nextToken(); // START_OBJECT - { jp.nextToken(); // FIELD_NAME - "error": @@ -384,7 +385,7 @@ public class AbfsHttpOperation { // Ignore errors that occur while attempting to parse the storage // error, since the response may have been handled by the HTTP driver // or for other reasons have an unexpected - this.LOG.debug("ExpectedError: ", ex); + LOG.debug("ExpectedError: ", ex); } } @@ -415,7 +416,7 @@ public class AbfsHttpOperation { final ObjectMapper objectMapper = new ObjectMapper(); this.listResultSchema = objectMapper.readValue(stream, ListResultSchema.class); } catch (IOException ex) { - this.LOG.error("Unable to deserialize list results", ex); + LOG.error("Unable to deserialize list results", ex); throw ex; } } 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 848ce8ac953..960579dfaa3 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 @@ -64,7 +64,7 @@ public class AbfsInputStream extends FSInputStream { this.path = path; this.contentLength = contentLength; this.bufferSize = bufferSize; - this.readAheadQueueDepth = (readAheadQueueDepth >= 0) ? readAheadQueueDepth : 2 * Runtime.getRuntime().availableProcessors(); + this.readAheadQueueDepth = (readAheadQueueDepth >= 0) ? readAheadQueueDepth : Runtime.getRuntime().availableProcessors(); this.eTag = eTag; this.tolerateOobAppends = false; this.readAheadEnabled = true; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 2dbcee57f59..b69ec835d6d 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 @@ -43,6 +43,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable { private final String path; private long position; private boolean closed; + private boolean supportFlush; private volatile IOException lastError; private long lastFlushOffset; @@ -61,11 +62,13 @@ public class AbfsOutputStream extends OutputStream implements Syncable { final AbfsClient client, final String path, final long position, - final int bufferSize) { + final int bufferSize, + final boolean supportFlush) { this.client = client; this.path = path; this.position = position; this.closed = false; + this.supportFlush = supportFlush; this.lastError = null; this.lastFlushOffset = 0; this.bufferSize = bufferSize; @@ -162,7 +165,9 @@ public class AbfsOutputStream extends OutputStream implements Syncable { */ @Override public void flush() throws IOException { - flushInternalAsync(); + if (supportFlush) { + flushInternalAsync(); + } } /** Similar to posix fsync, flush out the data in client's user buffer @@ -171,7 +176,9 @@ public class AbfsOutputStream extends OutputStream implements Syncable { */ @Override public void hsync() throws IOException { - flushInternal(); + if (supportFlush) { + flushInternal(); + } } /** Flush out the data in client's user buffer. After the return of @@ -180,7 +187,9 @@ public class AbfsOutputStream extends OutputStream implements Syncable { */ @Override public void hflush() throws IOException { - flushInternal(); + if (supportFlush) { + flushInternal(); + } } /** @@ -262,7 +271,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable { writeOperation.task.get(); } catch (Exception ex) { if (ex.getCause() instanceof AzureBlobFileSystemException) { - ex = (AzureBlobFileSystemException)ex.getCause(); + ex = (AzureBlobFileSystemException) ex.getCause(); } lastError = new IOException(ex); throw lastError; @@ -277,8 +286,6 @@ public class AbfsOutputStream extends OutputStream implements Syncable { if (this.lastTotalAppendOffset > this.lastFlushOffset) { this.flushWrittenBytesToServiceInternal(this.lastTotalAppendOffset, true); } - - this.lastTotalAppendOffset = 0; } private synchronized void flushWrittenBytesToServiceInternal(final long offset, @@ -304,7 +311,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable { } } catch (Exception e) { if (e.getCause() instanceof AzureBlobFileSystemException) { - lastError = (AzureBlobFileSystemException)e.getCause(); + lastError = (AzureBlobFileSystemException) e.getCause(); } else { lastError = new IOException(e); } @@ -322,7 +329,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable { try { completionService.take(); } catch (InterruptedException e) { - lastError = (IOException)new InterruptedIOException(e.toString()).initCause(e); + lastError = (IOException) new InterruptedIOException(e.toString()).initCause(e); throw lastError; } } 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 61263985002..6dd32fafb7a 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 @@ -121,7 +121,7 @@ public class AbfsRestOperation { } } - if (result.getStatusCode() > HttpURLConnection.HTTP_BAD_REQUEST) { + if (result.getStatusCode() >= HttpURLConnection.HTTP_BAD_REQUEST) { throw new AbfsRestOperationException(result.getStatusCode(), result.getStorageErrorCode(), result.getStorageErrorMessage(), null, result); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java index 36248533125..a200b406a55 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java @@ -22,6 +22,7 @@ import java.util.HashMap; import java.util.Map; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; /** * The UrlQueryBuilder for Rest AbfsClient. @@ -51,7 +52,12 @@ public class AbfsUriQueryBuilder { } else { sb.append(AbfsHttpConstants.AND_MARK); } - sb.append(entry.getKey()).append(AbfsHttpConstants.EQUAL).append(entry.getValue()); + try { + sb.append(entry.getKey()).append(AbfsHttpConstants.EQUAL).append(AbfsClient.urlEncode(entry.getValue())); + } + catch (AzureBlobFileSystemException ex) { + throw new IllegalArgumentException("Query string param is not encode-able: " + entry.getKey() + "=" + entry.getValue()); + } } return sb.toString(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/KeyProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/KeyProvider.java new file mode 100644 index 00000000000..27f76f8594f --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/KeyProvider.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; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException; + +/** + * The interface that every Azure file system key provider must implement. + */ +public interface KeyProvider { + /** + * Key providers must implement this method. Given a list of configuration + * parameters for the specified Azure storage account, retrieve the plaintext + * storage account key. + * + * @param accountName + * the storage account name + * @param conf + * Hadoop configuration parameters + * @return the plaintext storage account key + * @throws KeyProviderException + */ + String getStorageAccountKey(String accountName, Configuration conf) + throws KeyProviderException; +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ShellDecryptionKeyProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ShellDecryptionKeyProvider.java new file mode 100644 index 00000000000..3fc05ff3e5e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ShellDecryptionKeyProvider.java @@ -0,0 +1,63 @@ +/** + * 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.Arrays; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException; +import org.apache.hadoop.util.Shell; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Shell decryption key provider which invokes an external script that will + * perform the key decryption. + */ +public class ShellDecryptionKeyProvider extends SimpleKeyProvider { + private static final Logger LOG = LoggerFactory.getLogger(ShellDecryptionKeyProvider.class); + + @Override + public String getStorageAccountKey(String accountName, Configuration conf) + throws KeyProviderException { + String envelope = super.getStorageAccountKey(accountName, conf); + + final String command = conf.get(ConfigurationKeys.AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT); + if (command == null) { + throw new KeyProviderException( + "Script path is not specified via fs.azure.shellkeyprovider.script"); + } + + String[] cmd = command.split(" "); + String[] cmdWithEnvelope = Arrays.copyOf(cmd, cmd.length + 1); + cmdWithEnvelope[cmdWithEnvelope.length - 1] = envelope; + + String decryptedKey = null; + try { + decryptedKey = Shell.execCommand(cmdWithEnvelope); + } catch (IOException ex) { + throw new KeyProviderException(ex); + } + + // trim any whitespace + return decryptedKey.trim(); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SimpleKeyProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SimpleKeyProvider.java new file mode 100644 index 00000000000..cedae57f1bb --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SimpleKeyProvider.java @@ -0,0 +1,54 @@ +/** + * 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.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException; +import org.apache.hadoop.security.ProviderUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Key provider that simply returns the storage account key from the + * configuration as plaintext. + */ +public class SimpleKeyProvider implements KeyProvider { + private static final Logger LOG = LoggerFactory.getLogger(SimpleKeyProvider.class); + + @Override + public String getStorageAccountKey(String accountName, Configuration conf) + throws KeyProviderException { + String key = null; + try { + Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders( + conf, AzureBlobFileSystem.class); + char[] keyChars = c.getPassword(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + accountName); + if (keyChars != null) { + key = new String(keyChars); + } + } catch(IOException ioe) { + LOG.warn("Unable to get key from credential providers. {}", ioe); + } + return key; + } +} 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 106fa09e438..b1f14856cf1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -27,10 +27,6 @@ import java.util.concurrent.Callable; import com.google.common.base.Preconditions; import org.junit.After; import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.rules.TestName; -import org.junit.rules.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,9 +37,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azure.AbstractWasbTestWithTimeout; import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore; import org.apache.hadoop.fs.azure.NativeAzureFileSystem; -import org.apache.hadoop.fs.azure.integration.AzureTestConstants; import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation; -import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.utils.UriUtils; @@ -175,6 +169,17 @@ public abstract class AbstractAbfsIntegrationTest extends return abfs; } + public AzureBlobFileSystem getFileSystem(Configuration configuration) throws Exception{ + final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(configuration); + return fs; + } + + public AzureBlobFileSystem getFileSystem(String abfsUri) throws Exception { + configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, abfsUri); + final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(configuration); + return fs; + } + /** * Creates the filesystem; updates the {@link #abfs} field. * @return the created filesystem. diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java new file mode 100644 index 00000000000..9c369bb2bf4 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.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; + +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.AbfsRestOperation; +import org.junit.Assert; +import org.junit.Test; + +/** + * Test continuation token which has equal sign. + */ +public final class ITestAbfsClient extends AbstractAbfsIntegrationTest { + private static final int LIST_MAX_RESULTS = 5000; + @Test + public void testContinuationTokenHavingEqualSign() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + AbfsClient abfsClient = fs.getAbfsClient(); + + try { + AbfsRestOperation op = abfsClient.listPath("/", true, LIST_MAX_RESULTS, "==========="); + Assert.assertTrue(false); + } catch (AbfsRestOperationException ex) { + Assert.assertEquals("InvalidQueryParameterValue", ex.getErrorCode().getErrorCode()); + } + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java similarity index 90% rename from hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.java rename to hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index dd06fe3edfc..f62ea6e529e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.azurebfs.services; +package org.apache.hadoop.fs.azurebfs; import java.util.Arrays; import java.util.Random; @@ -28,8 +28,6 @@ import org.junit.runners.Parameterized; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.azurebfs.AbstractAbfsScaleTest; -import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; 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; @@ -72,13 +70,13 @@ public class ITestAbfsReadWriteAndSeek extends AbstractAbfsScaleTest { final byte[] b = new byte[2 * bufferSize]; new Random().nextBytes(b); - try(final FSDataOutputStream stream = fs.create(TEST_PATH)) { + try (FSDataOutputStream stream = fs.create(TEST_PATH)) { stream.write(b); } final byte[] readBuffer = new byte[2 * bufferSize]; int result; - try(final FSDataInputStream inputStream = fs.open(TEST_PATH)) { + try (FSDataInputStream inputStream = fs.open(TEST_PATH)) { inputStream.seek(bufferSize); result = inputStream.read(readBuffer, bufferSize, bufferSize); assertNotEquals(-1, result); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java index 057dfa03115..f1800c00c16 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 @@ -108,7 +108,7 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest { final byte[] b = new byte[1024 * 1000]; new Random().nextBytes(b); - try(final FSDataOutputStream stream = fs.create(TEST_FILE)) { + try (FSDataOutputStream stream = fs.create(TEST_FILE)) { stream.write(b, TEST_OFFSET, b.length - TEST_OFFSET); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java index 04690de2403..522b635e9d7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java @@ -91,7 +91,7 @@ public class ITestAzureBlobFileSystemE2EScale extends final FileSystem.Statistics abfsStatistics; int testBufferSize; final byte[] sourceData; - try(final FSDataOutputStream stream = fs.create(TEST_FILE)) { + try (FSDataOutputStream stream = fs.create(TEST_FILE)) { abfsStatistics = fs.getFsStatistics(); abfsStatistics.reset(); @@ -112,7 +112,7 @@ public class ITestAzureBlobFileSystemE2EScale extends remoteData.length, abfsStatistics.getBytesRead()); assertEquals("bytes written in " + stats, sourceData.length, abfsStatistics.getBytesWritten()); - assertEquals("bytesRead from read() call", testBufferSize, bytesRead ); + assertEquals("bytesRead from read() call", testBufferSize, bytesRead); assertArrayEquals("round tripped data", sourceData, remoteData); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java new file mode 100644 index 00000000000..e4acbaefc61 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java @@ -0,0 +1,60 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs; + +import java.lang.ref.WeakReference; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; + +/** + * Test finalize() method when "fs.abfs.impl.disable.cache" is enabled. + */ +public class ITestAzureBlobFileSystemFinalize extends AbstractAbfsScaleTest{ + static final String DISABLE_CACHE_KEY = "fs.abfs.impl.disable.cache"; + + public ITestAzureBlobFileSystemFinalize() throws Exception { + super(); + } + + @Test + public void testFinalize() throws Exception { + // Disable the cache for filesystem to make sure there is no reference. + Configuration configuration = this.getConfiguration(); + configuration.setBoolean(this.DISABLE_CACHE_KEY, true); + + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(configuration); + + WeakReference ref = new WeakReference(fs); + fs = null; + + int i = 0; + int maxTries = 1000; + while (ref.get() != null && i < maxTries) { + System.gc(); + System.runFinalization(); + i++; + } + + Assert.assertTrue("testFinalizer didn't get cleaned up within maxTries", ref.get() == null); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index d90f0186da1..2f40b6444fd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -20,12 +20,20 @@ package org.apache.hadoop.fs.azurebfs; import java.util.ArrayList; import java.util.List; +import java.util.EnumSet; import java.util.Random; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.io.IOException; +import com.microsoft.azure.storage.blob.BlockEntry; +import com.microsoft.azure.storage.blob.BlockListingFilter; +import com.microsoft.azure.storage.blob.CloudBlockBlob; +import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount; +import org.hamcrest.core.IsEqual; +import org.hamcrest.core.IsNot; import org.junit.Test; import org.apache.hadoop.fs.FSDataInputStream; @@ -46,6 +54,8 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { private static final int THREAD_SLEEP_TIME = 6000; private static final Path TEST_FILE_PATH = new Path("/testfile"); + private static final int TEST_FILE_LENGTH = 1024 * 1024 * 8; + private static final int WAITING_TIME = 4000; public ITestAzureBlobFileSystemFlush() { super(); @@ -55,7 +65,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { public void testAbfsOutputStreamAsyncFlushWithRetainUncommittedData() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); final byte[] b; - try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { + try (FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { b = new byte[TEST_BUFFER_SIZE]; new Random().nextBytes(b); @@ -70,7 +80,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { } final byte[] r = new byte[TEST_BUFFER_SIZE]; - try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB)) { + try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB)) { while (inputStream.available() != 0) { int result = inputStream.read(r); @@ -84,7 +94,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { public void testAbfsOutputStreamSyncFlush() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); final byte[] b; - try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { + try (FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { b = new byte[TEST_BUFFER_SIZE]; new Random().nextBytes(b); stream.write(b); @@ -97,7 +107,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { } final byte[] r = new byte[TEST_BUFFER_SIZE]; - try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB)) { + try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB)) { int result = inputStream.read(r); assertNotEquals(-1, result); @@ -111,7 +121,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { final AzureBlobFileSystem fs = getFileSystem(); final FileSystem.Statistics abfsStatistics; ExecutorService es; - try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { + try (FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { abfsStatistics = fs.getFsStatistics(); abfsStatistics.reset(); @@ -160,7 +170,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { public void testWriteHeavyBytesToFileAsyncFlush() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); ExecutorService es = Executors.newFixedThreadPool(10); - try(final FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { + try (FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { final byte[] b = new byte[TEST_BUFFER_SIZE]; new Random().nextBytes(b); @@ -196,4 +206,118 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH); assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, fileStatus.getLen()); } + + @Test + public void testFlushWithFlushEnabled() throws Exception { + AzureBlobStorageTestAccount testAccount = createWasbTestAccount(); + String wasbUrl = testAccount.getFileSystem().getName(); + String abfsUrl = wasbUrlToAbfsUrl(wasbUrl); + final AzureBlobFileSystem fs = this.getFileSystem(abfsUrl); + byte[] buffer = getRandomBytesArray(); + CloudBlockBlob blob = testAccount.getBlobReference(TEST_FILE_PATH.toString().substring(1)); + try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, true)) { + // Wait for write request to be executed + Thread.sleep(WAITING_TIME); + stream.flush(); + ArrayList blockList = blob.downloadBlockList( + BlockListingFilter.COMMITTED, null, null, null); + // verify block has been committed + assertEquals(1, blockList.size()); + } + } + + @Test + public void testFlushWithFlushDisabled() throws Exception { + AzureBlobStorageTestAccount testAccount = createWasbTestAccount(); + String wasbUrl = testAccount.getFileSystem().getName(); + String abfsUrl = wasbUrlToAbfsUrl(wasbUrl); + final AzureBlobFileSystem fs = this.getFileSystem(abfsUrl); + byte[] buffer = getRandomBytesArray(); + CloudBlockBlob blob = testAccount.getBlobReference(TEST_FILE_PATH.toString().substring(1)); + try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, false)) { + // Wait for write request to be executed + Thread.sleep(WAITING_TIME); + stream.flush(); + ArrayList blockList = blob.downloadBlockList( + BlockListingFilter.COMMITTED, null, null, null); + // verify block has not been committed + assertEquals(0, blockList.size()); + } + } + + @Test + public void testHflushWithFlushEnabled() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + byte[] buffer = getRandomBytesArray(); + try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, true)) { + stream.hflush(); + validate(fs, TEST_FILE_PATH, buffer, true); + } + } + + @Test + public void testHflushWithFlushDisabled() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + byte[] buffer = getRandomBytesArray(); + try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, false)) { + stream.hflush(); + validate(fs, TEST_FILE_PATH, buffer, false); + } + } + + @Test + public void testHsyncWithFlushEnabled() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + byte[] buffer = getRandomBytesArray(); + try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, true)) { + stream.hsync(); + validate(fs, TEST_FILE_PATH, buffer, true); + } + } + + @Test + public void testHsyncWithFlushDisabled() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + byte[] buffer = getRandomBytesArray(); + try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, false)) { + stream.hsync(); + validate(fs, TEST_FILE_PATH, buffer, false); + } + } + + private byte[] getRandomBytesArray() { + final byte[] b = new byte[TEST_FILE_LENGTH]; + new Random().nextBytes(b); + return b; + } + + private FSDataOutputStream getStreamAfterWrite(AzureBlobFileSystem fs, Path path, byte[] buffer, boolean enableFlush) throws IOException { + fs.getAbfsStore().getAbfsConfiguration().setEnableFlush(enableFlush); + FSDataOutputStream stream = fs.create(path); + stream.write(buffer); + return stream; + } + + private AzureBlobStorageTestAccount createWasbTestAccount() throws Exception { + return AzureBlobStorageTestAccount.create("", EnumSet.of(AzureBlobStorageTestAccount.CreateOptions.CreateContainer), + this.getConfiguration()); + } + + private void validate(FileSystem fs, Path path, byte[] writeBuffer, boolean isEqual) throws IOException { + String filePath = path.toUri().toString(); + try (FSDataInputStream inputStream = fs.open(path)) { + byte[] readBuffer = new byte[TEST_FILE_LENGTH]; + int numBytesRead = inputStream.read(readBuffer, 0, readBuffer.length); + if (isEqual) { + assertArrayEquals( + String.format("Bytes read do not match bytes written to %1$s", filePath), writeBuffer, readBuffer); + } else { + assertThat( + String.format("Bytes read unexpectedly match bytes written to %1$s", + filePath), + readBuffer, + IsNot.not(IsEqual.equalTo(writeBuffer))); + } + } + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java index 5a6e46db016..874a8a34c59 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java @@ -20,10 +20,10 @@ package org.apache.hadoop.fs.azurebfs; import java.io.FileNotFoundException; -import org.junit.Test; - import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.junit.Test; + import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; /** 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 1a0edaf54e8..07426c4ab22 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 @@ -30,7 +30,6 @@ import org.junit.Test; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory; 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; @@ -86,7 +85,7 @@ public class ITestAzureBlobFileSystemRename extends assertRenameOutcome(fs, test1, new Path("testDir/test10"), true); - assertPathDoesNotExist(fs, "rename source dir", test1 ); + assertPathDoesNotExist(fs, "rename source dir", test1); } @Test diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java index 1c71125ffaf..7a7e3279da6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java @@ -20,7 +20,6 @@ package org.apache.hadoop.fs.azurebfs; import java.util.Hashtable; -import org.junit.Ignore; import org.junit.Test; import org.apache.hadoop.fs.FSDataInputStream; @@ -28,8 +27,6 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import static org.junit.Assert.assertEquals; - /** * Test FileSystemProperties. */ @@ -62,7 +59,6 @@ public class ITestFileSystemProperties extends AbstractAbfsIntegrationTest { } @Test - @Ignore("JDK7 doesn't support PATCH, so PUT is used. Fix is applied in latest test tenant") public void testBase64FileSystemProperties() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java similarity index 97% rename from hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java rename to hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java index ebaafa42e02..fb667ddc8a8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.azurebfs.services; +package org.apache.hadoop.fs.azurebfs; import java.lang.reflect.Field; @@ -48,7 +48,7 @@ import org.junit.Test; * Test ConfigurationServiceFieldsValidation. */ public class TestAbfsConfigurationFieldsValidation { - private final AbfsConfiguration abfsConfiguration; + private AbfsConfiguration abfsConfiguration; private static final String INT_KEY= "intKey"; private static final String LONG_KEY= "longKey"; @@ -89,10 +89,12 @@ public class TestAbfsConfigurationFieldsValidation { private boolean boolField; public TestAbfsConfigurationFieldsValidation() throws Exception { + super(); Base64 base64 = new Base64(); this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8); this.encodedAccountKey = new String(base64.encode("someAccountKey".getBytes(Charsets.UTF_8)), Charsets.UTF_8); - Configuration configuration = new Configuration(false); + Configuration configuration = new Configuration(); + configuration.addResource("azure-bfs-test.xml"); configuration.set(INT_KEY, "1234565"); configuration.set(LONG_KEY, "4194304"); configuration.set(STRING_KEY, "stringValue"); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java index d365e6e9169..c0c5f91fabc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java @@ -43,9 +43,8 @@ public class AbfsFileSystemContract extends AbstractBondedFSContract { @Override public String getScheme() { - return isSecure ? - FileSystemUriSchemes.ABFS_SECURE_SCHEME - : FileSystemUriSchemes.ABFS_SCHEME; + return isSecure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME + : FileSystemUriSchemes.ABFS_SCHEME; } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java new file mode 100644 index 00000000000..0b335a53e06 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -0,0 +1,60 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.azurebfs.services; + +import java.net.URL; +import java.util.regex.Pattern; + +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.junit.Assert; +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; + +/** + * Test useragent of abfs client. + * + */ +public final class TestAbfsClient { + + @Test + public void verifyUnknownUserAgent() throws Exception { + String expectedUserAgentPattern = "Azure Blob FS\\/1.0 \\(JavaJRE ([^\\)]+)\\)"; + final Configuration configuration = new Configuration(); + configuration.unset(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY); + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration); + AbfsClient abfsClient = new AbfsClient(new URL("http://azure.com"), null, abfsConfiguration, null); + String userAgent = abfsClient.initializeUserAgent(abfsConfiguration); + Pattern pattern = Pattern.compile(expectedUserAgentPattern); + Assert.assertTrue(pattern.matcher(userAgent).matches()); + } + + @Test + public void verifyUserAgent() throws Exception { + String expectedUserAgentPattern = "Azure Blob FS\\/1.0 \\(JavaJRE ([^\\)]+)\\) Partner Service"; + final Configuration configuration = new Configuration(); + configuration.set(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, "Partner Service"); + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration); + AbfsClient abfsClient = new AbfsClient(new URL("http://azure.com"), null, abfsConfiguration, null); + String userAgent = abfsClient.initializeUserAgent(abfsConfiguration); + Pattern pattern = Pattern.compile(expectedUserAgentPattern); + Assert.assertTrue(pattern.matcher(userAgent).matches()); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java new file mode 100644 index 00000000000..d17e767724c --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java @@ -0,0 +1,89 @@ +/** + * 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.File; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException; +import org.apache.hadoop.util.Shell; + +import static org.junit.Assert.assertEquals; + +/** + * Test ShellDecryptionKeyProvider. + * + */ +public class TestShellDecryptionKeyProvider { + public static final Log LOG = LogFactory + .getLog(TestShellDecryptionKeyProvider.class); + private static final File TEST_ROOT_DIR = new File(System.getProperty( + "test.build.data", "/tmp"), "TestShellDecryptionKeyProvider"); + + @Test + public void testScriptPathNotSpecified() throws Exception { + if (!Shell.WINDOWS) { + return; + } + ShellDecryptionKeyProvider provider = new ShellDecryptionKeyProvider(); + Configuration conf = new Configuration(); + String account = "testacct"; + String key = "key"; + + conf.set(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + account, key); + try { + provider.getStorageAccountKey(account, conf); + Assert + .fail("fs.azure.shellkeyprovider.script is not specified, we should throw"); + } catch (KeyProviderException e) { + LOG.info("Received an expected exception: " + e.getMessage()); + } + } + + @Test + public void testValidScript() throws Exception { + if (!Shell.WINDOWS) { + return; + } + String expectedResult = "decretedKey"; + + // Create a simple script which echoes the given key plus the given + // expected result (so that we validate both script input and output) + File scriptFile = new File(TEST_ROOT_DIR, "testScript.cmd"); + FileUtils.writeStringToFile(scriptFile, "@echo %1 " + expectedResult); + + ShellDecryptionKeyProvider provider = new ShellDecryptionKeyProvider(); + Configuration conf = new Configuration(); + String account = "testacct"; + String key = "key1"; + conf.set(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + account, key); + conf.set(ConfigurationKeys.AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT, + "cmd /c " + scriptFile.getAbsolutePath()); + + String result = provider.getStorageAccountKey(account, conf); + assertEquals(key + " " + expectedResult, result); + } +} From cc5cc60c4162a2d788c80ebbbe69ca49f3eb90e6 Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Sat, 11 Aug 2018 05:32:30 +0000 Subject: [PATCH 05/29] Fixing issue due to commit 2b2399d6 after rebase onto trunk. --- .../java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 b0a30a0b55c..2cb517b4b10 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -40,7 +40,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.commons.lang.ArrayUtils; +import org.apache.commons.lang3.ArrayUtils; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; From d6a4f39bd5f192e5e3377630887a6998d4d400c2 Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Thu, 16 Aug 2018 16:19:13 +0000 Subject: [PATCH 06/29] HADOOP-15669. ABFS: Improve HTTPS Performance. Contributed by Vishwajeet Dusane. --- hadoop-project/pom.xml | 7 + hadoop-tools/hadoop-azure/pom.xml | 7 +- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 8 + .../azurebfs/constants/ConfigurationKeys.java | 1 + .../constants/FileSystemConfigurations.java | 4 + .../fs/azurebfs/services/AbfsClient.java | 43 +++- .../azurebfs/services/AbfsHttpOperation.java | 11 + .../fs/azurebfs/utils/SSLSocketFactoryEx.java | 240 ++++++++++++++++++ ...TestAbfsConfigurationFieldsValidation.java | 42 ++- .../fs/azurebfs/services/TestAbfsClient.java | 40 ++- 10 files changed, 375 insertions(+), 28 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/SSLSocketFactoryEx.java diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 184cb3d1ac8..a13283a0ac6 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -1349,6 +1349,13 @@ 7.0.0 + + + org.wildfly.openssl + wildfly-openssl + 1.0.4.Final + + org.threadly threadly diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml index 7d0406c6cf8..7152f6383a3 100644 --- a/hadoop-tools/hadoop-azure/pom.xml +++ b/hadoop-tools/hadoop-azure/pom.xml @@ -197,13 +197,18 @@ jackson-mapper-asl compile + org.codehaus.jackson jackson-core-asl compile - + + org.wildfly.openssl + wildfly-openssl + compile + 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 1fb5df9aa39..e647ae8f250 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 @@ -44,6 +44,10 @@ import org.apache.hadoop.fs.azurebfs.diagnostics.LongConfigurationBasicValidator import org.apache.hadoop.fs.azurebfs.diagnostics.StringConfigurationBasicValidator; import org.apache.hadoop.fs.azurebfs.services.KeyProvider; import org.apache.hadoop.fs.azurebfs.services.SimpleKeyProvider; +import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx; + +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SSL_CHANNEL_MODE_KEY; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_FS_AZURE_SSL_CHANNEL_MODE; /** * Configuration for Azure Blob FileSystem. @@ -270,6 +274,10 @@ public class AbfsConfiguration{ return this.userAgentId; } + public SSLSocketFactoryEx.SSLChannelMode getPreferredSSLFactoryOption() { + return configuration.getEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, DEFAULT_FS_AZURE_SSL_CHANNEL_MODE); + } + void validateStorageAccountKeys() throws InvalidConfigurationValueException { Base64StringConfigurationBasicValidator validator = new Base64StringConfigurationBasicValidator( ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME, "", true); 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 9c805a2a7c2..16ddd900edd 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 @@ -55,6 +55,7 @@ public final class ConfigurationKeys { public static final String FS_AZURE_READ_AHEAD_QUEUE_DEPTH = "fs.azure.readaheadqueue.depth"; public static final String FS_AZURE_ENABLE_FLUSH = "fs.azure.enable.flush"; public static final String FS_AZURE_USER_AGENT_PREFIX_KEY = "fs.azure.user.agent.prefix"; + public static final String FS_AZURE_SSL_CHANNEL_MODE_KEY = "fs.azure.ssl.channel.mode"; public static final String AZURE_KEY_ACCOUNT_KEYPROVIDER_PREFIX = "fs.azure.account.keyprovider."; public static final String AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT = "fs.azure.shellkeyprovider.script"; 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 1655d040493..5b92dddf732 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -20,6 +20,7 @@ package org.apache.hadoop.fs.azurebfs.constants; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx; /** * Responsible to keep all the Azure Blob File System related configurations. @@ -57,5 +58,8 @@ public final class FileSystemConfigurations { public static final int DEFAULT_READ_AHEAD_QUEUE_DEPTH = -1; public static final boolean DEFAULT_ENABLE_FLUSH = true; + public static final SSLSocketFactoryEx.SSLChannelMode DEFAULT_FS_AZURE_SSL_CHANNEL_MODE + = SSLSocketFactoryEx.SSLChannelMode.Default; + private FileSystemConfigurations() {} } \ No newline at end of file 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 60369be9bc7..e003ffd31d9 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.azurebfs.services; +import java.io.IOException; import java.io.UnsupportedEncodingException; import java.net.MalformedURLException; import java.net.URL; @@ -27,6 +28,7 @@ import java.util.List; import java.util.Locale; import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,6 +37,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*; import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.*; @@ -60,7 +63,19 @@ public class AbfsClient { this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(FORWARD_SLASH) + 1); this.abfsConfiguration = abfsConfiguration; this.retryPolicy = exponentialRetryPolicy; - this.userAgent = initializeUserAgent(abfsConfiguration); + + String sslProviderName = null; + + if (this.baseUrl.toString().startsWith(HTTPS_SCHEME)) { + try { + SSLSocketFactoryEx.initializeDefaultFactory(this.abfsConfiguration.getPreferredSSLFactoryOption()); + sslProviderName = SSLSocketFactoryEx.getDefaultFactory().getProviderName(); + } catch (IOException e) { + // Suppress exception. Failure to init SSLSocketFactoryEx would have only performance impact. + } + } + + this.userAgent = initializeUserAgent(abfsConfiguration, sslProviderName); } public String getFileSystem() { @@ -395,16 +410,26 @@ public class AbfsClient { } @VisibleForTesting - String initializeUserAgent(final AbfsConfiguration abfsConfiguration) { - final String userAgentComment = String.format(Locale.ROOT, - "(JavaJRE %s; %s %s)", - System.getProperty(JAVA_VERSION), - System.getProperty(OS_NAME) - .replaceAll(SINGLE_WHITE_SPACE, EMPTY_STRING), - System.getProperty(OS_VERSION)); + String initializeUserAgent(final AbfsConfiguration abfsConfiguration, + final String sslProviderName) { + StringBuilder sb = new StringBuilder(); + sb.append("(JavaJRE "); + sb.append(System.getProperty(JAVA_VERSION)); + sb.append("; "); + sb.append( + System.getProperty(OS_NAME).replaceAll(SINGLE_WHITE_SPACE, EMPTY_STRING)); + sb.append(" "); + sb.append(System.getProperty(OS_VERSION)); + if (sslProviderName != null && !sslProviderName.isEmpty()) { + sb.append("; "); + sb.append(sslProviderName); + } + sb.append(")"); + final String userAgentComment = sb.toString(); String customUserAgentId = abfsConfiguration.getCustomUserAgentPrefix(); if (customUserAgentId != null && !customUserAgentId.isEmpty()) { - return String.format(Locale.ROOT, CLIENT_VERSION + " %s %s", userAgentComment, customUserAgentId); + return String.format(Locale.ROOT, CLIENT_VERSION + " %s %s", + userAgentComment, customUserAgentId); } return String.format(CLIENT_VERSION + " %s", userAgentComment); } 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 2bfcff25003..f493298f2c1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -26,6 +26,10 @@ import java.net.URL; import java.util.List; import java.util.UUID; +import javax.net.ssl.HttpsURLConnection; +import javax.net.ssl.SSLSocketFactory; + +import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx; import org.codehaus.jackson.JsonFactory; import org.codehaus.jackson.JsonParser; import org.codehaus.jackson.JsonToken; @@ -174,6 +178,13 @@ public class AbfsHttpOperation { this.clientRequestId = UUID.randomUUID().toString(); this.connection = openConnection(); + if (this.connection instanceof HttpsURLConnection) { + HttpsURLConnection secureConn = (HttpsURLConnection) this.connection; + SSLSocketFactory sslSocketFactory = SSLSocketFactoryEx.getDefaultFactory(); + if (sslSocketFactory != null) { + secureConn.setSSLSocketFactory(sslSocketFactory); + } + } this.connection.setConnectTimeout(CONNECT_TIMEOUT); this.connection.setReadTimeout(READ_TIMEOUT); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/SSLSocketFactoryEx.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/SSLSocketFactoryEx.java new file mode 100644 index 00000000000..202e1850603 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/SSLSocketFactoryEx.java @@ -0,0 +1,240 @@ +/** + * 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.io.IOException; +import java.net.InetAddress; +import java.net.Socket; +import java.net.SocketException; +import java.security.KeyManagementException; +import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; + +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLSocket; +import javax.net.ssl.SSLSocketFactory; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.wildfly.openssl.OpenSSLProvider; + + +/** + * Extension to use native OpenSSL library instead of JSSE for better + * performance. + * + */ +public class SSLSocketFactoryEx extends SSLSocketFactory { + + /** + * Default indicates Ordered, preferred OpenSSL, if failed to load then fall + * back to Default_JSSE + */ + public enum SSLChannelMode { + OpenSSL, + Default, + Default_JSSE + } + + private static SSLSocketFactoryEx instance = null; + private static final Logger LOG = LoggerFactory.getLogger( + SSLSocketFactoryEx.class); + private String providerName; + private SSLContext ctx; + private String[] ciphers; + private SSLChannelMode channelMode; + + /** + * Initialize a singleton SSL socket factory. + * + * @param preferredMode applicable only if the instance is not initialized. + * @throws IOException + */ + public synchronized static void initializeDefaultFactory( + SSLChannelMode preferredMode) throws IOException { + if (instance == null) { + instance = new SSLSocketFactoryEx(preferredMode); + } + } + + /** + * Singletone instance of the SSLSocketFactory. + * + * SSLSocketFactory must be initialized with appropriate SSLChannelMode + * using initializeDefaultFactory method. + * + * @return instance of the SSLSocketFactory, instance must be initialized by + * initializeDefaultFactory. + */ + public static SSLSocketFactoryEx getDefaultFactory() { + return instance; + } + + static { + OpenSSLProvider.register(); + } + + private SSLSocketFactoryEx(SSLChannelMode preferredChannelMode) + throws IOException { + try { + initializeSSLContext(preferredChannelMode); + } catch (NoSuchAlgorithmException e) { + throw new IOException(e); + } catch (KeyManagementException e) { + throw new IOException(e); + } + + // Get list of supported cipher suits from the SSL factory. + SSLSocketFactory factory = ctx.getSocketFactory(); + String[] defaultCiphers = factory.getSupportedCipherSuites(); + String version = System.getProperty("java.version"); + + ciphers = (channelMode == SSLChannelMode.Default_JSSE + && version.startsWith("1.8")) + ? alterCipherList(defaultCiphers) : defaultCiphers; + + providerName = ctx.getProvider().getName() + "-" + + ctx.getProvider().getVersion(); + } + + private void initializeSSLContext(SSLChannelMode preferredChannelMode) + throws NoSuchAlgorithmException, KeyManagementException { + switch (preferredChannelMode) { + case Default: + try { + ctx = SSLContext.getInstance("openssl.TLS"); + ctx.init(null, null, null); + channelMode = SSLChannelMode.OpenSSL; + } catch (NoSuchAlgorithmException e) { + LOG.warn("Failed to load OpenSSL. Falling back to the JSSE default."); + ctx = SSLContext.getDefault(); + channelMode = SSLChannelMode.Default_JSSE; + } + break; + case OpenSSL: + ctx = SSLContext.getInstance("openssl.TLS"); + ctx.init(null, null, null); + channelMode = SSLChannelMode.OpenSSL; + break; + case Default_JSSE: + ctx = SSLContext.getDefault(); + channelMode = SSLChannelMode.Default_JSSE; + break; + default: + throw new AssertionError("Unknown channel mode: " + + preferredChannelMode); + } + } + + public String getProviderName() { + return providerName; + } + + @Override + public String[] getDefaultCipherSuites() { + return ciphers.clone(); + } + + @Override + public String[] getSupportedCipherSuites() { + return ciphers.clone(); + } + + public Socket createSocket() throws IOException { + SSLSocketFactory factory = ctx.getSocketFactory(); + SSLSocket ss = (SSLSocket) factory.createSocket(); + configureSocket(ss); + return ss; + } + + @Override + public Socket createSocket(Socket s, String host, int port, + boolean autoClose) throws IOException { + SSLSocketFactory factory = ctx.getSocketFactory(); + SSLSocket ss = (SSLSocket) factory.createSocket(s, host, port, autoClose); + + configureSocket(ss); + return ss; + } + + @Override + public Socket createSocket(InetAddress address, int port, + InetAddress localAddress, int localPort) + throws IOException { + SSLSocketFactory factory = ctx.getSocketFactory(); + SSLSocket ss = (SSLSocket) factory + .createSocket(address, port, localAddress, localPort); + + configureSocket(ss); + return ss; + } + + @Override + public Socket createSocket(String host, int port, InetAddress localHost, + int localPort) throws IOException { + SSLSocketFactory factory = ctx.getSocketFactory(); + SSLSocket ss = (SSLSocket) factory + .createSocket(host, port, localHost, localPort); + + configureSocket(ss); + + return ss; + } + + @Override + public Socket createSocket(InetAddress host, int port) throws IOException { + SSLSocketFactory factory = ctx.getSocketFactory(); + SSLSocket ss = (SSLSocket) factory.createSocket(host, port); + + configureSocket(ss); + + return ss; + } + + @Override + public Socket createSocket(String host, int port) throws IOException { + SSLSocketFactory factory = ctx.getSocketFactory(); + SSLSocket ss = (SSLSocket) factory.createSocket(host, port); + + configureSocket(ss); + + return ss; + } + + private void configureSocket(SSLSocket ss) throws SocketException { + ss.setEnabledCipherSuites(ciphers); + } + + private String[] alterCipherList(String[] defaultCiphers) { + + ArrayList preferredSuits = new ArrayList<>(); + + // Remove GCM mode based ciphers from the supported list. + for (int i = 0; i < defaultCiphers.length; i++) { + if (defaultCiphers[i].contains("_GCM_")) { + LOG.debug("Removed Cipher - " + defaultCiphers[i]); + } else { + preferredSuits.add(defaultCiphers[i]); + } + } + + ciphers = preferredSuits.toArray(new String[0]); + return ciphers; + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java index fb667ddc8a8..556dcdb0422 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java @@ -30,6 +30,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidati import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SSL_CHANNEL_MODE_KEY; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS; @@ -41,7 +42,10 @@ import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.A import org.apache.commons.codec.binary.Base64; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; +import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx; import org.junit.Test; /** @@ -50,11 +54,11 @@ import org.junit.Test; public class TestAbfsConfigurationFieldsValidation { private AbfsConfiguration abfsConfiguration; - private static final String INT_KEY= "intKey"; - private static final String LONG_KEY= "longKey"; - private static final String STRING_KEY= "stringKey"; - private static final String BASE64_KEY= "base64Key"; - private static final String BOOLEAN_KEY= "booleanKey"; + private static final String INT_KEY = "intKey"; + private static final String LONG_KEY = "longKey"; + private static final String STRING_KEY = "stringKey"; + private static final String BASE64_KEY = "base64Key"; + private static final String BOOLEAN_KEY = "booleanKey"; private static final int DEFAULT_INT = 4194304; private static final int DEFAULT_LONG = 4194304; @@ -77,15 +81,15 @@ public class TestAbfsConfigurationFieldsValidation { private int longField; @StringConfigurationValidatorAnnotation(ConfigurationKey = STRING_KEY, - DefaultValue = "default") + DefaultValue = "default") private String stringField; @Base64StringConfigurationValidatorAnnotation(ConfigurationKey = BASE64_KEY, - DefaultValue = "base64") + DefaultValue = "base64") private String base64Field; @BooleanConfigurationValidatorAnnotation(ConfigurationKey = BOOLEAN_KEY, - DefaultValue = false) + DefaultValue = false) private boolean boolField; public TestAbfsConfigurationFieldsValidation() throws Exception { @@ -142,8 +146,26 @@ public class TestAbfsConfigurationFieldsValidation { assertEquals(this.encodedAccountKey, accountKey); } - @Test (expected = ConfigurationPropertyNotFoundException.class) + @Test(expected = ConfigurationPropertyNotFoundException.class) public void testGetAccountKeyWithNonExistingAccountName() throws Exception { abfsConfiguration.getStorageAccountKey("bogusAccountName"); } -} + + @Test + public void testSSLSocketFactoryConfiguration() throws InvalidConfigurationValueException, IllegalAccessException { + assertEquals(SSLSocketFactoryEx.SSLChannelMode.Default, abfsConfiguration.getPreferredSSLFactoryOption()); + assertNotEquals(SSLSocketFactoryEx.SSLChannelMode.Default_JSSE, abfsConfiguration.getPreferredSSLFactoryOption()); + assertNotEquals(SSLSocketFactoryEx.SSLChannelMode.OpenSSL, abfsConfiguration.getPreferredSSLFactoryOption()); + + Configuration configuration = new Configuration(); + configuration.setEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, SSLSocketFactoryEx.SSLChannelMode.Default_JSSE); + AbfsConfiguration localAbfsConfiguration = new AbfsConfiguration(configuration); + assertEquals(SSLSocketFactoryEx.SSLChannelMode.Default_JSSE, localAbfsConfiguration.getPreferredSSLFactoryOption()); + + configuration = new Configuration(); + configuration.setEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, SSLSocketFactoryEx.SSLChannelMode.OpenSSL); + localAbfsConfiguration = new AbfsConfiguration(configuration); + assertEquals(SSLSocketFactoryEx.SSLChannelMode.OpenSSL, localAbfsConfiguration.getPreferredSSLFactoryOption()); + } + +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index 0b335a53e06..7bb27fc4514 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -22,6 +22,7 @@ import java.net.URL; import java.util.regex.Pattern; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx; import org.junit.Assert; import org.junit.Test; @@ -34,16 +35,29 @@ import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; */ public final class TestAbfsClient { + private void validateUserAgent(String expectedPattern, + URL baseUrl, + AbfsConfiguration config, + boolean includeSSLProvider) { + AbfsClient client = new AbfsClient(baseUrl, null, + config, null); + String sslProviderName = null; + if (includeSSLProvider) { + sslProviderName = SSLSocketFactoryEx.getDefaultFactory().getProviderName(); + } + String userAgent = client.initializeUserAgent(config, sslProviderName); + Pattern pattern = Pattern.compile(expectedPattern); + Assert.assertTrue(pattern.matcher(userAgent).matches()); + } + @Test public void verifyUnknownUserAgent() throws Exception { String expectedUserAgentPattern = "Azure Blob FS\\/1.0 \\(JavaJRE ([^\\)]+)\\)"; final Configuration configuration = new Configuration(); configuration.unset(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY); AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration); - AbfsClient abfsClient = new AbfsClient(new URL("http://azure.com"), null, abfsConfiguration, null); - String userAgent = abfsClient.initializeUserAgent(abfsConfiguration); - Pattern pattern = Pattern.compile(expectedUserAgentPattern); - Assert.assertTrue(pattern.matcher(userAgent).matches()); + validateUserAgent(expectedUserAgentPattern, new URL("http://azure.com"), + abfsConfiguration, false); } @Test @@ -52,9 +66,19 @@ public final class TestAbfsClient { final Configuration configuration = new Configuration(); configuration.set(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, "Partner Service"); AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration); - AbfsClient abfsClient = new AbfsClient(new URL("http://azure.com"), null, abfsConfiguration, null); - String userAgent = abfsClient.initializeUserAgent(abfsConfiguration); - Pattern pattern = Pattern.compile(expectedUserAgentPattern); - Assert.assertTrue(pattern.matcher(userAgent).matches()); + validateUserAgent(expectedUserAgentPattern, new URL("http://azure.com"), + abfsConfiguration, false); + } + + @Test + public void verifyUserAgentWithSSLProvider() throws Exception { + String expectedUserAgentPattern = "Azure Blob FS\\/1.0 \\(JavaJRE ([^\\)]+) SunJSSE-1.8\\) Partner Service"; + final Configuration configuration = new Configuration(); + configuration.set(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, "Partner Service"); + configuration.set(ConfigurationKeys.FS_AZURE_SSL_CHANNEL_MODE_KEY, + SSLSocketFactoryEx.SSLChannelMode.Default_JSSE.name()); + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration); + validateUserAgent(expectedUserAgentPattern, new URL("https://azure.com"), + abfsConfiguration, true); } } \ No newline at end of file From 9149b9703e3ab09abdc087db129e82ad3f4cefa1 Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Sat, 18 Aug 2018 18:53:32 +0000 Subject: [PATCH 07/29] HADOOP-15660. ABFS: Add support for OAuth Contributed by Da Zhou, Rajeev Bansal, and Junhua Gu. --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 149 ++++++-- .../fs/azurebfs/AzureBlobFileSystemStore.java | 28 +- .../azurebfs/constants/ConfigurationKeys.java | 19 + .../TokenAccessProviderException.java | 36 ++ .../services/AzureServiceErrorCode.java | 1 + .../services/ListResultEntrySchema.java | 89 ++++- .../azurebfs/oauth2/AccessTokenProvider.java | 98 +++++ .../azurebfs/oauth2/AzureADAuthenticator.java | 344 ++++++++++++++++++ .../fs/azurebfs/oauth2/AzureADToken.java | 47 +++ .../oauth2/ClientCredsTokenProvider.java | 62 ++++ .../oauth2/CustomTokenProviderAdaptee.java | 75 ++++ .../oauth2/CustomTokenProviderAdapter.java | 57 +++ .../fs/azurebfs/oauth2/MsiTokenProvider.java | 48 +++ .../fs/azurebfs/oauth2/QueryParams.java | 69 ++++ .../RefreshTokenBasedTokenProvider.java | 57 +++ .../oauth2/UserPasswordTokenProvider.java | 66 ++++ .../fs/azurebfs/oauth2/package-info.java | 18 + .../fs/azurebfs/services/AbfsClient.java | 18 +- .../fs/azurebfs/services/AbfsHttpHeader.java | 2 +- .../azurebfs/services/AbfsRestOperation.java | 19 +- .../hadoop/fs/azurebfs/services/AuthType.java | 27 ++ .../azurebfs/AbstractAbfsIntegrationTest.java | 35 +- .../hadoop/fs/azurebfs/ITestAbfsClient.java | 2 +- .../ITestAzureBlobFileSystemBackCompat.java | 4 + .../ITestAzureBlobFileSystemFileStatus.java | 3 - .../ITestAzureBlobFileSystemFinalize.java | 8 +- .../ITestAzureBlobFileSystemFlush.java | 8 +- .../ITestAzureBlobFileSystemOauth.java | 176 +++++++++ .../ITestAzureBlobFileSystemRandomRead.java | 3 + .../ITestFileSystemInitialization.java | 5 +- .../azurebfs/ITestFileSystemRegistration.java | 11 +- .../azurebfs/ITestWasbAbfsCompatibility.java | 2 + .../constants/TestConfigurationKeys.java | 6 + .../contract/ABFSContractTestBinding.java | 14 +- .../ITestAbfsFileSystemContractAppend.java | 19 +- .../ITestAbfsFileSystemContractConcat.java | 17 +- .../ITestAbfsFileSystemContractCreate.java | 17 +- .../ITestAbfsFileSystemContractDelete.java | 17 +- .../ITestAbfsFileSystemContractDistCp.java | 2 +- ...stAbfsFileSystemContractGetFileStatus.java | 17 +- .../ITestAbfsFileSystemContractMkdir.java | 17 +- .../ITestAbfsFileSystemContractOpen.java | 17 +- .../ITestAbfsFileSystemContractRename.java | 17 +- ...stAbfsFileSystemContractRootDirectory.java | 16 +- ...estAbfsFileSystemContractSecureDistCp.java | 2 +- .../ITestAbfsFileSystemContractSeek.java | 17 +- .../ITestAbfsFileSystemContractSetTimes.java | 17 +- .../ITestAzureBlobFileSystemBasics.java | 2 +- .../fs/azurebfs/services/TestAbfsClient.java | 6 +- .../fs/azurebfs/services/TestQueryParams.java | 72 ++++ .../utils/CleanUpAbfsTestContainer.java | 13 +- .../src/test/resources/azure-bfs-test.xml | 128 ++++++- 52 files changed, 1769 insertions(+), 250 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/TokenAccessProviderException.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AccessTokenProvider.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADToken.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/ClientCredsTokenProvider.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdaptee.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdapter.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/MsiTokenProvider.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/QueryParams.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/RefreshTokenBasedTokenProvider.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/UserPasswordTokenProvider.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AuthType.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestQueryParams.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index e647ae8f250..f26f56204ca 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.azurebfs; +import java.io.IOException; import java.lang.reflect.Field; import java.util.Map; @@ -26,7 +27,6 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation; @@ -37,16 +37,26 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemExc import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException; import org.apache.hadoop.fs.azurebfs.diagnostics.Base64StringConfigurationBasicValidator; import org.apache.hadoop.fs.azurebfs.diagnostics.BooleanConfigurationBasicValidator; import org.apache.hadoop.fs.azurebfs.diagnostics.IntegerConfigurationBasicValidator; import org.apache.hadoop.fs.azurebfs.diagnostics.LongConfigurationBasicValidator; import org.apache.hadoop.fs.azurebfs.diagnostics.StringConfigurationBasicValidator; +import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; +import org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider; +import org.apache.hadoop.fs.azurebfs.oauth2.CustomTokenProviderAdaptee; +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.azurebfs.services.KeyProvider; import org.apache.hadoop.fs.azurebfs.services.SimpleKeyProvider; import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx; +import org.apache.hadoop.util.ReflectionUtils; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SSL_CHANNEL_MODE_KEY; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.*; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_FS_AZURE_SSL_CHANNEL_MODE; /** @@ -58,81 +68,81 @@ public class AbfsConfiguration{ private final Configuration configuration; private final boolean isSecure; - @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_WRITE_BUFFER_SIZE, + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_WRITE_BUFFER_SIZE, MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE, MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE, DefaultValue = FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE) private int writeBufferSize; - @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_READ_BUFFER_SIZE, + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_READ_BUFFER_SIZE, MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE, MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE, DefaultValue = FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE) private int readBufferSize; - @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MIN_BACKOFF_INTERVAL, + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_MIN_BACKOFF_INTERVAL, DefaultValue = FileSystemConfigurations.DEFAULT_MIN_BACKOFF_INTERVAL) private int minBackoffInterval; - @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MAX_BACKOFF_INTERVAL, + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_MAX_BACKOFF_INTERVAL, DefaultValue = FileSystemConfigurations.DEFAULT_MAX_BACKOFF_INTERVAL) private int maxBackoffInterval; - @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BACKOFF_INTERVAL, + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_BACKOFF_INTERVAL, DefaultValue = FileSystemConfigurations.DEFAULT_BACKOFF_INTERVAL) private int backoffInterval; - @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_MAX_IO_RETRIES, + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_MAX_IO_RETRIES, MinValue = 0, DefaultValue = FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS) private int maxIoRetries; - @LongConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BLOCK_SIZE_PROPERTY_NAME, + @LongConfigurationValidatorAnnotation(ConfigurationKey = AZURE_BLOCK_SIZE_PROPERTY_NAME, MinValue = 0, MaxValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE, DefaultValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE) private long azureBlockSize; - @StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME, + @StringConfigurationValidatorAnnotation(ConfigurationKey = AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME, DefaultValue = FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT) private String azureBlockLocationHost; - @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CONCURRENT_CONNECTION_VALUE_OUT, + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_CONCURRENT_CONNECTION_VALUE_OUT, MinValue = 1, DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_WRITE_THREADS) private int maxConcurrentWriteThreads; - @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CONCURRENT_CONNECTION_VALUE_IN, + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_CONCURRENT_CONNECTION_VALUE_IN, MinValue = 1, DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_READ_THREADS) private int maxConcurrentReadThreads; - @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_TOLERATE_CONCURRENT_APPEND, + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = AZURE_TOLERATE_CONCURRENT_APPEND, DefaultValue = FileSystemConfigurations.DEFAULT_READ_TOLERATE_CONCURRENT_APPEND) private boolean tolerateOobAppends; - @StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_ATOMIC_RENAME_KEY, + @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ATOMIC_RENAME_KEY, DefaultValue = FileSystemConfigurations.DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES) private String azureAtomicDirs; - @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, DefaultValue = FileSystemConfigurations.DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION) private boolean createRemoteFileSystemDuringInitialization; - @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION, + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION, DefaultValue = FileSystemConfigurations.DEFAULT_AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION) private boolean skipUserGroupMetadataDuringInitialization; - @IntegerConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH, + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_READ_AHEAD_QUEUE_DEPTH, DefaultValue = FileSystemConfigurations.DEFAULT_READ_AHEAD_QUEUE_DEPTH) private int readAheadQueueDepth; - @BooleanConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_ENABLE_FLUSH, + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ENABLE_FLUSH, DefaultValue = FileSystemConfigurations.DEFAULT_ENABLE_FLUSH) private boolean enableFlush; - @StringConfigurationValidatorAnnotation(ConfigurationKey = ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, + @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_USER_AGENT_PREFIX_KEY, DefaultValue = "") private String userAgentId; @@ -140,7 +150,7 @@ public class AbfsConfiguration{ public AbfsConfiguration(final Configuration configuration) throws IllegalAccessException, InvalidConfigurationValueException { this.configuration = configuration; - this.isSecure = this.configuration.getBoolean(ConfigurationKeys.FS_AZURE_SECURE_MODE, false); + this.isSecure = this.configuration.getBoolean(FS_AZURE_SECURE_MODE, false); validateStorageAccountKeys(); Field[] fields = this.getClass().getDeclaredFields(); @@ -161,17 +171,17 @@ public class AbfsConfiguration{ } public boolean isEmulator() { - return this.getConfiguration().getBoolean(ConfigurationKeys.FS_AZURE_EMULATOR_ENABLED, false); + return this.getConfiguration().getBoolean(FS_AZURE_EMULATOR_ENABLED, false); } public boolean isSecureMode() { - return this.isSecure; + return isSecure; } public String getStorageAccountKey(final String accountName) throws AzureBlobFileSystemException { String key; String keyProviderClass = - configuration.get(ConfigurationKeys.AZURE_KEY_ACCOUNT_KEYPROVIDER_PREFIX + accountName); + configuration.get(AZURE_KEY_ACCOUNT_KEYPROVIDER_PREFIX + accountName); KeyProvider keyProvider; if (keyProviderClass == null) { @@ -278,19 +288,88 @@ public class AbfsConfiguration{ return configuration.getEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, DEFAULT_FS_AZURE_SSL_CHANNEL_MODE); } + public AuthType getAuthType(final String accountName) { + return configuration.getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME + accountName, AuthType.SharedKey); + } + + public AccessTokenProvider getTokenProvider(final String accountName) throws TokenAccessProviderException { + AuthType authType = configuration.getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME + accountName, AuthType.SharedKey); + if (authType == AuthType.OAuth) { + try { + Class tokenProviderClass = + configuration.getClass(FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME + accountName, null, + AccessTokenProvider.class); + AccessTokenProvider tokenProvider = null; + if (tokenProviderClass == ClientCredsTokenProvider.class) { + String authEndpoint = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT + accountName); + String clientId = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID + accountName); + String clientSecret = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET + accountName); + tokenProvider = new ClientCredsTokenProvider(authEndpoint, clientId, clientSecret); + } else if (tokenProviderClass == UserPasswordTokenProvider.class) { + String authEndpoint = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT + accountName); + String username = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_USER_NAME + accountName); + String password = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_USER_PASSWORD + accountName); + tokenProvider = new UserPasswordTokenProvider(authEndpoint, username, password); + } else if (tokenProviderClass == MsiTokenProvider.class) { + String tenantGuid = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT + accountName); + String clientId = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID + accountName); + tokenProvider = new MsiTokenProvider(tenantGuid, clientId); + } else if (tokenProviderClass == RefreshTokenBasedTokenProvider.class) { + String refreshToken = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN + accountName); + String clientId = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID + accountName); + tokenProvider = new RefreshTokenBasedTokenProvider(clientId, refreshToken); + } else { + throw new IllegalArgumentException("Failed to initialize " + tokenProviderClass); + } + return tokenProvider; + } catch(IllegalArgumentException e) { + throw e; + } catch (Exception e) { + throw new TokenAccessProviderException("Unable to load key provider class.", e); + } + + } else if (authType == AuthType.Custom) { + try { + String configKey = FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME + accountName; + Class customTokenProviderClass = + configuration.getClass(configKey, null, + CustomTokenProviderAdaptee.class); + if (customTokenProviderClass == null) { + throw new IllegalArgumentException( + String.format("The configuration value for \"%s\" is invalid.", configKey)); + } + CustomTokenProviderAdaptee azureTokenProvider = ReflectionUtils + .newInstance(customTokenProviderClass, configuration); + if (azureTokenProvider == null) { + throw new IllegalArgumentException("Failed to initialize " + customTokenProviderClass); + } + azureTokenProvider.initialize(configuration, accountName); + return new CustomTokenProviderAdapter(azureTokenProvider); + } catch(IllegalArgumentException e) { + throw e; + } catch (Exception e) { + throw new TokenAccessProviderException("Unable to load custom token provider class.", e); + } + + } else { + throw new TokenAccessProviderException(String.format( + "Invalid auth type: %s is being used, expecting OAuth", authType)); + } + } + void validateStorageAccountKeys() throws InvalidConfigurationValueException { Base64StringConfigurationBasicValidator validator = new Base64StringConfigurationBasicValidator( - ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME, "", true); - this.storageAccountKeys = this.configuration.getValByRegex(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX); + FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME, "", true); + this.storageAccountKeys = configuration.getValByRegex(FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX); - for (Map.Entry account : this.storageAccountKeys.entrySet()) { + for (Map.Entry account : storageAccountKeys.entrySet()) { validator.validate(account.getValue()); } } int validateInt(Field field) throws IllegalAccessException, InvalidConfigurationValueException { IntegerConfigurationValidatorAnnotation validator = field.getAnnotation(IntegerConfigurationValidatorAnnotation.class); - String value = this.configuration.get(validator.ConfigurationKey()); + String value = configuration.get(validator.ConfigurationKey()); // validate return new IntegerConfigurationBasicValidator( @@ -303,7 +382,7 @@ public class AbfsConfiguration{ long validateLong(Field field) throws IllegalAccessException, InvalidConfigurationValueException { LongConfigurationValidatorAnnotation validator = field.getAnnotation(LongConfigurationValidatorAnnotation.class); - String value = this.configuration.get(validator.ConfigurationKey()); + String value = configuration.get(validator.ConfigurationKey()); // validate return new LongConfigurationBasicValidator( @@ -316,7 +395,7 @@ public class AbfsConfiguration{ String validateString(Field field) throws IllegalAccessException, InvalidConfigurationValueException { StringConfigurationValidatorAnnotation validator = field.getAnnotation(StringConfigurationValidatorAnnotation.class); - String value = this.configuration.get(validator.ConfigurationKey()); + String value = configuration.get(validator.ConfigurationKey()); // validate return new StringConfigurationBasicValidator( @@ -327,7 +406,7 @@ public class AbfsConfiguration{ String validateBase64String(Field field) throws IllegalAccessException, InvalidConfigurationValueException { Base64StringConfigurationValidatorAnnotation validator = field.getAnnotation((Base64StringConfigurationValidatorAnnotation.class)); - String value = this.configuration.get(validator.ConfigurationKey()); + String value = configuration.get(validator.ConfigurationKey()); // validate return new Base64StringConfigurationBasicValidator( @@ -338,7 +417,7 @@ public class AbfsConfiguration{ boolean validateBoolean(Field field) throws IllegalAccessException, InvalidConfigurationValueException { BooleanConfigurationValidatorAnnotation validator = field.getAnnotation(BooleanConfigurationValidatorAnnotation.class); - String value = this.configuration.get(validator.ConfigurationKey()); + String value = configuration.get(validator.ConfigurationKey()); // validate return new BooleanConfigurationBasicValidator( @@ -347,6 +426,14 @@ public class AbfsConfiguration{ validator.ThrowIfInvalid()).validate(value); } + String getPasswordString(String key) throws IOException { + char[] passchars = configuration.getPassword(key); + if (passchars != null) { + return new String(passchars); + } + return null; + } + @VisibleForTesting void setReadBufferSize(int bufferSize) { this.readBufferSize = bufferSize; 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 ba721497d74..b8da35b0b10 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 @@ -67,10 +67,12 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TimeoutException; import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema; import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; +import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; +import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy; import org.apache.hadoop.fs.azurebfs.services.SharedKeyCredentials; import org.apache.hadoop.fs.permission.FsAction; @@ -487,16 +489,22 @@ public class AzureBlobFileSystemStore { throw new InvalidUriException(uri.toString()); } - int dotIndex = accountName.indexOf(AbfsHttpConstants.DOT); - if (dotIndex <= 0) { - throw new InvalidUriException( - uri.toString() + " - account name is not fully qualified."); - } - SharedKeyCredentials creds = - new SharedKeyCredentials(accountName.substring(0, dotIndex), - this.abfsConfiguration.getStorageAccountKey(accountName)); + SharedKeyCredentials creds = null; + AccessTokenProvider tokenProvider = null; - this.client = new AbfsClient(baseUrl, creds, abfsConfiguration, new ExponentialRetryPolicy()); + if (abfsConfiguration.getAuthType(accountName) == AuthType.SharedKey) { + int dotIndex = accountName.indexOf(AbfsHttpConstants.DOT); + if (dotIndex <= 0) { + throw new InvalidUriException( + uri.toString() + " - account name is not fully qualified."); + } + creds = new SharedKeyCredentials(accountName.substring(0, dotIndex), + abfsConfiguration.getStorageAccountKey(accountName)); + } else { + tokenProvider = abfsConfiguration.getTokenProvider(accountName); + } + + this.client = new AbfsClient(baseUrl, creds, abfsConfiguration, new ExponentialRetryPolicy(), tokenProvider); } private String getRelativePath(final Path path) { @@ -537,7 +545,7 @@ public class AzureBlobFileSystemStore { Date utcDate = new SimpleDateFormat(DATE_TIME_PATTERN).parse(lastModifiedTime); parsedTime = utcDate.getTime(); } catch (ParseException e) { - LOG.error("Failed to parse the date {0}", lastModifiedTime); + LOG.error("Failed to parse the date {}", lastModifiedTime); } finally { return parsedTime; } 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 16ddd900edd..ffdf700b78f 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 @@ -60,5 +60,24 @@ public final class ConfigurationKeys { public static final String AZURE_KEY_ACCOUNT_KEYPROVIDER_PREFIX = "fs.azure.account.keyprovider."; public static final String AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT = "fs.azure.shellkeyprovider.script"; + /** Prefix for auth type properties: {@value}. */ + public static final String FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME = "fs.azure.account.auth.type."; + /** Prefix for oauth token provider type: {@value}. */ + public static final String FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME = "fs.azure.account.oauth.provider.type."; + /** Prefix for oauth AAD client id: {@value}. */ + public static final String FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID = "fs.azure.account.oauth2.client.id."; + /** Prefix for oauth AAD client secret: {@value}. */ + public static final String FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET = "fs.azure.account.oauth2.client.secret."; + /** Prefix for oauth AAD client endpoint: {@value}. */ + public static final String FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT = "fs.azure.account.oauth2.client.endpoint."; + /** Prefix for oauth msi tenant id: {@value}. */ + public static final String FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT = "fs.azure.account.oauth2.msi.tenant."; + /** Prefix for oauth user name: {@value}. */ + public static final String FS_AZURE_ACCOUNT_OAUTH_USER_NAME = "fs.azure.account.oauth2.user.name."; + /** Prefix for oauth user password: {@value}. */ + public static final String FS_AZURE_ACCOUNT_OAUTH_USER_PASSWORD = "fs.azure.account.oauth2.user.password."; + /** Prefix for oauth refresh token: {@value}. */ + public static final String FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN = "fs.azure.account.oauth2.refresh.token."; + private ConfigurationKeys() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/TokenAccessProviderException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/TokenAccessProviderException.java new file mode 100644 index 00000000000..b40b34ac13e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/TokenAccessProviderException.java @@ -0,0 +1,36 @@ +/** + * 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.contracts.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; + +/** + * Thrown if there is a problem instantiating a TokenAccessProvider or retrieving a configuration + * using a TokenAccessProvider object. + */ +@InterfaceAudience.Private +public class TokenAccessProviderException extends AzureBlobFileSystemException { + + public TokenAccessProviderException(String message) { + super(message); + } + + public TokenAccessProviderException(String message, Throwable cause) { + super(message); + } +} \ No newline at end of file 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 a89f339967c..63bf8d03fda 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 @@ -44,6 +44,7 @@ public enum AzureServiceErrorCode { INGRESS_OVER_ACCOUNT_LIMIT(null, HttpURLConnection.HTTP_UNAVAILABLE, "Ingress is over the account limit."), EGRESS_OVER_ACCOUNT_LIMIT(null, HttpURLConnection.HTTP_UNAVAILABLE, "Egress is over the account limit."), INVALID_QUERY_PARAMETER_VALUE("InvalidQueryParameterValue", HttpURLConnection.HTTP_BAD_REQUEST, null), + AUTHORIZATION_PERMISSION_MISS_MATCH("AuthorizationPermissionMismatch", HttpURLConnection.HTTP_FORBIDDEN, null), UNKNOWN(null, -1, null); private final String errorCode; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java index 903ff69e9e3..1de9dfaeeb9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java @@ -57,13 +57,31 @@ public class ListResultEntrySchema { @JsonProperty(value = "contentLength") private Long contentLength; + /** + * The owner property. + */ + @JsonProperty(value = "owner") + private String owner; + + /** + * The group property. + */ + @JsonProperty(value = "group") + private String group; + + /** + * The permissions property. + */ + @JsonProperty(value = "permissions") + private String permissions; + /** * Get the name value. * * @return the name value */ public String name() { - return this.name; + return name; } /** @@ -83,7 +101,7 @@ public class ListResultEntrySchema { * @return the isDirectory value */ public Boolean isDirectory() { - return this.isDirectory; + return isDirectory; } /** @@ -103,7 +121,7 @@ public class ListResultEntrySchema { * @return the lastModified value */ public String lastModified() { - return this.lastModified; + return lastModified; } /** @@ -123,7 +141,7 @@ public class ListResultEntrySchema { * @return the etag value */ public String eTag() { - return this.eTag; + return eTag; } /** @@ -143,7 +161,7 @@ public class ListResultEntrySchema { * @return the contentLength value */ public Long contentLength() { - return this.contentLength; + return contentLength; } /** @@ -157,4 +175,65 @@ public class ListResultEntrySchema { return this; } + /** + * + Get the owner value. + * + * @return the owner value + */ + public String owner() { + return owner; + } + + /** + * Set the owner value. + * + * @param owner the owner value to set + * @return the ListEntrySchema object itself. + */ + public ListResultEntrySchema withOwner(final String owner) { + this.owner = owner; + return this; + } + + /** + * Get the group value. + * + * @return the group value + */ + public String group() { + return group; + } + + /** + * Set the group value. + * + * @param group the group value to set + * @return the ListEntrySchema object itself. + */ + public ListResultEntrySchema withGroup(final String group) { + this.group = group; + return this; + } + + /** + * Get the permissions value. + * + * @return the permissions value + */ + public String permissions() { + return permissions; + } + + /** + * Set the permissions value. + * + * @param permissions the permissions value to set + * @return the ListEntrySchema object itself. + */ + public ListResultEntrySchema withPermissions(final String permissions) { + this.permissions = permissions; + return this; + } + } \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AccessTokenProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AccessTokenProvider.java new file mode 100644 index 00000000000..72f37a1dc1a --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AccessTokenProvider.java @@ -0,0 +1,98 @@ +/** + * 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.oauth2; + +import java.io.IOException; +import java.util.Date; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Returns an Azure Active Directory token when requested. The provider can + * cache the token if it has already retrieved one. If it does, then the + * provider is responsible for checking expiry and refreshing as needed. + * + * In other words, this is is a token cache that fetches tokens when + * requested, if the cached token has expired. + * + */ +public abstract class AccessTokenProvider { + + private AzureADToken token; + private static final Logger LOG = LoggerFactory.getLogger(AccessTokenProvider.class); + + /** + * returns the {@link AzureADToken} cached (or retrieved) by this instance. + * + * @return {@link AzureADToken} containing the access token + * @throws IOException if there is an error fetching the token + */ + public synchronized AzureADToken getToken() throws IOException { + if (isTokenAboutToExpire()) { + LOG.debug("AAD Token is missing or expired:" + + " Calling refresh-token from abstract base class"); + token = refreshToken(); + } + return token; + } + + /** + * the method to fetch the access token. Derived classes should override + * this method to actually get the token from Azure Active Directory. + * + * This method will be called initially, and then once when the token + * is about to expire. + * + * + * @return {@link AzureADToken} containing the access token + * @throws IOException if there is an error fetching the token + */ + protected abstract AzureADToken refreshToken() throws IOException; + + /** + * Checks if the token is about to expire in the next 5 minutes. + * The 5 minute allowance is to allow for clock skew and also to + * allow for token to be refreshed in that much time. + * + * @return true if the token is expiring in next 5 minutes + */ + private boolean isTokenAboutToExpire() { + if (token == null) { + LOG.debug("AADToken: no token. Returning expiring=true"); + return true; // no token should have same response as expired token + } + boolean expiring = false; + // allow 5 minutes for clock skew + long approximatelyNow = System.currentTimeMillis() + FIVE_MINUTES; + if (token.getExpiry().getTime() < approximatelyNow) { + expiring = true; + } + if (expiring) { + LOG.debug("AADToken: token expiring: " + + token.getExpiry().toString() + + " : Five-minute window: " + + new Date(approximatelyNow).toString()); + } + + return expiring; + } + + // 5 minutes in milliseconds + private static final long FIVE_MINUTES = 300 * 1000; +} 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 new file mode 100644 index 00000000000..e82dc954f59 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java @@ -0,0 +1,344 @@ +/** + * 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.oauth2; + +import java.io.IOException; +import java.io.InputStream; +import java.net.HttpURLConnection; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.util.Date; +import java.util.Hashtable; +import java.util.Map; + +import com.google.common.base.Preconditions; +import org.codehaus.jackson.JsonFactory; +import org.codehaus.jackson.JsonParser; +import org.codehaus.jackson.JsonToken; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy; + +/** + * This class provides convenience methods to obtain AAD tokens. + * While convenient, it is not necessary to use these methods to + * obtain the tokens. Customers can use any other method + * (e.g., using the adal4j client) to obtain tokens. + */ + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public final class AzureADAuthenticator { + + private static final Logger LOG = LoggerFactory.getLogger(AzureADAuthenticator.class); + private static final String RESOURCE_NAME = "https://storage.azure.com/"; + private static final int CONNECT_TIMEOUT = 30 * 1000; + private static final int READ_TIMEOUT = 30 * 1000; + + private AzureADAuthenticator() { + // no operation + } + + /** + * gets Azure Active Directory token using the user ID and password of + * a service principal (that is, Web App in Azure Active Directory). + * + * Azure Active Directory allows users to set up a web app as a + * service principal. Users can optionally obtain service principal keys + * from AAD. This method gets a token using a service principal's client ID + * and keys. In addition, it needs the token endpoint associated with the + * user's directory. + * + * + * @param authEndpoint the OAuth 2.0 token endpoint associated + * with the user's directory (obtain from + * Active Directory configuration) + * @param clientId the client ID (GUID) of the client web app + * btained from Azure Active Directory configuration + * @param clientSecret the secret key of the client web app + * @return {@link AzureADToken} obtained using the creds + * @throws IOException throws IOException if there is a failure in connecting to Azure AD + */ + public static AzureADToken getTokenUsingClientCreds(String authEndpoint, + String clientId, String clientSecret) + throws IOException { + Preconditions.checkNotNull(authEndpoint, "authEndpoint"); + Preconditions.checkNotNull(clientId, "clientId"); + Preconditions.checkNotNull(clientSecret, "clientSecret"); + + QueryParams qp = new QueryParams(); + qp.add("resource", RESOURCE_NAME); + qp.add("grant_type", "client_credentials"); + qp.add("client_id", clientId); + qp.add("client_secret", clientSecret); + LOG.debug("AADToken: starting to fetch token using client creds for client ID " + clientId); + + return getTokenCall(authEndpoint, qp.serialize(), null, null); + } + + /** + * Gets AAD token from the local virtual machine's VM extension. This only works on + * an Azure VM with MSI extension + * enabled. + * + * @param tenantGuid (optional) The guid of the AAD tenant. Can be {@code null}. + * @param clientId (optional) The clientId guid of the MSI service + * principal to use. Can be {@code null}. + * @param bypassCache {@code boolean} specifying whether a cached token is acceptable or a fresh token + * request should me made to AAD + * @return {@link AzureADToken} obtained using the creds + * @throws IOException throws IOException if there is a failure in obtaining the token + */ + public static AzureADToken getTokenFromMsi(String tenantGuid, String clientId, + boolean bypassCache) throws IOException { + Preconditions.checkNotNull(tenantGuid, "tenantGuid"); + Preconditions.checkNotNull(clientId, "clientId"); + + String authEndpoint = "http://169.254.169.254/metadata/identity/oauth2/token"; + + QueryParams qp = new QueryParams(); + qp.add("api-version", "2018-02-01"); + qp.add("resource", RESOURCE_NAME); + + + if (tenantGuid.length() > 0) { + String authority = "https://login.microsoftonline.com/" + tenantGuid; + qp.add("authority", authority); + } + + if (clientId.length() > 0) { + qp.add("client_id", clientId); + } + + if (bypassCache) { + qp.add("bypass_cache", "true"); + } + + Hashtable headers = new Hashtable<>(); + headers.put("Metadata", "true"); + + LOG.debug("AADToken: starting to fetch token using MSI"); + return getTokenCall(authEndpoint, qp.serialize(), headers, "GET"); + } + + /** + * Gets Azure Active Directory token using refresh token. + * + * @param clientId the client ID (GUID) of the client web app obtained from Azure Active Directory configuration + * @param refreshToken the refresh token + * @return {@link AzureADToken} obtained using the refresh token + * @throws IOException throws IOException if there is a failure in connecting to Azure AD + */ + public static AzureADToken getTokenUsingRefreshToken(String clientId, + String refreshToken) throws IOException { + String authEndpoint = "https://login.microsoftonline.com/Common/oauth2/token"; + QueryParams qp = new QueryParams(); + qp.add("grant_type", "refresh_token"); + qp.add("refresh_token", refreshToken); + if (clientId != null) { + qp.add("client_id", clientId); + } + LOG.debug("AADToken: starting to fetch token using refresh token for client ID " + clientId); + return getTokenCall(authEndpoint, qp.serialize(), null, null); + } + + private static class HttpException extends IOException { + private int httpErrorCode; + private String requestId; + + public int getHttpErrorCode() { + return this.httpErrorCode; + } + + public String getRequestId() { + return this.requestId; + } + + HttpException(int httpErrorCode, String requestId, String message) { + super(message); + this.httpErrorCode = httpErrorCode; + this.requestId = requestId; + } + } + + private static AzureADToken getTokenCall(String authEndpoint, String body, + Hashtable headers, String httpMethod) + throws IOException { + AzureADToken token = null; + ExponentialRetryPolicy retryPolicy + = new ExponentialRetryPolicy(3, 0, 1000, 2); + + int httperror = 0; + String requestId; + String httpExceptionMessage = null; + IOException ex = null; + boolean succeeded = false; + int retryCount = 0; + do { + httperror = 0; + requestId = ""; + ex = null; + try { + token = getTokenSingleCall(authEndpoint, body, headers, httpMethod); + } catch (HttpException e) { + httperror = e.httpErrorCode; + requestId = e.requestId; + httpExceptionMessage = e.getMessage(); + } catch (IOException e) { + ex = e; + } + succeeded = ((httperror == 0) && (ex == null)); + retryCount++; + } while (!succeeded && retryPolicy.shouldRetry(retryCount, httperror)); + if (!succeeded) { + if (ex != null) { + throw ex; + } + if (httperror != 0) { + throw new IOException(httpExceptionMessage); + } + } + return token; + } + + private static AzureADToken getTokenSingleCall( + String authEndpoint, String payload, Hashtable headers, String httpMethod) + throws IOException { + + AzureADToken token = null; + HttpURLConnection conn = null; + String urlString = authEndpoint; + + httpMethod = (httpMethod == null) ? "POST" : httpMethod; + if (httpMethod.equals("GET")) { + urlString = urlString + "?" + payload; + } + + try { + URL url = new URL(urlString); + conn = (HttpURLConnection) url.openConnection(); + conn.setRequestMethod(httpMethod); + conn.setReadTimeout(READ_TIMEOUT); + conn.setConnectTimeout(CONNECT_TIMEOUT); + + if (headers != null && headers.size() > 0) { + for (Map.Entry entry : headers.entrySet()) { + conn.setRequestProperty(entry.getKey(), entry.getValue()); + } + } + conn.setRequestProperty("Connection", "close"); + + if (httpMethod.equals("POST")) { + conn.setDoOutput(true); + conn.getOutputStream().write(payload.getBytes("UTF-8")); + } + + int httpResponseCode = conn.getResponseCode(); + String requestId = conn.getHeaderField("x-ms-request-id"); + String responseContentType = conn.getHeaderField("Content-Type"); + long responseContentLength = conn.getHeaderFieldLong("Content-Length", 0); + + requestId = requestId == null ? "" : requestId; + if (httpResponseCode == HttpURLConnection.HTTP_OK + && responseContentType.startsWith("application/json") && responseContentLength > 0) { + InputStream httpResponseStream = conn.getInputStream(); + token = parseTokenFromStream(httpResponseStream); + } else { + String responseBody = consumeInputStream(conn.getInputStream(), 1024); + String proxies = "none"; + String httpProxy = System.getProperty("http.proxy"); + String httpsProxy = System.getProperty("https.proxy"); + if (httpProxy != null || httpsProxy != null) { + proxies = "http:" + httpProxy + "; https:" + httpsProxy; + } + String logMessage = + "AADToken: HTTP connection failed for getting token from AzureAD. Http response: " + + httpResponseCode + " " + conn.getResponseMessage() + + " Content-Type: " + responseContentType + + " Content-Length: " + responseContentLength + + " Request ID: " + requestId.toString() + + " Proxies: " + proxies + + " First 1K of Body: " + responseBody; + LOG.debug(logMessage); + throw new HttpException(httpResponseCode, requestId, logMessage); + } + } finally { + if (conn != null) { + conn.disconnect(); + } + } + return token; + } + + private static AzureADToken parseTokenFromStream(InputStream httpResponseStream) throws IOException { + AzureADToken token = new AzureADToken(); + try { + int expiryPeriod = 0; + + JsonFactory jf = new JsonFactory(); + JsonParser jp = jf.createJsonParser(httpResponseStream); + String fieldName, fieldValue; + jp.nextToken(); + while (jp.hasCurrentToken()) { + if (jp.getCurrentToken() == JsonToken.FIELD_NAME) { + fieldName = jp.getCurrentName(); + jp.nextToken(); // field value + fieldValue = jp.getText(); + + if (fieldName.equals("access_token")) { + token.setAccessToken(fieldValue); + } + if (fieldName.equals("expires_in")) { + expiryPeriod = Integer.parseInt(fieldValue); + } + } + jp.nextToken(); + } + jp.close(); + long expiry = System.currentTimeMillis(); + expiry = expiry + expiryPeriod * 1000L; // convert expiryPeriod to milliseconds and add + token.setExpiry(new Date(expiry)); + LOG.debug("AADToken: fetched token with expiry " + token.getExpiry().toString()); + } catch (Exception ex) { + LOG.debug("AADToken: got exception when parsing json token " + ex.toString()); + throw ex; + } finally { + httpResponseStream.close(); + } + return token; + } + + private static String consumeInputStream(InputStream inStream, int length) throws IOException { + byte[] b = new byte[length]; + int totalBytesRead = 0; + int bytesRead = 0; + + do { + bytesRead = inStream.read(b, totalBytesRead, length - totalBytesRead); + if (bytesRead > 0) { + totalBytesRead += bytesRead; + } + } while (bytesRead >= 0 && totalBytesRead < length); + + return new String(b, 0, totalBytesRead, StandardCharsets.UTF_8); + } +} 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 new file mode 100644 index 00000000000..daa5a93bf6c --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADToken.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.oauth2; + +import java.util.Date; + + +/** + * Object representing the AAD access token to use when making HTTP requests to Azure Data Lake Storage. + */ +public class AzureADToken { + private String accessToken; + private Date expiry; + + public String getAccessToken() { + return this.accessToken; + } + + public void setAccessToken(String accessToken) { + this.accessToken = accessToken; + } + + public Date getExpiry() { + return new Date(this.expiry.getTime()); + } + + public void setExpiry(Date expiry) { + this.expiry = new Date(expiry.getTime()); + } + +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/ClientCredsTokenProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/ClientCredsTokenProvider.java new file mode 100644 index 00000000000..9a46018ec62 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/ClientCredsTokenProvider.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.oauth2; + +import java.io.IOException; + +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Provides tokens based on client credentials. + */ +public class ClientCredsTokenProvider extends AccessTokenProvider { + + private final String authEndpoint; + + private final String clientId; + + private final String clientSecret; + + private static final Logger LOG = LoggerFactory.getLogger(AccessTokenProvider.class); + + + public ClientCredsTokenProvider(final String authEndpoint, + final String clientId, final String clientSecret) { + + Preconditions.checkNotNull(authEndpoint, "authEndpoint"); + Preconditions.checkNotNull(clientId, "clientId"); + Preconditions.checkNotNull(clientSecret, "clientSecret"); + + this.authEndpoint = authEndpoint; + this.clientId = clientId; + this.clientSecret = clientSecret; + } + + + @Override + protected AzureADToken refreshToken() throws IOException { + LOG.debug("AADToken: refreshing client-credential based token"); + return AzureADAuthenticator.getTokenUsingClientCreds(authEndpoint, clientId, clientSecret); + } + + +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdaptee.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdaptee.java new file mode 100644 index 00000000000..7366a8d7950 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdaptee.java @@ -0,0 +1,75 @@ +/** + * 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.oauth2; + +import java.io.IOException; +import java.util.Date; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; + + +/** + * This interface provides an extensibility model for customizing the acquisition + * of Azure Active Directory Access Tokens. When "fs.azure.account.auth.type" is + * set to "Custom", implementors may use the + * "fs.azure.account.oauth.provider.type.{accountName}" configuration property + * to specify a class with a custom implementation of CustomTokenProviderAdaptee. + * This class will be dynamically loaded, initialized, and invoked to provide + * AAD Access Tokens and their Expiry. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface CustomTokenProviderAdaptee { + + /** + * Initialize with supported configuration. This method is invoked when the + * (URI, Configuration)} method is invoked. + * + * @param configuration Configuration object + * @param accountName Account Name + * @throws IOException if instance can not be configured. + */ + void initialize(Configuration configuration, String accountName) + throws IOException; + + /** + * Obtain the access token that should be added to https connection's header. + * Will be called depending upon {@link #getExpiryTime()} expiry time is set, + * so implementations should be performant. Implementations are responsible + * for any refreshing of the token. + * + * @return String containing the access token + * @throws IOException if there is an error fetching the token + */ + String getAccessToken() throws IOException; + + /** + * Obtain expiry time of the token. If implementation is performant enough to + * maintain expiry and expect {@link #getAccessToken()} call for every + * connection then safe to return current or past time. + * + * However recommended to use the token expiry time received from Azure Active + * Directory. + * + * @return Date to expire access token retrieved from AAD. + */ + Date getExpiryTime(); +} \ No newline at end of file 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 new file mode 100644 index 00000000000..7bae415daf6 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdapter.java @@ -0,0 +1,57 @@ +/** + * 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.oauth2; + + +import java.io.IOException; + +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Provides tokens based on custom implementation, following the Adapter Design + * Pattern. + */ +public final class CustomTokenProviderAdapter extends AccessTokenProvider { + + private CustomTokenProviderAdaptee adaptee; + private static final Logger LOG = LoggerFactory.getLogger(AccessTokenProvider.class); + + /** + * Constructs a token provider based on the custom token provider. + * + * @param adaptee the custom token provider + */ + public CustomTokenProviderAdapter(CustomTokenProviderAdaptee adaptee) { + Preconditions.checkNotNull(adaptee, "adaptee"); + this.adaptee = adaptee; + } + + protected AzureADToken refreshToken() throws IOException { + LOG.debug("AADToken: refreshing custom based token"); + + AzureADToken azureADToken = new AzureADToken(); + azureADToken.setAccessToken(adaptee.getAccessToken()); + azureADToken.setExpiry(adaptee.getExpiryTime()); + + return azureADToken; + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/MsiTokenProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/MsiTokenProvider.java new file mode 100644 index 00000000000..2deb9d246d1 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/MsiTokenProvider.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.oauth2; + +import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Provides tokens based on Azure VM's Managed Service Identity. + */ +public class MsiTokenProvider extends AccessTokenProvider { + + private final String tenantGuid; + + private final String clientId; + + private static final Logger LOG = LoggerFactory.getLogger(AccessTokenProvider.class); + + public MsiTokenProvider(final String tenantGuid, final String clientId) { + this.tenantGuid = tenantGuid; + this.clientId = clientId; + } + + @Override + protected AzureADToken refreshToken() throws IOException { + LOG.debug("AADToken: refreshing token from MSI"); + AzureADToken token = AzureADAuthenticator.getTokenFromMsi(tenantGuid, clientId, false); + return token; + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/QueryParams.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/QueryParams.java new file mode 100644 index 00000000000..ff6e06f9501 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/QueryParams.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.oauth2; + +import java.io.UnsupportedEncodingException; +import java.net.URLEncoder; +import java.util.HashMap; +import java.util.Map; + +/** + * Utilities class http query parameters. + */ +public class QueryParams { + private Map params = new HashMap<>(); + private String apiVersion = null; + private String separator = ""; + private String serializedString = null; + + public void add(String name, String value) { + params.put(name, value); + serializedString = null; + } + + public void setApiVersion(String apiVersion) { + this.apiVersion = apiVersion; + serializedString = null; + } + + public String serialize() { + if (serializedString == null) { + StringBuilder sb = new StringBuilder(); + for (Map.Entry entry : params.entrySet()) { + String name = entry.getKey(); + try { + sb.append(separator); + sb.append(URLEncoder.encode(name, "UTF-8")); + sb.append('='); + sb.append(URLEncoder.encode(entry.getValue(), "UTF-8")); + separator = "&"; + } catch (UnsupportedEncodingException ex) { + } + } + + if (apiVersion != null) { + sb.append(separator); + sb.append("api-version="); + sb.append(apiVersion); + separator = "&"; + } + serializedString = sb.toString(); + } + return serializedString; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/RefreshTokenBasedTokenProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/RefreshTokenBasedTokenProvider.java new file mode 100644 index 00000000000..949d5bf1d80 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/RefreshTokenBasedTokenProvider.java @@ -0,0 +1,57 @@ +/** + * 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.oauth2; + +import java.io.IOException; + +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Provides tokens based on refresh token. + */ +public class RefreshTokenBasedTokenProvider extends AccessTokenProvider { + private static final Logger LOG = LoggerFactory.getLogger(AccessTokenProvider.class); + + private final String clientId; + + private final String refreshToken; + + /** + * Constructs a token provider based on the refresh token provided. + * + * @param clientId the client ID (GUID) of the client web app obtained from Azure Active Directory configuration + * @param refreshToken the refresh token + */ + public RefreshTokenBasedTokenProvider(String clientId, String refreshToken) { + Preconditions.checkNotNull(clientId, "clientId"); + Preconditions.checkNotNull(refreshToken, "refreshToken"); + this.clientId = clientId; + this.refreshToken = refreshToken; + } + + + @Override + protected AzureADToken refreshToken() throws IOException { + LOG.debug("AADToken: refreshing refresh-token based token"); + return AzureADAuthenticator.getTokenUsingRefreshToken(clientId, refreshToken); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/UserPasswordTokenProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/UserPasswordTokenProvider.java new file mode 100644 index 00000000000..7504e9d527e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/UserPasswordTokenProvider.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.oauth2; + +import java.io.IOException; + +import com.google.common.base.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +/** + * Provides tokens based on username and password. + */ +public class UserPasswordTokenProvider extends AccessTokenProvider { + + private final String authEndpoint; + + private final String username; + + private final String password; + + private static final Logger LOG = LoggerFactory.getLogger(AccessTokenProvider.class); + + public UserPasswordTokenProvider(final String authEndpoint, + final String username, final String password) { + Preconditions.checkNotNull(authEndpoint, "authEndpoint"); + Preconditions.checkNotNull(username, "username"); + Preconditions.checkNotNull(password, "password"); + + this.authEndpoint = authEndpoint; + this.username = username; + this.password = password; + } + + @Override + protected AzureADToken refreshToken() throws IOException { + LOG.debug("AADToken: refreshing user-password based token"); + return AzureADAuthenticator.getTokenUsingClientCreds(authEndpoint, username, password); + } + + private static String getPasswordString(Configuration conf, String key) + throws IOException { + char[] passchars = conf.getPassword(key); + if (passchars == null) { + throw new IOException("Password " + key + " not found"); + } + return new String(passchars); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/package-info.java new file mode 100644 index 00000000000..bad1a85b31d --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/package-info.java @@ -0,0 +1,18 @@ +/** + * 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.oauth2; \ No newline at end of file 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 e003ffd31d9..f5c9f18fc61 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -35,6 +35,7 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME; @@ -42,7 +43,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.* import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.*; /** - * AbfsClient + * AbfsClient. */ public class AbfsClient { public static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); @@ -54,9 +55,13 @@ public class AbfsClient { private final AbfsConfiguration abfsConfiguration; private final String userAgent; + private final AccessTokenProvider tokenProvider; + + public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, final AbfsConfiguration abfsConfiguration, - final ExponentialRetryPolicy exponentialRetryPolicy) { + final ExponentialRetryPolicy exponentialRetryPolicy, + final AccessTokenProvider tokenProvider) { this.baseUrl = baseUrl; this.sharedKeyCredentials = sharedKeyCredentials; String baseUrlString = baseUrl.toString(); @@ -76,6 +81,7 @@ public class AbfsClient { } this.userAgent = initializeUserAgent(abfsConfiguration, sslProviderName); + this.tokenProvider = tokenProvider; } public String getFileSystem() { @@ -409,6 +415,14 @@ public class AbfsClient { return encodedString; } + public synchronized String getAccessToken() throws IOException { + if (tokenProvider != null) { + return "Bearer " + tokenProvider.getToken().getAccessToken(); + } else { + return null; + } + } + @VisibleForTesting String initializeUserAgent(final AbfsConfiguration abfsConfiguration, final String sslProviderName) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java index 46b4c6d8442..0067b755460 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java @@ -19,7 +19,7 @@ package org.apache.hadoop.fs.azurebfs.services; /** - * The Http Request / Response Headers for Rest AbfsClient + * The Http Request / Response Headers for Rest AbfsClient. */ public class AbfsHttpHeader { private final String name; 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 6dd32fafb7a..c0407f58d29 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 @@ -30,6 +30,7 @@ import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; /** * The AbfsRestOperation for Rest AbfsClient. @@ -48,7 +49,7 @@ public class AbfsRestOperation { // request body and all the download methods have a response body. private final boolean hasRequestBody; - private final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); + private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); // For uploads, this is the request entity body. For downloads, // this will hold the response entity body. @@ -139,9 +140,15 @@ public class AbfsRestOperation { httpOperation = new AbfsHttpOperation(url, method, requestHeaders); // sign the HTTP request - client.getSharedKeyCredentials().signRequest( - httpOperation.getConnection(), - hasRequestBody ? bufferLength : 0); + if (client.getAccessToken() == null) { + // sign the HTTP request + client.getSharedKeyCredentials().signRequest( + httpOperation.getConnection(), + hasRequestBody ? bufferLength : 0); + } else { + httpOperation.getConnection().setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION, + client.getAccessToken()); + } if (hasRequestBody) { // HttpUrlConnection requires @@ -163,9 +170,7 @@ public class AbfsRestOperation { return false; } - if (LOG.isDebugEnabled()) { - LOG.debug("HttpRequest: " + httpOperation.toString()); - } + LOG.debug("HttpRequest: " + httpOperation.toString()); if (client.getRetryPolicy().shouldRetry(retryCount, httpOperation.getStatusCode())) { return false; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AuthType.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AuthType.java new file mode 100644 index 00000000000..c95b92cbe61 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AuthType.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. + */ +package org.apache.hadoop.fs.azurebfs.services; + +/** + * Auth Type Enum. + */ +public enum AuthType { + SharedKey, + OAuth, + Custom +} 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 b1f14856cf1..e0afeb4e234 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 @@ -34,6 +34,7 @@ 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.services.AuthType; import org.apache.hadoop.fs.azure.AbstractWasbTestWithTimeout; import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore; import org.apache.hadoop.fs.azure.NativeAzureFileSystem; @@ -62,7 +63,7 @@ public abstract class AbstractAbfsIntegrationTest extends private static final Logger LOG = LoggerFactory.getLogger(AbstractAbfsIntegrationTest.class); - private final boolean isEmulator; + private boolean isEmulator; private NativeAzureFileSystem wasb; private AzureBlobFileSystem abfs; private String abfsScheme; @@ -71,20 +72,18 @@ public abstract class AbstractAbfsIntegrationTest extends private String fileSystemName; private String accountName; private String testUrl; - - protected AbstractAbfsIntegrationTest(final boolean secure) { - this(secure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME); - } + private AuthType authType; protected AbstractAbfsIntegrationTest() { - this(FileSystemUriSchemes.ABFS_SCHEME); - } - - private AbstractAbfsIntegrationTest(final String scheme) { - abfsScheme = scheme; fileSystemName = ABFS_TEST_CONTAINER_PREFIX + UUID.randomUUID().toString(); configuration = new Configuration(); configuration.addResource(ABFS_TEST_RESOURCE_XML); + this.accountName = this.configuration.get(FS_AZURE_TEST_ACCOUNT_NAME); + + authType = configuration.getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME + + accountName, AuthType.SharedKey); + abfsScheme = authType == AuthType.SharedKey ? FileSystemUriSchemes.ABFS_SCHEME + : FileSystemUriSchemes.ABFS_SECURE_SCHEME; String accountName = configuration.get(FS_AZURE_TEST_ACCOUNT_NAME, ""); assumeTrue("Not set: " + FS_AZURE_TEST_ACCOUNT_NAME, @@ -94,8 +93,13 @@ public abstract class AbstractAbfsIntegrationTest extends accountName, containsString("dfs.core.windows.net")); String fullKey = FS_AZURE_TEST_ACCOUNT_KEY_PREFIX + accountName; - assumeTrue("Not set: " + fullKey, - configuration.get(fullKey) != null); + + if (authType == AuthType.SharedKey) { + assumeTrue("Not set: " + fullKey, configuration.get(fullKey) != null); + } else { + String accessTokenProviderKey = FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME + accountName; + assumeTrue("Not set: " + accessTokenProviderKey, configuration.get(accessTokenProviderKey) != null); + } final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName(); URI defaultUri = null; @@ -110,7 +114,6 @@ public abstract class AbstractAbfsIntegrationTest extends configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); configuration.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true); this.isEmulator = this.configuration.getBoolean(FS_AZURE_EMULATOR_ENABLED, false); - this.accountName = this.configuration.get(FS_AZURE_TEST_ACCOUNT_NAME); } @@ -119,7 +122,7 @@ public abstract class AbstractAbfsIntegrationTest extends //Create filesystem first to make sure getWasbFileSystem() can return an existing filesystem. createFileSystem(); - if (!isEmulator) { + if (!isEmulator && authType == AuthType.SharedKey) { final URI wasbUri = new URI(abfsUrlToWasbUrl(getTestUrl())); final AzureNativeFileSystemStore azureNativeFileSystemStore = new AzureNativeFileSystemStore(); @@ -234,6 +237,10 @@ public abstract class AbstractAbfsIntegrationTest extends return isEmulator; } + protected AuthType getAuthType() { + return this.authType; + } + /** * Write a buffer to a file. * @param path path diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java index 9c369bb2bf4..1c2083d5038 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java @@ -19,7 +19,6 @@ package org.apache.hadoop.fs.azurebfs; 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.AbfsRestOperation; import org.junit.Assert; @@ -30,6 +29,7 @@ import org.junit.Test; */ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest { private static final int LIST_MAX_RESULTS = 5000; + @Test public void testContinuationTokenHavingEqualSign() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java index d6964814cc6..6207a47a452 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java @@ -22,8 +22,11 @@ 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.blob.CloudBlockBlob; + +import org.junit.Assume; import org.junit.Test; +import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -34,6 +37,7 @@ public class ITestAzureBlobFileSystemBackCompat extends AbstractAbfsIntegrationTest { public ITestAzureBlobFileSystemBackCompat() { super(); + Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); } @Test 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 791694bf0f1..13abaf88bd9 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 @@ -34,9 +34,6 @@ public class ITestAzureBlobFileSystemFileStatus extends AbstractAbfsIntegrationTest { private static final Path TEST_FILE = new Path("testFile"); private static final Path TEST_FOLDER = new Path("testDir"); - public ITestAzureBlobFileSystemFileStatus() { - super(); - } @Test public void testEnsureStatusWorksForRoot() throws Exception { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java index e4acbaefc61..c1022b01978 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java @@ -25,12 +25,14 @@ import org.junit.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.azurebfs.services.AuthType; /** * Test finalize() method when "fs.abfs.impl.disable.cache" is enabled. */ public class ITestAzureBlobFileSystemFinalize extends AbstractAbfsScaleTest{ - static final String DISABLE_CACHE_KEY = "fs.abfs.impl.disable.cache"; + static final String DISABLE_ABFS_CACHE_KEY = "fs.abfs.impl.disable.cache"; + static final String DISABLE_ABFSSS_CACHE_KEY = "fs.abfss.impl.disable.cache"; public ITestAzureBlobFileSystemFinalize() throws Exception { super(); @@ -40,7 +42,9 @@ public class ITestAzureBlobFileSystemFinalize extends AbstractAbfsScaleTest{ public void testFinalize() throws Exception { // Disable the cache for filesystem to make sure there is no reference. Configuration configuration = this.getConfiguration(); - configuration.setBoolean(this.DISABLE_CACHE_KEY, true); + configuration.setBoolean( + this.getAuthType() == AuthType.SharedKey ? DISABLE_ABFS_CACHE_KEY : DISABLE_ABFSSS_CACHE_KEY, + true); AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(configuration); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index 2f40b6444fd..b02d723c9ca 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -31,9 +31,9 @@ import java.io.IOException; import com.microsoft.azure.storage.blob.BlockEntry; import com.microsoft.azure.storage.blob.BlockListingFilter; import com.microsoft.azure.storage.blob.CloudBlockBlob; -import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount; import org.hamcrest.core.IsEqual; import org.hamcrest.core.IsNot; +import org.junit.Assume; import org.junit.Test; import org.apache.hadoop.fs.FSDataInputStream; @@ -42,6 +42,9 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount; +import org.apache.hadoop.fs.azurebfs.services.AuthType; + /** * Test flush operation. */ @@ -209,6 +212,8 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { @Test public void testFlushWithFlushEnabled() throws Exception { + Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); + AzureBlobStorageTestAccount testAccount = createWasbTestAccount(); String wasbUrl = testAccount.getFileSystem().getName(); String abfsUrl = wasbUrlToAbfsUrl(wasbUrl); @@ -228,6 +233,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { @Test public void testFlushWithFlushDisabled() throws Exception { + Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); AzureBlobStorageTestAccount testAccount = createWasbTestAccount(); String wasbUrl = testAccount.getFileSystem().getName(); String abfsUrl = wasbUrlToAbfsUrl(wasbUrl); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java new file mode 100644 index 00000000000..f60740fce0b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java @@ -0,0 +1,176 @@ +/** + * 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.io.InputStream; +import java.util.Map; + +import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; +import org.junit.Assume; +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +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.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; +import org.apache.hadoop.fs.azurebfs.services.AuthType; + +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.TestConfigurationKeys.FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_ID; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_SECRET; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_DATA_READER_CLIENT_ID; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET; + +/** + * Test Azure Oauth with Blob Data contributor role and Blob Data Reader role. + * The Test AAD client need to be configured manually through Azure Portal, then save their properties in + * configuration files. + */ +public class ITestAzureBlobFileSystemOauth extends AbstractAbfsIntegrationTest{ + + private static final Path FILE_PATH = new Path("/testFile"); + private static final Path EXISTED_FILE_PATH = new Path("/existedFile"); + private static final Path EXISTED_FOLDER_PATH = new Path("/existedFolder"); + + public ITestAzureBlobFileSystemOauth() { + Assume.assumeTrue(this.getAuthType() == AuthType.OAuth); + } + /* + * BLOB DATA CONTRIBUTOR should have full access to the container and blobs in the container. + * */ + @Test + public void testBlobDataContributor() throws Exception { + String clientId = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_ID); + Assume.assumeTrue("Contributor client id not provided", clientId != null); + String secret = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_SECRET); + Assume.assumeTrue("Contributor client secret not provided", secret != null); + + prepareFiles(); + + final AzureBlobFileSystem fs = getBlobConributor(); + + // create and write into file in current container/fs + try(FSDataOutputStream stream = fs.create(FILE_PATH)) { + stream.write(0); + } + assertTrue(fs.exists(FILE_PATH)); + FileStatus fileStatus = fs.getFileStatus(FILE_PATH); + assertEquals(1, fileStatus.getLen()); + // delete file + assertTrue(fs.delete(FILE_PATH, true)); + assertFalse(fs.exists(FILE_PATH)); + + // Verify Blob Data Contributor has full access to existed folder, file + + // READ FOLDER + assertTrue(fs.exists(EXISTED_FOLDER_PATH)); + + //DELETE FOLDER + fs.delete(EXISTED_FOLDER_PATH, true); + assertFalse(fs.exists(EXISTED_FOLDER_PATH)); + + // READ FILE + try (FSDataInputStream stream = fs.open(EXISTED_FILE_PATH)) { + assertTrue(stream.read() != 0); + } + + assertEquals(0, fs.getFileStatus(EXISTED_FILE_PATH).getLen()); + + // WRITE FILE + try (FSDataOutputStream stream = fs.append(EXISTED_FILE_PATH)) { + stream.write(0); + } + + assertEquals(1, fs.getFileStatus(EXISTED_FILE_PATH).getLen()); + + // REMOVE FILE + fs.delete(EXISTED_FILE_PATH, true); + assertFalse(fs.exists(EXISTED_FILE_PATH)); + } + + /* + * BLOB DATA READER should have only READ access to the container and blobs in the container. + * */ + @Test + public void testBlobDataReader() throws Exception { + String clientId = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_READER_CLIENT_ID); + Assume.assumeTrue("Reader client id not provided", clientId != null); + String secret = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET); + Assume.assumeTrue("Reader client secret not provided", secret != null); + + prepareFiles(); + final AzureBlobFileSystem fs = getBlobReader(); + + // Use abfsStore in this test to verify the ERROR code in AbfsRestOperationException + AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); + // TEST READ FS + Map properties = abfsStore.getFilesystemProperties(); + // TEST READ FOLDER + assertTrue(fs.exists(EXISTED_FOLDER_PATH)); + + // TEST DELETE FOLDER + try { + abfsStore.delete(EXISTED_FOLDER_PATH, true); + } catch (AbfsRestOperationException e) { + assertEquals(AzureServiceErrorCode.AUTHORIZATION_PERMISSION_MISS_MATCH, e.getErrorCode()); + } + + // TEST READ FILE + try (InputStream inputStream = abfsStore.openFileForRead(EXISTED_FILE_PATH, null)) { + assertTrue(inputStream.read() != 0); + } + + // TEST WRITE FILE + try { + abfsStore.openFileForWrite(EXISTED_FILE_PATH, true); + } catch (AbfsRestOperationException e) { + assertEquals(AzureServiceErrorCode.AUTHORIZATION_PERMISSION_MISS_MATCH, e.getErrorCode()); + } + + } + + private void prepareFiles() throws IOException { + // create test files/folders to verify access control diff between + // Blob data contributor and Blob data reader + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.create(EXISTED_FILE_PATH); + assertTrue(fs.exists(EXISTED_FILE_PATH)); + fs.mkdirs(EXISTED_FOLDER_PATH); + assertTrue(fs.exists(EXISTED_FOLDER_PATH)); + } + + private AzureBlobFileSystem getBlobConributor() throws Exception { + Configuration configuration = this.getConfiguration(); + configuration.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID + this.getAccountName(), configuration.get(FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_ID)); + configuration.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET + this.getAccountName(), configuration.get(FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_SECRET)); + return getFileSystem(configuration); + } + + private AzureBlobFileSystem getBlobReader() throws Exception { + Configuration configuration = this.getConfiguration(); + configuration.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID + this.getAccountName(), configuration.get(FS_AZURE_BLOB_DATA_READER_CLIENT_ID)); + configuration.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET + this.getAccountName(), configuration.get(FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET)); + return getFileSystem(configuration); + } +} 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 c61de6764ea..13c5bc8f31f 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 @@ -23,6 +23,7 @@ import java.io.IOException; import java.util.Random; import java.util.concurrent.Callable; +import org.junit.Assume; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,6 +35,7 @@ import org.apache.hadoop.fs.FileStatus; 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.services.AuthType; import org.apache.hadoop.fs.contract.ContractTestUtils; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -66,6 +68,7 @@ public class ITestAzureBlobFileSystemRandomRead extends public ITestAzureBlobFileSystemRandomRead() throws Exception { super(); + Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); } @Test 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 3a44909a314..50b1828a828 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 @@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; +import org.apache.hadoop.fs.azurebfs.services.AuthType; /** * Test AzureBlobFileSystem initialization. @@ -41,8 +42,10 @@ public class ITestFileSystemInitialization extends AbstractAbfsIntegrationTest { final 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(FileSystemUriSchemes.ABFS_SCHEME, + new URI(scheme, filesystem + "@" + accountName, null, null, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java index 5d1cf91a9c3..56a91d3eaf9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; +import org.apache.hadoop.fs.azurebfs.services.AuthType; /** * Test AzureBlobFileSystem registration. @@ -79,8 +80,14 @@ public class ITestFileSystemRegistration extends AbstractAbfsIntegrationTest { AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(getConfiguration()); assertNotNull("filesystem", fs); - Abfs afs = (Abfs) FileContext.getFileContext(getConfiguration()).getDefaultFileSystem(); - assertNotNull("filecontext", afs); + if (this.getAuthType() == AuthType.OAuth) { + Abfss afs = (Abfss) FileContext.getFileContext(getConfiguration()).getDefaultFileSystem(); + assertNotNull("filecontext", afs); + } else { + Abfs afs = (Abfs) FileContext.getFileContext(getConfiguration()).getDefaultFileSystem(); + assertNotNull("filecontext", afs); + } + } @Test diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java index a89c0443350..ff28d3e0fdd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java @@ -31,6 +31,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azure.NativeAzureFileSystem; import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.azurebfs.services.AuthType; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory; @@ -50,6 +51,7 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest { public ITestWasbAbfsCompatibility() throws Exception { Assume.assumeFalse("Emulator is not supported", isEmulator()); + Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); } @Test 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 fc7312aadac..67301c7f8e3 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 @@ -28,6 +28,12 @@ public final class TestConfigurationKeys { public static final String FS_AZURE_TEST_HOST_PORT = "fs.azure.test.host.port"; public static final String FS_AZURE_CONTRACT_TEST_URI = "fs.contract.test.fs.abfs"; + public static final String FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_ID = "fs.azure.account.oauth2.contributor.client.id"; + public static final String FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_SECRET = "fs.azure.account.oauth2.contributor.client.secret"; + + public static final String FS_AZURE_BLOB_DATA_READER_CLIENT_ID = "fs.azure.account.oauth2.reader.client.id"; + public static final String FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET = "fs.azure.account.oauth2.reader.client.secret"; + public static final String ABFS_TEST_RESOURCE_XML = "azure-bfs-test.xml"; public static final String ABFS_TEST_CONTAINER_PREFIX = "abfs-testcontainer-"; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java index ffd5babc5bf..5505e6ae0f3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java @@ -25,6 +25,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.services.AuthType; /** * Bind ABFS contract tests to the Azure test setup/teardown. @@ -32,18 +33,17 @@ import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; public class ABFSContractTestBinding extends AbstractAbfsIntegrationTest { private final URI testUri; - public ABFSContractTestBinding(final boolean secure) throws Exception { - this(secure, true); + public ABFSContractTestBinding() throws Exception { + this(true); } - public ABFSContractTestBinding(final boolean secure, + public ABFSContractTestBinding( final boolean useExistingFileSystem) throws Exception{ - super(secure); if (useExistingFileSystem) { Configuration configuration = getConfiguration(); String testUrl = configuration.get(TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI); - if (secure) { + if (getAuthType() != AuthType.SharedKey) { testUrl = testUrl.replaceFirst(FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME); } setTestUrl(testUrl); @@ -61,4 +61,8 @@ public class ABFSContractTestBinding extends AbstractAbfsIntegrationTest { public Configuration getConfiguration() { return super.getConfiguration(); } + + public boolean isSecureMode() { + return this.getAuthType() == AuthType.SharedKey ? false : true; + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java index a302fccfd9c..8a955bc6062 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java @@ -18,34 +18,23 @@ package org.apache.hadoop.fs.azurebfs.contract; -import java.util.Arrays; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.contract.AbstractContractAppendTest; import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.junit.Test; import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; /** * Contract test for open operation. */ -@RunWith(Parameterized.class) public class ITestAbfsFileSystemContractAppend extends AbstractContractAppendTest { - @Parameterized.Parameters(name = "SecureMode={0}") - public static Iterable secure() { - return Arrays.asList(new Object[][] { {true}, {false} }); - } - private final boolean isSecure; private final ABFSContractTestBinding binding; - public ITestAbfsFileSystemContractAppend(final boolean secure) throws Exception { - this.isSecure = secure; - binding = new ABFSContractTestBinding(this.isSecure); + public ITestAbfsFileSystemContractAppend() throws Exception { + binding = new ABFSContractTestBinding(); + this.isSecure = binding.isSecureMode(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java index c31a6d2a5aa..383528b75a8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java @@ -17,11 +17,6 @@ */ package org.apache.hadoop.fs.azurebfs.contract; -import java.util.Arrays; - -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.contract.AbstractContractConcatTest; import org.apache.hadoop.fs.contract.AbstractFSContract; @@ -29,19 +24,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract; /** * Contract test for concat operation. */ -@RunWith(Parameterized.class) public class ITestAbfsFileSystemContractConcat extends AbstractContractConcatTest{ - @Parameterized.Parameters(name = "SecureMode={0}") - public static Iterable secure() { - return Arrays.asList(new Object[][] { {true}, {false} }); - } - private final boolean isSecure; private final ABFSContractTestBinding binding; - public ITestAbfsFileSystemContractConcat(final boolean secure) throws Exception { - isSecure = secure; - binding = new ABFSContractTestBinding(isSecure); + public ITestAbfsFileSystemContractConcat() throws Exception { + binding = new ABFSContractTestBinding(); + this.isSecure = binding.isSecureMode(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java index ce4d22963e1..3c3e9490365 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java @@ -18,11 +18,6 @@ package org.apache.hadoop.fs.azurebfs.contract; -import java.util.Arrays; - -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.contract.AbstractContractCreateTest; import org.apache.hadoop.fs.contract.AbstractFSContract; @@ -30,19 +25,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract; /** * Contract test for create operation. */ -@RunWith(Parameterized.class) public class ITestAbfsFileSystemContractCreate extends AbstractContractCreateTest{ - @Parameterized.Parameters(name = "SecureMode={0}") - public static Iterable secure() { - return Arrays.asList(new Object[][] { {true}, {false} }); - } - private final boolean isSecure; private final ABFSContractTestBinding binding; - public ITestAbfsFileSystemContractCreate(final boolean secure) throws Exception { - this.isSecure = secure; - binding = new ABFSContractTestBinding(this.isSecure); + public ITestAbfsFileSystemContractCreate() throws Exception { + binding = new ABFSContractTestBinding(); + this.isSecure = binding.isSecureMode(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java index 310731cc0eb..1d1136c3538 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java @@ -18,11 +18,6 @@ package org.apache.hadoop.fs.azurebfs.contract; -import java.util.Arrays; - -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.contract.AbstractContractDeleteTest; import org.apache.hadoop.fs.contract.AbstractFSContract; @@ -30,19 +25,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract; /** * Contract test for delete operation. */ -@RunWith(Parameterized.class) public class ITestAbfsFileSystemContractDelete extends AbstractContractDeleteTest { - @Parameterized.Parameters(name = "SecureMode={0}") - public static Iterable secure() { - return Arrays.asList(new Object[][] { {true}, {false} }); - } - private final boolean isSecure; private final ABFSContractTestBinding binding; - public ITestAbfsFileSystemContractDelete(final boolean secure) throws Exception { - this.isSecure = secure; - binding = new ABFSContractTestBinding(isSecure); + public ITestAbfsFileSystemContractDelete() throws Exception { + binding = new ABFSContractTestBinding(); + this.isSecure = binding.isSecureMode(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java index c2cf25569b2..544bbbfbdb1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java @@ -28,7 +28,7 @@ public class ITestAbfsFileSystemContractDistCp extends AbstractContractDistCpTes private final ABFSContractTestBinding binding; public ITestAbfsFileSystemContractDistCp() throws Exception { - binding = new ABFSContractTestBinding(false); + binding = new ABFSContractTestBinding(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java index 9ad3b215e10..08b7eefe198 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java @@ -17,11 +17,6 @@ */ package org.apache.hadoop.fs.azurebfs.contract; -import java.util.Arrays; - -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest; import org.apache.hadoop.fs.contract.AbstractFSContract; @@ -29,19 +24,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract; /** * Contract test for getFileStatus operation. */ -@RunWith(Parameterized.class) public class ITestAbfsFileSystemContractGetFileStatus extends AbstractContractGetFileStatusTest { - @Parameterized.Parameters(name = "SecureMode={0}") - public static Iterable secure() { - return Arrays.asList(new Object[][] { {true}, {false} }); - } - private final boolean isSecure; private final ABFSContractTestBinding binding; - public ITestAbfsFileSystemContractGetFileStatus(final boolean secure) throws Exception { - this.isSecure = secure; - binding = new ABFSContractTestBinding(isSecure); + public ITestAbfsFileSystemContractGetFileStatus() throws Exception { + binding = new ABFSContractTestBinding(); + this.isSecure = binding.isSecureMode(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java index 6265ca1f92c..7b785753f2c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java @@ -18,11 +18,6 @@ package org.apache.hadoop.fs.azurebfs.contract; -import java.util.Arrays; - -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.contract.AbstractContractMkdirTest; import org.apache.hadoop.fs.contract.AbstractFSContract; @@ -30,19 +25,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract; /** * Contract test for mkdir operation. */ -@RunWith(Parameterized.class) public class ITestAbfsFileSystemContractMkdir extends AbstractContractMkdirTest { - @Parameterized.Parameters(name = "SecureMode={0}") - public static Iterable secure() { - return Arrays.asList(new Object[][] { {true}, {false} }); - } - private final boolean isSecure; private final ABFSContractTestBinding binding; - public ITestAbfsFileSystemContractMkdir(final boolean secure) throws Exception { - this.isSecure = secure; - binding = new ABFSContractTestBinding(secure); + public ITestAbfsFileSystemContractMkdir() throws Exception { + binding = new ABFSContractTestBinding(); + this.isSecure = binding.isSecureMode(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java index ae4bb2a4fc6..41f691d512f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java @@ -18,11 +18,6 @@ package org.apache.hadoop.fs.azurebfs.contract; -import java.util.Arrays; - -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.contract.AbstractContractOpenTest; import org.apache.hadoop.fs.contract.AbstractFSContract; @@ -30,19 +25,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract; /** * Contract test for open operation. */ -@RunWith(Parameterized.class) public class ITestAbfsFileSystemContractOpen extends AbstractContractOpenTest { - @Parameterized.Parameters(name = "SecureMode={0}") - public static Iterable secure() { - return Arrays.asList(new Object[][] { {true}, {false} }); - } - private final boolean isSecure; private final ABFSContractTestBinding binding; - public ITestAbfsFileSystemContractOpen(final boolean secure) throws Exception { - this.isSecure = secure; - binding = new ABFSContractTestBinding(this.isSecure); + public ITestAbfsFileSystemContractOpen() throws Exception { + binding = new ABFSContractTestBinding(); + this.isSecure = binding.isSecureMode(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java index 6e6a7280350..82f104a44bf 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java @@ -18,11 +18,6 @@ package org.apache.hadoop.fs.azurebfs.contract; -import java.util.Arrays; - -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.contract.AbstractContractRenameTest; import org.apache.hadoop.fs.contract.AbstractFSContract; @@ -30,19 +25,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract; /** * Contract test for rename operation. */ -@RunWith(Parameterized.class) public class ITestAbfsFileSystemContractRename extends AbstractContractRenameTest { - @Parameterized.Parameters(name = "SecureMode={0}") - public static Iterable secure() { - return Arrays.asList(new Object[][] { {true}, {false} }); - } - private final boolean isSecure; private final ABFSContractTestBinding binding; - public ITestAbfsFileSystemContractRename(final boolean secure) throws Exception { - this.isSecure = secure; - binding = new ABFSContractTestBinding(this.isSecure); + public ITestAbfsFileSystemContractRename() throws Exception { + binding = new ABFSContractTestBinding(); + this.isSecure = binding.isSecureMode(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java index 01dea2d164d..5b5493fdcd0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java @@ -17,31 +17,21 @@ */ package org.apache.hadoop.fs.azurebfs.contract; -import java.util.Arrays; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest; import org.apache.hadoop.fs.contract.AbstractFSContract; import org.junit.Ignore; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; /** * Contract test for root directory operation. */ -@RunWith(Parameterized.class) public class ITestAbfsFileSystemContractRootDirectory extends AbstractContractRootDirectoryTest { - @Parameterized.Parameters(name = "SecureMode={0}") - public static Iterable secure() { - return Arrays.asList(new Object[][] { {true}, {false} }); - } - private final boolean isSecure; private final ABFSContractTestBinding binding; - public ITestAbfsFileSystemContractRootDirectory(final boolean secure) throws Exception { - this.isSecure = secure; - binding = new ABFSContractTestBinding(secure); + public ITestAbfsFileSystemContractRootDirectory() throws Exception { + binding = new ABFSContractTestBinding(); + this.isSecure = binding.isSecureMode(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java index 5ed74668408..fc235e36c61 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java @@ -28,7 +28,7 @@ public class ITestAbfsFileSystemContractSecureDistCp extends AbstractContractDis private final ABFSContractTestBinding binding; public ITestAbfsFileSystemContractSecureDistCp() throws Exception { - binding = new ABFSContractTestBinding(true); + binding = new ABFSContractTestBinding(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java index 5e0ea0c94ea..4529e752b01 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java @@ -18,11 +18,6 @@ package org.apache.hadoop.fs.azurebfs.contract; -import java.util.Arrays; - -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.contract.AbstractContractSeekTest; import org.apache.hadoop.fs.contract.AbstractFSContract; @@ -30,19 +25,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract; /** * Contract test for seek operation. */ -@RunWith(Parameterized.class) public class ITestAbfsFileSystemContractSeek extends AbstractContractSeekTest{ - @Parameterized.Parameters(name = "SecureMode={0}") - public static Iterable secure() { - return Arrays.asList(new Object[][] { {true}, {false} }); - } - private final boolean isSecure; private final ABFSContractTestBinding binding; - public ITestAbfsFileSystemContractSeek(final boolean secure) throws Exception { - this.isSecure = secure; - binding = new ABFSContractTestBinding(this.isSecure); + public ITestAbfsFileSystemContractSeek() throws Exception { + binding = new ABFSContractTestBinding(); + this.isSecure = binding.isSecureMode(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java index 8d23b0bbffd..6c4f9badc0f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java @@ -17,11 +17,6 @@ */ package org.apache.hadoop.fs.azurebfs.contract; -import java.util.Arrays; - -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.contract.AbstractContractSetTimesTest; import org.apache.hadoop.fs.contract.AbstractFSContract; @@ -29,19 +24,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract; /** * Contract test for setTimes operation. */ -@RunWith(Parameterized.class) public class ITestAbfsFileSystemContractSetTimes extends AbstractContractSetTimesTest { - @Parameterized.Parameters(name = "SecureMode={0}") - public static Iterable secure() { - return Arrays.asList(new Object[][] { {true}, {false} }); - } - private final boolean isSecure; private final ABFSContractTestBinding binding; - public ITestAbfsFileSystemContractSetTimes(final boolean secure) throws Exception { - this.isSecure = secure; - binding = new ABFSContractTestBinding(this.isSecure); + public ITestAbfsFileSystemContractSetTimes() throws Exception { + binding = new ABFSContractTestBinding(); + this.isSecure = binding.isSecureMode(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java index d8854a2b0a0..a9fa2d77194 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java @@ -40,7 +40,7 @@ public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest { public ITestAzureBlobFileSystemBasics() throws Exception { // If all contract tests are running in parallel, some root level tests in FileSystemContractBaseTest will fail // due to the race condition. Hence for this contract test it should be tested in different container - binding = new ABFSContractTestBinding(false, false); + binding = new ABFSContractTestBinding(false); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index 7bb27fc4514..a2fdd095ac5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -21,11 +21,11 @@ package org.apache.hadoop.fs.azurebfs.services; import java.net.URL; import java.util.regex.Pattern; -import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; -import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx; import org.junit.Assert; import org.junit.Test; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; @@ -40,7 +40,7 @@ public final class TestAbfsClient { AbfsConfiguration config, boolean includeSSLProvider) { AbfsClient client = new AbfsClient(baseUrl, null, - config, null); + config, null, null); String sslProviderName = null; if (includeSSLProvider) { sslProviderName = SSLSocketFactoryEx.getDefaultFactory().getProviderName(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestQueryParams.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestQueryParams.java new file mode 100644 index 00000000000..e6c6993b1dc --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestQueryParams.java @@ -0,0 +1,72 @@ +/** + * 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.junit.Assert; +import org.junit.Test; + +import org.apache.hadoop.fs.azurebfs.oauth2.QueryParams; +/** + * Test query params serialization. + */ +public class TestQueryParams { + private static final String SEPARATOR = "&"; + private static final String[][] PARAM_ARRAY = {{"K0", "V0"}, {"K1", "V1"}, {"K2", "V2"}}; + + @Test + public void testOneParam() { + String key = PARAM_ARRAY[0][0]; + String value = PARAM_ARRAY[0][1]; + + Map paramMap = new HashMap<>(); + paramMap.put(key, value); + + QueryParams qp = new QueryParams(); + qp.add(key, value); + Assert.assertEquals(key + "=" + value, qp.serialize()); + } + + @Test + public void testMultipleParams() { + QueryParams qp = new QueryParams(); + for (String[] entry : PARAM_ARRAY) { + qp.add(entry[0], entry[1]); + } + Map paramMap = constructMap(qp.serialize()); + Assert.assertEquals(PARAM_ARRAY.length, paramMap.size()); + + for (String[] entry : PARAM_ARRAY) { + Assert.assertTrue(paramMap.containsKey(entry[0])); + Assert.assertEquals(entry[1], paramMap.get(entry[0])); + } + } + + private Map constructMap(String input) { + String[] entries = input.split(SEPARATOR); + Map paramMap = new HashMap<>(); + for (String entry : entries) { + String[] keyValue = entry.split("="); + paramMap.put(keyValue[0], keyValue[1]); + } + return paramMap; + } + +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java index 9051a72e127..ef4ddb92157 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java @@ -21,9 +21,13 @@ import com.microsoft.azure.storage.CloudStorageAccount; import com.microsoft.azure.storage.blob.CloudBlobClient; import com.microsoft.azure.storage.blob.CloudBlobContainer; -import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount; +import org.junit.Assume; import org.junit.Test; +import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount; +import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; +import org.apache.hadoop.fs.azurebfs.services.AuthType; + import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.ABFS_TEST_CONTAINER_PREFIX; /** @@ -31,7 +35,12 @@ import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.ABFS * In that case, dev can use this tool to list and delete all test containers. * By default, all test container used in E2E tests sharing same prefix: "abfs-testcontainer-" */ -public class CleanUpAbfsTestContainer { +public class CleanUpAbfsTestContainer extends AbstractAbfsIntegrationTest{ + + public CleanUpAbfsTestContainer() { + Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); + } + @Test public void testEnumContainers() throws Throwable { int count = 0; diff --git a/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml b/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml index 464a8e670e9..9b908b06fd7 100644 --- a/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml +++ b/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml @@ -22,6 +22,27 @@ {YOURACCOUNT} + + fs.azure.account.auth.type.{YOURACCOUNT}.dfs.core.windows.net + {AUTH TYPE} + The auth type can be : SharedKey, OAuth, Custom. By default "SharedKey" is used. + + + + fs.contract.test.fs.abfs + abfs://{CONTAINERNAME}@{ACCOUNTNAME}.dfs.core.windows.net/value> + The name of the azure file system for testing. + + + + fs.contract.test.fs.abfss + abfss://{CONTAINERNAME}@{ACCOUNTNAME}.dfs.core.windows.net/value> + The name of the azure file system for testing. + + --> + + + + + + + + + + + + + + + + + + + + + + + + From 9c1e4e81399913f180131f4faa95604087c6d962 Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Wed, 22 Aug 2018 18:31:47 +0000 Subject: [PATCH 08/29] HADOOP-15661. ABFS: Add support for ACL. Contributed by Junhua Gu and Da Zhou. --- .../fs/azurebfs/AzureBlobFileSystem.java | 199 ++- .../fs/azurebfs/AzureBlobFileSystemStore.java | 351 +++++- .../azurebfs/constants/AbfsHttpConstants.java | 15 + .../constants/HttpHeaderConfigurations.java | 6 + .../InvalidAclOperationException.java | 33 + .../services/AzureServiceErrorCode.java | 1 + .../fs/azurebfs/services/AbfsAclHelper.java | 202 ++++ .../fs/azurebfs/services/AbfsClient.java | 119 +- .../fs/azurebfs/services/AbfsPermission.java | 114 ++ .../ITestAzureBlobFileSystemBackCompat.java | 2 + .../ITestAzureBlobFileSystemFileStatus.java | 43 +- .../ITestAzureBlobFileSystemFlush.java | 6 + .../ITestAzureBlobFileSystemPermission.java | 109 ++ .../ITestAzureBlobFileSystemRandomRead.java | 4 +- .../ITestAzureBlobFileSystemRename.java | 14 + .../azurebfs/ITestAzureBlobFilesystemAcl.java | 1071 +++++++++++++++++ .../azurebfs/ITestWasbAbfsCompatibility.java | 12 + .../fs/azurebfs/utils/AclTestHelpers.java | 119 ++ .../fs/azurebfs/utils/Parallelized.java | 60 + 19 files changed, 2422 insertions(+), 58 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAclOperationException.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsAclHelper.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsPermission.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AclTestHelpers.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/Parallelized.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 2cb517b4b10..6bec7cb2a3b 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.io.OutputStream; import java.net.HttpURLConnection; import java.net.URI; import java.net.URISyntaxException; +import java.util.List; import java.util.ArrayList; import java.util.EnumSet; import java.util.concurrent.Callable; @@ -60,6 +61,8 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.FileSystemOperationUnh import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; +import org.apache.hadoop.fs.permission.AclEntry; +import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Progressable; @@ -154,7 +157,8 @@ public class AzureBlobFileSystem extends FileSystem { blockSize); try { - OutputStream outputStream = abfsStore.createFile(makeQualified(f), overwrite); + OutputStream outputStream = abfsStore.createFile(makeQualified(f), overwrite, + permission == null ? FsPermission.getFileDefault() : permission, FsPermission.getUMask(getConf())); return new FSDataOutputStream(outputStream, statistics); } catch(AzureBlobFileSystemException ex) { checkException(f, ex); @@ -253,7 +257,8 @@ public class AzureBlobFileSystem extends FileSystem { AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH, AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND, AzureServiceErrorCode.INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE, - AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND); + AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND, + AzureServiceErrorCode.INTERNAL_OPERATION_ABORT); return false; } @@ -308,7 +313,8 @@ public class AzureBlobFileSystem extends FileSystem { } try { - abfsStore.createDirectory(makeQualified(f)); + abfsStore.createDirectory(makeQualified(f), permission == null ? FsPermission.getDirDefault() : permission, + FsPermission.getUMask(getConf())); return true; } catch (AzureBlobFileSystemException ex) { checkException(f, ex, AzureServiceErrorCode.PATH_ALREADY_EXISTS); @@ -457,6 +463,188 @@ public class AzureBlobFileSystem extends FileSystem { return true; } + /** + * Set owner of a path (i.e. a file or a directory). + * The parameters owner and group cannot both be null. + * + * @param path The path + * @param owner If it is null, the original username remains unchanged. + * @param group If it is null, the original groupname remains unchanged. + */ + @Override + public void setOwner(final Path path, final String owner, final String group) + throws IOException { + LOG.debug( + "AzureBlobFileSystem.setOwner path: {}", path); + + if ((owner == null || owner.isEmpty()) && (group == null || group.isEmpty())) { + throw new IllegalArgumentException("A valid owner or group must be specified."); + } + + try { + abfsStore.setOwner(makeQualified(path), + owner, + group); + } catch (AzureBlobFileSystemException ex) { + checkException(path, ex); + } + } + + /** + * Set permission of a path. + * + * @param path The path + * @param permission Access permission + */ + @Override + public void setPermission(final Path path, final FsPermission permission) + throws IOException { + LOG.debug("AzureBlobFileSystem.setPermission path: {}", path); + + if (permission == null) { + throw new IllegalArgumentException("The permission can't be null"); + } + + try { + abfsStore.setPermission(makeQualified(path), + permission); + } catch (AzureBlobFileSystemException ex) { + checkException(path, ex); + } + } + + /** + * Modifies ACL entries of files and directories. This method can add new ACL + * entries or modify the permissions on existing ACL entries. All existing + * ACL entries that are not specified in this call are retained without + * changes. (Modifications are merged into the current ACL.) + * + * @param path Path to modify + * @param aclSpec List of AbfsAclEntry describing modifications + * @throws IOException if an ACL could not be modified + */ + @Override + public void modifyAclEntries(final Path path, final List aclSpec) + throws IOException { + LOG.debug("AzureBlobFileSystem.modifyAclEntries path: {}", path.toString()); + + if (aclSpec == null || aclSpec.isEmpty()) { + throw new IllegalArgumentException("The value of the aclSpec parameter is invalid."); + } + + try { + abfsStore.modifyAclEntries(makeQualified(path), + aclSpec); + } catch (AzureBlobFileSystemException ex) { + checkException(path, ex); + } + } + + /** + * Removes ACL entries from files and directories. Other ACL entries are + * retained. + * + * @param path Path to modify + * @param aclSpec List of AclEntry describing entries to remove + * @throws IOException if an ACL could not be modified + */ + @Override + public void removeAclEntries(final Path path, final List aclSpec) + throws IOException { + LOG.debug("AzureBlobFileSystem.removeAclEntries path: {}", path); + + if (aclSpec == null || aclSpec.isEmpty()) { + throw new IllegalArgumentException("The aclSpec argument is invalid."); + } + + try { + abfsStore.removeAclEntries(makeQualified(path), aclSpec); + } catch (AzureBlobFileSystemException ex) { + checkException(path, ex); + } + } + + /** + * Removes all default ACL entries from files and directories. + * + * @param path Path to modify + * @throws IOException if an ACL could not be modified + */ + @Override + public void removeDefaultAcl(final Path path) throws IOException { + LOG.debug("AzureBlobFileSystem.removeDefaultAcl path: {}", path); + + try { + abfsStore.removeDefaultAcl(makeQualified(path)); + } catch (AzureBlobFileSystemException ex) { + checkException(path, ex); + } + } + + /** + * Removes all but the base ACL entries of files and directories. The entries + * for user, group, and others are retained for compatibility with permission + * bits. + * + * @param path Path to modify + * @throws IOException if an ACL could not be removed + */ + @Override + public void removeAcl(final Path path) throws IOException { + LOG.debug("AzureBlobFileSystem.removeAcl path: {}", path); + + try { + abfsStore.removeAcl(makeQualified(path)); + } catch (AzureBlobFileSystemException ex) { + checkException(path, ex); + } + } + + /** + * Fully replaces ACL of files and directories, discarding all existing + * entries. + * + * @param path Path to modify + * @param aclSpec List of AclEntry describing modifications, must include + * entries for user, group, and others for compatibility with + * permission bits. + * @throws IOException if an ACL could not be modified + */ + @Override + public void setAcl(final Path path, final List aclSpec) + throws IOException { + LOG.debug("AzureBlobFileSystem.setAcl path: {}", path); + + if (aclSpec == null || aclSpec.size() == 0) { + throw new IllegalArgumentException("The aclSpec argument is invalid."); + } + + try { + abfsStore.setAcl(makeQualified(path), aclSpec); + } catch (AzureBlobFileSystemException ex) { + checkException(path, ex); + } + } + + /** + * Gets the ACL of a file or directory. + * + * @param path Path to get + * @return AbfsAclStatus describing the ACL of the file or directory + * @throws IOException if an ACL could not be read + */ + @Override + public AclStatus getAclStatus(final Path path) throws IOException { + LOG.debug("AzureBlobFileSystem.getAclStatus path: {}", path.toString()); + + try { + return abfsStore.getAclStatus(makeQualified(path)); + } catch (AzureBlobFileSystemException ex) { + checkException(path, ex); + return null; + } + } + private FileStatus tryGetFileStatus(final Path f) { try { return getFileStatus(f); @@ -656,4 +844,9 @@ public class AzureBlobFileSystem extends FileSystem { AbfsClient getAbfsClient() { return abfsStore.getClient(); } + + @VisibleForTesting + boolean getIsNamespaceEnabeld() throws AzureBlobFileSystemException { + return abfsStore.getIsNamespaceEnabled(); + } } 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 b8da35b0b10..58df914ec8b 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 @@ -36,8 +36,10 @@ import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; import java.util.Date; +import java.util.HashMap; import java.util.HashSet; import java.util.Hashtable; +import java.util.List; import java.util.Map; import java.util.Set; import javax.xml.bind.DatatypeConverter; @@ -68,13 +70,18 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema; import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; +import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; +import org.apache.hadoop.fs.azurebfs.services.AbfsPermission; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy; import org.apache.hadoop.fs.azurebfs.services.SharedKeyCredentials; +import org.apache.hadoop.fs.permission.AclEntry; +import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.security.UserGroupInformation; @@ -85,7 +92,7 @@ import org.slf4j.LoggerFactory; import static org.apache.hadoop.util.Time.now; /** - * Provides the bridging logic between Hadoop's abstract filesystem and Azure Storage + * Provides the bridging logic between Hadoop's abstract filesystem and Azure Storage. */ @InterfaceAudience.Public @InterfaceStability.Evolving @@ -103,7 +110,8 @@ public class AzureBlobFileSystemStore { private final AbfsConfiguration abfsConfiguration; private final Set azureAtomicRenameDirSet; - + private boolean isNamespaceEnabledSet; + private boolean isNamespaceEnabled; public AzureBlobFileSystemStore(URI uri, boolean isSecure, Configuration configuration, UserGroupInformation userGroupInformation) throws AzureBlobFileSystemException { @@ -121,6 +129,20 @@ public class AzureBlobFileSystemStore { initializeClient(uri, isSecure); } + public boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException { + if (!isNamespaceEnabledSet) { + LOG.debug("getFilesystemProperties for filesystem: {}", + client.getFileSystem()); + + final AbfsRestOperation op = client.getFilesystemProperties(); + isNamespaceEnabled = Boolean.parseBoolean( + op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_NAMESPACE_ENABLED)); + isNamespaceEnabledSet = true; + } + + return isNamespaceEnabled; + } + @VisibleForTesting URIBuilder getURIBuilder(final String hostName, boolean isSecure) { String scheme = isSecure ? FileSystemUriSchemes.HTTPS_SCHEME : FileSystemUriSchemes.HTTP_SCHEME; @@ -197,7 +219,7 @@ public class AzureBlobFileSystemStore { } catch (CharacterCodingException ex) { throw new InvalidAbfsRestOperationException(ex); } - client.setPathProperties("/" + getRelativePath(path), commaSeparatedProperties); + client.setPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), commaSeparatedProperties); } public void createFilesystem() throws AzureBlobFileSystemException { @@ -214,13 +236,20 @@ public class AzureBlobFileSystemStore { client.deleteFilesystem(); } - public OutputStream createFile(final Path path, final boolean overwrite) throws AzureBlobFileSystemException { - LOG.debug("createFile filesystem: {} path: {} overwrite: {}", + public OutputStream createFile(final Path path, final boolean overwrite, final FsPermission permission, + final FsPermission umask) throws AzureBlobFileSystemException { + boolean isNamespaceEnabled = getIsNamespaceEnabled(); + LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}", client.getFileSystem(), path, - overwrite); + overwrite, + permission.toString(), + umask.toString(), + isNamespaceEnabled); - client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), true, overwrite); + client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), true, overwrite, + isNamespaceEnabled ? getOctalNotation(permission) : null, + isNamespaceEnabled ? getOctalNotation(umask) : null); final OutputStream outputStream; outputStream = new FSDataOutputStream( @@ -229,16 +258,23 @@ public class AzureBlobFileSystemStore { return outputStream; } - public void createDirectory(final Path path) throws AzureBlobFileSystemException { - LOG.debug("createDirectory filesystem: {} path: {}", + public void createDirectory(final Path path, final FsPermission permission, final FsPermission umask) + throws AzureBlobFileSystemException { + boolean isNamespaceEnabled = getIsNamespaceEnabled(); + LOG.debug("createDirectory filesystem: {} path: {} permission: {} umask: {} isNamespaceEnabled: {}", client.getFileSystem(), - path); + path, + permission, + umask, + isNamespaceEnabled); - client.createPath("/" + getRelativePath(path), false, true); + client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), false, true, + isNamespaceEnabled ? getOctalNotation(permission) : null, + isNamespaceEnabled ? getOctalNotation(umask) : null); } - public InputStream openFileForRead(final Path path, final FileSystem.Statistics statistics) throws AzureBlobFileSystemException { - + public InputStream openFileForRead(final Path path, final FileSystem.Statistics statistics) + throws AzureBlobFileSystemException { LOG.debug("openFileForRead filesystem: {} path: {}", client.getFileSystem(), path); @@ -327,7 +363,6 @@ public class AzureBlobFileSystemStore { public void delete(final Path path, final boolean recursive) throws AzureBlobFileSystemException { - LOG.debug("delete filesystem: {} path: {} recursive: {}", client.getFileSystem(), path, @@ -351,19 +386,31 @@ public class AzureBlobFileSystemStore { } public FileStatus getFileStatus(final Path path) throws IOException { - - LOG.debug("getFileStatus filesystem: {} path: {}", + boolean isNamespaceEnabled = getIsNamespaceEnabled(); + LOG.debug("getFileStatus filesystem: {} path: {} isNamespaceEnabled: {}", client.getFileSystem(), - path); + path, + isNamespaceEnabled); if (path.isRoot()) { - AbfsRestOperation op = client.getFilesystemProperties(); + final AbfsRestOperation op = isNamespaceEnabled + ? client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + AbfsHttpConstants.ROOT_PATH) + : client.getFilesystemProperties(); + final long blockSize = abfsConfiguration.getAzureBlockSize(); + final String owner = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_OWNER); + final String group = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_GROUP); + final String permissions = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PERMISSIONS); final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); final String lastModified = op.getResult().getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED); + final boolean hasAcl = AbfsPermission.isExtendedAcl(permissions); + return new VersionedFileStatus( - userGroupInformation.getUserName(), - userGroupInformation.getPrimaryGroupName(), + owner == null ? userGroupInformation.getUserName() : owner, + group == null ? userGroupInformation.getPrimaryGroupName() : group, + permissions == null ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL) + : AbfsPermission.valueOf(permissions), + hasAcl, 0, true, 1, @@ -375,14 +422,22 @@ public class AzureBlobFileSystemStore { AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path)); final long blockSize = abfsConfiguration.getAzureBlockSize(); - final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); - final String lastModified = op.getResult().getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED); - final String contentLength = op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH); - final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); + final AbfsHttpOperation result = op.getResult(); + final String eTag = result.getResponseHeader(HttpHeaderConfigurations.ETAG); + final String lastModified = result.getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED); + final String contentLength = result.getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH); + final String resourceType = result.getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); + final String owner = result.getResponseHeader(HttpHeaderConfigurations.X_MS_OWNER); + final String group = result.getResponseHeader(HttpHeaderConfigurations.X_MS_GROUP); + final String permissions = result.getResponseHeader((HttpHeaderConfigurations.X_MS_PERMISSIONS)); + final boolean hasAcl = AbfsPermission.isExtendedAcl(permissions); return new VersionedFileStatus( - userGroupInformation.getUserName(), - userGroupInformation.getPrimaryGroupName(), + owner == null ? userGroupInformation.getUserName() : owner, + group == null ? userGroupInformation.getPrimaryGroupName() : group, + permissions == null ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL) + : AbfsPermission.valueOf(permissions), + hasAcl, parseContentLength(contentLength), parseIsDirectory(resourceType), 1, @@ -417,6 +472,13 @@ public class AzureBlobFileSystemStore { long blockSize = abfsConfiguration.getAzureBlockSize(); for (ListResultEntrySchema entry : retrievedSchema.paths()) { + final String owner = entry.owner() == null ? userGroupInformation.getUserName() : entry.owner(); + final String group = entry.group() == null ? userGroupInformation.getPrimaryGroupName() : entry.group(); + final FsPermission fsPermission = entry.permissions() == null + ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL) + : AbfsPermission.valueOf(entry.permissions()); + final boolean hasAcl = AbfsPermission.isExtendedAcl(entry.permissions()); + long lastModifiedMillis = 0; long contentLength = entry.contentLength() == null ? 0 : entry.contentLength(); boolean isDirectory = entry.isDirectory() == null ? false : entry.isDirectory(); @@ -429,8 +491,10 @@ public class AzureBlobFileSystemStore { fileStatuses.add( new VersionedFileStatus( - userGroupInformation.getUserName(), - userGroupInformation.getPrimaryGroupName(), + owner, + group, + fsPermission, + hasAcl, contentLength, isDirectory, 1, @@ -445,6 +509,211 @@ public class AzureBlobFileSystemStore { return fileStatuses.toArray(new FileStatus[0]); } + public void setOwner(final Path path, final String owner, final String group) throws + AzureBlobFileSystemException { + if (!getIsNamespaceEnabled()) { + throw new UnsupportedOperationException( + "This operation is only valid for storage accounts with the hierarchical namespace enabled."); + } + + LOG.debug( + "setOwner filesystem: {} path: {} owner: {} group: {}", + client.getFileSystem(), + path.toString(), + owner, + group); + client.setOwner(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), owner, group); + } + + public void setPermission(final Path path, final FsPermission permission) throws + AzureBlobFileSystemException { + if (!getIsNamespaceEnabled()) { + throw new UnsupportedOperationException( + "This operation is only valid for storage accounts with the hierarchical namespace enabled."); + } + + LOG.debug( + "setPermission filesystem: {} path: {} permission: {}", + client.getFileSystem(), + path.toString(), + permission.toString()); + client.setPermission(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), + String.format(AbfsHttpConstants.PERMISSION_FORMAT, permission.toOctal())); + } + + public void modifyAclEntries(final Path path, final List aclSpec) throws + AzureBlobFileSystemException { + if (!getIsNamespaceEnabled()) { + throw new UnsupportedOperationException( + "This operation is only valid for storage accounts with the hierarchical namespace enabled."); + } + + LOG.debug( + "modifyAclEntries filesystem: {} path: {} aclSpec: {}", + client.getFileSystem(), + path.toString(), + AclEntry.aclSpecToString(aclSpec)); + + final Map modifyAclEntries = AbfsAclHelper.deserializeAclSpec(AclEntry.aclSpecToString(aclSpec)); + + final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true)); + final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + + final Map aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); + + for (Map.Entry modifyAclEntry : modifyAclEntries.entrySet()) { + aclEntries.put(modifyAclEntry.getKey(), modifyAclEntry.getValue()); + } + + if (!modifyAclEntries.containsKey(AbfsHttpConstants.ACCESS_MASK)) { + aclEntries.remove(AbfsHttpConstants.ACCESS_MASK); + } + + if (!modifyAclEntries.containsKey(AbfsHttpConstants.DEFAULT_MASK)) { + aclEntries.remove(AbfsHttpConstants.DEFAULT_MASK); + } + + client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), + AbfsAclHelper.serializeAclSpec(aclEntries), eTag); + } + + public void removeAclEntries(final Path path, final List aclSpec) throws AzureBlobFileSystemException { + if (!getIsNamespaceEnabled()) { + throw new UnsupportedOperationException( + "This operation is only valid for storage accounts with the hierarchical namespace enabled."); + } + + LOG.debug( + "removeAclEntries filesystem: {} path: {} aclSpec: {}", + client.getFileSystem(), + path.toString(), + AclEntry.aclSpecToString(aclSpec)); + + final Map removeAclEntries = AbfsAclHelper.deserializeAclSpec(AclEntry.aclSpecToString(aclSpec)); + final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true)); + final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + + final Map aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); + + AbfsAclHelper.removeAclEntriesInternal(aclEntries, removeAclEntries); + + client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), + AbfsAclHelper.serializeAclSpec(aclEntries), eTag); + } + + public void removeDefaultAcl(final Path path) throws AzureBlobFileSystemException { + if (!getIsNamespaceEnabled()) { + throw new UnsupportedOperationException( + "This operation is only valid for storage accounts with the hierarchical namespace enabled."); + } + + LOG.debug( + "removeDefaultAcl filesystem: {} path: {}", + client.getFileSystem(), + path.toString()); + + final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true)); + final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + final Map aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); + final Map defaultAclEntries = new HashMap<>(); + + for (Map.Entry aclEntry : aclEntries.entrySet()) { + if (aclEntry.getKey().startsWith("default:")) { + defaultAclEntries.put(aclEntry.getKey(), aclEntry.getValue()); + } + } + + for (Map.Entry defaultAclEntry : defaultAclEntries.entrySet()) { + aclEntries.remove(defaultAclEntry.getKey()); + } + + client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), + AbfsAclHelper.serializeAclSpec(aclEntries), eTag); + } + + public void removeAcl(final Path path) throws AzureBlobFileSystemException { + if (!getIsNamespaceEnabled()) { + throw new UnsupportedOperationException( + "This operation is only valid for storage accounts with the hierarchical namespace enabled."); + } + + LOG.debug( + "removeAcl filesystem: {} path: {}", + client.getFileSystem(), + path.toString()); + final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true)); + final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + + final Map aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); + final Map newAclEntries = new HashMap<>(); + + newAclEntries.put(AbfsHttpConstants.ACCESS_USER, aclEntries.get(AbfsHttpConstants.ACCESS_USER)); + newAclEntries.put(AbfsHttpConstants.ACCESS_GROUP, aclEntries.get(AbfsHttpConstants.ACCESS_GROUP)); + newAclEntries.put(AbfsHttpConstants.ACCESS_OTHER, aclEntries.get(AbfsHttpConstants.ACCESS_OTHER)); + + client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), + AbfsAclHelper.serializeAclSpec(newAclEntries), eTag); + } + + public void setAcl(final Path path, final List aclSpec) throws AzureBlobFileSystemException { + if (!getIsNamespaceEnabled()) { + throw new UnsupportedOperationException( + "This operation is only valid for storage accounts with the hierarchical namespace enabled."); + } + + LOG.debug( + "setAcl filesystem: {} path: {} aclspec: {}", + client.getFileSystem(), + path.toString(), + AclEntry.aclSpecToString(aclSpec)); + final Map aclEntries = AbfsAclHelper.deserializeAclSpec(AclEntry.aclSpecToString(aclSpec)); + final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true)); + final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG); + + final Map getAclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL)); + for (Map.Entry ace : getAclEntries.entrySet()) { + if (ace.getKey().startsWith("default:") && (ace.getKey() != AbfsHttpConstants.DEFAULT_MASK) + && !aclEntries.containsKey(ace.getKey())) { + aclEntries.put(ace.getKey(), ace.getValue()); + } + } + + client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), + AbfsAclHelper.serializeAclSpec(aclEntries), eTag); + } + + public AclStatus getAclStatus(final Path path) throws IOException { + if (!getIsNamespaceEnabled()) { + throw new UnsupportedOperationException( + "This operation is only valid for storage accounts with the hierarchical namespace enabled."); + } + + LOG.debug( + "getAclStatus filesystem: {} path: {}", + client.getFileSystem(), + path.toString()); + AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true)); + AbfsHttpOperation result = op.getResult(); + + final String owner = result.getResponseHeader(HttpHeaderConfigurations.X_MS_OWNER); + final String group = result.getResponseHeader(HttpHeaderConfigurations.X_MS_GROUP); + final String permissions = result.getResponseHeader(HttpHeaderConfigurations.X_MS_PERMISSIONS); + final String aclSpecString = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL); + + final List processedAclEntries = AclEntry.parseAclSpec(AbfsAclHelper.processAclString(aclSpecString), true); + final FsPermission fsPermission = permissions == null ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL) + : AbfsPermission.valueOf(permissions); + + final AclStatus.Builder aclStatusBuilder = new AclStatus.Builder(); + aclStatusBuilder.owner(owner == null ? userGroupInformation.getUserName() : owner); + aclStatusBuilder.group(group == null ? userGroupInformation.getPrimaryGroupName() : group); + + aclStatusBuilder.setPermission(fsPermission); + aclStatusBuilder.stickyBit(fsPermission.getStickyBit()); + aclStatusBuilder.addEntries(processedAclEntries); + return aclStatusBuilder.build(); + } + public boolean isAtomicRenameKey(String key) { return isKeyForDirectorySet(key, azureAtomicRenameDirSet); } @@ -507,19 +776,24 @@ public class AzureBlobFileSystemStore { this.client = new AbfsClient(baseUrl, creds, abfsConfiguration, new ExponentialRetryPolicy(), tokenProvider); } + private String getOctalNotation(FsPermission fsPermission) { + Preconditions.checkNotNull(fsPermission, "fsPermission"); + return String.format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal()); + } + private String getRelativePath(final Path path) { + return getRelativePath(path, false); + } + + private String getRelativePath(final Path path, final boolean allowRootPath) { Preconditions.checkNotNull(path, "path"); final String relativePath = path.toUri().getPath(); - if (relativePath.isEmpty()) { - return relativePath; + if (relativePath.length() == 0 || (relativePath.length() == 1 && relativePath.charAt(0) == Path.SEPARATOR_CHAR)) { + return allowRootPath ? AbfsHttpConstants.ROOT_PATH : AbfsHttpConstants.EMPTY_STRING; } if (relativePath.charAt(0) == Path.SEPARATOR_CHAR) { - if (relativePath.length() == 1) { - return AbfsHttpConstants.EMPTY_STRING; - } - return relativePath.substring(1); } @@ -644,15 +918,17 @@ public class AzureBlobFileSystemStore { private final String version; VersionedFileStatus( - final String owner, final String group, + final String owner, final String group, final FsPermission fsPermission, final boolean hasAcl, final long length, final boolean isdir, final int blockReplication, final long blocksize, final long modificationTime, final Path path, String version) { super(length, isdir, blockReplication, blocksize, modificationTime, 0, - new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL), + fsPermission, owner, group, - path); + null, + path, + hasAcl, false, false); this.version = version; } @@ -717,5 +993,4 @@ public class AzureBlobFileSystemStore { AbfsClient getClient() { return this.client; } - } 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 f80bc605cb6..447b6819e3b 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 @@ -34,6 +34,8 @@ public final class AbfsHttpConstants { public static final String APPEND_ACTION = "append"; public static final String FLUSH_ACTION = "flush"; public static final String SET_PROPERTIES_ACTION = "setProperties"; + public static final String SET_ACCESS_CONTROL = "setAccessControl"; + public static final String GET_ACCESS_CONTROL = "getAccessControl"; public static final String DEFAULT_TIMEOUT = "90"; public static final String JAVA_VERSION = "java.version"; @@ -58,6 +60,7 @@ public final class AbfsHttpConstants { public static final String PLUS = "+"; public static final String STAR = "*"; public static final String COMMA = ","; + public static final String COLON = ":"; public static final String EQUAL = "="; public static final String QUESTION_MARK = "?"; public static final String AND_MARK = "&"; @@ -72,5 +75,17 @@ public final class AbfsHttpConstants { public static final String APPLICATION_JSON = "application/json"; public static final String APPLICATION_OCTET_STREAM = "application/octet-stream"; + public static final String ROOT_PATH = "/"; + public static final String ACCESS_MASK = "mask:"; + public static final String ACCESS_USER = "user:"; + public static final String ACCESS_GROUP = "group:"; + public static final String ACCESS_OTHER = "other:"; + public static final String DEFAULT_MASK = "default:mask:"; + public static final String DEFAULT_USER = "default:user:"; + public static final String DEFAULT_GROUP = "default:group:"; + public static final String DEFAULT_OTHER = "default:other:"; + public static final String DEFAULT_SCOPE = "default:"; + public static final String PERMISSION_FORMAT = "%04d"; + private AbfsHttpConstants() {} } 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 4603b5fd03e..c8d43904deb 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 @@ -52,6 +52,12 @@ public final class HttpHeaderConfigurations { public static final String X_MS_PROPERTIES = "x-ms-properties"; public static final String X_MS_RENAME_SOURCE = "x-ms-rename-source"; public static final String LAST_MODIFIED = "Last-Modified"; + public static final String X_MS_OWNER = "x-ms-owner"; + public static final String X_MS_GROUP = "x-ms-group"; + public static final String X_MS_ACL = "x-ms-acl"; + public static final String X_MS_PERMISSIONS = "x-ms-permissions"; + public static final String X_MS_UMASK = "x-ms-umask"; + public static final String X_MS_NAMESPACE_ENABLED = "x-ms-namespace-enabled"; private HttpHeaderConfigurations() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAclOperationException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAclOperationException.java new file mode 100644 index 00000000000..9c186baab9d --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAclOperationException.java @@ -0,0 +1,33 @@ +/** + * 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.contracts.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Thrown when there is an attempt to perform an invalid operation on an ACL. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class InvalidAclOperationException extends AzureBlobFileSystemException { + public InvalidAclOperationException(String message) { + super(message); + } +} \ No newline at end of file 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 63bf8d03fda..60e7f92d270 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 @@ -33,6 +33,7 @@ import org.apache.hadoop.classification.InterfaceStability; public enum AzureServiceErrorCode { FILE_SYSTEM_ALREADY_EXISTS("FilesystemAlreadyExists", HttpURLConnection.HTTP_CONFLICT, null), PATH_ALREADY_EXISTS("PathAlreadyExists", HttpURLConnection.HTTP_CONFLICT, null), + INTERNAL_OPERATION_ABORT("InternalOperationAbortError", HttpURLConnection.HTTP_CONFLICT, null), PATH_CONFLICT("PathConflict", HttpURLConnection.HTTP_CONFLICT, null), FILE_SYSTEM_NOT_FOUND("FilesystemNotFound", HttpURLConnection.HTTP_NOT_FOUND, null), PATH_NOT_FOUND("PathNotFound", HttpURLConnection.HTTP_NOT_FOUND, null), diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsAclHelper.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsAclHelper.java new file mode 100644 index 00000000000..c28da2c4b4f --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsAclHelper.java @@ -0,0 +1,202 @@ +/** + * 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.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAclOperationException; +import org.apache.hadoop.fs.permission.FsAction; + +/** + * AbfsAclHelper provides convenience methods to implement modifyAclEntries / removeAclEntries / removeAcl / removeDefaultAcl + * from setAcl and getAcl. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public final class AbfsAclHelper { + + private AbfsAclHelper() { + // not called + } + + public static Map deserializeAclSpec(final String aclSpecString) { + final Map aclEntries = new HashMap<>(); + final String[] aclArray = aclSpecString.split(AbfsHttpConstants.COMMA); + for (String acl : aclArray) { + int idx = acl.lastIndexOf(AbfsHttpConstants.COLON); + aclEntries.put(acl.substring(0, idx), acl.substring(idx + 1)); + } + return aclEntries; + } + + public static String serializeAclSpec(final Map aclEntries) { + final StringBuilder sb = new StringBuilder(); + for (Map.Entry aclEntry : aclEntries.entrySet()) { + sb.append(aclEntry.getKey() + AbfsHttpConstants.COLON + aclEntry.getValue() + AbfsHttpConstants.COMMA); + } + if (sb.length() > 0) { + sb.setLength(sb.length() - 1); + } + return sb.toString(); + } + + public static String processAclString(final String aclSpecString) { + final List aclEntries = Arrays.asList(aclSpecString.split(AbfsHttpConstants.COMMA)); + final StringBuilder sb = new StringBuilder(); + + boolean containsMask = false; + for (int i = aclEntries.size() - 1; i >= 0; i--) { + String ace = aclEntries.get(i); + if (ace.startsWith(AbfsHttpConstants.ACCESS_OTHER)|| ace.startsWith(AbfsHttpConstants.ACCESS_USER + AbfsHttpConstants.COLON)) { + // skip + } else if (ace.startsWith(AbfsHttpConstants.ACCESS_MASK)) { + containsMask = true; + // skip + } else if (ace.startsWith(AbfsHttpConstants.ACCESS_GROUP + AbfsHttpConstants.COLON) && !containsMask) { + // skip + } else { + sb.insert(0, ace + AbfsHttpConstants.COMMA); + } + } + + return sb.length() == 0 ? AbfsHttpConstants.EMPTY_STRING : sb.substring(0, sb.length() - 1); + } + + public static void removeAclEntriesInternal(Map aclEntries, Map toRemoveEntries) + throws AzureBlobFileSystemException { + boolean accessAclTouched = false; + boolean defaultAclTouched = false; + + final Set removeIndicationSet = new HashSet<>(); + + for (String entryKey : toRemoveEntries.keySet()) { + final boolean isDefaultAcl = isDefaultAce(entryKey); + if (removeNamedAceAndUpdateSet(entryKey, isDefaultAcl, removeIndicationSet, aclEntries)) { + if (isDefaultAcl) { + defaultAclTouched = true; + } else { + accessAclTouched = true; + } + } + } + if (accessAclTouched) { + if (removeIndicationSet.contains(AbfsHttpConstants.ACCESS_MASK)) { + aclEntries.remove(AbfsHttpConstants.ACCESS_MASK); + } + recalculateMask(aclEntries, false); + } + if (defaultAclTouched) { + if (removeIndicationSet.contains(AbfsHttpConstants.DEFAULT_MASK)) { + aclEntries.remove(AbfsHttpConstants.DEFAULT_MASK); + } + if (removeIndicationSet.contains(AbfsHttpConstants.DEFAULT_USER)) { + aclEntries.put(AbfsHttpConstants.DEFAULT_USER, aclEntries.get(AbfsHttpConstants.ACCESS_USER)); + } + if (removeIndicationSet.contains(AbfsHttpConstants.DEFAULT_GROUP)) { + aclEntries.put(AbfsHttpConstants.DEFAULT_GROUP, aclEntries.get(AbfsHttpConstants.ACCESS_GROUP)); + } + if (removeIndicationSet.contains(AbfsHttpConstants.DEFAULT_OTHER)) { + aclEntries.put(AbfsHttpConstants.DEFAULT_OTHER, aclEntries.get(AbfsHttpConstants.ACCESS_OTHER)); + } + recalculateMask(aclEntries, true); + } + } + + private static boolean removeNamedAceAndUpdateSet(String entry, boolean isDefaultAcl, Set removeIndicationSet, + Map aclEntries) + throws AzureBlobFileSystemException { + final int startIndex = isDefaultAcl ? 1 : 0; + final String[] entryParts = entry.split(AbfsHttpConstants.COLON); + final String tag = isDefaultAcl ? AbfsHttpConstants.DEFAULT_SCOPE + entryParts[startIndex] + AbfsHttpConstants.COLON + : entryParts[startIndex] + AbfsHttpConstants.COLON; + + if ((entry.equals(AbfsHttpConstants.ACCESS_USER) || entry.equals(AbfsHttpConstants.ACCESS_GROUP) + || entry.equals(AbfsHttpConstants.ACCESS_OTHER)) + && !isNamedAce(entry)) { + throw new InvalidAclOperationException("Cannot remove user, group or other entry from access ACL."); + } + + boolean touched = false; + if (!isNamedAce(entry)) { + removeIndicationSet.add(tag); // this must not be a access user, group or other + touched = true; + } else { + if (aclEntries.remove(entry) != null) { + touched = true; + } + } + return touched; + } + + private static void recalculateMask(Map aclEntries, boolean isDefaultMask) { + FsAction umask = FsAction.NONE; + if (!isExtendAcl(aclEntries, isDefaultMask)) { + return; + } + + for (Map.Entry aclEntry : aclEntries.entrySet()) { + if (isDefaultMask) { + if ((isDefaultAce(aclEntry.getKey()) && isNamedAce(aclEntry.getKey())) + || aclEntry.getKey().equals(AbfsHttpConstants.DEFAULT_GROUP)) { + umask = umask.or(FsAction.getFsAction(aclEntry.getValue())); + } + } else { + if ((!isDefaultAce(aclEntry.getKey()) && isNamedAce(aclEntry.getKey())) + || aclEntry.getKey().equals(AbfsHttpConstants.ACCESS_GROUP)) { + umask = umask.or(FsAction.getFsAction(aclEntry.getValue())); + } + } + } + + aclEntries.put(isDefaultMask ? AbfsHttpConstants.DEFAULT_MASK : AbfsHttpConstants.ACCESS_MASK, umask.SYMBOL); + } + + private static boolean isExtendAcl(Map aclEntries, boolean checkDefault) { + for (String entryKey : aclEntries.keySet()) { + if (checkDefault && !(entryKey.equals(AbfsHttpConstants.DEFAULT_USER) + || entryKey.equals(AbfsHttpConstants.DEFAULT_GROUP) + || entryKey.equals(AbfsHttpConstants.DEFAULT_OTHER) || !isDefaultAce(entryKey))) { + return true; + } + if (!checkDefault && !(entryKey.equals(AbfsHttpConstants.ACCESS_USER) + || entryKey.equals(AbfsHttpConstants.ACCESS_GROUP) + || entryKey.equals(AbfsHttpConstants.ACCESS_OTHER) || isDefaultAce(entryKey))) { + return true; + } + } + return false; + } + + private static boolean isDefaultAce(String entry) { + return entry.startsWith(AbfsHttpConstants.DEFAULT_SCOPE); + } + + private static boolean isNamedAce(String entry) { + return entry.charAt(entry.length() - 1) != AbfsHttpConstants.COLON.charAt(0); + } +} \ No newline at end of file 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 f5c9f18fc61..18773b6881b 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 @@ -29,6 +29,9 @@ import java.util.Locale; import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -159,7 +162,8 @@ public class AbfsClient { final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_DIRECTORY, relativePath == null ? "" : relativePath); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_DIRECTORY, relativePath == null ? AbfsHttpConstants.EMPTY_STRING + : relativePath); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RECURSIVE, String.valueOf(recursive)); abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation); abfsUriQueryBuilder.addQuery(QUERY_PARAM_MAXRESULTS, String.valueOf(listMaxResults)); @@ -206,11 +210,19 @@ public class AbfsClient { return op; } - public AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite) - throws AzureBlobFileSystemException { + public AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite, + final String permission, final String umask) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); if (!overwrite) { - requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, "*")); + requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, AbfsHttpConstants.STAR)); + } + + if (permission != null && !permission.isEmpty()) { + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PERMISSIONS, permission)); + } + + if (umask != null && !umask.isEmpty()) { + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_UMASK, umask)); } final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); @@ -269,7 +281,6 @@ public class AbfsClient { return op; } - public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); @@ -373,6 +384,104 @@ public class AbfsClient { return op; } + public AbfsRestOperation setOwner(final String path, final String owner, final String group) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + // JDK7 does not support PATCH, so to workaround the issue we will use + // PUT and specify the real method in the X-Http-Method-Override header. + requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, + HTTP_METHOD_PATCH)); + + if (owner != null && !owner.isEmpty()) { + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_OWNER, owner)); + } + if (group != null && !group.isEmpty()) { + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_GROUP, group)); + } + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_ACCESS_CONTROL); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + this, + AbfsHttpConstants.HTTP_METHOD_PUT, + url, + requestHeaders); + op.execute(); + return op; + } + + public AbfsRestOperation setPermission(final String path, final String permission) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + // JDK7 does not support PATCH, so to workaround the issue we will use + // PUT and specify the real method in the X-Http-Method-Override header. + requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, + HTTP_METHOD_PATCH)); + + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PERMISSIONS, permission)); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_ACCESS_CONTROL); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + this, + AbfsHttpConstants.HTTP_METHOD_PUT, + url, + requestHeaders); + op.execute(); + return op; + } + + public AbfsRestOperation setAcl(final String path, final String aclSpecString) throws AzureBlobFileSystemException { + return setAcl(path, aclSpecString, AbfsHttpConstants.EMPTY_STRING); + } + + public AbfsRestOperation setAcl(final String path, final String aclSpecString, final String eTag) + throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + // JDK7 does not support PATCH, so to workaround the issue we will use + // PUT and specify the real method in the X-Http-Method-Override header. + requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, + HTTP_METHOD_PATCH)); + + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_ACL, aclSpecString)); + + if (eTag != null && !eTag.isEmpty()) { + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_MATCH, eTag)); + } + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_ACCESS_CONTROL); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + this, + AbfsHttpConstants.HTTP_METHOD_PUT, + url, + requestHeaders); + op.execute(); + return op; + } + + public AbfsRestOperation getAclStatus(final String path) throws AzureBlobFileSystemException { + final List requestHeaders = createDefaultHeaders(); + + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.GET_ACCESS_CONTROL); + + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final AbfsRestOperation op = new AbfsRestOperation( + this, + AbfsHttpConstants.HTTP_METHOD_HEAD, + url, + requestHeaders); + op.execute(); + return op; + } + private URL createRequestUrl(final String query) throws AzureBlobFileSystemException { return createRequestUrl(EMPTY_STRING, query); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsPermission.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsPermission.java new file mode 100644 index 00000000000..9c44610ff59 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsPermission.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 org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.permission.FsPermission; + +/** + * The AbfsPermission for AbfsClient. + */ +public class AbfsPermission extends FsPermission { + private static final int STICKY_BIT_OCTAL_VALUE = 01000; + private final boolean aclBit; + + public AbfsPermission(Short aShort, boolean aclBitStatus) { + super(aShort); + this.aclBit = aclBitStatus; + } + + public AbfsPermission(FsAction u, FsAction g, FsAction o) { + super(u, g, o, false); + this.aclBit = false; + } + + /** + * Returns true if there is also an ACL (access control list). + * + * @return boolean true if there is also an ACL (access control list). + * @deprecated Get acl bit from the {@link org.apache.hadoop.fs.FileStatus} + * object. + */ + public boolean getAclBit() { + return aclBit; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof FsPermission) { + FsPermission that = (FsPermission) obj; + return this.getUserAction() == that.getUserAction() + && this.getGroupAction() == that.getGroupAction() + && this.getOtherAction() == that.getOtherAction() + && this.getStickyBit() == that.getStickyBit(); + } + return false; + } + + /** + * Create a AbfsPermission from a abfs symbolic permission string + * @param abfsSymbolicPermission e.g. "rw-rw-rw-+" / "rw-rw-rw-" + * @return a permission object for the provided string representation + */ + public static AbfsPermission valueOf(final String abfsSymbolicPermission) { + if (abfsSymbolicPermission == null) { + return null; + } + + final boolean isExtendedAcl = abfsSymbolicPermission.charAt(abfsSymbolicPermission.length() - 1) == '+'; + + final String abfsRawSymbolicPermission = isExtendedAcl ? abfsSymbolicPermission.substring(0, abfsSymbolicPermission.length() - 1) + : abfsSymbolicPermission; + + int n = 0; + for (int i = 0; i < abfsRawSymbolicPermission.length(); i++) { + n = n << 1; + char c = abfsRawSymbolicPermission.charAt(i); + n += (c == '-' || c == 'T' || c == 'S') ? 0: 1; + } + + // Add sticky bit value if set + if (abfsRawSymbolicPermission.charAt(abfsRawSymbolicPermission.length() - 1) == 't' + || abfsRawSymbolicPermission.charAt(abfsRawSymbolicPermission.length() - 1) == 'T') { + n += STICKY_BIT_OCTAL_VALUE; + } + + return new AbfsPermission((short) n, isExtendedAcl); + } + + /** + * Check whether abfs symbolic permission string is a extended Acl + * @param abfsSymbolicPermission e.g. "rw-rw-rw-+" / "rw-rw-rw-" + * @return true if the permission string indicates the existence of an + * extended ACL; otherwise false. + */ + public static boolean isExtendedAcl(final String abfsSymbolicPermission) { + if (abfsSymbolicPermission == null) { + return false; + } + + return abfsSymbolicPermission.charAt(abfsSymbolicPermission.length() - 1) == '+'; + } + + @Override + public int hashCode() { + return toShort(); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java index 6207a47a452..800b95abb34 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java @@ -43,6 +43,8 @@ public class ITestAzureBlobFileSystemBackCompat extends @Test public void testBlobBackCompat() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + // test only valid for non-namespace enabled account + Assume.assumeFalse(fs.getIsNamespaceEnabeld()); String storageConnectionString = getBlobConnectionString(); CloudStorageAccount storageAccount = CloudStorageAccount.parse(storageConnectionString); CloudBlobClient blobClient = storageAccount.createCloudBlobClient(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java index 13abaf88bd9..9daac2ae516 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,11 +20,12 @@ package org.apache.hadoop.fs.azurebfs; import java.io.IOException; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.junit.Test; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; /** @@ -32,9 +33,17 @@ import org.apache.hadoop.fs.permission.FsPermission; */ public class ITestAzureBlobFileSystemFileStatus extends AbstractAbfsIntegrationTest { + private static final String DEFAULT_FILE_PERMISSION_VALUE = "640"; + private static final String DEFAULT_DIR_PERMISSION_VALUE = "750"; + private static final String DEFAULT_UMASK_VALUE = "027"; + private static final Path TEST_FILE = new Path("testFile"); private static final Path TEST_FOLDER = new Path("testDir"); + public ITestAzureBlobFileSystemFileStatus() { + super(); + } + @Test public void testEnsureStatusWorksForRoot() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); @@ -48,20 +57,32 @@ public class ITestAzureBlobFileSystemFileStatus extends public void testFileStatusPermissionsAndOwnerAndGroup() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); touch(TEST_FILE); - validateStatus(fs, TEST_FILE); + validateStatus(fs, TEST_FILE, false); } - private FileStatus validateStatus(final AzureBlobFileSystem fs, final Path name) + private FileStatus validateStatus(final AzureBlobFileSystem fs, final Path name, final boolean isDir) throws IOException { FileStatus fileStatus = fs.getFileStatus(name); + fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, DEFAULT_UMASK_VALUE); + String errorInStatus = "error in " + fileStatus + " from " + fs; - assertEquals(errorInStatus + ": permission", - new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL), - fileStatus.getPermission()); - assertEquals(errorInStatus + ": owner", - fs.getOwnerUser(), fileStatus.getOwner()); - assertEquals(errorInStatus + ": group", - fs.getOwnerUserPrimaryGroup(), fileStatus.getGroup()); + + // When running with Oauth, the owner and group info retrieved from server will be digit ids. + if (this.getAuthType() != AuthType.OAuth && !fs.isSecure()) { + assertEquals(errorInStatus + ": owner", + fs.getOwnerUser(), fileStatus.getOwner()); + assertEquals(errorInStatus + ": group", + fs.getOwnerUserPrimaryGroup(), fileStatus.getGroup()); + } else { + if (isDir) { + assertEquals(errorInStatus + ": permission", + new FsPermission(DEFAULT_DIR_PERMISSION_VALUE), fileStatus.getPermission()); + } else { + assertEquals(errorInStatus + ": permission", + new FsPermission(DEFAULT_FILE_PERMISSION_VALUE), fileStatus.getPermission()); + } + } + return fileStatus; } @@ -70,7 +91,7 @@ public class ITestAzureBlobFileSystemFileStatus extends final AzureBlobFileSystem fs = this.getFileSystem(); fs.mkdirs(TEST_FOLDER); - validateStatus(fs, TEST_FOLDER); + validateStatus(fs, TEST_FOLDER, true); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index b02d723c9ca..8a6207a0109 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -218,6 +218,9 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { String wasbUrl = testAccount.getFileSystem().getName(); String abfsUrl = wasbUrlToAbfsUrl(wasbUrl); final AzureBlobFileSystem fs = this.getFileSystem(abfsUrl); + // test only valid for non-namespace enabled account + Assume.assumeFalse(fs.getIsNamespaceEnabeld()); + byte[] buffer = getRandomBytesArray(); CloudBlockBlob blob = testAccount.getBlobReference(TEST_FILE_PATH.toString().substring(1)); try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, true)) { @@ -238,6 +241,9 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { String wasbUrl = testAccount.getFileSystem().getName(); String abfsUrl = wasbUrlToAbfsUrl(wasbUrl); final AzureBlobFileSystem fs = this.getFileSystem(abfsUrl); + // test only valid for non-namespace enabled account + Assume.assumeFalse(fs.getIsNamespaceEnabeld()); + byte[] buffer = getRandomBytesArray(); CloudBlockBlob blob = testAccount.getBlobReference(TEST_FILE_PATH.toString().substring(1)); try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, false)) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java new file mode 100644 index 00000000000..2f265d1865b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java @@ -0,0 +1,109 @@ +/** + * 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.ArrayList; +import java.util.Collection; +import java.util.UUID; + +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.services.AuthType; +import org.junit.Assert; +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.permission.FsAction; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.azurebfs.utils.Parallelized; + +/** + * Test permission operations. + */ +@RunWith(Parallelized.class) +public class ITestAzureBlobFileSystemPermission extends AbstractAbfsIntegrationTest{ + + private static Path testRoot = new Path("/test"); + private static final String DEFAULT_UMASK_VALUE = "027"; + private static final FsPermission DEFAULT_UMASK_PERMISSION = new FsPermission(DEFAULT_UMASK_VALUE); + private static final int KILOBYTE = 1024; + private FsPermission permission; + + private Path path; + + public ITestAzureBlobFileSystemPermission(FsPermission testPermission) throws Exception { + super(); + permission = testPermission; + + Assume.assumeTrue(this.getAuthType() == AuthType.OAuth); + } + + @Parameterized.Parameters(name = "{0}") + public static Collection abfsCreateNonRecursiveTestData() + throws Exception { + /* + Test Data + File/Folder name, User permission, Group permission, Other Permission, + Parent already exist + shouldCreateSucceed, expectedExceptionIfFileCreateFails + */ + final Collection datas = new ArrayList<>(); + for (FsAction g : FsAction.values()) { + for (FsAction o : FsAction.values()) { + datas.add(new Object[] {new FsPermission(FsAction.ALL, g, o)}); + } + } + return datas; + } + + @Test + public void testFilePermission() throws Exception { + + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, DEFAULT_UMASK_VALUE); + path = new Path(testRoot, UUID.randomUUID().toString()); + + fs.mkdirs(path.getParent(), + new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE)); + fs.removeDefaultAcl(path.getParent()); + + fs.create(path, permission, true, KILOBYTE, (short) 1, KILOBYTE - 1, null); + FileStatus status = fs.getFileStatus(path); + Assert.assertEquals(permission.applyUMask(DEFAULT_UMASK_PERMISSION), status.getPermission()); + } + + @Test + public void testFolderPermission() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "027"); + + path = new Path(testRoot, UUID.randomUUID().toString()); + + fs.mkdirs(path.getParent(), + new FsPermission(FsAction.ALL, FsAction.WRITE, FsAction.NONE)); + fs.removeDefaultAcl(path.getParent()); + + fs.mkdirs(path, permission); + FileStatus status = fs.getFileStatus(path); + Assert.assertEquals(permission.applyUMask(DEFAULT_UMASK_PERMISSION), status.getPermission()); + } +} \ No newline at end of file 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 13c5bc8f31f..38e7133ed8e 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 @@ -17,7 +17,6 @@ */ package org.apache.hadoop.fs.azurebfs; - import java.io.EOFException; import java.io.IOException; import java.util.Random; @@ -524,6 +523,9 @@ public class ITestAzureBlobFileSystemRandomRead extends } private void createTestFile() throws Exception { + final AzureBlobFileSystem abFs = this.getFileSystem(); + // test only valid for non-namespace enabled account + Assume.assumeFalse(abFs.getIsNamespaceEnabeld()); FileSystem fs = this.getWasbFileSystem(); if (fs.exists(TEST_FILE_PATH)) { 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 07426c4ab22..c97e84052d4 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 @@ -25,6 +25,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import org.junit.Assert; import org.junit.Test; import org.apache.hadoop.fs.FileStatus; @@ -133,4 +134,17 @@ public class ITestAzureBlobFileSystemRename extends new Path(fs.getUri().toString() + "/s"), false); } + + @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"))); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java new file mode 100644 index 00000000000..a13b73e2724 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java @@ -0,0 +1,1071 @@ +/** + * 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 com.google.common.collect.Lists; + +import java.io.FileNotFoundException; +import java.util.List; +import java.util.UUID; + +import org.apache.hadoop.fs.azurebfs.services.AuthType; +import org.junit.Assume; +import org.junit.Ignore; +import org.junit.Test; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers; +import org.apache.hadoop.fs.permission.AclEntry; +import org.apache.hadoop.fs.permission.AclStatus; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.permission.FsPermission; + +import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS; +import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT; +import static org.apache.hadoop.fs.permission.AclEntryType.USER; +import static org.apache.hadoop.fs.permission.AclEntryType.GROUP; +import static org.apache.hadoop.fs.permission.AclEntryType.OTHER; +import static org.apache.hadoop.fs.permission.AclEntryType.MASK; +import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry; + +/** + * Test acl operations. + */ +public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { + private static final FsAction ALL = FsAction.ALL; + private static final FsAction NONE = FsAction.NONE; + private static final FsAction READ = FsAction.READ; + private static final FsAction READ_EXECUTE = FsAction.READ_EXECUTE; + private static final FsAction READ_WRITE = FsAction.READ_WRITE; + private static Path testRoot = new Path("/test"); + private Path path; + + public ITestAzureBlobFilesystemAcl() throws Exception { + super(); + + Assume.assumeTrue(this.getAuthType() == AuthType.OAuth); + } + + @Test + public void testModifyAclEntries() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + fs.mkdirs(path, FsPermission.createImmutable((short) 0750)); + + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE), + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + + aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, "foo", READ_EXECUTE), + aclEntry(DEFAULT, USER, "foo", READ_EXECUTE)); + fs.modifyAclEntries(path, aclSpec); + + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", READ_EXECUTE), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, USER, "foo", READ_EXECUTE), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, MASK, READ_EXECUTE), + aclEntry(DEFAULT, OTHER, NONE) }, returned); + assertPermission(fs, (short) 0750); + } + + @Test + public void testModifyAclEntriesOnlyAccess() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + fs.create(path).close(); + fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE)); + fs.setAcl(path, aclSpec); + aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, "foo", READ_EXECUTE)); + fs.modifyAclEntries(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", READ_EXECUTE), + aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned); + assertPermission(fs, (short) 0750); + } + + @Test + public void testModifyAclEntriesOnlyDefault() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, "foo", READ_EXECUTE)); + fs.modifyAclEntries(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, USER, "foo", READ_EXECUTE), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, MASK, READ_EXECUTE), + aclEntry(DEFAULT, OTHER, NONE) }, returned); + assertPermission(fs, (short) 0750); + } + + @Test + public void testModifyAclEntriesMinimal() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + fs.create(path).close(); + fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, "foo", READ_WRITE)); + fs.modifyAclEntries(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", READ_WRITE), + aclEntry(ACCESS, GROUP, READ) }, returned); + assertPermission(fs, (short) 0660); + } + + @Test + public void testModifyAclEntriesMinimalDefault() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, OTHER, NONE)); + fs.modifyAclEntries(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, OTHER, NONE) }, returned); + assertPermission(fs, (short) 0750); + } + + @Test + public void testModifyAclEntriesCustomMask() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + fs.create(path).close(); + fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, MASK, NONE)); + fs.modifyAclEntries(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ) }, returned); + assertPermission(fs, (short) 0600); + } + + @Test + public void testModifyAclEntriesStickyBit() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE), + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, "foo", READ_EXECUTE), + aclEntry(DEFAULT, USER, "foo", READ_EXECUTE)); + fs.modifyAclEntries(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", READ_EXECUTE), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, USER, "foo", READ_EXECUTE), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, MASK, READ_EXECUTE), + aclEntry(DEFAULT, OTHER, NONE) }, returned); + assertPermission(fs, (short) 01750); + } + + @Test(expected=FileNotFoundException.class) + public void testModifyAclEntriesPathNotFound() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + // Path has not been created. + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE)); + fs.modifyAclEntries(path, aclSpec); + } + + @Test (expected=Exception.class) + public void testModifyAclEntriesDefaultOnFile() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + fs.create(path).close(); + fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.modifyAclEntries(path, aclSpec); + } + + @Test + public void testRemoveAclEntries() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE), + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, "foo"), + aclEntry(DEFAULT, USER, "foo")); + fs.removeAclEntries(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, MASK, READ_EXECUTE), + aclEntry(DEFAULT, OTHER, NONE) }, returned); + assertPermission(fs, (short) 0750); + } + + @Test + public void testRemoveAclEntriesOnlyAccess() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + fs.create(path).close(); + fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, USER, "bar", READ_WRITE), + aclEntry(ACCESS, GROUP, READ_WRITE), + aclEntry(ACCESS, OTHER, NONE)); + fs.setAcl(path, aclSpec); + aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, "foo")); + fs.removeAclEntries(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "bar", READ_WRITE), + aclEntry(ACCESS, GROUP, READ_WRITE) }, returned); + assertPermission(fs, (short) 0760); + } + + @Test + public void testRemoveAclEntriesOnlyDefault() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE), + aclEntry(DEFAULT, USER, "foo", ALL), + aclEntry(DEFAULT, USER, "bar", READ_EXECUTE)); + fs.setAcl(path, aclSpec); + aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, "foo")); + fs.removeAclEntries(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, USER, "bar", READ_EXECUTE), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, MASK, READ_EXECUTE), + aclEntry(DEFAULT, OTHER, NONE) }, returned); + assertPermission(fs, (short) 0750); + } + + @Test + public void testRemoveAclEntriesMinimal() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + fs.create(path).close(); + fs.setPermission(path, FsPermission.createImmutable((short) 0760)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_WRITE), + aclEntry(ACCESS, OTHER, NONE)); + fs.setAcl(path, aclSpec); + aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, "foo"), + aclEntry(ACCESS, MASK)); + fs.removeAclEntries(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { }, returned); + assertPermission(fs, (short) 0760); + } + + @Test + public void testRemoveAclEntriesMinimalDefault() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE), + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, "foo"), + aclEntry(ACCESS, MASK), + aclEntry(DEFAULT, USER, "foo"), + aclEntry(DEFAULT, MASK)); + fs.removeAclEntries(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, OTHER, NONE) }, returned); + assertPermission(fs, (short) 0750); + } + + @Test + public void testRemoveAclEntriesStickyBit() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE), + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, "foo"), + aclEntry(DEFAULT, USER, "foo")); + fs.removeAclEntries(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, MASK, READ_EXECUTE), + aclEntry(DEFAULT, OTHER, NONE) }, returned); + assertPermission(fs, (short) 01750); + } + + @Test(expected=FileNotFoundException.class) + public void testRemoveAclEntriesPathNotFound() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + // Path has not been created. + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, "foo")); + fs.removeAclEntries(path, aclSpec); + } + + @Test + public void testRemoveDefaultAcl() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE), + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + fs.removeDefaultAcl(path); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned); + assertPermission(fs, (short) 0770); + } + + @Test + public void testRemoveDefaultAclOnlyAccess() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + fs.create(path).close(); + fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE)); + fs.setAcl(path, aclSpec); + fs.removeDefaultAcl(path); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned); + assertPermission(fs, (short) 0770); + } + + @Test + public void testRemoveDefaultAclOnlyDefault() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + fs.removeDefaultAcl(path); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { }, returned); + assertPermission(fs, (short) 0750); + } + + @Test + public void testRemoveDefaultAclMinimal() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + fs.removeDefaultAcl(path); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { }, returned); + assertPermission(fs, (short) 0750); + } + + @Test + public void testRemoveDefaultAclStickyBit() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE), + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + fs.removeDefaultAcl(path); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned); + assertPermission(fs, (short) 01770); + } + + @Test(expected=FileNotFoundException.class) + public void testRemoveDefaultAclPathNotFound() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + // Path has not been created. + fs.removeDefaultAcl(path); + } + + @Test + public void testRemoveAcl() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE), + aclEntry(DEFAULT, USER, "foo", ALL)); + + fs.setAcl(path, aclSpec); + fs.removeAcl(path); + + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { }, returned); + assertPermission(fs, (short) 0750); + } + + @Test + public void testRemoveAclMinimalAcl() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + fs.create(path).close(); + fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + fs.removeAcl(path); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { }, returned); + assertPermission(fs, (short) 0640); + } + + @Test + public void testRemoveAclStickyBit() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE), + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + fs.removeAcl(path); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { }, returned); + assertPermission(fs, (short) 01750); + } + + @Test + public void testRemoveAclOnlyDefault() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE), + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + fs.removeAcl(path); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { }, returned); + assertPermission(fs, (short) 0750); + } + + @Test(expected=FileNotFoundException.class) + public void testRemoveAclPathNotFound() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + // Path has not been created. + fs.removeAcl(path); + } + + @Test + public void testSetAcl() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE), + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, USER, "foo", ALL), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, MASK, ALL), + aclEntry(DEFAULT, OTHER, NONE) }, returned); + assertPermission(fs, (short) 0770); + } + + @Test + public void testSetAclOnlyAccess() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + fs.create(path).close(); + fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, READ_WRITE), + aclEntry(ACCESS, USER, "foo", READ), + aclEntry(ACCESS, GROUP, READ), + aclEntry(ACCESS, OTHER, NONE)); + fs.setAcl(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", READ), + aclEntry(ACCESS, GROUP, READ) }, returned); + assertPermission(fs, (short) 0640); + } + + @Test + public void testSetAclOnlyDefault() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, USER, "foo", ALL), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, MASK, ALL), + aclEntry(DEFAULT, OTHER, NONE) }, returned); + assertPermission(fs, (short) 0750); + } + + @Test + public void testSetAclMinimal() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + fs.create(path).close(); + fs.setPermission(path, FsPermission.createImmutable((short) 0644)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, READ_WRITE), + aclEntry(ACCESS, USER, "foo", READ), + aclEntry(ACCESS, GROUP, READ), + aclEntry(ACCESS, OTHER, NONE)); + fs.setAcl(path, aclSpec); + aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, READ_WRITE), + aclEntry(ACCESS, GROUP, READ), + aclEntry(ACCESS, OTHER, NONE)); + fs.setAcl(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { }, returned); + assertPermission(fs, (short) 0640); + } + + @Test + public void testSetAclMinimalDefault() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, OTHER, NONE)); + fs.setAcl(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, OTHER, NONE) }, returned); + assertPermission(fs, (short) 0750); + } + + @Test + public void testSetAclCustomMask() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + fs.create(path).close(); + fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, READ_WRITE), + aclEntry(ACCESS, USER, "foo", READ), + aclEntry(ACCESS, GROUP, READ), + aclEntry(ACCESS, MASK, ALL), + aclEntry(ACCESS, OTHER, NONE)); + fs.setAcl(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", READ), + aclEntry(ACCESS, GROUP, READ) }, returned); + assertPermission(fs, (short) 0670); + } + + @Test + public void testSetAclStickyBit() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE), + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, USER, "foo", ALL), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, MASK, ALL), + aclEntry(DEFAULT, OTHER, NONE) }, returned); + assertPermission(fs, (short) 01770); + } + + @Test(expected=FileNotFoundException.class) + public void testSetAclPathNotFound() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + // Path has not been created. + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, READ_WRITE), + aclEntry(ACCESS, USER, "foo", READ), + aclEntry(ACCESS, GROUP, READ), + aclEntry(ACCESS, OTHER, NONE)); + fs.setAcl(path, aclSpec); + } + + @Test(expected=Exception.class) + public void testSetAclDefaultOnFile() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + fs.create(path).close(); + fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + } + + @Test + public void testSetPermission() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE), + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + fs.setPermission(path, FsPermission.createImmutable((short) 0700)); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, USER, "foo", ALL), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, MASK, ALL), + aclEntry(DEFAULT, OTHER, NONE) }, returned); + assertPermission(fs, (short) 0700); + } + + @Test + public void testSetPermissionOnlyAccess() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + fs.create(path).close(); + fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, READ_WRITE), + aclEntry(ACCESS, USER, "foo", READ), + aclEntry(ACCESS, GROUP, READ), + aclEntry(ACCESS, OTHER, NONE)); + fs.setAcl(path, aclSpec); + fs.setPermission(path, FsPermission.createImmutable((short) 0600)); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", READ), + aclEntry(ACCESS, GROUP, READ) }, returned); + assertPermission(fs, (short) 0600); + } + + @Test + public void testSetPermissionOnlyDefault() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(ACCESS, OTHER, NONE), + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + fs.setPermission(path, FsPermission.createImmutable((short) 0700)); + AclStatus s = fs.getAclStatus(path); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, USER, "foo", ALL), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, MASK, ALL), + aclEntry(DEFAULT, OTHER, NONE) }, returned); + assertPermission(fs, (short) 0700); + } + + @Test + public void testDefaultAclNewFile() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + Path filePath = new Path(path, "file1"); + fs.create(filePath).close(); + AclStatus s = fs.getAclStatus(filePath); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned); + assertPermission(fs, filePath, (short) 0640); + } + + @Test + @Ignore // wait umask fix to be deployed + public void testOnlyAccessAclNewFile() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, "foo", ALL)); + fs.modifyAclEntries(path, aclSpec); + Path filePath = new Path(path, "file1"); + fs.create(filePath).close(); + AclStatus s = fs.getAclStatus(filePath); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { }, returned); + assertPermission(fs, filePath, (short) 0644); + } + + @Test + @Ignore // wait investigation in service + public void testDefaultMinimalAclNewFile() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, OTHER, NONE)); + fs.setAcl(path, aclSpec); + Path filePath = new Path(path, "file1"); + fs.create(filePath).close(); + AclStatus s = fs.getAclStatus(filePath); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { }, returned); + assertPermission(fs, filePath, (short) 0640); + } + + @Test + public void testDefaultAclNewDir() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + + Path dirPath = new Path(path, "dir1"); + fs.mkdirs(dirPath); + + AclStatus s = fs.getAclStatus(dirPath); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, USER, "foo", ALL), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, MASK, ALL), + aclEntry(DEFAULT, OTHER, NONE) }, returned); + assertPermission(fs, dirPath, (short) 0770); + } + + @Test + @Ignore // wait umask fix to be deployed + public void testOnlyAccessAclNewDir() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(ACCESS, USER, "foo", ALL)); + fs.modifyAclEntries(path, aclSpec); + Path dirPath = new Path(path, "dir1"); + fs.mkdirs(dirPath); + AclStatus s = fs.getAclStatus(dirPath); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { }, returned); + assertPermission(fs, dirPath, (short) 0755); + } + + @Test + @Ignore // wait investigation in service + public void testDefaultMinimalAclNewDir() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, OTHER, NONE)); + fs.setAcl(path, aclSpec); + Path dirPath = new Path(path, "dir1"); + fs.mkdirs(dirPath); + AclStatus s = fs.getAclStatus(dirPath); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, OTHER, NONE) }, returned); + assertPermission(fs, dirPath, (short) 0750); + } + + @Test + public void testDefaultAclNewFileWithMode() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0755)); + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + Path filePath = new Path(path, "file1"); + int bufferSize = 4 * 1024 * 1024; + fs.create(filePath, new FsPermission((short) 0740), false, bufferSize, + fs.getDefaultReplication(filePath), fs.getDefaultBlockSize(path), null) + .close(); + AclStatus s = fs.getAclStatus(filePath); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned); + assertPermission(fs, filePath, (short) 0740); + } + + @Test + public void testDefaultAclNewDirWithMode() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0755)); + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(path, aclSpec); + Path dirPath = new Path(path, "dir1"); + fs.mkdirs(dirPath, new FsPermission((short) 0740)); + AclStatus s = fs.getAclStatus(dirPath); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(new AclEntry[] { + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, USER, "foo", ALL), + aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, MASK, ALL), + aclEntry(DEFAULT, OTHER, READ_EXECUTE) }, returned); + assertPermission(fs, dirPath, (short) 0740); + } + + @Test + public void testDefaultAclRenamedFile() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + Path dirPath = new Path(path, "dir"); + FileSystem.mkdirs(fs, dirPath, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(dirPath, aclSpec); + Path filePath = new Path(path, "file1"); + fs.create(filePath).close(); + fs.setPermission(filePath, FsPermission.createImmutable((short) 0640)); + Path renamedFilePath = new Path(dirPath, "file1"); + fs.rename(filePath, renamedFilePath); + AclEntry[] expected = new AclEntry[] { }; + AclStatus s = fs.getAclStatus(renamedFilePath); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(expected, returned); + assertPermission(fs, renamedFilePath, (short) 0640); + } + + @Test + public void testDefaultAclRenamedDir() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + path = new Path(testRoot, UUID.randomUUID().toString()); + Path dirPath = new Path(path, "dir"); + FileSystem.mkdirs(fs, dirPath, FsPermission.createImmutable((short) 0750)); + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, USER, "foo", ALL)); + fs.setAcl(dirPath, aclSpec); + Path subdirPath = new Path(path, "subdir"); + FileSystem.mkdirs(fs, subdirPath, FsPermission.createImmutable((short) 0750)); + Path renamedSubdirPath = new Path(dirPath, "subdir"); + fs.rename(subdirPath, renamedSubdirPath); + AclEntry[] expected = new AclEntry[] { }; + AclStatus s = fs.getAclStatus(renamedSubdirPath); + AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); + assertArrayEquals(expected, returned); + assertPermission(fs, renamedSubdirPath, (short) 0750); + } + + @Test + public void testEnsureAclOperationWorksForRoot() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + + Path rootPath = new Path("/"); + + List aclSpec1 = Lists.newArrayList( + aclEntry(DEFAULT, GROUP, "foo", ALL), + aclEntry(ACCESS, GROUP, "bar", ALL)); + fs.setAcl(rootPath, aclSpec1); + fs.getAclStatus(rootPath); + + fs.setOwner(rootPath, "", "testgroup"); + fs.setPermission(rootPath, new FsPermission("777")); + + List aclSpec2 = Lists.newArrayList( + aclEntry(DEFAULT, USER, "foo", ALL), + aclEntry(ACCESS, USER, "bar", ALL)); + fs.modifyAclEntries(rootPath, aclSpec2); + fs.removeAclEntries(rootPath, aclSpec2); + fs.removeDefaultAcl(rootPath); + fs.removeAcl(rootPath); + } + + private void assertPermission(FileSystem fs, short perm) throws Exception { + assertPermission(fs, path, perm); + } + + private void assertPermission(FileSystem fs, Path pathToCheck, short perm) + throws Exception { + AclTestHelpers.assertPermission(fs, pathToCheck, perm); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java index ff28d3e0fdd..e4f3ea3d6aa 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java @@ -58,6 +58,9 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest { public void testListFileStatus() throws Exception { // crate file using abfs AzureBlobFileSystem fs = getFileSystem(); + // test only valid for non-namespace enabled account + Assume.assumeFalse(fs.getIsNamespaceEnabeld()); + NativeAzureFileSystem wasb = getWasbFileSystem(); Path path1 = new Path("/testfiles/~12/!008/3/abFsTestfile"); @@ -89,6 +92,9 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest { boolean[] readFileWithAbfs = new boolean[]{false, true, true, false}; AzureBlobFileSystem abfs = getFileSystem(); + // test only valid for non-namespace enabled account + Assume.assumeFalse(abfs.getIsNamespaceEnabeld()); + NativeAzureFileSystem wasb = getWasbFileSystem(); for (int i = 0; i< 4; i++) { @@ -125,6 +131,9 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest { boolean[] readDirWithAbfs = new boolean[]{false, true, true, false}; AzureBlobFileSystem abfs = getFileSystem(); + // test only valid for non-namespace enabled account + Assume.assumeFalse(abfs.getIsNamespaceEnabeld()); + NativeAzureFileSystem wasb = getWasbFileSystem(); for (int i = 0; i < 4; i++) { @@ -156,6 +165,9 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest { public void testSetWorkingDirectory() throws Exception { //create folders AzureBlobFileSystem abfs = getFileSystem(); + // test only valid for non-namespace enabled account + Assume.assumeFalse(abfs.getIsNamespaceEnabeld()); + NativeAzureFileSystem wasb = getWasbFileSystem(); Path d1d4 = new Path("/d1/d2/d3/d4"); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AclTestHelpers.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AclTestHelpers.java new file mode 100644 index 00000000000..2ec97220492 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AclTestHelpers.java @@ -0,0 +1,119 @@ +/** + * 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.io.IOException; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.AclEntry; +import org.apache.hadoop.fs.permission.AclEntryScope; +import org.apache.hadoop.fs.permission.AclEntryType; +import org.apache.hadoop.fs.permission.FsAction; + +import static org.junit.Assert.assertEquals; + +/** + * Helper methods useful for writing ACL tests. + */ +public final class AclTestHelpers { + + /** + * Create a new AclEntry with scope, type and permission (no name). + * + * @param scope AclEntryScope scope of the ACL entry + * @param type AclEntryType ACL entry type + * @param permission FsAction set of permissions in the ACL entry + * @return AclEntry new AclEntry + */ + public static AclEntry aclEntry(AclEntryScope scope, AclEntryType type, + FsAction permission) { + return new AclEntry.Builder() + .setScope(scope) + .setType(type) + .setPermission(permission) + .build(); + } + + /** + * Create a new AclEntry with scope, type, name and permission. + * + * @param scope AclEntryScope scope of the ACL entry + * @param type AclEntryType ACL entry type + * @param name String optional ACL entry name + * @param permission FsAction set of permissions in the ACL entry + * @return AclEntry new AclEntry + */ + public static AclEntry aclEntry(AclEntryScope scope, AclEntryType type, + String name, FsAction permission) { + return new AclEntry.Builder() + .setScope(scope) + .setType(type) + .setName(name) + .setPermission(permission) + .build(); + } + + /** + * Create a new AclEntry with scope, type and name (no permission). + * + * @param scope AclEntryScope scope of the ACL entry + * @param type AclEntryType ACL entry type + * @param name String optional ACL entry name + * @return AclEntry new AclEntry + */ + public static AclEntry aclEntry(AclEntryScope scope, AclEntryType type, + String name) { + return new AclEntry.Builder() + .setScope(scope) + .setType(type) + .setName(name) + .build(); + } + + /** + * Create a new AclEntry with scope and type (no name or permission). + * + * @param scope AclEntryScope scope of the ACL entry + * @param type AclEntryType ACL entry type + * @return AclEntry new AclEntry + */ + public static AclEntry aclEntry(AclEntryScope scope, AclEntryType type) { + return new AclEntry.Builder() + .setScope(scope) + .setType(type) + .build(); + } + + /** + * Asserts the value of the FsPermission bits on the inode of a specific path. + * + * @param fs FileSystem to use for check + * @param pathToCheck Path inode to check + * @param perm short expected permission bits + * @throws IOException thrown if there is an I/O error + */ + public static void assertPermission(FileSystem fs, Path pathToCheck, + short perm) throws IOException { + assertEquals(perm, fs.getFileStatus(pathToCheck).getPermission().toShort()); + } + + private AclTestHelpers() { + // Not called. + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/Parallelized.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/Parallelized.java new file mode 100644 index 00000000000..994b8ec07b6 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/Parallelized.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.hadoop.fs.azurebfs.utils; + +import org.junit.runners.Parameterized; +import org.junit.runners.model.RunnerScheduler; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +/** + * Provided for convenience to execute parametrized test cases concurrently. + */ +public class Parallelized extends Parameterized { + + public Parallelized(Class classObj) throws Throwable { + super(classObj); + setScheduler(new ThreadPoolScheduler()); + } + + private static class ThreadPoolScheduler implements RunnerScheduler { + private ExecutorService executor; + + ThreadPoolScheduler() { + int numThreads = 10; + executor = Executors.newFixedThreadPool(numThreads); + } + + public void finished() { + executor.shutdown(); + try { + executor.awaitTermination(10, TimeUnit.MINUTES); + } catch (InterruptedException exc) { + throw new RuntimeException(exc); + } + } + + public void schedule(Runnable childStatement) { + executor.submit(childStatement); + } + } +} \ No newline at end of file From 6b6f8cc2bedefc98028d875398ce022edaf77933 Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Thu, 23 Aug 2018 20:43:52 +0000 Subject: [PATCH 09/29] HADOOP 15688. ABFS: InputStream wrapped in FSDataInputStream twice. Contributed by Sean Mackrory. --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 34 +++++++++---------- 1 file changed, 16 insertions(+), 18 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 58df914ec8b..fc60127c51e 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 @@ -19,7 +19,6 @@ package org.apache.hadoop.fs.azurebfs; import java.io.File; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.net.MalformedURLException; import java.net.URI; @@ -50,8 +49,6 @@ import com.google.common.base.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -251,11 +248,12 @@ public class AzureBlobFileSystemStore { isNamespaceEnabled ? getOctalNotation(permission) : null, isNamespaceEnabled ? getOctalNotation(umask) : null); - final OutputStream outputStream; - outputStream = new FSDataOutputStream( - new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), 0, - abfsConfiguration.getWriteBufferSize(), abfsConfiguration.isFlushEnabled()), null); - return outputStream; + return new AbfsOutputStream( + client, + AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), + 0, + abfsConfiguration.getWriteBufferSize(), + abfsConfiguration.isFlushEnabled()); } public void createDirectory(final Path path, final FsPermission permission, final FsPermission umask) @@ -273,7 +271,7 @@ public class AzureBlobFileSystemStore { isNamespaceEnabled ? getOctalNotation(umask) : null); } - public InputStream openFileForRead(final Path path, final FileSystem.Statistics statistics) + public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statistics statistics) throws AzureBlobFileSystemException { LOG.debug("openFileForRead filesystem: {} path: {}", client.getFileSystem(), @@ -294,10 +292,9 @@ public class AzureBlobFileSystemStore { } // Add statistics for InputStream - return new FSDataInputStream( - new AbfsInputStream(client, statistics, - AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength, - abfsConfiguration.getReadBufferSize(), abfsConfiguration.getReadAheadQueueDepth(), eTag)); + return new AbfsInputStream(client, statistics, + AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength, + abfsConfiguration.getReadBufferSize(), abfsConfiguration.getReadAheadQueueDepth(), eTag); } public OutputStream openFileForWrite(final Path path, final boolean overwrite) throws @@ -322,11 +319,12 @@ public class AzureBlobFileSystemStore { final long offset = overwrite ? 0 : contentLength; - final OutputStream outputStream; - outputStream = new FSDataOutputStream( - new AbfsOutputStream(client, AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), - offset, abfsConfiguration.getWriteBufferSize(), abfsConfiguration.isFlushEnabled()), null); - return outputStream; + return new AbfsOutputStream( + client, + AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), + offset, + abfsConfiguration.getWriteBufferSize(), + abfsConfiguration.isFlushEnabled()); } public void rename(final Path source, final Path destination) throws From dd2b22fa31b4e8c9a9a1da05ae214605fdb5c25e Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Fri, 24 Aug 2018 01:28:17 +0000 Subject: [PATCH 10/29] HADOOP-15682. ABFS: Add support for StreamCapabilities. Fix javadoc and checkstyle. Contributed by Thomas Marquardt. --- .../diagnostics/ConfigurationValidator.java | 4 ++- .../azurebfs/services/AbfsHttpOperation.java | 6 ++++- .../azurebfs/services/AbfsOutputStream.java | 21 ++++++++++++++- .../fs/azurebfs/services/KeyProvider.java | 3 ++- .../fs/azurebfs/utils/SSLSocketFactoryEx.java | 6 ++--- .../ITestAzureBlobFileSystemFlush.java | 27 +++++++++++++++++++ 6 files changed, 60 insertions(+), 7 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java index d61229ee803..e0121b612fe 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java @@ -28,8 +28,10 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationVa @InterfaceStability.Evolving public interface ConfigurationValidator { /** - * Validates the configValue. + * Validates a configuration value. + * @param configValue the configuration value to be validated. * @return validated value of type T + * @throws InvalidConfigurationValueException if the configuration value is invalid. */ T validate(String configValue) throws InvalidConfigurationValueException; } 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 f493298f2c1..de38b347248 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 @@ -198,7 +198,7 @@ public class AbfsHttpOperation { this.connection.setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, clientRequestId); } - /** + /** * Sends the HTTP request. Note that HttpUrlConnection requires that an * empty buffer be sent in order to set the "Content-Length: 0" header, which * is required by our endpoint. @@ -242,6 +242,10 @@ public class AbfsHttpOperation { /** * Gets and processes the HTTP response. * + * @param buffer a buffer to hold the response entity body + * @param offset an offset in the buffer where the data will being. + * @param length the number of bytes to be written to the buffer. + * * @throws IOException if an error occurs. */ public void processResponse(final byte[] buffer, final int offset, final int length) throws IOException { 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 b69ec835d6d..92e081eaa12 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 @@ -21,6 +21,7 @@ package org.apache.hadoop.fs.azurebfs.services; import java.io.IOException; import java.io.InterruptedIOException; import java.io.OutputStream; +import java.util.Locale; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ExecutorCompletionService; @@ -32,13 +33,14 @@ import java.util.concurrent.TimeUnit; import com.google.common.base.Preconditions; import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.Syncable; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; /** * The BlobFsOutputStream for Rest AbfsClient. */ -public class AbfsOutputStream extends OutputStream implements Syncable { +public class AbfsOutputStream extends OutputStream implements Syncable, StreamCapabilities { private final AbfsClient client; private final String path; private long position; @@ -87,6 +89,23 @@ public class AbfsOutputStream extends OutputStream implements Syncable { this.completionService = new ExecutorCompletionService<>(this.threadExecutor); } + /** + * Query the stream for a specific capability. + * + * @param capability string to query the stream support for. + * @return true for hsync and hflush. + */ + @Override + public boolean hasCapability(String capability) { + switch (capability.toLowerCase(Locale.ENGLISH)) { + case StreamCapabilities.HSYNC: + case StreamCapabilities.HFLUSH: + return supportFlush; + default: + return false; + } + } + /** * Writes the specified byte to this output stream. The general contract for * write is that one byte is written to the output stream. The byte to be diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/KeyProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/KeyProvider.java index 27f76f8594f..09491c520bd 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/KeyProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/KeyProvider.java @@ -35,7 +35,8 @@ public interface KeyProvider { * @param conf * Hadoop configuration parameters * @return the plaintext storage account key - * @throws KeyProviderException + * @throws KeyProviderException if an error occurs while attempting to get + * the storage account key. */ String getStorageAccountKey(String accountName, Configuration conf) throws KeyProviderException; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/SSLSocketFactoryEx.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/SSLSocketFactoryEx.java index 202e1850603..00e7786fa4a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/SSLSocketFactoryEx.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/SSLSocketFactoryEx.java @@ -40,7 +40,7 @@ import org.wildfly.openssl.OpenSSLProvider; * performance. * */ -public class SSLSocketFactoryEx extends SSLSocketFactory { +public final class SSLSocketFactoryEx extends SSLSocketFactory { /** * Default indicates Ordered, preferred OpenSSL, if failed to load then fall @@ -64,9 +64,9 @@ public class SSLSocketFactoryEx extends SSLSocketFactory { * Initialize a singleton SSL socket factory. * * @param preferredMode applicable only if the instance is not initialized. - * @throws IOException + * @throws IOException if an error occurs. */ - public synchronized static void initializeDefaultFactory( + public static synchronized void initializeDefaultFactory( SSLChannelMode preferredMode) throws IOException { if (instance == null) { instance = new SSLSocketFactoryEx(preferredMode); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index 8a6207a0109..7c6bbb5c607 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -31,6 +31,7 @@ import java.io.IOException; import com.microsoft.azure.storage.blob.BlockEntry; import com.microsoft.azure.storage.blob.BlockListingFilter; import com.microsoft.azure.storage.blob.CloudBlockBlob; +import org.apache.hadoop.fs.StreamCapabilities; import org.hamcrest.core.IsEqual; import org.hamcrest.core.IsNot; import org.junit.Assume; @@ -287,6 +288,32 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { } } + @Test + public void testStreamCapabilitiesWithFlushDisabled() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + byte[] buffer = getRandomBytesArray(); + try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, false)) { + assertFalse(stream.hasCapability(StreamCapabilities.HFLUSH)); + assertFalse(stream.hasCapability(StreamCapabilities.HSYNC)); + assertFalse(stream.hasCapability(StreamCapabilities.DROPBEHIND)); + assertFalse(stream.hasCapability(StreamCapabilities.READAHEAD)); + assertFalse(stream.hasCapability(StreamCapabilities.UNBUFFER)); + } + } + + @Test + public void testStreamCapabilitiesWithFlushEnabled() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + byte[] buffer = getRandomBytesArray(); + try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, true)) { + assertTrue(stream.hasCapability(StreamCapabilities.HFLUSH)); + assertTrue(stream.hasCapability(StreamCapabilities.HSYNC)); + assertFalse(stream.hasCapability(StreamCapabilities.DROPBEHIND)); + assertFalse(stream.hasCapability(StreamCapabilities.READAHEAD)); + assertFalse(stream.hasCapability(StreamCapabilities.UNBUFFER)); + } + } + @Test public void testHsyncWithFlushDisabled() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); From df57c6c3b12117788b78c30b9d0703c5e9d88458 Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Wed, 29 Aug 2018 05:59:44 +0000 Subject: [PATCH 11/29] HADOOP-15692. ABFS: extensible support for custom oauth. Contributed by Junhua Gu and Rajeev Bansal. --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 63 +++++++----- .../fs/azurebfs/AzureBlobFileSystem.java | 28 ++++++ .../azurebfs/constants/ConfigurationKeys.java | 3 + .../constants/FileSystemConfigurations.java | 1 + .../CustomDelegationTokenManager.java | 66 +++++++++++++ .../CustomTokenProviderAdaptee.java | 8 +- .../fs/azurebfs/extensions/package-info.java | 21 ++++ .../oauth2/CustomTokenProviderAdapter.java | 1 + .../AbfsDelegationTokenIdentifier.java | 49 ++++++++++ .../security/AbfsDelegationTokenManager.java | 88 +++++++++++++++++ .../azurebfs/security/AbfsTokenRenewer.java | 96 +++++++++++++++++++ .../fs/azurebfs/security/package-info.java | 23 +++++ ...ache.hadoop.security.token.TokenIdentifier | 1 + ....apache.hadoop.security.token.TokenRenewer | 1 + .../ITestAzureBlobFileSystemFileStatus.java | 3 +- 15 files changed, 421 insertions(+), 31 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomDelegationTokenManager.java rename hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/{oauth2 => extensions}/CustomTokenProviderAdaptee.java (94%) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsDelegationTokenIdentifier.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsDelegationTokenManager.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsTokenRenewer.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/package-info.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index f26f56204ca..069f17a1a7b 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 @@ -27,7 +27,6 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation; @@ -43,13 +42,14 @@ import org.apache.hadoop.fs.azurebfs.diagnostics.BooleanConfigurationBasicValida import org.apache.hadoop.fs.azurebfs.diagnostics.IntegerConfigurationBasicValidator; import org.apache.hadoop.fs.azurebfs.diagnostics.LongConfigurationBasicValidator; import org.apache.hadoop.fs.azurebfs.diagnostics.StringConfigurationBasicValidator; +import org.apache.hadoop.fs.azurebfs.extensions.CustomTokenProviderAdaptee; import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider; -import org.apache.hadoop.fs.azurebfs.oauth2.CustomTokenProviderAdaptee; 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.security.AbfsDelegationTokenManager; import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.azurebfs.services.KeyProvider; import org.apache.hadoop.fs.azurebfs.services.SimpleKeyProvider; @@ -57,7 +57,7 @@ import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx; import org.apache.hadoop.util.ReflectionUtils; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.*; -import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_FS_AZURE_SSL_CHANNEL_MODE; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.*; /** * Configuration for Azure Blob FileSystem. @@ -69,83 +69,86 @@ public class AbfsConfiguration{ private final boolean isSecure; @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_WRITE_BUFFER_SIZE, - MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE, - MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE, - DefaultValue = FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE) + MinValue = MIN_BUFFER_SIZE, + MaxValue = MAX_BUFFER_SIZE, + DefaultValue = DEFAULT_WRITE_BUFFER_SIZE) private int writeBufferSize; @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_READ_BUFFER_SIZE, - MinValue = FileSystemConfigurations.MIN_BUFFER_SIZE, - MaxValue = FileSystemConfigurations.MAX_BUFFER_SIZE, - DefaultValue = FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE) + MinValue = MIN_BUFFER_SIZE, + MaxValue = MAX_BUFFER_SIZE, + DefaultValue = DEFAULT_READ_BUFFER_SIZE) private int readBufferSize; @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_MIN_BACKOFF_INTERVAL, - DefaultValue = FileSystemConfigurations.DEFAULT_MIN_BACKOFF_INTERVAL) + DefaultValue = DEFAULT_MIN_BACKOFF_INTERVAL) private int minBackoffInterval; @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_MAX_BACKOFF_INTERVAL, - DefaultValue = FileSystemConfigurations.DEFAULT_MAX_BACKOFF_INTERVAL) + DefaultValue = DEFAULT_MAX_BACKOFF_INTERVAL) private int maxBackoffInterval; @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_BACKOFF_INTERVAL, - DefaultValue = FileSystemConfigurations.DEFAULT_BACKOFF_INTERVAL) + DefaultValue = DEFAULT_BACKOFF_INTERVAL) private int backoffInterval; @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_MAX_IO_RETRIES, MinValue = 0, - DefaultValue = FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS) + DefaultValue = DEFAULT_MAX_RETRY_ATTEMPTS) private int maxIoRetries; @LongConfigurationValidatorAnnotation(ConfigurationKey = AZURE_BLOCK_SIZE_PROPERTY_NAME, MinValue = 0, - MaxValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE, - DefaultValue = FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE) + MaxValue = MAX_AZURE_BLOCK_SIZE, + DefaultValue = MAX_AZURE_BLOCK_SIZE) private long azureBlockSize; @StringConfigurationValidatorAnnotation(ConfigurationKey = AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME, - DefaultValue = FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT) + DefaultValue = AZURE_BLOCK_LOCATION_HOST_DEFAULT) private String azureBlockLocationHost; @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_CONCURRENT_CONNECTION_VALUE_OUT, MinValue = 1, - DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_WRITE_THREADS) + DefaultValue = MAX_CONCURRENT_WRITE_THREADS) private int maxConcurrentWriteThreads; @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_CONCURRENT_CONNECTION_VALUE_IN, MinValue = 1, - DefaultValue = FileSystemConfigurations.MAX_CONCURRENT_READ_THREADS) + DefaultValue = MAX_CONCURRENT_READ_THREADS) private int maxConcurrentReadThreads; @BooleanConfigurationValidatorAnnotation(ConfigurationKey = AZURE_TOLERATE_CONCURRENT_APPEND, - DefaultValue = FileSystemConfigurations.DEFAULT_READ_TOLERATE_CONCURRENT_APPEND) + DefaultValue = DEFAULT_READ_TOLERATE_CONCURRENT_APPEND) private boolean tolerateOobAppends; @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ATOMIC_RENAME_KEY, - DefaultValue = FileSystemConfigurations.DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES) + DefaultValue = DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES) private String azureAtomicDirs; @BooleanConfigurationValidatorAnnotation(ConfigurationKey = AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, - DefaultValue = FileSystemConfigurations.DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION) + DefaultValue = DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION) private boolean createRemoteFileSystemDuringInitialization; @BooleanConfigurationValidatorAnnotation(ConfigurationKey = AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION, - DefaultValue = FileSystemConfigurations.DEFAULT_AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION) + DefaultValue = DEFAULT_AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION) private boolean skipUserGroupMetadataDuringInitialization; - @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_READ_AHEAD_QUEUE_DEPTH, - DefaultValue = FileSystemConfigurations.DEFAULT_READ_AHEAD_QUEUE_DEPTH) + DefaultValue = DEFAULT_READ_AHEAD_QUEUE_DEPTH) private int readAheadQueueDepth; @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ENABLE_FLUSH, - DefaultValue = FileSystemConfigurations.DEFAULT_ENABLE_FLUSH) + DefaultValue = DEFAULT_ENABLE_FLUSH) private boolean enableFlush; @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_USER_AGENT_PREFIX_KEY, - DefaultValue = "") + DefaultValue = "") private String userAgentId; + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ENABLE_DELEGATION_TOKEN, + DefaultValue = DEFAULT_ENABLE_DELEGATION_TOKEN) + private boolean enableDelegationToken; + private Map storageAccountKeys; public AbfsConfiguration(final Configuration configuration) throws IllegalAccessException, InvalidConfigurationValueException { @@ -292,6 +295,14 @@ public class AbfsConfiguration{ return configuration.getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME + accountName, AuthType.SharedKey); } + public boolean isDelegationTokenManagerEnabled() { + return enableDelegationToken; + } + + public AbfsDelegationTokenManager getDelegationTokenManager() throws IOException { + return new AbfsDelegationTokenManager(configuration); + } + public AccessTokenProvider getTokenProvider(final String accountName) throws TokenAccessProviderException { AuthType authType = configuration.getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME + accountName, AuthType.SharedKey); if (authType == AuthType.OAuth) { 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 6bec7cb2a3b..4bde9d80eb4 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 @@ -61,9 +61,11 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.FileSystemOperationUnh import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException; import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; +import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager; import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Progressable; @@ -82,6 +84,9 @@ public class AzureBlobFileSystem extends FileSystem { private AzureBlobFileSystemStore abfsStore; private boolean isClosed; + private boolean delegationTokenEnabled = false; + private AbfsDelegationTokenManager delegationTokenManager; + @Override public void initialize(URI uri, Configuration configuration) throws IOException { @@ -112,6 +117,15 @@ public class AzureBlobFileSystem extends FileSystem { //Provide a default group name this.primaryUserGroup = this.user; } + + if (UserGroupInformation.isSecurityEnabled()) { + this.delegationTokenEnabled = abfsStore.getAbfsConfiguration().isDelegationTokenManagerEnabled(); + + if(this.delegationTokenEnabled) { + LOG.debug("Initializing DelegationTokenManager for {}", uri); + this.delegationTokenManager = abfsStore.getAbfsConfiguration().getDelegationTokenManager(); + } + } } @Override @@ -815,6 +829,20 @@ public class AzureBlobFileSystem extends FileSystem { return result; } + /** + * Get a delegation token from remote service endpoint if + * 'fs.azure.enable.kerberos.support' is set to 'true', and + * 'fs.azure.enable.delegation.token' is set to 'true'. + * @param renewer the account name that is allowed to renew the token. + * @return delegation token + * @throws IOException thrown when getting the current user. + */ + @Override + public synchronized Token getDelegationToken(final String renewer) throws IOException { + return this.delegationTokenEnabled ? this.delegationTokenManager.getDelegationToken(renewer) + : super.getDelegationToken(renewer); + } + @VisibleForTesting FileSystem.Statistics getFsStatistics() { return this.statistics; 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 ffdf700b78f..8ad0dc61827 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 @@ -79,5 +79,8 @@ public final class ConfigurationKeys { /** Prefix for oauth refresh token: {@value}. */ public static final String FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN = "fs.azure.account.oauth2.refresh.token."; + public static final String FS_AZURE_ENABLE_DELEGATION_TOKEN = "fs.azure.enable.delegation.token"; + public static final String FS_AZURE_DELEGATION_TOKEN_PROVIDER_TYPE = "fs.azure.delegation.token.provider.type"; + 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 5b92dddf732..a921faf8d59 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 @@ -61,5 +61,6 @@ public final class FileSystemConfigurations { public static final SSLSocketFactoryEx.SSLChannelMode DEFAULT_FS_AZURE_SSL_CHANNEL_MODE = SSLSocketFactoryEx.SSLChannelMode.Default; + public static final boolean DEFAULT_ENABLE_DELEGATION_TOKEN = false; private FileSystemConfigurations() {} } \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomDelegationTokenManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomDelegationTokenManager.java new file mode 100644 index 00000000000..f5716e089c0 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomDelegationTokenManager.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.extensions; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier; +import org.apache.hadoop.security.token.Token; + +/** + * Interface for Managing the Delegation tokens. + */ +public interface CustomDelegationTokenManager { + + /** + * Initialize with supported configuration. This method is invoked when the + * (URI, Configuration)} method is invoked. + * + * @param configuration Configuration object + * @throws IOException if instance can not be configured. + */ + void initialize(Configuration configuration) + throws IOException; + + /** + * Get Delegation token. + * @param renewer delegation token renewer + * @return delegation token + * @throws IOException when error in getting the delegation token + */ + Token getDelegationToken(String renewer) + throws IOException; + + /** + * Renew the delegation token. + * @param token delegation token. + * @return renewed time. + * @throws IOException when error in renewing the delegation token + */ + long renewDelegationToken(Token token) throws IOException; + + /** + * Cancel the delegation token. + * @param token delegation token. + * @throws IOException when error in cancelling the delegation token. + */ + void cancelDelegationToken(Token token) throws IOException; +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdaptee.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomTokenProviderAdaptee.java similarity index 94% rename from hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdaptee.java rename to hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomTokenProviderAdaptee.java index 7366a8d7950..889d506c4a5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdaptee.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomTokenProviderAdaptee.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.azurebfs.oauth2; +package org.apache.hadoop.fs.azurebfs.extensions; import java.io.IOException; import java.util.Date; @@ -47,8 +47,8 @@ public interface CustomTokenProviderAdaptee { * @param accountName Account Name * @throws IOException if instance can not be configured. */ - void initialize(Configuration configuration, String accountName) - throws IOException; + void initialize(Configuration configuration, final String accountName) + throws IOException; /** * Obtain the access token that should be added to https connection's header. @@ -72,4 +72,4 @@ public interface CustomTokenProviderAdaptee { * @return Date to expire access token retrieved from AAD. */ Date getExpiryTime(); -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/package-info.java new file mode 100644 index 00000000000..667b7b2e07c --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/package-info.java @@ -0,0 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +@InterfaceAudience.Public +package org.apache.hadoop.fs.azurebfs.extensions; +import org.apache.hadoop.classification.InterfaceAudience; 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 7bae415daf6..6e9f6350a1e 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 @@ -25,6 +25,7 @@ import com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.azurebfs.extensions.CustomTokenProviderAdaptee; /** * Provides tokens based on custom implementation, following the Adapter Design diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsDelegationTokenIdentifier.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsDelegationTokenIdentifier.java new file mode 100644 index 00000000000..390c2f40319 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsDelegationTokenIdentifier.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.security; + +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier; + +/** + * Delegation token Identifier for ABFS delegation tokens. + */ +public class AbfsDelegationTokenIdentifier extends DelegationTokenIdentifier { + public static final Text TOKEN_KIND = new Text("ABFS delegation"); + + public AbfsDelegationTokenIdentifier(){ + super(TOKEN_KIND); + } + + public AbfsDelegationTokenIdentifier(Text kind) { + super(kind); + } + + public AbfsDelegationTokenIdentifier(Text kind, Text owner, Text renewer, + Text realUser) { + super(kind, owner, renewer, realUser); + } + + @Override + public Text getKind() { + return TOKEN_KIND; + } + +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsDelegationTokenManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsDelegationTokenManager.java new file mode 100644 index 00000000000..eb47f768a4b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsDelegationTokenManager.java @@ -0,0 +1,88 @@ +/** + * 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.security; + +import java.io.IOException; + +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.extensions.CustomDelegationTokenManager; +import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.util.ReflectionUtils; + +/** + * Class for delegation token Manager. + */ +public class AbfsDelegationTokenManager { + + private CustomDelegationTokenManager tokenManager; + private static final Logger LOG = + LoggerFactory.getLogger(AbfsDelegationTokenManager.class); + + public AbfsDelegationTokenManager(final Configuration conf) throws IOException { + + Preconditions.checkNotNull(conf, "conf"); + + Class customDelegationTokenMgrClass = + conf.getClass(ConfigurationKeys.FS_AZURE_DELEGATION_TOKEN_PROVIDER_TYPE, null, + CustomDelegationTokenManager.class); + + if (customDelegationTokenMgrClass == null) { + throw new IllegalArgumentException( + "The value for \"fs.azure.delegation.token.provider.type\" is not defined."); + } + + CustomDelegationTokenManager customTokenMgr = (CustomDelegationTokenManager) ReflectionUtils + .newInstance(customDelegationTokenMgrClass, conf); + if (customTokenMgr == null) { + throw new IllegalArgumentException(String.format("Failed to initialize %s.", customDelegationTokenMgrClass)); + } + + customTokenMgr.initialize(conf); + + tokenManager = customTokenMgr; + } + + public Token getDelegationToken( + String renewer) throws IOException { + + Token token = tokenManager.getDelegationToken(renewer); + + token.setKind(AbfsDelegationTokenIdentifier.TOKEN_KIND); + return token; + } + + public long renewDelegationToken(Token token) + throws IOException { + + return tokenManager.renewDelegationToken(token); + } + + public void cancelDelegationToken(Token token) + throws IOException { + + tokenManager.cancelDelegationToken(token); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsTokenRenewer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsTokenRenewer.java new file mode 100644 index 00000000000..ab51838f26f --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsTokenRenewer.java @@ -0,0 +1,96 @@ +/** + * 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.security; + +import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenRenewer; + +/** + * Token Renewer for renewing ABFS delegation tokens with remote service. + */ +public class AbfsTokenRenewer extends TokenRenewer { + public static final Logger LOG = + LoggerFactory.getLogger(AbfsTokenRenewer.class); + + /** + * Checks if this particular object handles the Kind of token passed. + * + * @param kind the kind of the token + * @return true if it handles passed token kind false otherwise. + */ + @Override + public boolean handleKind(Text kind) { + return AbfsDelegationTokenIdentifier.TOKEN_KIND.equals(kind); + } + + /** + * Checks if passed token is managed. + * + * @param token the token being checked + * @return true if it is managed. + * @throws IOException thrown when evaluating if token is managed. + */ + @Override + public boolean isManaged(Token token) throws IOException { + return true; + } + + /** + * Renew the delegation token. + * + * @param token token to renew. + * @param conf configuration object. + * @return extended expiry time of the token. + * @throws IOException thrown when trying get current user. + * @throws InterruptedException thrown when thread is interrupted + */ + @Override + public long renew(final Token token, Configuration conf) + throws IOException, InterruptedException { + LOG.debug("Renewing the delegation token"); + return getInstance(conf).renewDelegationToken(token); + } + + /** + * Cancel the delegation token. + * + * @param token token to cancel. + * @param conf configuration object. + * @throws IOException thrown when trying get current user. + * @throws InterruptedException thrown when thread is interrupted. + */ + @Override + public void cancel(final Token token, Configuration conf) + throws IOException, InterruptedException { + LOG.debug("Cancelling the delegation token"); + getInstance(conf).cancelDelegationToken(token); + } + + private AbfsDelegationTokenManager getInstance(Configuration conf) + throws IOException { + return new AbfsDelegationTokenManager(conf); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/package-info.java new file mode 100644 index 00000000000..7c3e37ae6eb --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/package-info.java @@ -0,0 +1,23 @@ +/** + * 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. + */ + +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.fs.azurebfs.security; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-tools/hadoop-azure/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier b/hadoop-tools/hadoop-azure/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier index 7ec8216deb0..90169185863 100644 --- a/hadoop-tools/hadoop-azure/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier +++ b/hadoop-tools/hadoop-azure/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier @@ -13,4 +13,5 @@ # See the License for the specific language governing permissions and # limitations under the License. +org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenIdentifier org.apache.hadoop.fs.azure.security.WasbDelegationTokenIdentifier \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer b/hadoop-tools/hadoop-azure/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer index f9c590aad8d..d889534c73c 100644 --- a/hadoop-tools/hadoop-azure/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer +++ b/hadoop-tools/hadoop-azure/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer @@ -13,4 +13,5 @@ # See the License for the specific language governing permissions and # limitations under the License. +org.apache.hadoop.fs.azurebfs.security.AbfsTokenRenewer org.apache.hadoop.fs.azure.security.WasbTokenRenewer \ No newline at end of file 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 9daac2ae516..88f77b0bcae 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 @@ -56,6 +56,7 @@ public class ITestAzureBlobFileSystemFileStatus extends @Test public void testFileStatusPermissionsAndOwnerAndGroup() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, DEFAULT_UMASK_VALUE); touch(TEST_FILE); validateStatus(fs, TEST_FILE, false); } @@ -63,7 +64,6 @@ public class ITestAzureBlobFileSystemFileStatus extends private FileStatus validateStatus(final AzureBlobFileSystem fs, final Path name, final boolean isDir) throws IOException { FileStatus fileStatus = fs.getFileStatus(name); - fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, DEFAULT_UMASK_VALUE); String errorInStatus = "error in " + fileStatus + " from " + fs; @@ -89,6 +89,7 @@ public class ITestAzureBlobFileSystemFileStatus extends @Test public void testFolderStatusPermissionsAndOwnerAndGroup() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, DEFAULT_UMASK_VALUE); fs.mkdirs(TEST_FOLDER); validateStatus(fs, TEST_FOLDER, true); From 81dc4a995c3837b721a0c1f897698b5ed47b8fb0 Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Fri, 31 Aug 2018 03:24:42 +0000 Subject: [PATCH 12/29] HADOOP-15663. ABFS: Simplify configuration. Contributed by Da Zhou. --- .../src/main/resources/core-default.xml | 12 + .../hadoop/fs/azurebfs/AbfsConfiguration.java | 4 - .../fs/azurebfs/AzureBlobFileSystemStore.java | 24 +- .../azurebfs/constants/ConfigurationKeys.java | 5 +- .../hadoop/fs/azurebfs/utils/UriUtils.java | 15 +- .../src/site/markdown/testing_azure.md | 233 ++++++++++++------ .../fs/azure/AzureBlobStorageTestAccount.java | 22 +- ...stFileSystemOperationExceptionMessage.java | 3 +- .../azure/ITestWasbUriAndConfiguration.java | 26 -- .../azure/integration/AzureTestConstants.java | 6 +- .../fs/azure/integration/AzureTestUtils.java | 18 ++ .../metrics/TestRollingWindowAverage.java | 4 +- .../azurebfs/AbstractAbfsIntegrationTest.java | 73 ++++-- .../azurebfs/AbstractAbfsTestWithTimeout.java | 70 ++++++ .../ITestAzureBlobFileSystemBackCompat.java | 2 +- .../azurebfs/ITestWasbAbfsCompatibility.java | 2 +- ...TestAbfsConfigurationFieldsValidation.java | 3 +- .../constants/TestConfigurationKeys.java | 13 +- .../contract/ABFSContractTestBinding.java | 3 + .../TestConfigurationValidators.java | 1 - ...sTestContainer.java => AbfsTestUtils.java} | 49 ++-- .../fs/azurebfs/utils/TestUriUtils.java | 12 +- .../src/test/resources/azure-bfs-test.xml | 188 -------------- .../src/test/resources/azure-test.xml | 56 ++--- 24 files changed, 431 insertions(+), 413 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java rename hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/{CleanUpAbfsTestContainer.java => AbfsTestUtils.java} (64%) delete mode 100644 hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml 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 471dacc9221..3fcdecb5959 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 @@ -1618,6 +1618,18 @@ + + fs.AbstractFileSystem.wasb.impl + org.apache.hadoop.fs.azure.Wasb + AbstractFileSystem implementation class of wasb:// + + + + fs.AbstractFileSystem.wasbs.impl + org.apache.hadoop.fs.azure.Wasbs + AbstractFileSystem implementation class of wasbs:// + + fs.wasb.impl org.apache.hadoop.fs.azure.NativeAzureFileSystem 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 069f17a1a7b..924bc3e8d9e 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 @@ -173,10 +173,6 @@ public class AbfsConfiguration{ } } - public boolean isEmulator() { - return this.getConfiguration().getBoolean(FS_AZURE_EMULATOR_ENABLED, false); - } - public boolean isSecureMode() { return isSecure; } 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 fc60127c51e..6542a648b7b 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 @@ -77,6 +77,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy; import org.apache.hadoop.fs.azurebfs.services.SharedKeyCredentials; +import org.apache.hadoop.fs.azurebfs.utils.UriUtils; import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.FsAction; @@ -86,6 +87,7 @@ import org.apache.http.client.utils.URIBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_ABFS_ENDPOINT; import static org.apache.hadoop.util.Time.now; /** @@ -146,7 +148,27 @@ public class AzureBlobFileSystemStore { final URIBuilder uriBuilder = new URIBuilder(); uriBuilder.setScheme(scheme); - uriBuilder.setHost(hostName); + + // For testing purposes, an IP address and port may be provided to override + // the host specified in the FileSystem URI. Also note that the format of + // the Azure Storage Service URI changes from + // http[s]://[account][domain-suffix]/[filesystem] to + // http[s]://[ip]:[port]/[account]/[filesystem]. + String endPoint = abfsConfiguration.getConfiguration().get(AZURE_ABFS_ENDPOINT); + if (endPoint == null || !endPoint.contains(AbfsHttpConstants.COLON)) { + uriBuilder.setHost(hostName); + return uriBuilder; + } + + // Split ip and port + String[] data = endPoint.split(AbfsHttpConstants.COLON); + if (data.length != 2) { + throw new RuntimeException(String.format("ABFS endpoint is not set correctly : %s, " + + "Do not specify scheme when using {IP}:{PORT}", endPoint)); + } + uriBuilder.setHost(data[0].trim()); + uriBuilder.setPort(Integer.parseInt(data[1].trim())); + uriBuilder.setPath("/" + UriUtils.extractAccountNameFromHostName(hostName)); return uriBuilder; } 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 8ad0dc61827..ca4c9c3005d 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 @@ -37,9 +37,6 @@ public final class ConfigurationKeys { public static final String AZURE_BACKOFF_INTERVAL = "fs.azure.io.retry.backoff.interval"; public static final String AZURE_MAX_IO_RETRIES = "fs.azure.io.retry.max.retries"; - // Remove this and use common azure storage emulator property for public release. - public static final String FS_AZURE_EMULATOR_ENABLED = "fs.azure.abfs.emulator.enabled"; - // Read and write buffer sizes defined by the user public static final String AZURE_WRITE_BUFFER_SIZE = "fs.azure.write.request.size"; public static final String AZURE_READ_BUFFER_SIZE = "fs.azure.read.request.size"; @@ -60,6 +57,8 @@ public final class ConfigurationKeys { public static final String AZURE_KEY_ACCOUNT_KEYPROVIDER_PREFIX = "fs.azure.account.keyprovider."; public static final String AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT = "fs.azure.shellkeyprovider.script"; + /** End point of ABFS account: {@value}. */ + public static final String AZURE_ABFS_ENDPOINT = "fs.azure.abfs.endpoint"; /** Prefix for auth type properties: {@value}. */ public static final String FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME = "fs.azure.account.auth.type."; /** Prefix for oauth token provider type: {@value}. */ 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 7652adf10e3..1bbc1b39e16 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 @@ -41,20 +41,21 @@ public final class UriUtils { } /** - * Extracts the raw account name from account name. - * @param accountName to extract the raw account name. - * @return extracted raw account name. + * Extracts the account name from the host name. + * @param hostName the fully-qualified domain name of the storage service + * endpoint (e.g. {account}.dfs.core.windows.net. + * @return the storage service account name. */ - public static String extractRawAccountFromAccountName(final String accountName) { - if (accountName == null || accountName.isEmpty()) { + public static String extractAccountNameFromHostName(final String hostName) { + if (hostName == null || hostName.isEmpty()) { return null; } - if (!containsAbfsUrl(accountName)) { + if (!containsAbfsUrl(hostName)) { return null; } - String[] splitByDot = accountName.split("\\."); + String[] splitByDot = hostName.split("\\."); if (splitByDot.length == 0) { return null; } 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 c148807aaa0..a4d920afde4 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md @@ -90,7 +90,7 @@ For example: - fs.azure.test.account.name + fs.azure.wasb.account.name {ACCOUNTNAME}.blob.core.windows.net @@ -126,7 +126,7 @@ Overall, to run all the tests using `mvn test`, a sample `azure-auth-keys.xml` - fs.azure.test.account.name + fs.azure.wasb.account.name {ACCOUNTNAME}.blob.core.windows.net @@ -576,77 +576,172 @@ This will delete the containers; the output log of the test run will provide the details and summary of the operation. -## Testing ABFS +## Testing the Azure ABFS Client -The ABFS Connector tests share the same account as the wasb tests; this is -needed for cross-connector compatibility tests. +Azure Data Lake Storage Gen 2 (ADLS Gen 2) is a set of capabilities dedicated to +big data analytics, built on top of Azure Blob Storage. The ABFS and ABFSS +schemes target the ADLS Gen 2 REST API, and the WASB and WASBS schemes target +the Azure Blob Storage REST API. ADLS Gen 2 offers better performance and +scalability. ADLS Gen 2 also offers authentication and authorization compatible +with the Hadoop Distributed File System permissions model when hierarchical +namespace is enabled for the storage account. Furthermore, the metadata and data +produced by ADLS Gen 2 REST API can be consumed by Blob REST API, and vice versa. + +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. -This makes for a somewhat complex set of configuration options. +```xml + + + + + fs.azure.abfs.account.name + {ACCOUNT_NAME}.dfs.core.windows.net + -Here are the settings for an account `ACCOUNTNAME` + + fs.azure.account.key.{ACCOUNT_NAME}.dfs.core.windows.net + {ACCOUNT_ACCESS_KEY} + + + + fs.azure.wasb.account.name + {ACCOUNT_NAME}.blob.core.windows.net + + + + fs.azure.account.key.{ACCOUNT_NAME}.blob.core.windows.net + {ACCOUNT_ACCESS_KEY} + + + + fs.contract.test.fs.abfs + abfs://{CONTAINER_NAME}@{ACCOUNT_NAME}.dfs.core.windows.net + A file system URI to be used by the contract tests. + + + + fs.contract.test.fs.wasb + wasb://{CONTAINER_NAME}@{ACCOUNT_NAME}.blob.core.windows.net + A file system URI to be used by the contract tests. + + +``` + +To run OAuth and ACL test cases you must use a storage account with the +hierarchical namespace enabled, and set the following configuration settings: + +```xml + + + + fs.azure.account.auth.type.{YOUR_ABFS_ACCOUNT_NAME} + {AUTH TYPE} + The authorization type can be SharedKey, OAuth, or Custom. The + default is SharedKey. + + + + + + + + + + + + + + + + + + + + + + + +``` + +If running tests against an endpoint that uses the URL format +http[s]://[ip]:[port]/[account]/[filesystem] instead of +http[s]://[account][domain-suffix]/[filesystem], please use the following: ```xml - abfs.account.name - ACCOUNTNAME + fs.azure.abfs.endpoint + {IP}:{PORT} - - - abfs.account.full.name - ${abfs.account.name}.dfs.core.windows.net - - - - abfs.account.key - SECRETKEY== - - - - fs.azure.account.key.ACCOUNTNAME.dfs.core.windows.net - ${abfs.account.key} - - - - fs.azure.account.key.ACCOUNTNAME.blob.core.windows.net - ${abfs.account.key} - - - - fs.azure.test.account.key.ACCOUNTNAME.dfs.core.windows.net - ${abfs.account.key} - - - - fs.azure.test.account.key.ACCOUNTNAME.blob.core.windows.net - ${abfs.account.key} - - - - fs.azure.account.key.ACCOUNTNAME - ${abfs.account.key} - - - - fs.azure.test.account.key.ACCOUNTNAME - ${abfs.account.key} - - - - fs.azure.test.account.name - ${abfs.account.full.name} - - - - fs.contract.test.fs.abfs - abfs://TESTCONTAINER@ACCOUNTNAME.dfs.core.windows.net - Container for contract tests - - - - fs.contract.test.fs.abfss - abfss://TESTCONTAINER@ACCOUNTNAME.dfs.core.windows.net - Container for contract tests - - - -``` +``` \ No newline at end of file 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 5b36c8793ca..b65ce78fcbb 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 @@ -48,6 +48,7 @@ import java.util.concurrent.ConcurrentLinkedQueue; import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.DEFAULT_STORAGE_EMULATOR_ACCOUNT_NAME; import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.KEY_USE_LOCAL_SAS_KEY_MODE; import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.KEY_USE_SECURE_MODE; +import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.verifyWasbAccountNameInConfig; /** * Helper class to create WASB file systems backed by either a mock in-memory @@ -58,11 +59,14 @@ public final class AzureBlobStorageTestAccount implements AutoCloseable, private static final Logger LOG = LoggerFactory.getLogger( AzureBlobStorageTestAccount.class); - private static final String ACCOUNT_KEY_PROPERTY_NAME = "fs.azure.account.key."; private static final String SAS_PROPERTY_NAME = "fs.azure.sas."; private static final String TEST_CONFIGURATION_FILE_NAME = "azure-test.xml"; - private static final String TEST_ACCOUNT_NAME_PROPERTY_NAME = "fs.azure.test.account.name"; + public static final String ACCOUNT_KEY_PROPERTY_NAME = "fs.azure.account.key."; + public static final String TEST_ACCOUNT_NAME_PROPERTY_NAME = "fs.azure.account.name"; + public static final String WASB_TEST_ACCOUNT_NAME_WITH_DOMAIN = "fs.azure.wasb.account.name"; public static final String MOCK_ACCOUNT_NAME = "mockAccount.blob.core.windows.net"; + public static final String WASB_ACCOUNT_NAME_DOMAIN_SUFFIX = ".blob.core.windows.net"; + public static final String WASB_ACCOUNT_NAME_DOMAIN_SUFFIX_REGEX = "\\.blob(\\.preprod)?\\.core\\.windows\\.net"; public static final String MOCK_CONTAINER_NAME = "mockContainer"; public static final String WASB_AUTHORITY_DELIMITER = "@"; public static final String WASB_SCHEME = "wasb"; @@ -379,7 +383,7 @@ public final class AzureBlobStorageTestAccount implements AutoCloseable, containerName); container.create(); - String accountName = conf.get(TEST_ACCOUNT_NAME_PROPERTY_NAME); + String accountName = verifyWasbAccountNameInConfig(conf); // Ensure that custom throttling is disabled and tolerate concurrent // out-of-band appends. @@ -525,7 +529,7 @@ public final class AzureBlobStorageTestAccount implements AutoCloseable, static CloudStorageAccount createTestAccount(Configuration conf) throws URISyntaxException, KeyProviderException { - String testAccountName = conf.get(TEST_ACCOUNT_NAME_PROPERTY_NAME); + String testAccountName = verifyWasbAccountNameInConfig(conf); if (testAccountName == null) { LOG.warn("Skipping live Azure test because of missing test account"); return null; @@ -570,16 +574,16 @@ public final class AzureBlobStorageTestAccount implements AutoCloseable, String containerName = useContainerSuffixAsContainerName ? containerNameSuffix : String.format( - "wasbtests-%s-%tQ%s", + "wasbtests-%s-%s%s", System.getProperty("user.name"), - new Date(), + UUID.randomUUID().toString(), containerNameSuffix); container = account.createCloudBlobClient().getContainerReference( containerName); if (createOptions.contains(CreateOptions.CreateContainer)) { container.createIfNotExists(); } - String accountName = conf.get(TEST_ACCOUNT_NAME_PROPERTY_NAME); + String accountName = verifyWasbAccountNameInConfig(conf); if (createOptions.contains(CreateOptions.UseSas)) { String sas = generateSAS(container, createOptions.contains(CreateOptions.Readonly)); @@ -741,7 +745,7 @@ public final class AzureBlobStorageTestAccount implements AutoCloseable, CloudBlobClient blobClient = account.createCloudBlobClient(); // Capture the account URL and the account name. - String accountName = conf.get(TEST_ACCOUNT_NAME_PROPERTY_NAME); + String accountName = verifyWasbAccountNameInConfig(conf); configureSecureModeTestSettings(conf); @@ -814,7 +818,7 @@ public final class AzureBlobStorageTestAccount implements AutoCloseable, CloudBlobClient blobClient = account.createCloudBlobClient(); // Capture the account URL and the account name. - String accountName = conf.get(TEST_ACCOUNT_NAME_PROPERTY_NAME); + String accountName = verifyWasbAccountNameInConfig(conf); configureSecureModeTestSettings(conf); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationExceptionMessage.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationExceptionMessage.java index 6d5e72e57b7..af570bdbea1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationExceptionMessage.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationExceptionMessage.java @@ -29,6 +29,7 @@ import com.microsoft.azure.storage.CloudStorageAccount; import org.junit.Test; import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.NO_ACCESS_TO_CONTAINER_MSG; +import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.verifyWasbAccountNameInConfig; /** * Test for error messages coming from SDK. @@ -46,7 +47,7 @@ public class ITestFileSystemOperationExceptionMessage AzureBlobStorageTestAccount.createTestAccount(conf); AzureTestUtils.assume("No test account", account != null); - String testStorageAccount = conf.get("fs.azure.test.account.name"); + String testStorageAccount = verifyWasbAccountNameInConfig(conf); conf = new Configuration(); conf.set("fs.AbstractFileSystem.wasb.impl", "org.apache.hadoop.fs.azure.Wasb"); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestWasbUriAndConfiguration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestWasbUriAndConfiguration.java index bee02206d60..7783684e9d2 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestWasbUriAndConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestWasbUriAndConfiguration.java @@ -499,32 +499,6 @@ public class ITestWasbUriAndConfiguration extends AbstractWasbTestWithTimeout { } } - @Test - public void testNoAbstractFileSystemImplementationSpecifiedForWasbsScheme() throws Exception { - try { - testAccount = AzureBlobStorageTestAccount.createMock(); - Configuration conf = testAccount.getFileSystem().getConf(); - String authority = testAccount.getFileSystem().getUri().getAuthority(); - URI defaultUri = new URI("wasbs", authority, null, null, null); - conf.set(FS_DEFAULT_NAME_KEY, defaultUri.toString()); - - FileSystem fs = FileSystem.get(conf); - assertTrue(fs instanceof NativeAzureFileSystem); - assertEquals("wasbs", fs.getScheme()); - - // should throw if 'fs.AbstractFileSystem.wasbs.impl'' is not specified - try{ - FileContext.getFileContext(conf).getDefaultFileSystem(); - fail("Should've thrown."); - }catch(UnsupportedFileSystemException e){ - } - - } finally { - testAccount.cleanup(); - FileSystem.closeAll(); - } - } - @Test public void testCredentialProviderPathExclusions() throws Exception { String providerPath = diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestConstants.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestConstants.java index 0b72f069410..82907c57475 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestConstants.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestConstants.java @@ -143,12 +143,10 @@ public interface AzureTestConstants { - String ACCOUNT_KEY_PROPERTY_NAME - = "fs.azure.account.key."; + String ACCOUNT_KEY_PROPERTY_NAME = "fs.azure.account.key."; + String ACCOUNT_NAME_PROPERTY_NAME = "fs.azure.account.name"; String SAS_PROPERTY_NAME = "fs.azure.sas."; String TEST_CONFIGURATION_FILE_NAME = "azure-test.xml"; - String TEST_ACCOUNT_NAME_PROPERTY_NAME - = "fs.azure.test.account.name"; String MOCK_ACCOUNT_NAME = "mockAccount.blob.core.windows.net"; String MOCK_CONTAINER_NAME = "mockContainer"; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestUtils.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestUtils.java index 8d2a104eb47..b438c8e94f7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestUtils.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestUtils.java @@ -36,10 +36,14 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount; import org.apache.hadoop.fs.azure.NativeAzureFileSystem; + +import static org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.WASB_ACCOUNT_NAME_DOMAIN_SUFFIX_REGEX; +import static org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.WASB_TEST_ACCOUNT_NAME_WITH_DOMAIN; import static org.apache.hadoop.fs.azure.integration.AzureTestConstants.*; import static org.apache.hadoop.test.MetricsAsserts.getLongCounter; import static org.apache.hadoop.test.MetricsAsserts.getLongGauge; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; +import static org.junit.Assume.assumeTrue; /** * Utilities for the Azure tests. Based on {@code S3ATestUtils}, so @@ -476,4 +480,18 @@ public final class AzureTestUtils extends Assert { + KEY_SCALE_TESTS_ENABLED, enabled); } + + /** + * Check the account name for WASB tests is set correctly and return. + */ + public static String verifyWasbAccountNameInConfig(Configuration conf) { + String accountName = conf.get(ACCOUNT_NAME_PROPERTY_NAME); + if (accountName == null) { + accountName = conf.get(WASB_TEST_ACCOUNT_NAME_WITH_DOMAIN); + } + assumeTrue("Account for WASB is missing or it is not in correct format", + accountName != null && !accountName.endsWith(WASB_ACCOUNT_NAME_DOMAIN_SUFFIX_REGEX)); + return accountName; + } + } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestRollingWindowAverage.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestRollingWindowAverage.java index 9b1fb8e60e1..cd8b6927a04 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestRollingWindowAverage.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestRollingWindowAverage.java @@ -18,8 +18,8 @@ package org.apache.hadoop.fs.azure.metrics; -import static org.junit.Assert.*; -import org.junit.*; +import static org.junit.Assert.assertEquals; +import org.junit.Test; public class TestRollingWindowAverage { /** 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 e0afeb4e234..f61954f4b3d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -35,7 +35,6 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.services.AuthType; -import org.apache.hadoop.fs.azure.AbstractWasbTestWithTimeout; import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore; import org.apache.hadoop.fs.azure.NativeAzureFileSystem; import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation; @@ -45,12 +44,12 @@ import org.apache.hadoop.fs.azurebfs.utils.UriUtils; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.io.IOUtils; +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.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.junit.Assume.assumeTrue; -import static org.hamcrest.CoreMatchers.*; /** * Base for AzureBlobFileSystem Integration tests. @@ -58,12 +57,12 @@ import static org.hamcrest.CoreMatchers.*; * Important: This is for integration tests only. */ public abstract class AbstractAbfsIntegrationTest extends - AbstractWasbTestWithTimeout { + AbstractAbfsTestWithTimeout { private static final Logger LOG = LoggerFactory.getLogger(AbstractAbfsIntegrationTest.class); - private boolean isEmulator; + private boolean isIPAddress; private NativeAzureFileSystem wasb; private AzureBlobFileSystem abfs; private String abfsScheme; @@ -75,27 +74,28 @@ public abstract class AbstractAbfsIntegrationTest extends private AuthType authType; protected AbstractAbfsIntegrationTest() { - fileSystemName = ABFS_TEST_CONTAINER_PREFIX + UUID.randomUUID().toString(); + fileSystemName = TEST_CONTAINER_PREFIX + UUID.randomUUID().toString(); configuration = new Configuration(); - configuration.addResource(ABFS_TEST_RESOURCE_XML); - this.accountName = this.configuration.get(FS_AZURE_TEST_ACCOUNT_NAME); + configuration.addResource(TEST_CONFIGURATION_FILE_NAME); + + this.accountName = this.configuration.get(FS_AZURE_ACCOUNT_NAME); + if (accountName == null) { + // check if accountName is set using different config key + accountName = configuration.get(FS_AZURE_ABFS_ACCOUNT_NAME); + } + assumeTrue("Not set: " + FS_AZURE_ABFS_ACCOUNT_NAME, + accountName != null && !accountName.isEmpty()); + authType = configuration.getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME + accountName, AuthType.SharedKey); abfsScheme = authType == AuthType.SharedKey ? FileSystemUriSchemes.ABFS_SCHEME : FileSystemUriSchemes.ABFS_SECURE_SCHEME; - String accountName = configuration.get(FS_AZURE_TEST_ACCOUNT_NAME, ""); - assumeTrue("Not set: " + FS_AZURE_TEST_ACCOUNT_NAME, - !accountName.isEmpty()); - assertThat("The key in " + FS_AZURE_TEST_ACCOUNT_KEY_PREFIX - + " is not bound to an ABFS account", - accountName, containsString("dfs.core.windows.net")); - String fullKey = FS_AZURE_TEST_ACCOUNT_KEY_PREFIX - + accountName; - if (authType == AuthType.SharedKey) { - assumeTrue("Not set: " + fullKey, configuration.get(fullKey) != null); + String keyProperty = FS_AZURE_ACCOUNT_KEY_PREFIX + accountName; + assumeTrue("Not set: " + keyProperty, configuration.get(keyProperty) != null); + // Update credentials } else { String accessTokenProviderKey = FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME + accountName; assumeTrue("Not set: " + accessTokenProviderKey, configuration.get(accessTokenProviderKey) != null); @@ -113,7 +113,17 @@ public abstract class AbstractAbfsIntegrationTest extends this.testUrl = defaultUri.toString(); configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); configuration.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true); - this.isEmulator = this.configuration.getBoolean(FS_AZURE_EMULATOR_ENABLED, false); + // For testing purposes, an IP address and port may be provided to override + // the host specified in the FileSystem URI. Also note that the format of + // the Azure Storage Service URI changes from + // http[s]://[account][domain-suffix]/[filesystem] to + // http[s]://[ip]:[port]/[account]/[filesystem]. + String endPoint = configuration.get(AZURE_ABFS_ENDPOINT); + if (endPoint != null && endPoint.contains(":") && endPoint.split(":").length == 2) { + this.isIPAddress = true; + } else { + this.isIPAddress = false; + } } @@ -122,13 +132,22 @@ public abstract class AbstractAbfsIntegrationTest extends //Create filesystem first to make sure getWasbFileSystem() can return an existing filesystem. createFileSystem(); - if (!isEmulator && authType == AuthType.SharedKey) { + if (!isIPAddress && authType == AuthType.SharedKey) { final URI wasbUri = new URI(abfsUrlToWasbUrl(getTestUrl())); final AzureNativeFileSystemStore azureNativeFileSystemStore = new AzureNativeFileSystemStore(); + + // update configuration with wasb credentials + String accountNameWithoutDomain = accountName.split("\\.")[0]; + String wasbAccountName = accountNameWithoutDomain + WASB_ACCOUNT_NAME_DOMAIN_SUFFIX; + String keyProperty = FS_AZURE_ACCOUNT_KEY_PREFIX + wasbAccountName; + if (configuration.get(keyProperty) == null) { + configuration.set(keyProperty, getAccountKey()); + } + azureNativeFileSystemStore.initialize( wasbUri, - getConfiguration(), + configuration, new AzureFileSystemInstrumentation(getConfiguration())); wasb = new NativeAzureFileSystem(azureNativeFileSystemStore); @@ -201,7 +220,9 @@ public abstract class AbstractAbfsIntegrationTest extends } protected String getHostName() { - return configuration.get(FS_AZURE_TEST_HOST_NAME); + // READ FROM ENDPOINT, THIS IS CALLED ONLY WHEN TESTING AGAINST DEV-FABRIC + String endPoint = configuration.get(AZURE_ABFS_ENDPOINT); + return endPoint.split(":")[0]; } protected void setTestUrl(String testUrl) { @@ -220,21 +241,21 @@ public abstract class AbstractAbfsIntegrationTest extends } protected String getAccountName() { - return configuration.get(FS_AZURE_TEST_ACCOUNT_NAME); + return this.accountName; } protected String getAccountKey() { return configuration.get( - FS_AZURE_TEST_ACCOUNT_KEY_PREFIX - + getAccountName()); + FS_AZURE_ACCOUNT_KEY_PREFIX + + accountName); } protected Configuration getConfiguration() { return configuration; } - protected boolean isEmulator() { - return isEmulator; + protected boolean isIPAddress() { + return isIPAddress; } protected AuthType getAuthType() { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java new file mode 100644 index 00000000000..fee90abeabc --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.fs.azurebfs; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.rules.TestName; +import org.junit.rules.Timeout; + +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_TIMEOUT; + +/** + * Base class for any ABFS test with timeouts & named threads. + * This class does not attempt to bind to Azure. + */ +public class AbstractAbfsTestWithTimeout extends Assert { + /** + * The name of the current method. + */ + @Rule + public TestName methodName = new TestName(); + /** + * Set the timeout for every test. + * This is driven by the value returned by {@link #getTestTimeoutMillis()}. + */ + @Rule + public Timeout testTimeout = new Timeout(getTestTimeoutMillis()); + + /** + * Name the junit thread for the class. This will overridden + * before the individual test methods are run. + */ + @BeforeClass + public static void nameTestThread() { + Thread.currentThread().setName("JUnit"); + } + + /** + * Name the thread to the current test method. + */ + @Before + public void nameThread() { + Thread.currentThread().setName("JUnit-" + methodName.getMethodName()); + } + + /** + * Override point: the test timeout in milliseconds. + * @return a timeout in milliseconds + */ + protected int getTestTimeoutMillis() { + return TEST_TIMEOUT; + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java index 800b95abb34..46ac8664b69 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java @@ -69,7 +69,7 @@ public class ITestAzureBlobFileSystemBackCompat extends private String getBlobConnectionString() { String connectionString; - if (isEmulator()) { + if (isIPAddress()) { connectionString = "DefaultEndpointsProtocol=http;BlobEndpoint=http://" + this.getHostName() + ":8880/" + this.getAccountName().split("\\.") [0] + ";AccountName=" + this.getAccountName().split("\\.")[0] diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java index e4f3ea3d6aa..c4bfee28c3d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java @@ -50,7 +50,7 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest { LoggerFactory.getLogger(ITestWasbAbfsCompatibility.class); public ITestWasbAbfsCompatibility() throws Exception { - Assume.assumeFalse("Emulator is not supported", isEmulator()); + Assume.assumeFalse("Emulator is not supported", isIPAddress()); Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java index 556dcdb0422..bedd4f36363 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java @@ -23,6 +23,7 @@ import java.lang.reflect.Field; import org.apache.commons.codec.Charsets; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation; @@ -98,7 +99,7 @@ public class TestAbfsConfigurationFieldsValidation { this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8); this.encodedAccountKey = new String(base64.encode("someAccountKey".getBytes(Charsets.UTF_8)), Charsets.UTF_8); Configuration configuration = new Configuration(); - configuration.addResource("azure-bfs-test.xml"); + configuration.addResource(TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME); configuration.set(INT_KEY, "1234565"); configuration.set(LONG_KEY, "4194304"); configuration.set(STRING_KEY, "stringValue"); 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 67301c7f8e3..7852f25b7ed 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 @@ -22,10 +22,9 @@ package org.apache.hadoop.fs.azurebfs.constants; * Responsible to keep all the Azure Blob File System configurations keys in Hadoop configuration file. */ public final class TestConfigurationKeys { - public static final String FS_AZURE_TEST_ACCOUNT_NAME = "fs.azure.test.account.name"; - public static final String FS_AZURE_TEST_ACCOUNT_KEY_PREFIX = "fs.azure.test.account.key."; - public static final String FS_AZURE_TEST_HOST_NAME = "fs.azure.test.host.name"; - public static final String FS_AZURE_TEST_HOST_PORT = "fs.azure.test.host.port"; + 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_ACCOUNT_KEY_PREFIX = "fs.azure.account.key."; public static final String FS_AZURE_CONTRACT_TEST_URI = "fs.contract.test.fs.abfs"; public static final String FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_ID = "fs.azure.account.oauth2.contributor.client.id"; @@ -34,9 +33,9 @@ public final class TestConfigurationKeys { public static final String FS_AZURE_BLOB_DATA_READER_CLIENT_ID = "fs.azure.account.oauth2.reader.client.id"; public static final String FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET = "fs.azure.account.oauth2.reader.client.secret"; - public static final String ABFS_TEST_RESOURCE_XML = "azure-bfs-test.xml"; - - public static final String ABFS_TEST_CONTAINER_PREFIX = "abfs-testcontainer-"; + public static final String TEST_CONFIGURATION_FILE_NAME = "azure-test.xml"; + public static final String TEST_CONTAINER_PREFIX = "abfs-testcontainer-"; + public static final int TEST_TIMEOUT = 10 * 60 * 1000; private TestConfigurationKeys() {} } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java index 5505e6ae0f3..6d089d14e01 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; import org.apache.hadoop.fs.azurebfs.services.AuthType; +import org.junit.Assume; /** * Bind ABFS contract tests to the Azure test setup/teardown. @@ -42,6 +43,8 @@ public class ABFSContractTestBinding extends AbstractAbfsIntegrationTest { if (useExistingFileSystem) { Configuration configuration = getConfiguration(); String testUrl = configuration.get(TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI); + Assume.assumeTrue("Contract tests are skipped because of missing config property :" + + TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI, testUrl != null); if (getAuthType() != AuthType.SharedKey) { testUrl = testUrl.replaceFirst(FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java index e9e90c993a7..a9556ab1c0f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java @@ -32,7 +32,6 @@ import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.M import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE; -import static org.junit.Assert.assertEquals; /** * Test configuration validators. diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AbfsTestUtils.java similarity index 64% rename from hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java rename to hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AbfsTestUtils.java index ef4ddb92157..9c636a2ce4f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AbfsTestUtils.java @@ -22,56 +22,59 @@ import com.microsoft.azure.storage.blob.CloudBlobClient; import com.microsoft.azure.storage.blob.CloudBlobContainer; import org.junit.Assume; -import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; import org.apache.hadoop.fs.azurebfs.services.AuthType; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.ABFS_TEST_CONTAINER_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONTAINER_PREFIX; -/** - * If unit tests were interrupted and crushed accidentally, the test containers won't be deleted. - * In that case, dev can use this tool to list and delete all test containers. - * By default, all test container used in E2E tests sharing same prefix: "abfs-testcontainer-" - */ -public class CleanUpAbfsTestContainer extends AbstractAbfsIntegrationTest{ + /* + * Some Utils for ABFS tests. + * */ +public final class AbfsTestUtils extends AbstractAbfsIntegrationTest{ + private static final Logger LOG = + LoggerFactory.getLogger(AbfsTestUtils.class); + /** + * If unit tests were interrupted and crushed accidentally, the test containers won't be deleted. + * In that case, dev can use this tool to list and delete all test containers. + * By default, all test container used in E2E tests sharing same prefix: "abfs-testcontainer-" + */ - public CleanUpAbfsTestContainer() { + public void checkContainers() throws Throwable { Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); - } - - @Test - public void testEnumContainers() throws Throwable { int count = 0; CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount(); CloudBlobClient blobClient = storageAccount.createCloudBlobClient(); Iterable containers - = blobClient.listContainers(ABFS_TEST_CONTAINER_PREFIX); + = blobClient.listContainers(TEST_CONTAINER_PREFIX); for (CloudBlobContainer container : containers) { count++; - System.out.println(String.format("Container %s URI %s", + LOG.info("Container {}, URI {}", container.getName(), - container.getUri())); + container.getUri()); } - System.out.println(String.format("Found %d test containers", count)); + LOG.info("Found {} test containers", count); } - @Test - public void testDeleteContainers() throws Throwable { + + public void deleteContainers() throws Throwable { + Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); int count = 0; CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount(); CloudBlobClient blobClient = storageAccount.createCloudBlobClient(); Iterable containers - = blobClient.listContainers(ABFS_TEST_CONTAINER_PREFIX); + = blobClient.listContainers(TEST_CONTAINER_PREFIX); for (CloudBlobContainer container : containers) { - System.out.println(String.format("Container %s URI %s", + LOG.info("Container {} URI {}", container.getName(), - container.getUri())); + container.getUri()); if (container.deleteIfExists()) { count++; } } - System.out.println(String.format("Deleted %s test containers", count)); + LOG.info("Deleted {} test containers", count); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TestUriUtils.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TestUriUtils.java index e71e31de69e..690e56c5105 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TestUriUtils.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TestUriUtils.java @@ -38,11 +38,11 @@ public final class TestUriUtils { @Test public void testExtractRawAccountName() throws Exception { - Assert.assertEquals("abfs", UriUtils.extractRawAccountFromAccountName("abfs.dfs.core.windows.net")); - Assert.assertEquals("abfs", UriUtils.extractRawAccountFromAccountName("abfs.dfs.preprod.core.windows.net")); - Assert.assertEquals(null, UriUtils.extractRawAccountFromAccountName("abfs.dfs.cores.windows.net")); - Assert.assertEquals(null, UriUtils.extractRawAccountFromAccountName("")); - Assert.assertEquals(null, UriUtils.extractRawAccountFromAccountName(null)); - Assert.assertEquals(null, UriUtils.extractRawAccountFromAccountName("abfs.dfs.cores.windows.net")); + Assert.assertEquals("abfs", UriUtils.extractAccountNameFromHostName("abfs.dfs.core.windows.net")); + Assert.assertEquals("abfs", UriUtils.extractAccountNameFromHostName("abfs.dfs.preprod.core.windows.net")); + Assert.assertEquals(null, UriUtils.extractAccountNameFromHostName("abfs.dfs.cores.windows.net")); + Assert.assertEquals(null, UriUtils.extractAccountNameFromHostName("")); + Assert.assertEquals(null, UriUtils.extractAccountNameFromHostName(null)); + Assert.assertEquals(null, UriUtils.extractAccountNameFromHostName("abfs.dfs.cores.windows.net")); } } diff --git a/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml b/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml deleted file mode 100644 index 9b908b06fd7..00000000000 --- a/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml +++ /dev/null @@ -1,188 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - 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 8cea256de8c..a36a391cd56 100644 --- a/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml +++ b/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml @@ -16,49 +16,39 @@ - + + + + fs.azure.test.emulator + false + - + + fs.azure.secure.mode + false + - - + + + fs.azure.user.agent.prefix MSFT - - - - + + + + - - - - fs.AbstractFileSystem.wasb.impl - org.apache.hadoop.fs.azure.Wasb - From 4410eacba7862ec24173356fe3fd468fd79aeb8f Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Sat, 1 Sep 2018 20:39:34 +0000 Subject: [PATCH 13/29] HADOOP-15664. ABFS: Reduce test run time via parallelization and grouping. Contributed by Da Zhou. --- hadoop-tools/hadoop-azure/pom.xml | 350 +++++++++++++++++- .../fs/azurebfs/AzureBlobFileSystem.java | 8 +- .../azurebfs/services/AbfsOutputStream.java | 6 + .../ITestNativeFileSystemStatistics.java | 99 +++++ .../azure/NativeAzureFileSystemBaseTest.java | 80 +--- .../fs/azure/integration/AzureTestUtils.java | 53 ++- .../ITestAzureBlobFileSystemE2EScale.java | 11 +- .../ITestAzureBlobFileSystemFileStatus.java | 3 + .../ITestAzureBlobFileSystemFlush.java | 169 +++++---- .../azurebfs/ITestWasbAbfsCompatibility.java | 2 +- 10 files changed, 632 insertions(+), 149 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeFileSystemStatistics.java diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml index 7152f6383a3..42f4d05d404 100644 --- a/hadoop-tools/hadoop-azure/pom.xml +++ b/hadoop-tools/hadoop-azure/pom.xml @@ -252,6 +252,351 @@ + + parallel-tests-wasb + + + parallel-tests-wasb + + + + + + maven-antrun-plugin + + + create-parallel-tests-dirs + test-compile + + + + + + + run + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + default-test + + test + + + 1 + ${testsThreadCount} + false + ${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true + ${fs.azure.scale.test.timeout} + + ${test.build.data}/${surefire.forkNumber} + ${test.build.dir}/${surefire.forkNumber} + ${hadoop.tmp.dir}/${surefire.forkNumber} + fork-${surefire.forkNumber} + ${fs.azure.scale.test.enabled} + ${fs.azure.scale.test.huge.filesize} + ${fs.azure.scale.test.huge.partitionsize} + ${fs.azure.scale.test.timeout} + ${fs.azure.scale.test.list.performance.threads} + ${fs.azure.scale.test.list.performance.files} + + + **/azure/Test*.java + **/azure/**/Test*.java + + + **/azure/**/TestRollingWindowAverage*.java + + + + + serialized-test-wasb + + test + + + 1 + false + ${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true + ${fs.azure.scale.test.timeout} + + ${test.build.data}/${surefire.forkNumber} + ${test.build.dir}/${surefire.forkNumber} + ${hadoop.tmp.dir}/${surefire.forkNumber} + fork-${surefire.forkNumber} + ${fs.azure.scale.test.enabled} + ${fs.azure.scale.test.huge.filesize} + ${fs.azure.scale.test.huge.partitionsize} + ${fs.azure.scale.test.timeout} + ${fs.azure.scale.test.list.performance.threads} + ${fs.azure.scale.test.list.performance.files} + + + **/azure/**/TestRollingWindowAverage*.java + + + + + + + org.apache.maven.plugins + maven-failsafe-plugin + + + default-integration-test-wasb + + integration-test + verify + + + 1 + ${testsThreadCount} + false + ${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true + ${fs.azure.scale.test.timeout} + false + + + true + ${test.build.data}/${surefire.forkNumber} + ${test.build.dir}/${surefire.forkNumber} + ${hadoop.tmp.dir}/${surefire.forkNumber} + + + + + + fork-${surefire.forkNumber} + + ${fs.azure.scale.test.enabled} + ${fs.azure.scale.test.huge.filesize} + ${fs.azure.scale.test.huge.partitionsize} + ${fs.azure.scale.test.timeout} + ${fs.azure.scale.test.list.performance.threads} + ${fs.azure.scale.test.list.performance.files} + + + + **/azure/ITest*.java + **/azure/**/ITest*.java + + + **/azure/ITestNativeFileSystemStatistics.java + + + + + + + sequential-integration-tests-wasb + + integration-test + verify + + + ${fs.azure.scale.test.timeout} + false + + false + ${fs.azure.scale.test.enabled} + ${fs.azure.scale.test.huge.filesize} + ${fs.azure.scale.test.huge.partitionsize} + ${fs.azure.scale.test.timeout} + ${fs.azure.scale.test.list.performance.threads} + ${fs.azure.scale.test.list.performance.files} + + + **/azure/ITestNativeFileSystemStatistics.java + + + + + + + + + + + parallel-tests-abfs + + + parallel-tests-abfs + + + + + + maven-antrun-plugin + + + create-parallel-tests-dirs + test-compile + + + + + + + run + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + default-test + + test + + + ${testsThreadCount} + false + ${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true + ${fs.azure.scale.test.timeout} + + ${test.build.data}/${surefire.forkNumber} + ${test.build.dir}/${surefire.forkNumber} + ${hadoop.tmp.dir}/${surefire.forkNumber} + fork-${surefire.forkNumber} + ${fs.azure.scale.test.enabled} + ${fs.azure.scale.test.huge.filesize} + ${fs.azure.scale.test.huge.partitionsize} + ${fs.azure.scale.test.timeout} + ${fs.azure.scale.test.list.performance.threads} + ${fs.azure.scale.test.list.performance.files} + + + **/azurebfs/Test*.java + **/azurebfs/**/Test*.java + + + + + + + org.apache.maven.plugins + maven-failsafe-plugin + + + integration-test-abfs-parallel-classesAndMethods + + integration-test + verify + + + ${testsThreadCount} + true + both + ${testsThreadCount} + ${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true + ${fs.azure.scale.test.timeout} + false + + + true + ${test.build.data}/${surefire.forkNumber} + ${test.build.dir}/${surefire.forkNumber} + ${hadoop.tmp.dir}/${surefire.forkNumber} + + + + + fork-${surefire.forkNumber} + + ${fs.azure.scale.test.enabled} + ${fs.azure.scale.test.timeout} + + + + **/azurebfs/ITest*.java + **/azurebfs/**/ITest*.java + + + **/azurebfs/contract/ITest*.java + **/azurebfs/ITestAzureBlobFileSystemE2EScale.java + **/azurebfs/ITestAbfsReadWriteAndSeek.java + **/azurebfs/ITestAzureBlobFileSystemListStatus.java + + + + + + integration-test-abfs-parallel-classes + + integration-test + verify + + + ${testsThreadCount} + false + + ${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true + ${fs.azure.scale.test.timeout} + false + + + true + ${test.build.data}/${surefire.forkNumber} + ${test.build.dir}/${surefire.forkNumber} + ${hadoop.tmp.dir}/${surefire.forkNumber} + + + + + + fork-${surefire.forkNumber} + + ${fs.azure.scale.test.enabled} + ${fs.azure.scale.test.timeout} + + + **/azurebfs/contract/ITest*.java + **/azurebfs/ITestAzureBlobFileSystemE2EScale.java + **/azurebfs/ITestAbfsReadWriteAndSeek.java + **/azurebfs/ITestAzureBlobFileSystemListStatus.java + + + + + + + + + parallel-tests @@ -417,6 +762,7 @@ **/ITestWasbRemoteCallHelper.java **/ITestBlockBlobInputStream.java **/ITestWasbAbfsCompatibility.java + **/ITestNativeFileSystemStatistics.java @@ -452,6 +798,7 @@ **/ITestAzureBlobFileSystemRandomRead.java **/ITestWasbRemoteCallHelper.java **/ITestBlockBlobInputStream.java + **/ITestNativeFileSystemStatistics.java @@ -460,11 +807,12 @@ + sequential-tests - !parallel-tests + sequential-tests 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 4bde9d80eb4..b8091921079 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 @@ -107,7 +107,11 @@ public class AzureBlobFileSystem extends FileSystem { if (abfsStore.getAbfsConfiguration().getCreateRemoteFileSystemDuringInitialization()) { if (!this.fileSystemExists()) { - this.createFileSystem(); + try { + this.createFileSystem(); + } catch (AzureBlobFileSystemException ex) { + checkException(null, ex, AzureServiceErrorCode.FILE_SYSTEM_ALREADY_EXISTS); + } } } @@ -121,7 +125,7 @@ public class AzureBlobFileSystem extends FileSystem { if (UserGroupInformation.isSecurityEnabled()) { this.delegationTokenEnabled = abfsStore.getAbfsConfiguration().isDelegationTokenManagerEnabled(); - if(this.delegationTokenEnabled) { + if (this.delegationTokenEnabled) { LOG.debug("Initializing DelegationTokenManager for {}", uri); this.delegationTokenManager = abfsStore.getAbfsConfiguration().getDelegationTokenManager(); } 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 92e081eaa12..7e43090a957 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 @@ -30,6 +30,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.apache.hadoop.fs.FSExceptionMessages; @@ -369,4 +370,9 @@ public class AbfsOutputStream extends OutputStream implements Syncable, StreamCa this.length = length; } } + + @VisibleForTesting + public synchronized void waitForPendingUploads() throws IOException { + waitForTaskToComplete(); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeFileSystemStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeFileSystemStatistics.java new file mode 100644 index 00000000000..cbb09ddff8b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeFileSystemStatistics.java @@ -0,0 +1,99 @@ +/* + * 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.azure; + +import org.junit.FixMethodOrder; +import org.junit.Test; +import org.junit.runners.MethodSorters; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import static org.junit.Assume.assumeNotNull; +import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.cleanupTestAccount; +import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.readStringFromFile; +import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.writeStringToFile; + +@FixMethodOrder(MethodSorters.NAME_ASCENDING) +/** + * Because FileSystem.Statistics is per FileSystem, so statistics can not be ran in + * parallel, hence in this test file, force them to run in sequential. + * */ +public class ITestNativeFileSystemStatistics extends AbstractWasbTestWithTimeout{ + + @Test + public void test_001_NativeAzureFileSystemMocked() throws Exception { + AzureBlobStorageTestAccount testAccount = AzureBlobStorageTestAccount.createMock(); + assumeNotNull(testAccount); + testStatisticsWithAccount(testAccount); + } + + @Test + public void test_002_NativeAzureFileSystemPageBlobLive() throws Exception { + Configuration conf = new Configuration(); + // Configure the page blob directories key so every file created is a page blob. + conf.set(AzureNativeFileSystemStore.KEY_PAGE_BLOB_DIRECTORIES, "/"); + + // Configure the atomic rename directories key so every folder will have + // atomic rename applied. + conf.set(AzureNativeFileSystemStore.KEY_ATOMIC_RENAME_DIRECTORIES, "/"); + AzureBlobStorageTestAccount testAccount = AzureBlobStorageTestAccount.create(conf); + assumeNotNull(testAccount); + testStatisticsWithAccount(testAccount); + } + + @Test + public void test_003_NativeAzureFileSystem() throws Exception { + AzureBlobStorageTestAccount testAccount = AzureBlobStorageTestAccount.create(); + assumeNotNull(testAccount); + testStatisticsWithAccount(testAccount); + } + + private void testStatisticsWithAccount(AzureBlobStorageTestAccount testAccount) throws Exception { + assumeNotNull(testAccount); + NativeAzureFileSystem fs = testAccount.getFileSystem(); + testStatistics(fs); + cleanupTestAccount(testAccount); + } + + /** + * When tests are ran in parallel, this tests will fail because + * FileSystem.Statistics is per FileSystem class. + */ + @SuppressWarnings("deprecation") + private void testStatistics(NativeAzureFileSystem fs) throws Exception { + FileSystem.clearStatistics(); + FileSystem.Statistics stats = FileSystem.getStatistics("wasb", + NativeAzureFileSystem.class); + assertEquals(0, stats.getBytesRead()); + assertEquals(0, stats.getBytesWritten()); + Path newFile = new Path("testStats"); + writeStringToFile(fs, newFile, "12345678"); + assertEquals(8, stats.getBytesWritten()); + assertEquals(0, stats.getBytesRead()); + String readBack = readStringFromFile(fs, newFile); + assertEquals("12345678", readBack); + assertEquals(8, stats.getBytesRead()); + assertEquals(8, stats.getBytesWritten()); + assertTrue(fs.delete(newFile, true)); + assertEquals(8, stats.getBytesRead()); + assertEquals(8, stats.getBytesWritten()); + } +} 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 726b5049b4c..19d370ebc99 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 @@ -18,14 +18,10 @@ package org.apache.hadoop.fs.azure; -import java.io.BufferedReader; -import java.io.BufferedWriter; import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; -import java.io.InputStreamReader; import java.io.OutputStream; -import java.io.OutputStreamWriter; import java.util.ArrayList; import java.util.Arrays; import java.util.Calendar; @@ -51,6 +47,9 @@ import com.microsoft.azure.storage.AccessCondition; import com.microsoft.azure.storage.StorageException; import com.microsoft.azure.storage.blob.CloudBlob; +import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.readStringFromFile; +import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.writeStringToFile; +import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.writeStringToStream; import static org.apache.hadoop.test.GenericTestUtils.*; /* @@ -329,12 +328,12 @@ public abstract class NativeAzureFileSystemBaseTest FileSystem localFs = FileSystem.get(new Configuration()); localFs.delete(localFilePath, true); try { - writeString(localFs, localFilePath, "Testing"); + writeStringToFile(localFs, localFilePath, "Testing"); Path dstPath = methodPath(); assertTrue(FileUtil.copy(localFs, localFilePath, fs, dstPath, false, fs.getConf())); assertPathExists("coied from local", dstPath); - assertEquals("Testing", readString(fs, dstPath)); + assertEquals("Testing", readStringFromFile(fs, dstPath)); fs.delete(dstPath, true); } finally { localFs.delete(localFilePath, true); @@ -363,26 +362,6 @@ public abstract class NativeAzureFileSystemBaseTest assertTrue(fs.delete(rootFolder, true)); } - @Test - public void testStatistics() throws Exception { - FileSystem.clearStatistics(); - FileSystem.Statistics stats = FileSystem.getStatistics("wasb", - NativeAzureFileSystem.class); - assertEquals(0, stats.getBytesRead()); - assertEquals(0, stats.getBytesWritten()); - Path newFile = new Path("testStats"); - writeString(newFile, "12345678"); - assertEquals(8, stats.getBytesWritten()); - assertEquals(0, stats.getBytesRead()); - String readBack = readString(newFile); - assertEquals("12345678", readBack); - assertEquals(8, stats.getBytesRead()); - assertEquals(8, stats.getBytesWritten()); - assertTrue(fs.delete(newFile, true)); - assertEquals(8, stats.getBytesRead()); - assertEquals(8, stats.getBytesWritten()); - } - @Test public void testUriEncoding() throws Exception { fs.create(new Path("p/t%5Fe")).close(); @@ -767,7 +746,7 @@ public abstract class NativeAzureFileSystemBaseTest Path renamePendingFile = new Path(renamePendingStr); FSDataOutputStream out = fs.create(renamePendingFile, true); assertTrue(out != null); - writeString(out, renameDescription); + writeStringToStream(out, renameDescription); // Redo the rename operation based on the contents of the -RenamePending.json file. // Trigger the redo by checking for existence of the original folder. It must appear @@ -831,7 +810,7 @@ public abstract class NativeAzureFileSystemBaseTest Path renamePendingFile = new Path(renamePendingStr); FSDataOutputStream out = fs.create(renamePendingFile, true); assertTrue(out != null); - writeString(out, pending.makeRenamePendingFileContents()); + writeStringToStream(out, pending.makeRenamePendingFileContents()); // Redo the rename operation based on the contents of the // -RenamePending.json file. Trigger the redo by checking for existence of @@ -886,7 +865,7 @@ public abstract class NativeAzureFileSystemBaseTest Path renamePendingFile = new Path(renamePendingStr); FSDataOutputStream out = fs.create(renamePendingFile, true); assertTrue(out != null); - writeString(out, pending.makeRenamePendingFileContents()); + writeStringToStream(out, pending.makeRenamePendingFileContents()); // Rename inner folder to simulate the scenario where rename has started and // only one directory has been renamed but not the files under it @@ -1000,7 +979,7 @@ public abstract class NativeAzureFileSystemBaseTest Path renamePendingFile = new Path(renamePendingStr); FSDataOutputStream out = fs.create(renamePendingFile, true); assertTrue(out != null); - writeString(out, pending.makeRenamePendingFileContents()); + writeStringToStream(out, pending.makeRenamePendingFileContents()); try { pending.redo(); @@ -1228,7 +1207,7 @@ public abstract class NativeAzureFileSystemBaseTest Path renamePendingFile = new Path(renamePendingStr); FSDataOutputStream out = fs.create(renamePendingFile, true); assertTrue(out != null); - writeString(out, renameDescription); + writeStringToStream(out, renameDescription); } // set whether a child is present or not @@ -1488,7 +1467,7 @@ public abstract class NativeAzureFileSystemBaseTest Calendar utc = Calendar.getInstance(TimeZone.getTimeZone("UTC")); long currentUtcTime = utc.getTime().getTime(); FileStatus fileStatus = fs.getFileStatus(testPath); - final long errorMargin = 10 * 1000; // Give it +/-10 seconds + final long errorMargin = 60 * 1000; // Give it +/-60 seconds assertTrue("Modification time " + new Date(fileStatus.getModificationTime()) + " is not close to now: " + utc.getTime(), @@ -1504,45 +1483,12 @@ public abstract class NativeAzureFileSystemBaseTest } private String readString(Path testFile) throws IOException { - return readString(fs, testFile); + return readStringFromFile(fs, testFile); } - private String readString(FileSystem fs, Path testFile) throws IOException { - FSDataInputStream inputStream = fs.open(testFile); - String ret = readString(inputStream); - inputStream.close(); - return ret; - } - - private String readString(FSDataInputStream inputStream) throws IOException { - BufferedReader reader = new BufferedReader(new InputStreamReader( - inputStream)); - final int BUFFER_SIZE = 1024; - char[] buffer = new char[BUFFER_SIZE]; - int count = reader.read(buffer, 0, BUFFER_SIZE); - if (count > BUFFER_SIZE) { - throw new IOException("Exceeded buffer size"); - } - inputStream.close(); - return new String(buffer, 0, count); - } private void writeString(Path path, String value) throws IOException { - writeString(fs, path, value); - } - - private void writeString(FileSystem fs, Path path, String value) - throws IOException { - FSDataOutputStream outputStream = fs.create(path, true); - writeString(outputStream, value); - } - - private void writeString(FSDataOutputStream outputStream, String value) - throws IOException { - BufferedWriter writer = new BufferedWriter(new OutputStreamWriter( - outputStream)); - writer.write(value); - writer.close(); + writeStringToFile(fs, path, value); } @Test diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestUtils.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestUtils.java index b438c8e94f7..c46320a4835 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestUtils.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestUtils.java @@ -18,7 +18,11 @@ package org.apache.hadoop.fs.azure.integration; +import java.io.BufferedReader; +import java.io.BufferedWriter; import java.io.IOException; +import java.io.InputStreamReader; +import java.io.OutputStreamWriter; import java.net.URI; import java.util.List; @@ -30,12 +34,15 @@ import org.slf4j.LoggerFactory; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount; import org.apache.hadoop.fs.azure.NativeAzureFileSystem; +import static org.junit.Assume.assumeTrue; import static org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.WASB_ACCOUNT_NAME_DOMAIN_SUFFIX_REGEX; import static org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.WASB_TEST_ACCOUNT_NAME_WITH_DOMAIN; @@ -43,7 +50,6 @@ import static org.apache.hadoop.fs.azure.integration.AzureTestConstants.*; import static org.apache.hadoop.test.MetricsAsserts.getLongCounter; import static org.apache.hadoop.test.MetricsAsserts.getLongGauge; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; -import static org.junit.Assume.assumeTrue; /** * Utilities for the Azure tests. Based on {@code S3ATestUtils}, so @@ -494,4 +500,49 @@ public final class AzureTestUtils extends Assert { return accountName; } + /** + * Write string into a file. + */ + public static void writeStringToFile(FileSystem fs, Path path, String value) + throws IOException { + FSDataOutputStream outputStream = fs.create(path, true); + writeStringToStream(outputStream, value); + } + + /** + * Write string into a file. + */ + public static void writeStringToStream(FSDataOutputStream outputStream, String value) + throws IOException { + BufferedWriter writer = new BufferedWriter(new OutputStreamWriter( + outputStream)); + writer.write(value); + writer.close(); + } + + /** + * Read string from a file. + */ + public static String readStringFromFile(FileSystem fs, Path testFile) throws IOException { + FSDataInputStream inputStream = fs.open(testFile); + String ret = readStringFromStream(inputStream); + inputStream.close(); + return ret; + } + + /** + * Read string from stream. + */ + public static String readStringFromStream(FSDataInputStream inputStream) throws IOException { + BufferedReader reader = new BufferedReader(new InputStreamReader( + inputStream)); + final int BUFFER_SIZE = 1024; + char[] buffer = new char[BUFFER_SIZE]; + int count = reader.read(buffer, 0, BUFFER_SIZE); + if (count > BUFFER_SIZE) { + throw new IOException("Exceeded buffer size"); + } + inputStream.close(); + return new String(buffer, 0, count); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java index 522b635e9d7..7ed9d42bb19 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java @@ -44,7 +44,6 @@ public class ITestAzureBlobFileSystemE2EScale extends private static final int BASE_SIZE = 1024; private static final int ONE_MB = 1024 * 1024; private static final int DEFAULT_WRITE_TIMES = 100; - private static final Path TEST_FILE = new Path("ITestAzureBlobFileSystemE2EScale"); public ITestAzureBlobFileSystemE2EScale() { } @@ -52,7 +51,8 @@ public class ITestAzureBlobFileSystemE2EScale extends @Test public void testWriteHeavyBytesToFileAcrossThreads() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); - final FSDataOutputStream stream = fs.create(TEST_FILE); + final Path testFile = path(methodName.getMethodName()); + final FSDataOutputStream stream = fs.create(testFile); ExecutorService es = Executors.newFixedThreadPool(TEN); int testWriteBufferSize = 2 * TEN * ONE_THOUSAND * BASE_SIZE; @@ -81,7 +81,7 @@ public class ITestAzureBlobFileSystemE2EScale extends stream.close(); es.shutdownNow(); - FileStatus fileStatus = fs.getFileStatus(TEST_FILE); + FileStatus fileStatus = fs.getFileStatus(testFile); assertEquals(testWriteBufferSize * operationCount, fileStatus.getLen()); } @@ -89,9 +89,10 @@ public class ITestAzureBlobFileSystemE2EScale extends public void testReadWriteHeavyBytesToFileWithStatistics() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); final FileSystem.Statistics abfsStatistics; + final Path testFile = path(methodName.getMethodName()); int testBufferSize; final byte[] sourceData; - try (FSDataOutputStream stream = fs.create(TEST_FILE)) { + try (FSDataOutputStream stream = fs.create(testFile)) { abfsStatistics = fs.getFsStatistics(); abfsStatistics.reset(); @@ -103,7 +104,7 @@ public class ITestAzureBlobFileSystemE2EScale extends final byte[] remoteData = new byte[testBufferSize]; int bytesRead; - try (FSDataInputStream inputStream = fs.open(TEST_FILE, 4 * ONE_MB)) { + try (FSDataInputStream inputStream = fs.open(testFile, 4 * ONE_MB)) { bytesRead = inputStream.read(remoteData); } 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 88f77b0bcae..dba10f5f13f 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 @@ -22,6 +22,7 @@ import java.io.IOException; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.azurebfs.services.AuthType; +import org.junit.Ignore; import org.junit.Test; import org.apache.hadoop.fs.FileStatus; @@ -53,6 +54,7 @@ public class ITestAzureBlobFileSystemFileStatus extends assertEquals("root listing", 0, rootls.length); } + @Ignore("When running against live abfs with Oauth account, this test will fail. Need to check the tenant.") @Test public void testFileStatusPermissionsAndOwnerAndGroup() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); @@ -86,6 +88,7 @@ public class ITestAzureBlobFileSystemFileStatus extends return fileStatus; } + @Ignore("When running against live abfs with Oauth account, this test will fail. Need to check the tenant.") @Test public void testFolderStatusPermissionsAndOwnerAndGroup() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index 7c6bbb5c607..337f95ce917 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -18,20 +18,19 @@ package org.apache.hadoop.fs.azurebfs; +import java.io.InputStream; import java.util.ArrayList; import java.util.List; -import java.util.EnumSet; import java.util.Random; +import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.io.IOException; -import com.microsoft.azure.storage.blob.BlockEntry; -import com.microsoft.azure.storage.blob.BlockListingFilter; -import com.microsoft.azure.storage.blob.CloudBlockBlob; import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.hamcrest.core.IsEqual; import org.hamcrest.core.IsNot; import org.junit.Assume; @@ -43,11 +42,12 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount; import org.apache.hadoop.fs.azurebfs.services.AuthType; /** * Test flush operation. + * This class cannot be run in parallel test mode--check comments in + * testWriteHeavyBytesToFileSyncFlush(). */ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { private static final int BASE_SIZE = 1024; @@ -55,11 +55,10 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { private static final int TEST_BUFFER_SIZE = 5 * ONE_THOUSAND * BASE_SIZE; private static final int ONE_MB = 1024 * 1024; private static final int FLUSH_TIMES = 200; - private static final int THREAD_SLEEP_TIME = 6000; + private static final int THREAD_SLEEP_TIME = 1000; - private static final Path TEST_FILE_PATH = new Path("/testfile"); private static final int TEST_FILE_LENGTH = 1024 * 1024 * 8; - private static final int WAITING_TIME = 4000; + private static final int WAITING_TIME = 1000; public ITestAzureBlobFileSystemFlush() { super(); @@ -68,8 +67,9 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { @Test public void testAbfsOutputStreamAsyncFlushWithRetainUncommittedData() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); + final Path testFilePath = path(methodName.getMethodName()); final byte[] b; - try (FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { + try (FSDataOutputStream stream = fs.create(testFilePath)) { b = new byte[TEST_BUFFER_SIZE]; new Random().nextBytes(b); @@ -84,7 +84,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { } final byte[] r = new byte[TEST_BUFFER_SIZE]; - try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB)) { + try (FSDataInputStream inputStream = fs.open(testFilePath, 4 * ONE_MB)) { while (inputStream.available() != 0) { int result = inputStream.read(r); @@ -97,8 +97,10 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { @Test public void testAbfsOutputStreamSyncFlush() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); + final Path testFilePath = path(methodName.getMethodName()); + final byte[] b; - try (FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { + try (FSDataOutputStream stream = fs.create(testFilePath)) { b = new byte[TEST_BUFFER_SIZE]; new Random().nextBytes(b); stream.write(b); @@ -111,7 +113,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { } final byte[] r = new byte[TEST_BUFFER_SIZE]; - try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH, 4 * ONE_MB)) { + try (FSDataInputStream inputStream = fs.open(testFilePath, 4 * ONE_MB)) { int result = inputStream.read(r); assertNotEquals(-1, result); @@ -123,12 +125,9 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { @Test public void testWriteHeavyBytesToFileSyncFlush() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); - final FileSystem.Statistics abfsStatistics; + final Path testFilePath = path(methodName.getMethodName()); ExecutorService es; - try (FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { - abfsStatistics = fs.getFsStatistics(); - abfsStatistics.reset(); - + try (FSDataOutputStream stream = fs.create(testFilePath)) { es = Executors.newFixedThreadPool(10); final byte[] b = new byte[TEST_BUFFER_SIZE]; @@ -163,18 +162,18 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { } es.shutdownNow(); - FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH); + FileStatus fileStatus = fs.getFileStatus(testFilePath); long expectedWrites = (long) TEST_BUFFER_SIZE * FLUSH_TIMES; - assertEquals("Wrong file length in " + fileStatus, expectedWrites, fileStatus.getLen()); - assertEquals("wrong bytes Written count in " + abfsStatistics, - expectedWrites, abfsStatistics.getBytesWritten()); + assertEquals("Wrong file length in " + testFilePath, expectedWrites, fileStatus.getLen()); } @Test public void testWriteHeavyBytesToFileAsyncFlush() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); ExecutorService es = Executors.newFixedThreadPool(10); - try (FSDataOutputStream stream = fs.create(TEST_FILE_PATH)) { + + final Path testFilePath = path(methodName.getMethodName()); + try (FSDataOutputStream stream = fs.create(testFilePath)) { final byte[] b = new byte[TEST_BUFFER_SIZE]; new Random().nextBytes(b); @@ -207,54 +206,50 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { } es.shutdownNow(); - FileStatus fileStatus = fs.getFileStatus(TEST_FILE_PATH); + FileStatus fileStatus = fs.getFileStatus(testFilePath); assertEquals((long) TEST_BUFFER_SIZE * FLUSH_TIMES, fileStatus.getLen()); } @Test public void testFlushWithFlushEnabled() throws Exception { - Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); - - AzureBlobStorageTestAccount testAccount = createWasbTestAccount(); - String wasbUrl = testAccount.getFileSystem().getName(); - String abfsUrl = wasbUrlToAbfsUrl(wasbUrl); - final AzureBlobFileSystem fs = this.getFileSystem(abfsUrl); - // test only valid for non-namespace enabled account - Assume.assumeFalse(fs.getIsNamespaceEnabeld()); - - byte[] buffer = getRandomBytesArray(); - CloudBlockBlob blob = testAccount.getBlobReference(TEST_FILE_PATH.toString().substring(1)); - try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, true)) { - // Wait for write request to be executed - Thread.sleep(WAITING_TIME); - stream.flush(); - ArrayList blockList = blob.downloadBlockList( - BlockListingFilter.COMMITTED, null, null, null); - // verify block has been committed - assertEquals(1, blockList.size()); - } + testFlush(true); } @Test public void testFlushWithFlushDisabled() throws Exception { - Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); - AzureBlobStorageTestAccount testAccount = createWasbTestAccount(); - String wasbUrl = testAccount.getFileSystem().getName(); - String abfsUrl = wasbUrlToAbfsUrl(wasbUrl); - final AzureBlobFileSystem fs = this.getFileSystem(abfsUrl); - // test only valid for non-namespace enabled account - Assume.assumeFalse(fs.getIsNamespaceEnabeld()); + testFlush(false); + } + private void testFlush(boolean flushEnabled) throws Exception { + Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); + + final AzureBlobFileSystem fs = (AzureBlobFileSystem) getFileSystem(); + + // Simulate setting "fs.azure.enable.flush" to true or false + fs.getAbfsStore().getAbfsConfiguration().setEnableFlush(flushEnabled); + + final Path testFilePath = path(methodName.getMethodName()); byte[] buffer = getRandomBytesArray(); - CloudBlockBlob blob = testAccount.getBlobReference(TEST_FILE_PATH.toString().substring(1)); - try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, false)) { - // Wait for write request to be executed - Thread.sleep(WAITING_TIME); + + // The test case must write "fs.azure.write.request.size" bytes + // to the stream in order for the data to be uploaded to storage. + assertEquals( + fs.getAbfsStore().getAbfsConfiguration().getWriteBufferSize(), + buffer.length); + + try (FSDataOutputStream stream = fs.create(testFilePath)) { + stream.write(buffer); + + // Write asynchronously uploads data, so we must wait for completion + AbfsOutputStream abfsStream = (AbfsOutputStream) stream.getWrappedStream(); + abfsStream.waitForPendingUploads(); + + // Flush commits the data so it can be read. stream.flush(); - ArrayList blockList = blob.downloadBlockList( - BlockListingFilter.COMMITTED, null, null, null); - // verify block has not been committed - assertEquals(0, blockList.size()); + + // Verify that the data can be read if flushEnabled is true; and otherwise + // cannot be read. + validate(fs.open(testFilePath), buffer, flushEnabled); } } @@ -262,9 +257,12 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { public void testHflushWithFlushEnabled() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); byte[] buffer = getRandomBytesArray(); - try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, true)) { + String fileName = UUID.randomUUID().toString(); + final Path testFilePath = path(fileName); + + try (FSDataOutputStream stream = getStreamAfterWrite(fs, testFilePath, buffer, true)) { stream.hflush(); - validate(fs, TEST_FILE_PATH, buffer, true); + validate(fs, testFilePath, buffer, true); } } @@ -272,9 +270,11 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { public void testHflushWithFlushDisabled() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); byte[] buffer = getRandomBytesArray(); - try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, false)) { + final Path testFilePath = path(methodName.getMethodName()); + + try (FSDataOutputStream stream = getStreamAfterWrite(fs, testFilePath, buffer, false)) { stream.hflush(); - validate(fs, TEST_FILE_PATH, buffer, false); + validate(fs, testFilePath, buffer, false); } } @@ -282,9 +282,12 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { public void testHsyncWithFlushEnabled() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); byte[] buffer = getRandomBytesArray(); - try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, true)) { + + final Path testFilePath = path(methodName.getMethodName()); + + try (FSDataOutputStream stream = getStreamAfterWrite(fs, testFilePath, buffer, true)) { stream.hsync(); - validate(fs, TEST_FILE_PATH, buffer, true); + validate(fs, testFilePath, buffer, true); } } @@ -292,7 +295,10 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { public void testStreamCapabilitiesWithFlushDisabled() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); byte[] buffer = getRandomBytesArray(); - try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, false)) { + + final Path testFilePath = path(methodName.getMethodName()); + + try (FSDataOutputStream stream = getStreamAfterWrite(fs, testFilePath, buffer, false)) { assertFalse(stream.hasCapability(StreamCapabilities.HFLUSH)); assertFalse(stream.hasCapability(StreamCapabilities.HSYNC)); assertFalse(stream.hasCapability(StreamCapabilities.DROPBEHIND)); @@ -305,7 +311,8 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { public void testStreamCapabilitiesWithFlushEnabled() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); byte[] buffer = getRandomBytesArray(); - try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, true)) { + final Path testFilePath = path(methodName.getMethodName()); + try (FSDataOutputStream stream = getStreamAfterWrite(fs, testFilePath, buffer, true)) { assertTrue(stream.hasCapability(StreamCapabilities.HFLUSH)); assertTrue(stream.hasCapability(StreamCapabilities.HSYNC)); assertFalse(stream.hasCapability(StreamCapabilities.DROPBEHIND)); @@ -318,9 +325,10 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { public void testHsyncWithFlushDisabled() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); byte[] buffer = getRandomBytesArray(); - try (FSDataOutputStream stream = getStreamAfterWrite(fs, TEST_FILE_PATH, buffer, false)) { + final Path testFilePath = path(methodName.getMethodName()); + try (FSDataOutputStream stream = getStreamAfterWrite(fs, testFilePath, buffer, false)) { stream.hsync(); - validate(fs, TEST_FILE_PATH, buffer, false); + validate(fs, testFilePath, buffer, false); } } @@ -337,11 +345,28 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { return stream; } - private AzureBlobStorageTestAccount createWasbTestAccount() throws Exception { - return AzureBlobStorageTestAccount.create("", EnumSet.of(AzureBlobStorageTestAccount.CreateOptions.CreateContainer), - this.getConfiguration()); - } + private void validate(InputStream stream, byte[] writeBuffer, boolean isEqual) + throws IOException { + try { + byte[] readBuffer = new byte[writeBuffer.length]; + int numBytesRead = stream.read(readBuffer, 0, readBuffer.length); + + if (isEqual) { + assertArrayEquals( + "Bytes read do not match bytes written.", + writeBuffer, + readBuffer); + } else { + assertThat( + "Bytes read unexpectedly match bytes written.", + readBuffer, + IsNot.not(IsEqual.equalTo(writeBuffer))); + } + } finally { + stream.close(); + } + } private void validate(FileSystem fs, Path path, byte[] writeBuffer, boolean isEqual) throws IOException { String filePath = path.toUri().toString(); try (FSDataInputStream inputStream = fs.open(path)) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java index c4bfee28c3d..33a5805ec98 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java @@ -98,7 +98,7 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest { NativeAzureFileSystem wasb = getWasbFileSystem(); for (int i = 0; i< 4; i++) { - Path path = new Path("/testfiles/~12/!008/testfile" + i); + Path path = new Path("/testReadFile/~12/!008/testfile" + i); final FileSystem createFs = createFileWithAbfs[i] ? abfs : wasb; // Write From 97f06b3fc70ad509e601076c015bc244daa1243f Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Mon, 3 Sep 2018 01:37:10 +0000 Subject: [PATCH 14/29] HADOOP-15703. ABFS - Implement client-side throttling. Contributed by Sneha Varma and Thomas Marquardt. --- .../fs/azure/ClientThrottlingAnalyzer.java | 2 +- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 8 + .../fs/azurebfs/AzureBlobFileSystem.java | 3 + .../azurebfs/constants/ConfigurationKeys.java | 2 +- .../constants/FileSystemConfigurations.java | 1 + .../fs/azurebfs/services/AbfsClient.java | 17 ++ .../AbfsClientThrottlingAnalyzer.java | 272 ++++++++++++++++++ .../AbfsClientThrottlingIntercept.java | 117 ++++++++ .../azurebfs/services/AbfsRestOperation.java | 15 +- .../services/AbfsRestOperationType.java | 42 +++ .../TestAbfsClientThrottlingAnalyzer.java | 159 ++++++++++ 11 files changed, 633 insertions(+), 5 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingAnalyzer.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationType.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClientThrottlingAnalyzer.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/ClientThrottlingAnalyzer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/ClientThrottlingAnalyzer.java index 850e552758d..859a608a1e1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/ClientThrottlingAnalyzer.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/ClientThrottlingAnalyzer.java @@ -99,7 +99,7 @@ class ClientThrottlingAnalyzer { this.blobMetrics = new AtomicReference( new BlobOperationMetrics(System.currentTimeMillis())); this.timer = new Timer( - String.format("wasb-timer-client-throttling-analyzer-%s", name)); + String.format("wasb-timer-client-throttling-analyzer-%s", name), true); this.timer.schedule(new TimerTaskImpl(), analysisPeriodMs, analysisPeriodMs); 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 924bc3e8d9e..518fef9495f 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 @@ -141,6 +141,10 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_ENABLE_FLUSH) private boolean enableFlush; + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ENABLE_AUTOTHROTTLING, + DefaultValue = DEFAULT_ENABLE_AUTOTHROTTLING) + private boolean enableAutoThrottling; + @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_USER_AGENT_PREFIX_KEY, DefaultValue = "") private String userAgentId; @@ -279,6 +283,10 @@ public class AbfsConfiguration{ return this.enableFlush; } + public boolean isAutoThrottlingEnabled() { + return this.enableAutoThrottling; + } + public String getCustomUserAgentPrefix() { return this.userAgentId; } 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 b8091921079..c0ecc355007 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 @@ -38,6 +38,7 @@ import java.util.concurrent.Future; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientThrottlingIntercept; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -130,6 +131,8 @@ public class AzureBlobFileSystem extends FileSystem { this.delegationTokenManager = abfsStore.getAbfsConfiguration().getDelegationTokenManager(); } } + + AbfsClientThrottlingIntercept.initializeSingleton(abfsStore.getAbfsConfiguration().isAutoThrottlingEnabled()); } @Override 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 ca4c9c3005d..52367193045 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 @@ -47,7 +47,7 @@ public final class ConfigurationKeys { public static final String AZURE_TOLERATE_CONCURRENT_APPEND = "fs.azure.io.read.tolerate.concurrent.append"; 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_AUTOTHROTTLING_ENABLE = "fs.azure.autothrottling.enable"; + public static final String FS_AZURE_ENABLE_AUTOTHROTTLING = "fs.azure.enable.autothrottling"; public static final String FS_AZURE_ATOMIC_RENAME_KEY = "fs.azure.atomic.rename.key"; public static final String FS_AZURE_READ_AHEAD_QUEUE_DEPTH = "fs.azure.readaheadqueue.depth"; public static final String FS_AZURE_ENABLE_FLUSH = "fs.azure.enable.flush"; 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 a921faf8d59..a9412a961c0 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 @@ -57,6 +57,7 @@ public final class FileSystemConfigurations { public static final int DEFAULT_READ_AHEAD_QUEUE_DEPTH = -1; public static final boolean DEFAULT_ENABLE_FLUSH = true; + public static final boolean DEFAULT_ENABLE_AUTOTHROTTLING = true; public static final SSLSocketFactoryEx.SSLChannelMode DEFAULT_FS_AZURE_SSL_CHANNEL_MODE = SSLSocketFactoryEx.SSLChannelMode.Default; 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 18773b6881b..258045a4738 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 @@ -125,6 +125,7 @@ public class AbfsClient { final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.CreateFileSystem, this, HTTP_METHOD_PUT, url, @@ -148,6 +149,7 @@ public class AbfsClient { final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.SetFileSystemProperties, this, HTTP_METHOD_PUT, url, @@ -170,6 +172,7 @@ public class AbfsClient { final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.ListPaths, this, HTTP_METHOD_GET, url, @@ -186,6 +189,7 @@ public class AbfsClient { final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.GetFileSystemProperties, this, HTTP_METHOD_HEAD, url, @@ -202,6 +206,7 @@ public class AbfsClient { final URL url = createRequestUrl(abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.DeleteFileSystem, this, HTTP_METHOD_DELETE, url, @@ -230,6 +235,7 @@ public class AbfsClient { final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.CreatePath, this, HTTP_METHOD_PUT, url, @@ -251,6 +257,7 @@ public class AbfsClient { final URL url = createRequestUrl(destination, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.RenamePath, this, HTTP_METHOD_PUT, url, @@ -273,6 +280,7 @@ public class AbfsClient { final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.Append, this, HTTP_METHOD_PUT, url, @@ -296,6 +304,7 @@ public class AbfsClient { final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.Flush, this, HTTP_METHOD_PUT, url, @@ -319,6 +328,7 @@ public class AbfsClient { final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.SetPathProperties, this, HTTP_METHOD_PUT, url, @@ -334,6 +344,7 @@ public class AbfsClient { final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.GetPathProperties, this, HTTP_METHOD_HEAD, url, @@ -354,6 +365,7 @@ public class AbfsClient { final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.ReadFile, this, HTTP_METHOD_GET, url, @@ -376,6 +388,7 @@ public class AbfsClient { final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.DeletePath, this, HTTP_METHOD_DELETE, url, @@ -404,6 +417,7 @@ public class AbfsClient { final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.SetOwner, this, AbfsHttpConstants.HTTP_METHOD_PUT, url, @@ -427,6 +441,7 @@ public class AbfsClient { final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.SetPermissions, this, AbfsHttpConstants.HTTP_METHOD_PUT, url, @@ -458,6 +473,7 @@ public class AbfsClient { final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.SetAcl, this, AbfsHttpConstants.HTTP_METHOD_PUT, url, @@ -474,6 +490,7 @@ public class AbfsClient { final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = new AbfsRestOperation( + AbfsRestOperationType.GetAcl, this, AbfsHttpConstants.HTTP_METHOD_HEAD, url, 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 new file mode 100644 index 00000000000..f1e5aaae683 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingAnalyzer.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.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class AbfsClientThrottlingAnalyzer { + private static final Logger LOG = LoggerFactory.getLogger( + AbfsClientThrottlingAnalyzer.class); + private static final int DEFAULT_ANALYSIS_PERIOD_MS = 10 * 1000; + 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; + private static final double SLEEP_DECREASE_FACTOR = .975; + private static final double SLEEP_INCREASE_FACTOR = 1.05; + private int analysisPeriodMs; + + private volatile int sleepDuration = 0; + private long consecutiveNoErrorCount = 0; + private String name = null; + private Timer timer = null; + private AtomicReference blobMetrics = null; + + 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. + * @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 { + Preconditions.checkArgument( + StringUtils.isNotEmpty(name), + "The argument 'name' cannot be null or empty."); + Preconditions.checkArgument( + 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.blobMetrics = new AtomicReference( + new AbfsOperationMetrics(System.currentTimeMillis())); + this.timer = new Timer( + String.format("abfs-timer-client-throttling-analyzer-%s", name), true); + this.timer.schedule(new TimerTaskImpl(), + analysisPeriodMs, + analysisPeriodMs); + } + + /** + * Updates metrics with results from the current storage operation. + * + * @param count The count of bytes transferred. + * @param isFailedOperation True if the operation failed; otherwise false. + */ + public void addBytesTransferred(long count, boolean isFailedOperation) { + AbfsOperationMetrics metrics = blobMetrics.get(); + if (isFailedOperation) { + metrics.bytesFailed.addAndGet(count); + metrics.operationsFailed.incrementAndGet(); + } else { + metrics.bytesSuccessful.addAndGet(count); + metrics.operationsSuccessful.incrementAndGet(); + } + } + + /** + * Suspends the current storage operation, as necessary, to reduce throughput. + */ + public void suspendIfNecessary() { + int duration = sleepDuration; + if (duration > 0) { + try { + Thread.sleep(duration); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + } + } + } + + @VisibleForTesting + int getSleepDuration() { + return sleepDuration; + } + + 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 errorPercentage = (bytesFailed <= 0) + ? 0 + : (percentageConversionFactor + * bytesFailed + / (bytesFailed + bytesSuccessful)); + long periodMs = metrics.endTime - metrics.startTime; + + double newSleepDuration; + + if (errorPercentage < MIN_ACCEPTABLE_ERROR_PERCENTAGE) { + ++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; + + newSleepDuration = sleepDuration * reductionFactor; + } else if (errorPercentage < MAX_EQUILIBRIUM_ERROR_PERCENTAGE) { + // Do not modify sleepDuration in order to stabilize throughput. + newSleepDuration = sleepDuration; + } else { + // Increase sleepDuration in order to minimize error rate. + consecutiveNoErrorCount = 0; + + // Increase sleep duration in order to reduce throughput and error rate. + // First, calculate target throughput: bytesSuccessful / periodMs. + // Next, calculate time required to send *all* data (assuming next period + // is similar to previous) at the target throughput: (bytesSuccessful + // + bytesFailed) * periodMs / bytesSuccessful. Next, subtract periodMs to + // get the total additional delay needed. + double additionalDelayNeeded = 5 * analysisPeriodMs; + if (bytesSuccessful > 0) { + additionalDelayNeeded = (bytesSuccessful + bytesFailed) + * periodMs + / bytesSuccessful + - periodMs; + } + + // amortize the additional delay needed across the estimated number of + // requests during the next period + newSleepDuration = additionalDelayNeeded + / (operationsFailed + operationsSuccessful); + + final double maxSleepDuration = analysisPeriodMs; + final double minSleepDuration = sleepDuration * SLEEP_INCREASE_FACTOR; + + // Add 1 ms to avoid rounding down and to decrease proximity to the server + // side ingress/egress limit. Ensure that the new sleep duration is + // larger than the current one to more quickly reduce the number of + // errors. Don't allow the sleep duration to grow unbounded, after a + // certain point throttling won't help, for example, if there are far too + // many tasks/containers/nodes no amount of throttling will help. + newSleepDuration = Math.max(newSleepDuration, minSleepDuration) + 1; + newSleepDuration = Math.min(newSleepDuration, maxSleepDuration); + } + + 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)); + } + + return (int) newSleepDuration; + } + + /** + * Timer callback implementation for periodically analyzing metrics. + */ + class TimerTaskImpl extends TimerTask { + private AtomicInteger doingWork = new AtomicInteger(0); + + /** + * Periodically analyzes a snapshot of the blob storage metrics and updates + * the sleepDuration in order to appropriately throttle storage operations. + */ + @Override + public void run() { + boolean doWork = false; + try { + doWork = doingWork.compareAndSet(0, 1); + + // prevent concurrent execution of this task + if (!doWork) { + return; + } + + long now = System.currentTimeMillis(); + if (now - blobMetrics.get().startTime >= analysisPeriodMs) { + AbfsOperationMetrics oldMetrics = blobMetrics.getAndSet( + new AbfsOperationMetrics(now)); + oldMetrics.endTime = now; + sleepDuration = analyzeMetricsAndUpdateSleepDuration(oldMetrics, + sleepDuration); + } + } finally { + if (doWork) { + doingWork.set(0); + } + } + } + } + + /** + * 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(); + } + } +} \ No newline at end of file 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 new file mode 100644 index 00000000000..e981d76ba18 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java @@ -0,0 +1,117 @@ +/** + * 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.net.HttpURLConnection; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Throttles Azure Blob File System read and write operations to achieve maximum + * throughput by minimizing errors. The errors occur when the account ingress + * or egress limits are exceeded and the server-side throttles requests. + * Server-side throttling causes the retry policy to be used, but the retry + * policy sleeps for long periods of time causing the total ingress or egress + * throughput to be as much as 35% lower than optimal. The retry policy is also + * after the fact, in that it applies after a request fails. On the other hand, + * the client-side throttling implemented here happens before requests are made + * and sleeps just enough to minimize errors, allowing optimal ingress and/or + * egress throughput. + */ +public final class AbfsClientThrottlingIntercept { + private static final Logger LOG = LoggerFactory.getLogger( + AbfsClientThrottlingIntercept.class); + private static AbfsClientThrottlingIntercept singleton = null; + private AbfsClientThrottlingAnalyzer readThrottler = null; + private AbfsClientThrottlingAnalyzer writeThrottler = null; + private static boolean isAutoThrottlingEnabled = false; + + // Hide default constructor + private AbfsClientThrottlingIntercept() { + readThrottler = new AbfsClientThrottlingAnalyzer("read"); + writeThrottler = new AbfsClientThrottlingAnalyzer("write"); + isAutoThrottlingEnabled = true; + LOG.debug("Client-side throttling is enabled for the ABFS file system."); + } + + public static synchronized void initializeSingleton(boolean isAutoThrottlingEnabled) { + if (!isAutoThrottlingEnabled) { + return; + } + if (singleton == null) { + singleton = new AbfsClientThrottlingIntercept(); + } + } + + static void updateMetrics(AbfsRestOperationType operationType, + AbfsHttpOperation abfsHttpOperation) { + if (!isAutoThrottlingEnabled || abfsHttpOperation == null) { + return; + } + + int status = abfsHttpOperation.getStatusCode(); + long contentLength = 0; + // If the socket is terminated prior to receiving a response, the HTTP + // status may be 0 or -1. A status less than 200 or greater than or equal + // to 500 is considered an error. + boolean isFailedOperation = (status < HttpURLConnection.HTTP_OK + || status >= HttpURLConnection.HTTP_INTERNAL_ERROR); + + switch (operationType) { + case Append: + contentLength = abfsHttpOperation.getBytesSent(); + if (contentLength > 0) { + singleton.writeThrottler.addBytesTransferred(contentLength, + isFailedOperation); + } + break; + case ReadFile: + contentLength = abfsHttpOperation.getBytesReceived(); + if (contentLength > 0) { + singleton.readThrottler.addBytesTransferred(contentLength, + isFailedOperation); + } + break; + default: + break; + } + } + + /** + * Called before the request is sent. Client-side throttling + * uses this to suspend the request, if necessary, to minimize errors and + * maximize throughput. + */ + static void sendingRequest(AbfsRestOperationType operationType) { + if (!isAutoThrottlingEnabled) { + return; + } + + switch (operationType) { + case ReadFile: + singleton.readThrottler.suspendIfNecessary(); + break; + case Append: + singleton.writeThrottler.suspendIfNecessary(); + break; + default: + break; + } + } +} \ No newline at end of file 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 c0407f58d29..9a7187996de 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 @@ -36,6 +36,8 @@ import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; * The AbfsRestOperation for Rest AbfsClient. */ public class AbfsRestOperation { + // The type of the REST operation (Append, ReadFile, etc) + private final AbfsRestOperationType operationType; // Blob FS client, which has the credentials, retry policy, and logs. private final AbfsClient client; // the HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE) @@ -71,10 +73,12 @@ public class AbfsRestOperation { * @param url The full URL including query string parameters. * @param requestHeaders The HTTP request headers. */ - AbfsRestOperation(final AbfsClient client, + AbfsRestOperation(final AbfsRestOperationType operationType, + final AbfsClient client, final String method, final URL url, final List requestHeaders) { + this.operationType = operationType; this.client = client; this.method = method; this.url = url; @@ -86,6 +90,7 @@ public class AbfsRestOperation { /** * Initializes a new REST operation. * + * @param operationType The type of the REST operation (Append, ReadFile, etc). * @param client The Blob FS client. * @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE). * @param url The full URL including query string parameters. @@ -95,14 +100,15 @@ public class AbfsRestOperation { * @param bufferOffset An offset into the buffer where the data beings. * @param bufferLength The length of the data in the buffer. */ - AbfsRestOperation(AbfsClient client, + AbfsRestOperation(AbfsRestOperationType operationType, + AbfsClient client, String method, URL url, List requestHeaders, byte[] buffer, int bufferOffset, int bufferLength) { - this(client, method, url, requestHeaders); + this(operationType, client, method, url, requestHeaders); this.buffer = buffer; this.bufferOffset = bufferOffset; this.bufferLength = bufferLength; @@ -152,6 +158,7 @@ public class AbfsRestOperation { if (hasRequestBody) { // HttpUrlConnection requires + AbfsClientThrottlingIntercept.sendingRequest(operationType); httpOperation.sendRequest(buffer, bufferOffset, bufferLength); } @@ -168,6 +175,8 @@ public class AbfsRestOperation { throw new InvalidAbfsRestOperationException(ex); } return false; + } finally { + AbfsClientThrottlingIntercept.updateMetrics(operationType, httpOperation); } LOG.debug("HttpRequest: " + httpOperation.toString()); 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 new file mode 100644 index 00000000000..eeea81750e6 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationType.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; + +/** + * The REST operation type (Read, Append, Other ). + */ +public enum AbfsRestOperationType { + CreateFileSystem, + GetFileSystemProperties, + SetFileSystemProperties, + ListPaths, + DeleteFileSystem, + CreatePath, + RenamePath, + GetAcl, + GetPathProperties, + SetAcl, + SetOwner, + SetPathProperties, + SetPermissions, + Append, + Flush, + ReadFile, + DeletePath +} 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 new file mode 100644 index 00000000000..5105b85d000 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClientThrottlingAnalyzer.java @@ -0,0 +1,159 @@ +package org.apache.hadoop.fs.azurebfs.services; + +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Tests for AbfsClientThrottlingAnalyzer. + */ +public class TestAbfsClientThrottlingAnalyzer { + private static final int ANALYSIS_PERIOD = 1000; + private static final int ANALYSIS_PERIOD_PLUS_10_PERCENT = ANALYSIS_PERIOD + + ANALYSIS_PERIOD / 10; + private static final long MEGABYTE = 1024 * 1024; + private static final int MAX_ACCEPTABLE_PERCENT_DIFFERENCE = 20; + + private void sleep(long milliseconds) { + try { + Thread.sleep(milliseconds); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + + private void fuzzyValidate(long expected, long actual, double percentage) { + final double lowerBound = Math.max(expected - percentage / 100 * expected, 0); + final double upperBound = expected + percentage / 100 * expected; + + assertTrue( + String.format( + "The actual value %1$d is not within the expected range: " + + "[%2$.2f, %3$.2f].", + actual, + lowerBound, + upperBound), + actual >= lowerBound && actual <= upperBound); + } + + private void validate(long expected, long actual) { + assertEquals( + String.format("The actual value %1$d is not the expected value %2$d.", + actual, + expected), + expected, actual); + } + + private void validateLessThanOrEqual(long maxExpected, long actual) { + assertTrue( + String.format( + "The actual value %1$d is not less than or equal to the maximum" + + " expected value %2$d.", + actual, + maxExpected), + actual < maxExpected); + } + + /** + * Ensure that there is no waiting (sleepDuration = 0) if the metrics have + * never been updated. This validates proper initialization of + * ClientThrottlingAnalyzer. + */ + @Test + public void testNoMetricUpdatesThenNoWaiting() { + AbfsClientThrottlingAnalyzer analyzer = new AbfsClientThrottlingAnalyzer( + "test", + ANALYSIS_PERIOD); + validate(0, analyzer.getSleepDuration()); + sleep(ANALYSIS_PERIOD_PLUS_10_PERCENT); + validate(0, analyzer.getSleepDuration()); + } + + /** + * Ensure that there is no waiting (sleepDuration = 0) if the metrics have + * only been updated with successful requests. + */ + @Test + public void testOnlySuccessThenNoWaiting() { + AbfsClientThrottlingAnalyzer analyzer = new AbfsClientThrottlingAnalyzer( + "test", + ANALYSIS_PERIOD); + analyzer.addBytesTransferred(8 * MEGABYTE, false); + validate(0, analyzer.getSleepDuration()); + sleep(ANALYSIS_PERIOD_PLUS_10_PERCENT); + validate(0, analyzer.getSleepDuration()); + } + + /** + * Ensure that there is waiting (sleepDuration != 0) if the metrics have + * only been updated with failed requests. Also ensure that the + * sleepDuration decreases over time. + */ + @Test + public void testOnlyErrorsAndWaiting() { + AbfsClientThrottlingAnalyzer analyzer = new AbfsClientThrottlingAnalyzer( + "test", + ANALYSIS_PERIOD); + validate(0, analyzer.getSleepDuration()); + analyzer.addBytesTransferred(4 * MEGABYTE, true); + sleep(ANALYSIS_PERIOD_PLUS_10_PERCENT); + final int expectedSleepDuration1 = 1100; + validateLessThanOrEqual(expectedSleepDuration1, analyzer.getSleepDuration()); + sleep(10 * ANALYSIS_PERIOD); + final int expectedSleepDuration2 = 900; + validateLessThanOrEqual(expectedSleepDuration2, analyzer.getSleepDuration()); + } + + /** + * Ensure that there is waiting (sleepDuration != 0) if the metrics have + * only been updated with both successful and failed requests. Also ensure + * that the sleepDuration decreases over time. + */ + @Test + public void testSuccessAndErrorsAndWaiting() { + AbfsClientThrottlingAnalyzer analyzer = new AbfsClientThrottlingAnalyzer( + "test", + ANALYSIS_PERIOD); + validate(0, analyzer.getSleepDuration()); + analyzer.addBytesTransferred(8 * MEGABYTE, false); + analyzer.addBytesTransferred(2 * MEGABYTE, true); + sleep(ANALYSIS_PERIOD_PLUS_10_PERCENT); + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); + analyzer.suspendIfNecessary(); + final int expectedElapsedTime = 126; + fuzzyValidate(expectedElapsedTime, + timer.elapsedTimeMs(), + MAX_ACCEPTABLE_PERCENT_DIFFERENCE); + sleep(10 * ANALYSIS_PERIOD); + final int expectedSleepDuration = 110; + validateLessThanOrEqual(expectedSleepDuration, analyzer.getSleepDuration()); + } + + /** + * Ensure that there is waiting (sleepDuration != 0) if the metrics have + * only been updated with many successful and failed requests. Also ensure + * that the sleepDuration decreases to zero over time. + */ + @Test + public void testManySuccessAndErrorsAndWaiting() { + AbfsClientThrottlingAnalyzer analyzer = new AbfsClientThrottlingAnalyzer( + "test", + ANALYSIS_PERIOD); + validate(0, analyzer.getSleepDuration()); + final int numberOfRequests = 20; + for (int i = 0; i < numberOfRequests; i++) { + analyzer.addBytesTransferred(8 * MEGABYTE, false); + analyzer.addBytesTransferred(2 * MEGABYTE, true); + } + sleep(ANALYSIS_PERIOD_PLUS_10_PERCENT); + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); + analyzer.suspendIfNecessary(); + fuzzyValidate(7, + timer.elapsedTimeMs(), + MAX_ACCEPTABLE_PERCENT_DIFFERENCE); + sleep(10 * ANALYSIS_PERIOD); + validate(0, analyzer.getSleepDuration()); + } +} \ No newline at end of file From 347a52a86654867be13e081c4f1309949d44449d Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Thu, 6 Sep 2018 01:35:49 +0000 Subject: [PATCH 15/29] Fixing findbugs and license issues related to: HADOOP-15703. ABFS - Implement client-side throttling. Contributed by Sneha Varma and Thomas Marquardt. --- .../AbfsClientThrottlingIntercept.java | 4 ++-- .../TestAbfsClientThrottlingAnalyzer.java | 18 ++++++++++++++++++ 2 files changed, 20 insertions(+), 2 deletions(-) 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 e981d76ba18..08922198d1d 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 @@ -46,8 +46,6 @@ public final class AbfsClientThrottlingIntercept { private AbfsClientThrottlingIntercept() { readThrottler = new AbfsClientThrottlingAnalyzer("read"); writeThrottler = new AbfsClientThrottlingAnalyzer("write"); - isAutoThrottlingEnabled = true; - LOG.debug("Client-side throttling is enabled for the ABFS file system."); } public static synchronized void initializeSingleton(boolean isAutoThrottlingEnabled) { @@ -56,6 +54,8 @@ public final class AbfsClientThrottlingIntercept { } if (singleton == null) { singleton = new AbfsClientThrottlingIntercept(); + isAutoThrottlingEnabled = true; + LOG.debug("Client-side throttling is enabled for the ABFS file system."); } } 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 5105b85d000..3f680e49930 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 @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.fs.azurebfs.services; import org.apache.hadoop.fs.contract.ContractTestUtils; From 6801b3073317000d5a9c24764aa93918955c27a6 Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Fri, 7 Sep 2018 03:45:35 +0000 Subject: [PATCH 16/29] HADOOP-15728. ABFS: Add backward compatibility to handle Unsupported Operation for storage account with no namespace feature. Contributed by Da Zhou. --- .../fs/azurebfs/AzureBlobFileSystem.java | 44 +++++ .../ITestAzureBlobFileSystemPermission.java | 4 +- .../azurebfs/ITestAzureBlobFilesystemAcl.java | 185 +++++++++++++++++- 3 files changed, 228 insertions(+), 5 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index c0ecc355007..7cbf4d7d3f7 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 @@ -497,6 +497,10 @@ public class AzureBlobFileSystem extends FileSystem { throws IOException { LOG.debug( "AzureBlobFileSystem.setOwner path: {}", path); + if (!getIsNamespaceEnabeld()) { + super.setOwner(path, owner, group); + return; + } if ((owner == null || owner.isEmpty()) && (group == null || group.isEmpty())) { throw new IllegalArgumentException("A valid owner or group must be specified."); @@ -521,6 +525,10 @@ public class AzureBlobFileSystem extends FileSystem { public void setPermission(final Path path, final FsPermission permission) throws IOException { LOG.debug("AzureBlobFileSystem.setPermission path: {}", path); + if (!getIsNamespaceEnabeld()) { + super.setPermission(path, permission); + return; + } if (permission == null) { throw new IllegalArgumentException("The permission can't be null"); @@ -549,6 +557,12 @@ public class AzureBlobFileSystem extends FileSystem { throws IOException { LOG.debug("AzureBlobFileSystem.modifyAclEntries path: {}", path.toString()); + if (!getIsNamespaceEnabeld()) { + throw new UnsupportedOperationException( + "modifyAclEntries is only supported by storage accounts " + + "with the hierarchical namespace enabled."); + } + if (aclSpec == null || aclSpec.isEmpty()) { throw new IllegalArgumentException("The value of the aclSpec parameter is invalid."); } @@ -574,6 +588,12 @@ public class AzureBlobFileSystem extends FileSystem { throws IOException { LOG.debug("AzureBlobFileSystem.removeAclEntries path: {}", path); + if (!getIsNamespaceEnabeld()) { + throw new UnsupportedOperationException( + "removeAclEntries is only supported by storage accounts " + + "with the hierarchical namespace enabled."); + } + if (aclSpec == null || aclSpec.isEmpty()) { throw new IllegalArgumentException("The aclSpec argument is invalid."); } @@ -595,6 +615,12 @@ public class AzureBlobFileSystem extends FileSystem { public void removeDefaultAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeDefaultAcl path: {}", path); + if (!getIsNamespaceEnabeld()) { + throw new UnsupportedOperationException( + "removeDefaultAcl is only supported by storage accounts" + + " with the hierarchical namespace enabled."); + } + try { abfsStore.removeDefaultAcl(makeQualified(path)); } catch (AzureBlobFileSystemException ex) { @@ -614,6 +640,12 @@ public class AzureBlobFileSystem extends FileSystem { public void removeAcl(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.removeAcl path: {}", path); + if (!getIsNamespaceEnabeld()) { + throw new UnsupportedOperationException( + "removeAcl is only supported by storage accounts" + + " with the hierarchical namespace enabled."); + } + try { abfsStore.removeAcl(makeQualified(path)); } catch (AzureBlobFileSystemException ex) { @@ -636,6 +668,12 @@ public class AzureBlobFileSystem extends FileSystem { throws IOException { LOG.debug("AzureBlobFileSystem.setAcl path: {}", path); + if (!getIsNamespaceEnabeld()) { + throw new UnsupportedOperationException( + "setAcl is only supported by storage accounts" + + " with the hierarchical namespace enabled."); + } + if (aclSpec == null || aclSpec.size() == 0) { throw new IllegalArgumentException("The aclSpec argument is invalid."); } @@ -658,6 +696,12 @@ public class AzureBlobFileSystem extends FileSystem { public AclStatus getAclStatus(final Path path) throws IOException { LOG.debug("AzureBlobFileSystem.getAclStatus path: {}", path.toString()); + if (!getIsNamespaceEnabeld()) { + throw new UnsupportedOperationException( + "getAclStatus is only supported by storage accounts" + + " with the hierarchical namespace enabled."); + } + try { return abfsStore.getAclStatus(makeQualified(path)); } catch (AzureBlobFileSystemException ex) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java index 2f265d1865b..e0e0758cf05 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java @@ -53,8 +53,6 @@ public class ITestAzureBlobFileSystemPermission extends AbstractAbfsIntegrationT public ITestAzureBlobFileSystemPermission(FsPermission testPermission) throws Exception { super(); permission = testPermission; - - Assume.assumeTrue(this.getAuthType() == AuthType.OAuth); } @Parameterized.Parameters(name = "{0}") @@ -79,6 +77,7 @@ public class ITestAzureBlobFileSystemPermission extends AbstractAbfsIntegrationT public void testFilePermission() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + Assume.assumeTrue(fs.getIsNamespaceEnabeld()); fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, DEFAULT_UMASK_VALUE); path = new Path(testRoot, UUID.randomUUID().toString()); @@ -94,6 +93,7 @@ public class ITestAzureBlobFileSystemPermission extends AbstractAbfsIntegrationT @Test public void testFolderPermission() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + Assume.assumeTrue(fs.getIsNamespaceEnabeld()); fs.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "027"); path = new Path(testRoot, UUID.randomUUID().toString()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java index a13b73e2724..67da26babcb 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java @@ -24,19 +24,21 @@ import java.io.FileNotFoundException; import java.util.List; import java.util.UUID; -import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.junit.Assume; import org.junit.Ignore; import org.junit.Test; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers; import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; +import static org.junit.Assume.assumeTrue; + import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS; import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT; import static org.apache.hadoop.fs.permission.AclEntryType.USER; @@ -59,13 +61,12 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { public ITestAzureBlobFilesystemAcl() throws Exception { super(); - - Assume.assumeTrue(this.getAuthType() == AuthType.OAuth); } @Test public void testModifyAclEntries() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.mkdirs(path, FsPermission.createImmutable((short) 0750)); @@ -98,6 +99,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testModifyAclEntriesOnlyAccess() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) 0640)); @@ -121,6 +123,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testModifyAclEntriesOnlyDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -143,6 +146,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testModifyAclEntriesMinimal() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) 0640)); @@ -160,6 +164,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testModifyAclEntriesMinimalDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -179,6 +184,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testModifyAclEntriesCustomMask() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) 0640)); @@ -197,6 +203,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testModifyAclEntriesStickyBit() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750)); List aclSpec = Lists.newArrayList( @@ -226,6 +233,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test(expected=FileNotFoundException.class) public void testModifyAclEntriesPathNotFound() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); // Path has not been created. List aclSpec = Lists.newArrayList( @@ -239,6 +247,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test (expected=Exception.class) public void testModifyAclEntriesDefaultOnFile() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) 0640)); @@ -250,6 +259,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testRemoveAclEntries() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -277,6 +287,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testRemoveAclEntriesOnlyAccess() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) 0640)); @@ -301,6 +312,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testRemoveAclEntriesOnlyDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -327,6 +339,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testRemoveAclEntriesMinimal() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) 0760)); @@ -349,6 +362,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testRemoveAclEntriesMinimalDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -376,6 +390,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testRemoveAclEntriesStickyBit() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750)); List aclSpec = Lists.newArrayList( @@ -403,6 +418,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test(expected=FileNotFoundException.class) public void testRemoveAclEntriesPathNotFound() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); // Path has not been created. List aclSpec = Lists.newArrayList( @@ -413,6 +429,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testRemoveDefaultAcl() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -434,6 +451,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testRemoveDefaultAclOnlyAccess() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) 0640)); @@ -455,6 +473,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testRemoveDefaultAclOnlyDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -470,6 +489,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testRemoveDefaultAclMinimal() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); fs.removeDefaultAcl(path); @@ -482,6 +502,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testRemoveDefaultAclStickyBit() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750)); List aclSpec = Lists.newArrayList( @@ -503,6 +524,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test(expected=FileNotFoundException.class) public void testRemoveDefaultAclPathNotFound() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); // Path has not been created. fs.removeDefaultAcl(path); @@ -511,6 +533,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testRemoveAcl() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -532,6 +555,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testRemoveAclMinimalAcl() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) 0640)); @@ -545,6 +569,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testRemoveAclStickyBit() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750)); List aclSpec = Lists.newArrayList( @@ -564,6 +589,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testRemoveAclOnlyDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -582,6 +608,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test(expected=FileNotFoundException.class) public void testRemoveAclPathNotFound() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); // Path has not been created. fs.removeAcl(path); @@ -590,6 +617,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testSetAcl() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -615,6 +643,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testSetAclOnlyAccess() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) 0640)); @@ -635,6 +664,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testSetAclOnlyDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -654,6 +684,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testSetAclMinimal() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) 0644)); @@ -677,6 +708,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testSetAclMinimalDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -696,6 +728,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testSetAclCustomMask() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) 0640)); @@ -717,6 +750,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testSetAclStickyBit() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 01750)); List aclSpec = Lists.newArrayList( @@ -742,6 +776,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test(expected=FileNotFoundException.class) public void testSetAclPathNotFound() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); // Path has not been created. List aclSpec = Lists.newArrayList( @@ -755,6 +790,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test(expected=Exception.class) public void testSetAclDefaultOnFile() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) 0640)); @@ -766,6 +802,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testSetPermission() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -792,6 +829,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testSetPermissionOnlyAccess() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); fs.setPermission(path, FsPermission.createImmutable((short) 0640)); @@ -813,6 +851,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testSetPermissionOnlyDefault() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -836,6 +875,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testDefaultAclNewFile() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -855,6 +895,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Ignore // wait umask fix to be deployed public void testOnlyAccessAclNewFile() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -872,6 +913,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Ignore // wait investigation in service public void testDefaultMinimalAclNewFile() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -890,6 +932,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testDefaultAclNewDir() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -916,6 +959,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Ignore // wait umask fix to be deployed public void testOnlyAccessAclNewDir() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -933,6 +977,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Ignore // wait investigation in service public void testDefaultMinimalAclNewDir() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); List aclSpec = Lists.newArrayList( @@ -954,6 +999,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testDefaultAclNewFileWithMode() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0755)); List aclSpec = Lists.newArrayList( @@ -975,6 +1021,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testDefaultAclNewDirWithMode() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0755)); List aclSpec = Lists.newArrayList( @@ -998,6 +1045,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testDefaultAclRenamedFile() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); Path dirPath = new Path(path, "dir"); FileSystem.mkdirs(fs, dirPath, FsPermission.createImmutable((short) 0750)); @@ -1019,6 +1067,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testDefaultAclRenamedDir() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); Path dirPath = new Path(path, "dir"); FileSystem.mkdirs(fs, dirPath, FsPermission.createImmutable((short) 0750)); @@ -1039,6 +1088,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { @Test public void testEnsureAclOperationWorksForRoot() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); + assumeTrue(fs.getIsNamespaceEnabeld()); Path rootPath = new Path("/"); @@ -1060,6 +1110,135 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { fs.removeAcl(rootPath); } + @Test + public void testSetOwnerForNonNamespaceEnabledAccount() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + Assume.assumeTrue(!fs.getIsNamespaceEnabeld()); + final Path filePath = new Path(methodName.getMethodName()); + fs.create(filePath); + + assertTrue(fs.exists(filePath)); + + FileStatus oldFileStatus = fs.getFileStatus(filePath); + fs.setOwner(filePath, "Alice", "testGroup"); + FileStatus newFileStatus = fs.getFileStatus(filePath); + + assertEquals(oldFileStatus.getOwner(), newFileStatus.getOwner()); + assertEquals(oldFileStatus.getGroup(), newFileStatus.getGroup()); + } + + @Test + public void testSetPermissionForNonNamespaceEnabledAccount() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + Assume.assumeTrue(!fs.getIsNamespaceEnabeld()); + final Path filePath = new Path(methodName.getMethodName()); + fs.create(filePath); + + assertTrue(fs.exists(filePath)); + FsPermission oldPermission = fs.getFileStatus(filePath).getPermission(); + // default permission for non-namespace enabled account is "777" + FsPermission newPermission = new FsPermission("557"); + + assertNotEquals(oldPermission, newPermission); + + fs.setPermission(filePath, newPermission); + FsPermission updatedPermission = fs.getFileStatus(filePath).getPermission(); + assertEquals(oldPermission, updatedPermission); + } + + @Test + public void testModifyAclEntriesForNonNamespaceEnabledAccount() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + Assume.assumeTrue(!fs.getIsNamespaceEnabeld()); + final Path filePath = new Path(methodName.getMethodName()); + fs.create(filePath); + try { + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, GROUP, "foo", ALL), + aclEntry(ACCESS, GROUP, "bar", ALL)); + fs.modifyAclEntries(filePath, aclSpec); + assertFalse("UnsupportedOperationException is expected", false); + } catch (UnsupportedOperationException ex) { + //no-op + } + } + + @Test + public void testRemoveAclEntriesEntriesForNonNamespaceEnabledAccount() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + Assume.assumeTrue(!fs.getIsNamespaceEnabeld()); + final Path filePath = new Path(methodName.getMethodName()); + fs.create(filePath); + try { + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, GROUP, "foo", ALL), + aclEntry(ACCESS, GROUP, "bar", ALL)); + fs.removeAclEntries(filePath, aclSpec); + assertFalse("UnsupportedOperationException is expected", false); + } catch (UnsupportedOperationException ex) { + //no-op + } + } + + @Test + public void testRemoveDefaultAclForNonNamespaceEnabledAccount() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + Assume.assumeTrue(!fs.getIsNamespaceEnabeld()); + final Path filePath = new Path(methodName.getMethodName()); + fs.create(filePath); + try { + fs.removeDefaultAcl(filePath); + assertFalse("UnsupportedOperationException is expected", false); + } catch (UnsupportedOperationException ex) { + //no-op + } + } + + @Test + public void testRemoveAclForNonNamespaceEnabledAccount() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + Assume.assumeTrue(!fs.getIsNamespaceEnabeld()); + final Path filePath = new Path(methodName.getMethodName()); + fs.create(filePath); + try { + fs.removeAcl(filePath); + assertFalse("UnsupportedOperationException is expected", false); + } catch (UnsupportedOperationException ex) { + //no-op + } + } + + @Test + public void testSetAclForNonNamespaceEnabledAccount() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + Assume.assumeTrue(!fs.getIsNamespaceEnabeld()); + final Path filePath = new Path(methodName.getMethodName()); + fs.create(filePath); + try { + List aclSpec = Lists.newArrayList( + aclEntry(DEFAULT, GROUP, "foo", ALL), + aclEntry(ACCESS, GROUP, "bar", ALL)); + fs.setAcl(filePath, aclSpec); + assertFalse("UnsupportedOperationException is expected", false); + } catch (UnsupportedOperationException ex) { + //no-op + } + } + + @Test + public void testGetAclStatusForNonNamespaceEnabledAccount() throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + Assume.assumeTrue(!fs.getIsNamespaceEnabeld()); + final Path filePath = new Path(methodName.getMethodName()); + fs.create(filePath); + try { + AclStatus aclSpec = fs.getAclStatus(filePath); + assertFalse("UnsupportedOperationException is expected", false); + } catch (UnsupportedOperationException ex) { + //no-op + } + } + private void assertPermission(FileSystem fs, short perm) throws Exception { assertPermission(fs, path, perm); } From 9475fd902a37e94fb7687877d33aa7dfff92d9eb Mon Sep 17 00:00:00 2001 From: Sean Mackrory Date: Wed, 12 Sep 2018 07:14:31 -0600 Subject: [PATCH 17/29] HADOOP-15745. ABFS: Add ABFS configuration to ConfigRedactor. --- .../apache/hadoop/fs/CommonConfigurationKeysPublic.java | 4 +++- .../hadoop-common/src/main/resources/core-default.xml | 4 +++- .../java/org/apache/hadoop/conf/TestConfigRedactor.java | 7 +++++++ 3 files changed, 13 insertions(+), 2 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java index b101b3b3096..b92d3253aa4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java @@ -886,7 +886,9 @@ public class CommonConfigurationKeysPublic { "fs.s3a.*.server-side-encryption.key", "fs.azure\\.account.key.*", "credential$", - "oauth.*token$", + "oauth.*secret", + "oauth.*password", + "oauth.*token", HADOOP_SECURITY_SENSITIVE_CONFIG_KEYS); /** 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 3fcdecb5959..f8eba04bbc0 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 @@ -603,7 +603,9 @@ fs.s3a.*.server-side-encryption.key fs.azure.account.key.* credential$ - oauth.*token$ + oauth.*secret + oauth.*password + oauth.*token hadoop.security.sensitive-config-keys A comma-separated or multi-line list of regular expressions to diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigRedactor.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigRedactor.java index 313394293c0..ca53fa7f2bf 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigRedactor.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigRedactor.java @@ -55,6 +55,13 @@ public class TestConfigRedactor { "fs.s3a.server-side-encryption.key", "fs.s3a.bucket.engineering.server-side-encryption.key", "fs.azure.account.key.abcdefg.blob.core.windows.net", + "fs.azure.account.key.abcdefg.dfs.core.windows.net", + "fs.azure.account.oauth2.client.secret", + "fs.azure.account.oauth2.client.secret.account.dfs.core.windows.net", + "fs.azure.account.oauth2.user.password", + "fs.azure.account.oauth2.user.password.account.dfs.core.windows.net", + "fs.azure.account.oauth2.refresh.token", + "fs.azure.account.oauth2.refresh.token.account.dfs.core.windows.net", "fs.adl.oauth2.refresh.token", "fs.adl.oauth2.credential", "dfs.adls.oauth2.refresh.token", From 13c70e9ba3c168b6aa2184e183291411b346d531 Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Wed, 12 Sep 2018 21:53:09 +0000 Subject: [PATCH 18/29] HADOOP-15740. ABFS: Check variable names during initialization of AbfsClientThrottlingIntercept. Contributed by Sneha Varma. --- .../fs/azurebfs/services/AbfsClientThrottlingIntercept.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 08922198d1d..97ea2a64861 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 @@ -48,8 +48,8 @@ public final class AbfsClientThrottlingIntercept { writeThrottler = new AbfsClientThrottlingAnalyzer("write"); } - public static synchronized void initializeSingleton(boolean isAutoThrottlingEnabled) { - if (!isAutoThrottlingEnabled) { + public static synchronized void initializeSingleton(boolean enableAutoThrottling) { + if (!enableAutoThrottling) { return; } if (singleton == null) { From e5593cbd8323399359b3e8da46bd58e8364cbf22 Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Wed, 12 Sep 2018 22:51:41 +0000 Subject: [PATCH 19/29] HADOOP-15694. ABFS: Allow OAuth credentials to not be tied to accounts. Contributed by Sean Mackrory. --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 208 ++++++++++--- .../fs/azurebfs/AzureBlobFileSystemStore.java | 69 +++-- .../azurebfs/constants/ConfigurationKeys.java | 44 +-- .../oauth2/UserPasswordTokenProvider.java | 10 - .../services/ShellDecryptionKeyProvider.java | 14 +- .../azurebfs/services/SimpleKeyProvider.java | 18 +- .../hadoop-azure/src/site/markdown/abfs.md | 10 + .../azurebfs/AbstractAbfsIntegrationTest.java | 63 ++-- .../fs/azurebfs/AbstractAbfsScaleTest.java | 8 +- .../hadoop/fs/azurebfs/ITestAbfsClient.java | 6 +- .../azurebfs/ITestAbfsReadWriteAndSeek.java | 4 +- .../ITestAzureBlobFileSystemAppend.java | 3 +- .../ITestAzureBlobFileSystemBackCompat.java | 3 +- .../ITestAzureBlobFileSystemCopy.java | 3 +- .../ITestAzureBlobFileSystemCreate.java | 3 +- .../ITestAzureBlobFileSystemDelete.java | 3 +- .../azurebfs/ITestAzureBlobFileSystemE2E.java | 5 +- .../ITestAzureBlobFileSystemE2EScale.java | 2 +- .../ITestAzureBlobFileSystemFileStatus.java | 2 +- .../ITestAzureBlobFileSystemFinalize.java | 8 +- .../ITestAzureBlobFileSystemFlush.java | 2 +- ...ITestAzureBlobFileSystemInitAndCreate.java | 2 +- .../ITestAzureBlobFileSystemListStatus.java | 3 +- .../ITestAzureBlobFileSystemMkDir.java | 3 +- .../ITestAzureBlobFileSystemOauth.java | 20 +- .../ITestAzureBlobFileSystemRename.java | 4 +- ...ITestAzureBlobFileSystemRenameUnicode.java | 2 +- .../ITestFileSystemInitialization.java | 8 +- .../azurebfs/ITestFileSystemProperties.java | 2 +- .../azurebfs/ITestFileSystemRegistration.java | 15 +- ...TestAbfsConfigurationFieldsValidation.java | 24 +- .../fs/azurebfs/TestAccountConfiguration.java | 273 ++++++++++++++++++ .../constants/TestConfigurationKeys.java | 2 +- .../contract/ABFSContractTestBinding.java | 8 +- .../ITestAbfsFileSystemContractAppend.java | 2 +- .../ITestAbfsFileSystemContractConcat.java | 2 +- .../ITestAbfsFileSystemContractCreate.java | 2 +- .../ITestAbfsFileSystemContractDelete.java | 2 +- ...stAbfsFileSystemContractGetFileStatus.java | 2 +- .../ITestAbfsFileSystemContractMkdir.java | 2 +- .../ITestAbfsFileSystemContractOpen.java | 2 +- .../ITestAbfsFileSystemContractRename.java | 2 +- ...stAbfsFileSystemContractRootDirectory.java | 2 +- .../ITestAbfsFileSystemContractSeek.java | 2 +- .../ITestAbfsFileSystemContractSetTimes.java | 2 +- .../fs/azurebfs/services/TestAbfsClient.java | 10 +- .../TestShellDecryptionKeyProvider.java | 1 + .../fs/azurebfs/utils/AbfsTestUtils.java | 5 + 48 files changed, 665 insertions(+), 227 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 518fef9495f..927a315be98 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 @@ -54,6 +54,7 @@ import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.azurebfs.services.KeyProvider; import org.apache.hadoop.fs.azurebfs.services.SimpleKeyProvider; import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx; +import org.apache.hadoop.security.ProviderUtils; import org.apache.hadoop.util.ReflectionUtils; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.*; @@ -65,7 +66,8 @@ import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.* @InterfaceAudience.Private @InterfaceStability.Evolving public class AbfsConfiguration{ - private final Configuration configuration; + private final Configuration rawConfig; + private final String accountName; private final boolean isSecure; @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_WRITE_BUFFER_SIZE, @@ -155,9 +157,12 @@ public class AbfsConfiguration{ private Map storageAccountKeys; - public AbfsConfiguration(final Configuration configuration) throws IllegalAccessException, InvalidConfigurationValueException { - this.configuration = configuration; - this.isSecure = this.configuration.getBoolean(FS_AZURE_SECURE_MODE, false); + public AbfsConfiguration(final Configuration rawConfig, String accountName) + throws IllegalAccessException, InvalidConfigurationValueException, IOException { + this.rawConfig = ProviderUtils.excludeIncompatibleCredentialProviders( + rawConfig, AzureBlobFileSystem.class); + this.accountName = accountName; + this.isSecure = getBoolean(FS_AZURE_SECURE_MODE, false); validateStorageAccountKeys(); Field[] fields = this.getClass().getDeclaredFields(); @@ -177,14 +182,130 @@ public class AbfsConfiguration{ } } + /** + * Appends an account name to a configuration key yielding the + * account-specific form. + * @param key Account-agnostic configuration key + * @return Account-specific configuration key + */ + public String accountConf(String key) { + return key + "." + accountName; + } + + /** + * Returns the account-specific value if it exists, then looks for an + * account-agnostic value. + * @param key Account-agnostic configuration key + * @return value if one exists, else null + */ + public String get(String key) { + return rawConfig.get(accountConf(key), rawConfig.get(key)); + } + + /** + * Returns the account-specific value if it exists, then looks for an + * account-agnostic value, and finally tries the default value. + * @param key Account-agnostic configuration key + * @param defaultValue Value returned if none is configured + * @return value if one exists, else the default value + */ + public boolean getBoolean(String key, boolean defaultValue) { + return rawConfig.getBoolean(accountConf(key), rawConfig.getBoolean(key, defaultValue)); + } + + /** + * Returns the account-specific value if it exists, then looks for an + * account-agnostic value, and finally tries the default value. + * @param key Account-agnostic configuration key + * @param defaultValue Value returned if none is configured + * @return value if one exists, else the default value + */ + public long getLong(String key, long defaultValue) { + return rawConfig.getLong(accountConf(key), rawConfig.getLong(key, defaultValue)); + } + + /** + * Returns the account-specific password in string form if it exists, then + * looks for an account-agnostic value. + * @param key Account-agnostic configuration key + * @return value in String form if one exists, else null + * @throws IOException + */ + public String getPasswordString(String key) throws IOException { + char[] passchars = rawConfig.getPassword(accountConf(key)); + if (passchars == null) { + passchars = rawConfig.getPassword(key); + } + if (passchars != null) { + return new String(passchars); + } + return null; + } + + /** + * Returns the account-specific Class if it exists, then looks for an + * account-agnostic value, and finally tries the default value. + * @param name Account-agnostic configuration key + * @param defaultValue Class returned if none is configured + * @param xface Interface shared by all possible values + * @return Highest-precedence Class object that was found + * @throws IOException + */ + public Class getClass(String name, Class defaultValue, Class xface) { + return rawConfig.getClass(accountConf(name), + rawConfig.getClass(name, defaultValue, xface), + xface); + } + + /** + * Returns the account-specific password in string form if it exists, then + * looks for an account-agnostic value. + * @param name Account-agnostic configuration key + * @param defaultValue Value returned if none is configured + * @return value in String form if one exists, else null + * @throws IOException + */ + public > T getEnum(String name, T defaultValue) { + return rawConfig.getEnum(accountConf(name), + rawConfig.getEnum(name, defaultValue)); + } + + /** + * Unsets parameter in the underlying Configuration object. + * Provided only as a convenience; does not add any account logic. + * @param key Configuration key + */ + public void unset(String key) { + rawConfig.unset(key); + } + + /** + * Sets String in the underlying Configuration object. + * Provided only as a convenience; does not add any account logic. + * @param key Configuration key + * @param value Configuration value + */ + public void set(String key, String value) { + rawConfig.set(key, value); + } + + /** + * Sets boolean in the underlying Configuration object. + * Provided only as a convenience; does not add any account logic. + * @param key Configuration key + * @param value Configuration value + */ + public void setBoolean(String key, boolean value) { + rawConfig.setBoolean(key, value); + } + public boolean isSecureMode() { return isSecure; } - public String getStorageAccountKey(final String accountName) throws AzureBlobFileSystemException { + public String getStorageAccountKey() throws AzureBlobFileSystemException { String key; - String keyProviderClass = - configuration.get(AZURE_KEY_ACCOUNT_KEYPROVIDER_PREFIX + accountName); + String keyProviderClass = get(AZURE_KEY_ACCOUNT_KEYPROVIDER); KeyProvider keyProvider; if (keyProviderClass == null) { @@ -195,7 +316,7 @@ public class AbfsConfiguration{ // implements KeyProvider Object keyProviderObject; try { - Class clazz = configuration.getClassByName(keyProviderClass); + Class clazz = rawConfig.getClassByName(keyProviderClass); keyProviderObject = clazz.newInstance(); } catch (Exception e) { throw new KeyProviderException("Unable to load key provider class.", e); @@ -206,7 +327,7 @@ public class AbfsConfiguration{ } keyProvider = (KeyProvider) keyProviderObject; } - key = keyProvider.getStorageAccountKey(accountName, configuration); + key = keyProvider.getStorageAccountKey(accountName, rawConfig); if (key == null) { throw new ConfigurationPropertyNotFoundException(accountName); @@ -215,8 +336,8 @@ public class AbfsConfiguration{ return key; } - public Configuration getConfiguration() { - return this.configuration; + public Configuration getRawConfiguration() { + return this.rawConfig; } public int getWriteBufferSize() { @@ -292,11 +413,11 @@ public class AbfsConfiguration{ } public SSLSocketFactoryEx.SSLChannelMode getPreferredSSLFactoryOption() { - return configuration.getEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, DEFAULT_FS_AZURE_SSL_CHANNEL_MODE); + return getEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, DEFAULT_FS_AZURE_SSL_CHANNEL_MODE); } - public AuthType getAuthType(final String accountName) { - return configuration.getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME + accountName, AuthType.SharedKey); + public AuthType getAuthType(String accountName) { + return getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey); } public boolean isDelegationTokenManagerEnabled() { @@ -304,34 +425,34 @@ public class AbfsConfiguration{ } public AbfsDelegationTokenManager getDelegationTokenManager() throws IOException { - return new AbfsDelegationTokenManager(configuration); + return new AbfsDelegationTokenManager(getRawConfiguration()); } - public AccessTokenProvider getTokenProvider(final String accountName) throws TokenAccessProviderException { - AuthType authType = configuration.getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME + accountName, AuthType.SharedKey); + public AccessTokenProvider getTokenProvider() throws TokenAccessProviderException { + AuthType authType = getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey); if (authType == AuthType.OAuth) { try { Class tokenProviderClass = - configuration.getClass(FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME + accountName, null, + getClass(FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME, null, AccessTokenProvider.class); AccessTokenProvider tokenProvider = null; if (tokenProviderClass == ClientCredsTokenProvider.class) { - String authEndpoint = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT + accountName); - String clientId = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID + accountName); - String clientSecret = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET + accountName); + String authEndpoint = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT); + String clientId = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID); + String clientSecret = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET); tokenProvider = new ClientCredsTokenProvider(authEndpoint, clientId, clientSecret); } else if (tokenProviderClass == UserPasswordTokenProvider.class) { - String authEndpoint = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT + accountName); - String username = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_USER_NAME + accountName); - String password = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_USER_PASSWORD + accountName); + String authEndpoint = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT); + String username = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_USER_NAME); + String password = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_USER_PASSWORD); tokenProvider = new UserPasswordTokenProvider(authEndpoint, username, password); } else if (tokenProviderClass == MsiTokenProvider.class) { - String tenantGuid = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT + accountName); - String clientId = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID + accountName); + String tenantGuid = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT); + String clientId = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID); tokenProvider = new MsiTokenProvider(tenantGuid, clientId); } else if (tokenProviderClass == RefreshTokenBasedTokenProvider.class) { - String refreshToken = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN + accountName); - String clientId = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID + accountName); + String refreshToken = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN); + String clientId = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID); tokenProvider = new RefreshTokenBasedTokenProvider(clientId, refreshToken); } else { throw new IllegalArgumentException("Failed to initialize " + tokenProviderClass); @@ -345,20 +466,19 @@ public class AbfsConfiguration{ } else if (authType == AuthType.Custom) { try { - String configKey = FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME + accountName; + String configKey = FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME; Class customTokenProviderClass = - configuration.getClass(configKey, null, - CustomTokenProviderAdaptee.class); + getClass(configKey, null, CustomTokenProviderAdaptee.class); if (customTokenProviderClass == null) { throw new IllegalArgumentException( String.format("The configuration value for \"%s\" is invalid.", configKey)); } CustomTokenProviderAdaptee azureTokenProvider = ReflectionUtils - .newInstance(customTokenProviderClass, configuration); + .newInstance(customTokenProviderClass, rawConfig); if (azureTokenProvider == null) { throw new IllegalArgumentException("Failed to initialize " + customTokenProviderClass); } - azureTokenProvider.initialize(configuration, accountName); + azureTokenProvider.initialize(rawConfig, accountName); return new CustomTokenProviderAdapter(azureTokenProvider); } catch(IllegalArgumentException e) { throw e; @@ -375,7 +495,7 @@ public class AbfsConfiguration{ void validateStorageAccountKeys() throws InvalidConfigurationValueException { Base64StringConfigurationBasicValidator validator = new Base64StringConfigurationBasicValidator( FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME, "", true); - this.storageAccountKeys = configuration.getValByRegex(FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX); + this.storageAccountKeys = rawConfig.getValByRegex(FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX); for (Map.Entry account : storageAccountKeys.entrySet()) { validator.validate(account.getValue()); @@ -384,7 +504,7 @@ public class AbfsConfiguration{ int validateInt(Field field) throws IllegalAccessException, InvalidConfigurationValueException { IntegerConfigurationValidatorAnnotation validator = field.getAnnotation(IntegerConfigurationValidatorAnnotation.class); - String value = configuration.get(validator.ConfigurationKey()); + String value = get(validator.ConfigurationKey()); // validate return new IntegerConfigurationBasicValidator( @@ -397,7 +517,7 @@ public class AbfsConfiguration{ long validateLong(Field field) throws IllegalAccessException, InvalidConfigurationValueException { LongConfigurationValidatorAnnotation validator = field.getAnnotation(LongConfigurationValidatorAnnotation.class); - String value = configuration.get(validator.ConfigurationKey()); + String value = rawConfig.get(validator.ConfigurationKey()); // validate return new LongConfigurationBasicValidator( @@ -410,7 +530,7 @@ public class AbfsConfiguration{ String validateString(Field field) throws IllegalAccessException, InvalidConfigurationValueException { StringConfigurationValidatorAnnotation validator = field.getAnnotation(StringConfigurationValidatorAnnotation.class); - String value = configuration.get(validator.ConfigurationKey()); + String value = rawConfig.get(validator.ConfigurationKey()); // validate return new StringConfigurationBasicValidator( @@ -421,7 +541,7 @@ public class AbfsConfiguration{ String validateBase64String(Field field) throws IllegalAccessException, InvalidConfigurationValueException { Base64StringConfigurationValidatorAnnotation validator = field.getAnnotation((Base64StringConfigurationValidatorAnnotation.class)); - String value = configuration.get(validator.ConfigurationKey()); + String value = rawConfig.get(validator.ConfigurationKey()); // validate return new Base64StringConfigurationBasicValidator( @@ -432,7 +552,7 @@ public class AbfsConfiguration{ boolean validateBoolean(Field field) throws IllegalAccessException, InvalidConfigurationValueException { BooleanConfigurationValidatorAnnotation validator = field.getAnnotation(BooleanConfigurationValidatorAnnotation.class); - String value = configuration.get(validator.ConfigurationKey()); + String value = rawConfig.get(validator.ConfigurationKey()); // validate return new BooleanConfigurationBasicValidator( @@ -441,14 +561,6 @@ public class AbfsConfiguration{ validator.ThrowIfInvalid()).validate(value); } - String getPasswordString(String key) throws IOException { - char[] passchars = configuration.getPassword(key); - if (passchars != null) { - return new String(passchars); - } - return null; - } - @VisibleForTesting void setReadBufferSize(int bufferSize) { this.readBufferSize = bufferSize; @@ -463,4 +575,4 @@ public class AbfsConfiguration{ void setEnableFlush(boolean enableFlush) { this.enableFlush = enableFlush; } -} \ No newline at end of file +} 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 6542a648b7b..e3bea3251d0 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 @@ -113,10 +113,15 @@ public class AzureBlobFileSystemStore { private boolean isNamespaceEnabled; public AzureBlobFileSystemStore(URI uri, boolean isSecure, Configuration configuration, UserGroupInformation userGroupInformation) - throws AzureBlobFileSystemException { + throws AzureBlobFileSystemException, IOException { this.uri = uri; + + String[] authorityParts = authorityParts(uri); + final String fileSystemName = authorityParts[0]; + final String accountName = authorityParts[1]; + try { - this.abfsConfiguration = new AbfsConfiguration(configuration); + this.abfsConfiguration = new AbfsConfiguration(configuration, accountName); } catch (IllegalAccessException exception) { throw new FileSystemOperationUnhandledException(exception); } @@ -125,7 +130,31 @@ public class AzureBlobFileSystemStore { this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList( abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA))); - initializeClient(uri, isSecure); + initializeClient(uri, fileSystemName, accountName, isSecure); + } + + private String[] authorityParts(URI uri) throws InvalidUriAuthorityException, InvalidUriException { + final String authority = uri.getRawAuthority(); + if (null == authority) { + throw new InvalidUriAuthorityException(uri.toString()); + } + + if (!authority.contains(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER)) { + throw new InvalidUriAuthorityException(uri.toString()); + } + + final String[] authorityParts = authority.split(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2); + + if (authorityParts.length < 2 || authorityParts[0] != null + && authorityParts[0].isEmpty()) { + final String errMsg = String + .format("'%s' has a malformed authority, expected container name. " + + "Authority takes the form " + + FileSystemUriSchemes.ABFS_SCHEME + "://[@]", + uri.toString()); + throw new InvalidUriException(errMsg); + } + return authorityParts; } public boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException { @@ -154,7 +183,7 @@ public class AzureBlobFileSystemStore { // the Azure Storage Service URI changes from // http[s]://[account][domain-suffix]/[filesystem] to // http[s]://[ip]:[port]/[account]/[filesystem]. - String endPoint = abfsConfiguration.getConfiguration().get(AZURE_ABFS_ENDPOINT); + String endPoint = abfsConfiguration.get(AZURE_ABFS_ENDPOINT); if (endPoint == null || !endPoint.contains(AbfsHttpConstants.COLON)) { uriBuilder.setHost(hostName); return uriBuilder; @@ -738,36 +767,12 @@ public class AzureBlobFileSystemStore { return isKeyForDirectorySet(key, azureAtomicRenameDirSet); } - private void initializeClient(URI uri, boolean isSeure) throws AzureBlobFileSystemException { + private void initializeClient(URI uri, String fileSystemName, String accountName, boolean isSecure) throws AzureBlobFileSystemException { if (this.client != null) { return; } - final String authority = uri.getRawAuthority(); - if (null == authority) { - throw new InvalidUriAuthorityException(uri.toString()); - } - - if (!authority.contains(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER)) { - throw new InvalidUriAuthorityException(uri.toString()); - } - - final String[] authorityParts = authority.split(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2); - - if (authorityParts.length < 2 || authorityParts[0] != null - && authorityParts[0].isEmpty()) { - final String errMsg = String - .format("'%s' has a malformed authority, expected container name. " - + "Authority takes the form " - + FileSystemUriSchemes.ABFS_SCHEME + "://[@]", - uri.toString()); - throw new InvalidUriException(errMsg); - } - - final String fileSystemName = authorityParts[0]; - final String accountName = authorityParts[1]; - - final URIBuilder uriBuilder = getURIBuilder(accountName, isSeure); + final URIBuilder uriBuilder = getURIBuilder(accountName, isSecure); final String url = uriBuilder.toString() + AbfsHttpConstants.FORWARD_SLASH + fileSystemName; @@ -788,9 +793,9 @@ public class AzureBlobFileSystemStore { uri.toString() + " - account name is not fully qualified."); } creds = new SharedKeyCredentials(accountName.substring(0, dotIndex), - abfsConfiguration.getStorageAccountKey(accountName)); + abfsConfiguration.getStorageAccountKey()); } else { - tokenProvider = abfsConfiguration.getTokenProvider(accountName); + tokenProvider = abfsConfiguration.getTokenProvider(); } this.client = new AbfsClient(baseUrl, creds, abfsConfiguration, new ExponentialRetryPolicy(), tokenProvider); 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 52367193045..13cdaeb4349 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 @@ -27,7 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability; @InterfaceAudience.Public @InterfaceStability.Evolving public final class ConfigurationKeys { - public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME = "fs.azure.account.key."; + 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"; @@ -54,29 +54,33 @@ public final class ConfigurationKeys { public static final String FS_AZURE_USER_AGENT_PREFIX_KEY = "fs.azure.user.agent.prefix"; public static final String FS_AZURE_SSL_CHANNEL_MODE_KEY = "fs.azure.ssl.channel.mode"; - public static final String AZURE_KEY_ACCOUNT_KEYPROVIDER_PREFIX = "fs.azure.account.keyprovider."; + 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"; /** End point of ABFS account: {@value}. */ public static final String AZURE_ABFS_ENDPOINT = "fs.azure.abfs.endpoint"; - /** Prefix for auth type properties: {@value}. */ - public static final String FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME = "fs.azure.account.auth.type."; - /** Prefix for oauth token provider type: {@value}. */ - public static final String FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME = "fs.azure.account.oauth.provider.type."; - /** Prefix for oauth AAD client id: {@value}. */ - public static final String FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID = "fs.azure.account.oauth2.client.id."; - /** Prefix for oauth AAD client secret: {@value}. */ - public static final String FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET = "fs.azure.account.oauth2.client.secret."; - /** Prefix for oauth AAD client endpoint: {@value}. */ - public static final String FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT = "fs.azure.account.oauth2.client.endpoint."; - /** Prefix for oauth msi tenant id: {@value}. */ - public static final String FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT = "fs.azure.account.oauth2.msi.tenant."; - /** Prefix for oauth user name: {@value}. */ - public static final String FS_AZURE_ACCOUNT_OAUTH_USER_NAME = "fs.azure.account.oauth2.user.name."; - /** Prefix for oauth user password: {@value}. */ - public static final String FS_AZURE_ACCOUNT_OAUTH_USER_PASSWORD = "fs.azure.account.oauth2.user.password."; - /** Prefix for oauth refresh token: {@value}. */ - public static final String FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN = "fs.azure.account.oauth2.refresh.token."; + /** Key for auth type properties: {@value}. */ + public static final String FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME = "fs.azure.account.auth.type"; + /** Key for oauth token provider type: {@value}. */ + public static final String FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME = "fs.azure.account.oauth.provider.type"; + /** Key for oauth AAD client id: {@value}. */ + public static final String FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID = "fs.azure.account.oauth2.client.id"; + /** Key for oauth AAD client secret: {@value}. */ + public static final String FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET = "fs.azure.account.oauth2.client.secret"; + /** Key for oauth AAD client endpoint: {@value}. */ + public static final String FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT = "fs.azure.account.oauth2.client.endpoint"; + /** Key for oauth msi tenant id: {@value}. */ + public static final String FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT = "fs.azure.account.oauth2.msi.tenant"; + /** Key for oauth user name: {@value}. */ + public static final String FS_AZURE_ACCOUNT_OAUTH_USER_NAME = "fs.azure.account.oauth2.user.name"; + /** Key for oauth user password: {@value}. */ + public static final String FS_AZURE_ACCOUNT_OAUTH_USER_PASSWORD = "fs.azure.account.oauth2.user.password"; + /** Key for oauth refresh token: {@value}. */ + public static final String FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN = "fs.azure.account.oauth2.refresh.token"; + + public static String accountProperty(String property, String account) { + return property + "." + account; + } public static final String FS_AZURE_ENABLE_DELEGATION_TOKEN = "fs.azure.enable.delegation.token"; public static final String FS_AZURE_DELEGATION_TOKEN_PROVIDER_TYPE = "fs.azure.delegation.token.provider.type"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/UserPasswordTokenProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/UserPasswordTokenProvider.java index 7504e9d527e..3dad32ec6f5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/UserPasswordTokenProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/UserPasswordTokenProvider.java @@ -24,7 +24,6 @@ import com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.conf.Configuration; /** * Provides tokens based on username and password. */ @@ -54,13 +53,4 @@ public class UserPasswordTokenProvider extends AccessTokenProvider { LOG.debug("AADToken: refreshing user-password based token"); return AzureADAuthenticator.getTokenUsingClientCreds(authEndpoint, username, password); } - - private static String getPasswordString(Configuration conf, String key) - throws IOException { - char[] passchars = conf.getPassword(key); - if (passchars == null) { - throw new IOException("Password " + key + " not found"); - } - return new String(passchars); - } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ShellDecryptionKeyProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ShellDecryptionKeyProvider.java index 3fc05ff3e5e..bdac922fb3a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ShellDecryptionKeyProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ShellDecryptionKeyProvider.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.Arrays; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException; import org.apache.hadoop.util.Shell; @@ -36,11 +37,18 @@ public class ShellDecryptionKeyProvider extends SimpleKeyProvider { private static final Logger LOG = LoggerFactory.getLogger(ShellDecryptionKeyProvider.class); @Override - public String getStorageAccountKey(String accountName, Configuration conf) + public String getStorageAccountKey(String accountName, Configuration rawConfig) throws KeyProviderException { - String envelope = super.getStorageAccountKey(accountName, conf); + String envelope = super.getStorageAccountKey(accountName, rawConfig); - final String command = conf.get(ConfigurationKeys.AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT); + AbfsConfiguration abfsConfig; + try { + abfsConfig = new AbfsConfiguration(rawConfig, accountName); + } catch(IllegalAccessException | IOException e) { + throw new KeyProviderException("Unable to get key from credential providers.", e); + } + + final String command = abfsConfig.get(ConfigurationKeys.AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT); if (command == null) { throw new KeyProviderException( "Script path is not specified via fs.azure.shellkeyprovider.script"); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SimpleKeyProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SimpleKeyProvider.java index cedae57f1bb..727e1b3fd3f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SimpleKeyProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SimpleKeyProvider.java @@ -21,10 +21,10 @@ package org.apache.hadoop.fs.azurebfs.services; import java.io.IOException; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException; -import org.apache.hadoop.security.ProviderUtils; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,19 +36,19 @@ public class SimpleKeyProvider implements KeyProvider { private static final Logger LOG = LoggerFactory.getLogger(SimpleKeyProvider.class); @Override - public String getStorageAccountKey(String accountName, Configuration conf) + public String getStorageAccountKey(String accountName, Configuration rawConfig) throws KeyProviderException { String key = null; + try { - Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders( - conf, AzureBlobFileSystem.class); - char[] keyChars = c.getPassword(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + accountName); - if (keyChars != null) { - key = new String(keyChars); - } + AbfsConfiguration abfsConfig = new AbfsConfiguration(rawConfig, accountName); + key = abfsConfig.getPasswordString(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME); + } catch(IllegalAccessException | InvalidConfigurationValueException e) { + throw new KeyProviderException("Failure to initialize configuration", e); } catch(IOException ioe) { LOG.warn("Unable to get key from credential providers. {}", ioe); } + return key; } } diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md index a4b3483de70..db55e67b766 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md @@ -62,6 +62,16 @@ The abfs client has a fully consistent view of the store, which has complete Cre * Directory Rename: `O(files)`. * Directory Delete: `O(files)`. +## Configuring ABFS + +Any configuration can be specified generally (or as the default when accessing all accounts) or can be tied to s a specific account. +For example, an OAuth identity can be configured for use regardless of which account is accessed with the property +"fs.azure.account.oauth2.client.id" +or you can configure an identity to be used only for a specific storage account with +"fs.azure.account.oauth2.client.id.\.dfs.core.windows.net". + +Note that it doesn't make sense to do this with some properties, like shared keys that are inherently account-specific. + ## Testing ABFS See the relevant section in [Testing Azure](testing_azure.html). 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 f61954f4b3d..52185cdc9af 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 @@ -67,38 +67,39 @@ public abstract class AbstractAbfsIntegrationTest extends private AzureBlobFileSystem abfs; private String abfsScheme; - private Configuration configuration; + private Configuration rawConfig; + private AbfsConfiguration abfsConfig; private String fileSystemName; private String accountName; private String testUrl; private AuthType authType; - protected AbstractAbfsIntegrationTest() { + protected AbstractAbfsIntegrationTest() throws Exception { fileSystemName = TEST_CONTAINER_PREFIX + UUID.randomUUID().toString(); - configuration = new Configuration(); - configuration.addResource(TEST_CONFIGURATION_FILE_NAME); + rawConfig = new Configuration(); + rawConfig.addResource(TEST_CONFIGURATION_FILE_NAME); - this.accountName = this.configuration.get(FS_AZURE_ACCOUNT_NAME); + this.accountName = rawConfig.get(FS_AZURE_ACCOUNT_NAME); if (accountName == null) { // check if accountName is set using different config key - accountName = configuration.get(FS_AZURE_ABFS_ACCOUNT_NAME); + accountName = rawConfig.get(FS_AZURE_ABFS_ACCOUNT_NAME); } assumeTrue("Not set: " + FS_AZURE_ABFS_ACCOUNT_NAME, accountName != null && !accountName.isEmpty()); + abfsConfig = new AbfsConfiguration(rawConfig, accountName); - authType = configuration.getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME - + accountName, AuthType.SharedKey); + authType = abfsConfig.getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey); abfsScheme = authType == AuthType.SharedKey ? FileSystemUriSchemes.ABFS_SCHEME : FileSystemUriSchemes.ABFS_SECURE_SCHEME; if (authType == AuthType.SharedKey) { - String keyProperty = FS_AZURE_ACCOUNT_KEY_PREFIX + accountName; - assumeTrue("Not set: " + keyProperty, configuration.get(keyProperty) != null); + assumeTrue("Not set: " + FS_AZURE_ACCOUNT_KEY, + abfsConfig.get(FS_AZURE_ACCOUNT_KEY) != null); // Update credentials } else { - String accessTokenProviderKey = FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME + accountName; - assumeTrue("Not set: " + accessTokenProviderKey, configuration.get(accessTokenProviderKey) != null); + assumeTrue("Not set: " + FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME, + abfsConfig.get(FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME) != null); } final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName(); @@ -111,14 +112,14 @@ public abstract class AbstractAbfsIntegrationTest extends } this.testUrl = defaultUri.toString(); - configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); - configuration.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true); + abfsConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); + abfsConfig.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true); // For testing purposes, an IP address and port may be provided to override // the host specified in the FileSystem URI. Also note that the format of // the Azure Storage Service URI changes from // http[s]://[account][domain-suffix]/[filesystem] to // http[s]://[ip]:[port]/[account]/[filesystem]. - String endPoint = configuration.get(AZURE_ABFS_ENDPOINT); + String endPoint = abfsConfig.get(AZURE_ABFS_ENDPOINT); if (endPoint != null && endPoint.contains(":") && endPoint.split(":").length == 2) { this.isIPAddress = true; } else { @@ -140,18 +141,18 @@ public abstract class AbstractAbfsIntegrationTest extends // update configuration with wasb credentials String accountNameWithoutDomain = accountName.split("\\.")[0]; String wasbAccountName = accountNameWithoutDomain + WASB_ACCOUNT_NAME_DOMAIN_SUFFIX; - String keyProperty = FS_AZURE_ACCOUNT_KEY_PREFIX + wasbAccountName; - if (configuration.get(keyProperty) == null) { - configuration.set(keyProperty, getAccountKey()); + String keyProperty = FS_AZURE_ACCOUNT_KEY + "." + wasbAccountName; + if (rawConfig.get(keyProperty) == null) { + rawConfig.set(keyProperty, getAccountKey()); } azureNativeFileSystemStore.initialize( wasbUri, - configuration, - new AzureFileSystemInstrumentation(getConfiguration())); + rawConfig, + new AzureFileSystemInstrumentation(rawConfig)); wasb = new NativeAzureFileSystem(azureNativeFileSystemStore); - wasb.initialize(wasbUri, configuration); + wasb.initialize(wasbUri, rawConfig); } } @@ -197,8 +198,8 @@ public abstract class AbstractAbfsIntegrationTest extends } public AzureBlobFileSystem getFileSystem(String abfsUri) throws Exception { - configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, abfsUri); - final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(configuration); + abfsConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, abfsUri); + final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(rawConfig); return fs; } @@ -210,7 +211,7 @@ public abstract class AbstractAbfsIntegrationTest extends public AzureBlobFileSystem createFileSystem() throws IOException { Preconditions.checkState(abfs == null, "existing ABFS instance exists: %s", abfs); - abfs = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + abfs = (AzureBlobFileSystem) FileSystem.newInstance(rawConfig); return abfs; } @@ -221,7 +222,7 @@ public abstract class AbstractAbfsIntegrationTest extends protected String getHostName() { // READ FROM ENDPOINT, THIS IS CALLED ONLY WHEN TESTING AGAINST DEV-FABRIC - String endPoint = configuration.get(AZURE_ABFS_ENDPOINT); + String endPoint = abfsConfig.get(AZURE_ABFS_ENDPOINT); return endPoint.split(":")[0]; } @@ -245,13 +246,15 @@ public abstract class AbstractAbfsIntegrationTest extends } protected String getAccountKey() { - return configuration.get( - FS_AZURE_ACCOUNT_KEY_PREFIX - + accountName); + return abfsConfig.get(FS_AZURE_ACCOUNT_KEY); } - protected Configuration getConfiguration() { - return configuration; + public AbfsConfiguration getConfiguration() { + return abfsConfig; + } + + public Configuration getRawConfiguration() { + return abfsConfig.getRawConfiguration(); } protected boolean isIPAddress() { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsScaleTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsScaleTest.java index cfda7a7eef9..14c9bff7bf8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsScaleTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsScaleTest.java @@ -21,6 +21,7 @@ package org.apache.hadoop.fs.azurebfs; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azure.integration.AzureTestConstants; import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.assumeScaleTestsEnabled; @@ -34,6 +35,10 @@ public class AbstractAbfsScaleTest extends AbstractAbfsIntegrationTest { protected static final Logger LOG = LoggerFactory.getLogger(AbstractAbfsScaleTest.class); + public AbstractAbfsScaleTest() throws Exception { + super(); + } + @Override protected int getTestTimeoutMillis() { return AzureTestConstants.SCALE_TEST_TIMEOUT_MILLIS; @@ -43,7 +48,8 @@ public class AbstractAbfsScaleTest extends AbstractAbfsIntegrationTest { public void setup() throws Exception { super.setup(); LOG.debug("Scale test operation count = {}", getOperationCount()); - assumeScaleTestsEnabled(getConfiguration()); + Configuration rawConfiguration = getRawConfiguration(); + assumeScaleTestsEnabled(rawConfiguration); } protected long getOperationCount() { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java index 1c2083d5038..f024f257b7d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java @@ -30,6 +30,10 @@ import org.junit.Test; public final class ITestAbfsClient extends AbstractAbfsIntegrationTest { private static final int LIST_MAX_RESULTS = 5000; + public ITestAbfsClient() throws Exception { + super(); + } + @Test public void testContinuationTokenHavingEqualSign() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); @@ -42,4 +46,4 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest { Assert.assertEquals("InvalidQueryParameterValue", ex.getErrorCode().getErrorCode()); } } -} \ No newline at end of file +} 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 f62ea6e529e..a270a00e913 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 @@ -51,7 +51,7 @@ public class ITestAbfsReadWriteAndSeek extends AbstractAbfsScaleTest { private final int size; - public ITestAbfsReadWriteAndSeek(final int size) { + public ITestAbfsReadWriteAndSeek(final int size) throws Exception { this.size = size; } @@ -62,7 +62,7 @@ public class ITestAbfsReadWriteAndSeek extends AbstractAbfsScaleTest { private void testReadWriteAndSeek(int bufferSize) throws Exception { final AzureBlobFileSystem fs = getFileSystem(); - final AbfsConfiguration abfsConfiguration = new AbfsConfiguration(getConfiguration()); + final AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration(); abfsConfiguration.setWriteBufferSize(bufferSize); abfsConfiguration.setReadBufferSize(bufferSize); 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 f2e26ec19f2..cbe19396d12 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 @@ -34,7 +34,8 @@ 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"); - public ITestAzureBlobFileSystemAppend() { + + public ITestAzureBlobFileSystemAppend() throws Exception { super(); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java index 46ac8664b69..d8940f7d753 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java @@ -35,7 +35,8 @@ import org.apache.hadoop.fs.Path; */ public class ITestAzureBlobFileSystemBackCompat extends AbstractAbfsIntegrationTest { - public ITestAzureBlobFileSystemBackCompat() { + + public ITestAzureBlobFileSystemBackCompat() throws Exception { super(); Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java index 90eff97854c..917ee9ce1b0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCopy.java @@ -39,7 +39,8 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile; * Test copy operation. */ public class ITestAzureBlobFileSystemCopy extends AbstractAbfsIntegrationTest { - public ITestAzureBlobFileSystemCopy() { + + public ITestAzureBlobFileSystemCopy() throws Exception { super(); } 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 1e43f9a3601..ab01166b9b3 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 @@ -38,7 +38,8 @@ 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"; - public ITestAzureBlobFileSystemCreate() { + + public ITestAzureBlobFileSystemCreate() throws Exception { super(); } 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 91d1723a752..486daca4f11 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 @@ -41,7 +41,8 @@ import static org.apache.hadoop.test.LambdaTestUtils.intercept; */ public class ITestAzureBlobFileSystemDelete extends AbstractAbfsIntegrationTest { - public ITestAzureBlobFileSystemDelete() { + + public ITestAzureBlobFileSystemDelete() throws Exception { super(); } 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 f1800c00c16..6021e5d51be 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 @@ -28,7 +28,6 @@ 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.conf.Configuration; import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; import static org.junit.Assert.assertEquals; @@ -45,9 +44,9 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest { private static final int TEST_DEFAULT_BUFFER_SIZE = 4 * 1024 * 1024; private static final int TEST_DEFAULT_READ_BUFFER_SIZE = 1023900; - public ITestAzureBlobFileSystemE2E() { + public ITestAzureBlobFileSystemE2E() throws Exception { super(); - Configuration configuration = this.getConfiguration(); + AbfsConfiguration configuration = this.getConfiguration(); configuration.set(ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH, "0"); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java index 7ed9d42bb19..fccd0632375 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2EScale.java @@ -45,7 +45,7 @@ public class ITestAzureBlobFileSystemE2EScale extends private static final int ONE_MB = 1024 * 1024; private static final int DEFAULT_WRITE_TIMES = 100; - public ITestAzureBlobFileSystemE2EScale() { + public ITestAzureBlobFileSystemE2EScale() throws Exception { } @Test 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 dba10f5f13f..b08b9202dd2 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 @@ -41,7 +41,7 @@ public class ITestAzureBlobFileSystemFileStatus extends private static final Path TEST_FILE = new Path("testFile"); private static final Path TEST_FOLDER = new Path("testDir"); - public ITestAzureBlobFileSystemFileStatus() { + public ITestAzureBlobFileSystemFileStatus() throws Exception { super(); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java index c1022b01978..9d1738857c6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java @@ -41,12 +41,12 @@ public class ITestAzureBlobFileSystemFinalize extends AbstractAbfsScaleTest{ @Test public void testFinalize() throws Exception { // Disable the cache for filesystem to make sure there is no reference. - Configuration configuration = this.getConfiguration(); - configuration.setBoolean( + Configuration rawConfig = this.getRawConfiguration(); + rawConfig.setBoolean( this.getAuthType() == AuthType.SharedKey ? DISABLE_ABFS_CACHE_KEY : DISABLE_ABFSSS_CACHE_KEY, true); - AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(configuration); + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(rawConfig); WeakReference ref = new WeakReference(fs); fs = null; @@ -61,4 +61,4 @@ public class ITestAzureBlobFileSystemFinalize extends AbstractAbfsScaleTest{ Assert.assertTrue("testFinalizer didn't get cleaned up within maxTries", ref.get() == null); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index 337f95ce917..23a1ab5bb72 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -60,7 +60,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest { private static final int TEST_FILE_LENGTH = 1024 * 1024 * 8; private static final int WAITING_TIME = 1000; - public ITestAzureBlobFileSystemFlush() { + public ITestAzureBlobFileSystemFlush() throws Exception { super(); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java index 874a8a34c59..5f08721ada0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemInitAndCreate.java @@ -31,8 +31,8 @@ import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; */ public class ITestAzureBlobFileSystemInitAndCreate extends AbstractAbfsIntegrationTest { - public ITestAzureBlobFileSystemInitAndCreate() { + public ITestAzureBlobFileSystemInitAndCreate() throws Exception { this.getConfiguration().unset(ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION); } 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 b87abe68355..60e0fbc7237 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 @@ -42,7 +42,8 @@ import static org.apache.hadoop.test.LambdaTestUtils.intercept; public class ITestAzureBlobFileSystemListStatus extends AbstractAbfsIntegrationTest { private static final int TEST_FILES_NUMBER = 6000; - public ITestAzureBlobFileSystemListStatus() { + + public ITestAzureBlobFileSystemListStatus() throws Exception { super(); } 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 1bb2c54b376..382d3966485 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 @@ -28,7 +28,8 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs; * Test mkdir operation. */ public class ITestAzureBlobFileSystemMkDir extends AbstractAbfsIntegrationTest { - public ITestAzureBlobFileSystemMkDir() { + + public ITestAzureBlobFileSystemMkDir() throws Exception { super(); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java index f60740fce0b..533f4712565 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java @@ -53,7 +53,7 @@ public class ITestAzureBlobFileSystemOauth extends AbstractAbfsIntegrationTest{ private static final Path EXISTED_FILE_PATH = new Path("/existedFile"); private static final Path EXISTED_FOLDER_PATH = new Path("/existedFolder"); - public ITestAzureBlobFileSystemOauth() { + public ITestAzureBlobFileSystemOauth() throws Exception { Assume.assumeTrue(this.getAuthType() == AuthType.OAuth); } /* @@ -161,16 +161,18 @@ public class ITestAzureBlobFileSystemOauth extends AbstractAbfsIntegrationTest{ } private AzureBlobFileSystem getBlobConributor() throws Exception { - Configuration configuration = this.getConfiguration(); - configuration.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID + this.getAccountName(), configuration.get(FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_ID)); - configuration.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET + this.getAccountName(), configuration.get(FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_SECRET)); - return getFileSystem(configuration); + AbfsConfiguration abfsConfig = this.getConfiguration(); + abfsConfig.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID + "." + this.getAccountName(), abfsConfig.get(FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_ID)); + abfsConfig.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET + "." + this.getAccountName(), abfsConfig.get(FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_SECRET)); + Configuration rawConfig = abfsConfig.getRawConfiguration(); + return getFileSystem(rawConfig); } private AzureBlobFileSystem getBlobReader() throws Exception { - Configuration configuration = this.getConfiguration(); - configuration.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID + this.getAccountName(), configuration.get(FS_AZURE_BLOB_DATA_READER_CLIENT_ID)); - configuration.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET + this.getAccountName(), configuration.get(FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET)); - return getFileSystem(configuration); + AbfsConfiguration abfsConfig = this.getConfiguration(); + abfsConfig.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID + "." + this.getAccountName(), abfsConfig.get(FS_AZURE_BLOB_DATA_READER_CLIENT_ID)); + abfsConfig.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET + "." + this.getAccountName(), abfsConfig.get(FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET)); + Configuration rawConfig = abfsConfig.getRawConfiguration(); + return getFileSystem(rawConfig); } } 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 c97e84052d4..e0e1d899a21 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 @@ -41,7 +41,9 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile; */ public class ITestAzureBlobFileSystemRename extends AbstractAbfsIntegrationTest { - public ITestAzureBlobFileSystemRename() { + + public ITestAzureBlobFileSystemRename() throws Exception { + super(); } @Test 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 0ac7fcf08f0..044c325c8c8 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 @@ -66,7 +66,7 @@ public class ITestAzureBlobFileSystemRenameUnicode extends }); } - public ITestAzureBlobFileSystemRenameUnicode() { + public ITestAzureBlobFileSystemRenameUnicode() throws Exception { } /** 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 50b1828a828..8b60dd801cb 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 @@ -32,7 +32,7 @@ import org.apache.hadoop.fs.azurebfs.services.AuthType; * Test AzureBlobFileSystem initialization. */ public class ITestFileSystemInitialization extends AbstractAbfsIntegrationTest { - public ITestFileSystemInitialization() { + public ITestFileSystemInitialization() throws Exception { super(); } @@ -62,10 +62,10 @@ public class ITestFileSystemInitialization extends AbstractAbfsIntegrationTest { null, null, null); - Configuration conf = getConfiguration(); - conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); + Configuration rawConfig = getRawConfiguration(); + rawConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); - try(SecureAzureBlobFileSystem fs = (SecureAzureBlobFileSystem) FileSystem.newInstance(conf)) { + try(SecureAzureBlobFileSystem fs = (SecureAzureBlobFileSystem) FileSystem.newInstance(rawConfig)) { assertEquals(fs.getUri(), new URI(FileSystemUriSchemes.ABFS_SECURE_SCHEME, filesystem + "@" + accountName, null, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java index 7a7e3279da6..e6b45c8f8b8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemProperties.java @@ -33,7 +33,7 @@ import org.apache.hadoop.fs.Path; public class ITestFileSystemProperties extends AbstractAbfsIntegrationTest { private static final int TEST_DATA = 100; private static final Path TEST_PATH = new Path("/testfile"); - public ITestFileSystemProperties() { + public ITestFileSystemProperties() throws Exception { } @Test diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java index 56a91d3eaf9..4393bd82b11 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java @@ -77,14 +77,15 @@ public class ITestFileSystemRegistration extends AbstractAbfsIntegrationTest { @Test public void ensureAzureBlobFileSystemIsDefaultFileSystem() throws Exception { - AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(getConfiguration()); + Configuration rawConfig = getRawConfiguration(); + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(rawConfig); assertNotNull("filesystem", fs); if (this.getAuthType() == AuthType.OAuth) { - Abfss afs = (Abfss) FileContext.getFileContext(getConfiguration()).getDefaultFileSystem(); + Abfss afs = (Abfss) FileContext.getFileContext(rawConfig).getDefaultFileSystem(); assertNotNull("filecontext", afs); } else { - Abfs afs = (Abfs) FileContext.getFileContext(getConfiguration()).getDefaultFileSystem(); + Abfs afs = (Abfs) FileContext.getFileContext(rawConfig).getDefaultFileSystem(); assertNotNull("filecontext", afs); } @@ -100,13 +101,13 @@ public class ITestFileSystemRegistration extends AbstractAbfsIntegrationTest { null, null, null); - getConfiguration().set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, + Configuration rawConfig = getRawConfiguration(); + rawConfig.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); - SecureAzureBlobFileSystem fs = (SecureAzureBlobFileSystem) FileSystem.get( - getConfiguration()); + SecureAzureBlobFileSystem fs = (SecureAzureBlobFileSystem) FileSystem.get(rawConfig); assertNotNull("filesystem", fs); - Abfss afs = (Abfss) FileContext.getFileContext(getConfiguration()).getDefaultFileSystem(); + Abfss afs = (Abfss) FileContext.getFileContext(rawConfig).getDefaultFileSystem(); assertNotNull("filecontext", afs); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java index bedd4f36363..50f7bd98bf3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.azurebfs; +import java.io.IOException; import java.lang.reflect.Field; import org.apache.commons.codec.Charsets; @@ -66,6 +67,7 @@ public class TestAbfsConfigurationFieldsValidation { private static final int TEST_INT = 1234565; private static final int TEST_LONG = 4194304; + private final String accountName; private final String encodedString; private final String encodedAccountKey; @@ -96,6 +98,7 @@ public class TestAbfsConfigurationFieldsValidation { public TestAbfsConfigurationFieldsValidation() throws Exception { super(); Base64 base64 = new Base64(); + this.accountName = "testaccount1.blob.core.windows.net"; this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8); this.encodedAccountKey = new String(base64.encode("someAccountKey".getBytes(Charsets.UTF_8)), Charsets.UTF_8); Configuration configuration = new Configuration(); @@ -105,8 +108,8 @@ public class TestAbfsConfigurationFieldsValidation { configuration.set(STRING_KEY, "stringValue"); configuration.set(BASE64_KEY, encodedString); configuration.set(BOOLEAN_KEY, "true"); - configuration.set(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + "testaccount1.blob.core.windows.net", this.encodedAccountKey); - abfsConfiguration = new AbfsConfiguration(configuration); + configuration.set(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + "." + accountName, this.encodedAccountKey); + abfsConfiguration = new AbfsConfiguration(configuration, accountName); } @Test @@ -143,30 +146,35 @@ public class TestAbfsConfigurationFieldsValidation { @Test public void testGetAccountKey() throws Exception { - String accountKey = abfsConfiguration.getStorageAccountKey("testaccount1.blob.core.windows.net"); + String accountKey = abfsConfiguration.getStorageAccountKey(); assertEquals(this.encodedAccountKey, accountKey); } @Test(expected = ConfigurationPropertyNotFoundException.class) public void testGetAccountKeyWithNonExistingAccountName() throws Exception { - abfsConfiguration.getStorageAccountKey("bogusAccountName"); + Configuration configuration = new Configuration(); + configuration.addResource(TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME); + configuration.unset(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME); + AbfsConfiguration abfsConfig = new AbfsConfiguration(configuration, "bogusAccountName"); + abfsConfig.getStorageAccountKey(); } @Test - public void testSSLSocketFactoryConfiguration() throws InvalidConfigurationValueException, IllegalAccessException { + public void testSSLSocketFactoryConfiguration() + throws InvalidConfigurationValueException, IllegalAccessException, IOException { assertEquals(SSLSocketFactoryEx.SSLChannelMode.Default, abfsConfiguration.getPreferredSSLFactoryOption()); assertNotEquals(SSLSocketFactoryEx.SSLChannelMode.Default_JSSE, abfsConfiguration.getPreferredSSLFactoryOption()); assertNotEquals(SSLSocketFactoryEx.SSLChannelMode.OpenSSL, abfsConfiguration.getPreferredSSLFactoryOption()); Configuration configuration = new Configuration(); configuration.setEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, SSLSocketFactoryEx.SSLChannelMode.Default_JSSE); - AbfsConfiguration localAbfsConfiguration = new AbfsConfiguration(configuration); + AbfsConfiguration localAbfsConfiguration = new AbfsConfiguration(configuration, accountName); assertEquals(SSLSocketFactoryEx.SSLChannelMode.Default_JSSE, localAbfsConfiguration.getPreferredSSLFactoryOption()); configuration = new Configuration(); configuration.setEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, SSLSocketFactoryEx.SSLChannelMode.OpenSSL); - localAbfsConfiguration = new AbfsConfiguration(configuration); + localAbfsConfiguration = new AbfsConfiguration(configuration, accountName); assertEquals(SSLSocketFactoryEx.SSLChannelMode.OpenSSL, localAbfsConfiguration.getPreferredSSLFactoryOption()); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java new file mode 100644 index 00000000000..425485cb3ac --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java @@ -0,0 +1,273 @@ +/** + * 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 org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import org.junit.Test; + +/** + * Tests correct precedence of various configurations that might be returned. + * Configuration can be specified with the account name as a suffix to the + * config key, or without one. Account-specific values should be returned + * whenever they exist. Account-agnostic values are returned if they do not. + * Default values are returned if neither exists. + * + * These tests are in 2 main groups: tests of methods that allow default values + * (such as get and getPasswordString) are of one form, while tests of methods + * that do allow default values (all others) follow another form. + */ +public class TestAccountConfiguration { + + @Test + public void testStringPrecedence() + throws IllegalAccessException, IOException, InvalidConfigurationValueException { + AbfsConfiguration abfsConf; + final Configuration conf = new Configuration(); + + final String accountName1 = "account1"; + final String accountName2 = "account2"; + final String accountName3 = "account3"; + + final String globalKey = "fs.azure.configuration"; + final String accountKey1 = globalKey + "." + accountName1; + final String accountKey2 = globalKey + "." + accountName2; + final String accountKey3 = globalKey + "." + accountName3; + + final String globalValue = "global"; + final String accountValue1 = "one"; + final String accountValue2 = "two"; + + conf.set(accountKey1, accountValue1); + conf.set(accountKey2, accountValue2); + conf.set(globalKey, globalValue); + + abfsConf = new AbfsConfiguration(conf, accountName1); + assertEquals("Wrong value returned when account-specific value was requested", + abfsConf.get(accountKey1), accountValue1); + assertEquals("Account-specific value was not returned when one existed", + abfsConf.get(globalKey), accountValue1); + + abfsConf = new AbfsConfiguration(conf, accountName2); + assertEquals("Wrong value returned when a different account-specific value was requested", + abfsConf.get(accountKey1), accountValue1); + assertEquals("Wrong value returned when account-specific value was requested", + abfsConf.get(accountKey2), accountValue2); + assertEquals("Account-agnostic value return even though account-specific value was set", + abfsConf.get(globalKey), accountValue2); + + abfsConf = new AbfsConfiguration(conf, accountName3); + assertNull("Account-specific value returned when none was set", + abfsConf.get(accountKey3)); + assertEquals("Account-agnostic value not returned when no account-specific value was set", + abfsConf.get(globalKey), globalValue); + } + + @Test + public void testPasswordPrecedence() + throws IllegalAccessException, IOException, InvalidConfigurationValueException { + AbfsConfiguration abfsConf; + final Configuration conf = new Configuration(); + + final String accountName1 = "account1"; + final String accountName2 = "account2"; + final String accountName3 = "account3"; + + final String globalKey = "fs.azure.password"; + final String accountKey1 = globalKey + "." + accountName1; + final String accountKey2 = globalKey + "." + accountName2; + final String accountKey3 = globalKey + "." + accountName3; + + final String globalValue = "global"; + final String accountValue1 = "one"; + final String accountValue2 = "two"; + + conf.set(accountKey1, accountValue1); + conf.set(accountKey2, accountValue2); + conf.set(globalKey, globalValue); + + abfsConf = new AbfsConfiguration(conf, accountName1); + assertEquals("Wrong value returned when account-specific value was requested", + abfsConf.getPasswordString(accountKey1), accountValue1); + assertEquals("Account-specific value was not returned when one existed", + abfsConf.getPasswordString(globalKey), accountValue1); + + abfsConf = new AbfsConfiguration(conf, accountName2); + assertEquals("Wrong value returned when a different account-specific value was requested", + abfsConf.getPasswordString(accountKey1), accountValue1); + assertEquals("Wrong value returned when account-specific value was requested", + abfsConf.getPasswordString(accountKey2), accountValue2); + assertEquals("Account-agnostic value return even though account-specific value was set", + abfsConf.getPasswordString(globalKey), accountValue2); + + abfsConf = new AbfsConfiguration(conf, accountName3); + assertNull("Account-specific value returned when none was set", + abfsConf.getPasswordString(accountKey3)); + assertEquals("Account-agnostic value not returned when no account-specific value was set", + abfsConf.getPasswordString(globalKey), globalValue); + } + + @Test + public void testBooleanPrecedence() + throws IllegalAccessException, IOException, InvalidConfigurationValueException { + + final String accountName = "account"; + final String globalKey = "fs.azure.bool"; + final String accountKey = globalKey + "." + accountName; + + final Configuration conf = new Configuration(); + final AbfsConfiguration abfsConf = new AbfsConfiguration(conf, accountName); + + conf.setBoolean(globalKey, false); + assertEquals("Default value returned even though account-agnostic config was set", + abfsConf.getBoolean(globalKey, true), false); + conf.unset(globalKey); + assertEquals("Default value not returned even though config was unset", + abfsConf.getBoolean(globalKey, true), true); + + conf.setBoolean(accountKey, false); + assertEquals("Default value returned even though account-specific config was set", + abfsConf.getBoolean(globalKey, true), false); + conf.unset(accountKey); + assertEquals("Default value not returned even though config was unset", + abfsConf.getBoolean(globalKey, true), true); + + conf.setBoolean(accountKey, true); + conf.setBoolean(globalKey, false); + assertEquals("Account-agnostic or default value returned even though account-specific config was set", + abfsConf.getBoolean(globalKey, false), true); + } + + @Test + public void testLongPrecedence() + throws IllegalAccessException, IOException, InvalidConfigurationValueException { + + final String accountName = "account"; + final String globalKey = "fs.azure.long"; + final String accountKey = globalKey + "." + accountName; + + final Configuration conf = new Configuration(); + final AbfsConfiguration abfsConf = new AbfsConfiguration(conf, accountName); + + conf.setLong(globalKey, 0); + assertEquals("Default value returned even though account-agnostic config was set", + abfsConf.getLong(globalKey, 1), 0); + conf.unset(globalKey); + assertEquals("Default value not returned even though config was unset", + abfsConf.getLong(globalKey, 1), 1); + + conf.setLong(accountKey, 0); + assertEquals("Default value returned even though account-specific config was set", + abfsConf.getLong(globalKey, 1), 0); + conf.unset(accountKey); + assertEquals("Default value not returned even though config was unset", + abfsConf.getLong(globalKey, 1), 1); + + conf.setLong(accountKey, 1); + conf.setLong(globalKey, 0); + assertEquals("Account-agnostic or default value returned even though account-specific config was set", + abfsConf.getLong(globalKey, 0), 1); + } + + public enum GetEnumType { + TRUE, FALSE + } + + @Test + public void testEnumPrecedence() + throws IllegalAccessException, IOException, InvalidConfigurationValueException { + + final String accountName = "account"; + final String globalKey = "fs.azure.enum"; + final String accountKey = globalKey + "." + accountName; + + final Configuration conf = new Configuration(); + final AbfsConfiguration abfsConf = new AbfsConfiguration(conf, accountName); + + conf.setEnum(globalKey, GetEnumType.FALSE); + assertEquals("Default value returned even though account-agnostic config was set", + abfsConf.getEnum(globalKey, GetEnumType.TRUE), GetEnumType.FALSE); + conf.unset(globalKey); + assertEquals("Default value not returned even though config was unset", + abfsConf.getEnum(globalKey, GetEnumType.TRUE), GetEnumType.TRUE); + + conf.setEnum(accountKey, GetEnumType.FALSE); + assertEquals("Default value returned even though account-specific config was set", + abfsConf.getEnum(globalKey, GetEnumType.TRUE), GetEnumType.FALSE); + conf.unset(accountKey); + assertEquals("Default value not returned even though config was unset", + abfsConf.getEnum(globalKey, GetEnumType.TRUE), GetEnumType.TRUE); + + conf.setEnum(accountKey, GetEnumType.TRUE); + conf.setEnum(globalKey, GetEnumType.FALSE); + assertEquals("Account-agnostic or default value returned even though account-specific config was set", + abfsConf.getEnum(globalKey, GetEnumType.FALSE), GetEnumType.TRUE); + } + + interface GetClassInterface { + } + + private class GetClassImpl0 implements GetClassInterface { + } + + private class GetClassImpl1 implements GetClassInterface { + } + + @Test + public void testClassPrecedence() + throws IllegalAccessException, IOException, InvalidConfigurationValueException { + + final String accountName = "account"; + final String globalKey = "fs.azure.class"; + final String accountKey = globalKey + "." + accountName; + + final Configuration conf = new Configuration(); + final AbfsConfiguration abfsConf = new AbfsConfiguration(conf, accountName); + + final Class class0 = GetClassImpl0.class; + final Class class1 = GetClassImpl1.class; + final Class xface = GetClassInterface.class; + + conf.setClass(globalKey, class0, xface); + assertEquals("Default value returned even though account-agnostic config was set", + abfsConf.getClass(globalKey, class1, xface), class0); + conf.unset(globalKey); + assertEquals("Default value not returned even though config was unset", + abfsConf.getClass(globalKey, class1, xface), class1); + + conf.setClass(accountKey, class0, xface); + assertEquals("Default value returned even though account-specific config was set", + abfsConf.getClass(globalKey, class1, xface), class0); + conf.unset(accountKey); + assertEquals("Default value not returned even though config was unset", + abfsConf.getClass(globalKey, class1, xface), class1); + + conf.setClass(accountKey, class1, xface); + conf.setClass(globalKey, class0, xface); + assertEquals("Account-agnostic or default value returned even though account-specific config was set", + abfsConf.getClass(globalKey, class0, xface), class1); + } + +} 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 7852f25b7ed..49a0e2ee761 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,7 +24,7 @@ package org.apache.hadoop.fs.azurebfs.constants; 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_ACCOUNT_KEY_PREFIX = "fs.azure.account.key."; + 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_BLOB_DATA_CONTRIBUTOR_CLIENT_ID = "fs.azure.account.oauth2.contributor.client.id"; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java index 6d089d14e01..79e295a99fd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java @@ -20,8 +20,8 @@ package org.apache.hadoop.fs.azurebfs.contract; import java.net.URI; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes; import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys; @@ -41,7 +41,7 @@ public class ABFSContractTestBinding extends AbstractAbfsIntegrationTest { public ABFSContractTestBinding( final boolean useExistingFileSystem) throws Exception{ if (useExistingFileSystem) { - Configuration configuration = getConfiguration(); + AbfsConfiguration configuration = getConfiguration(); String testUrl = configuration.get(TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI); Assume.assumeTrue("Contract tests are skipped because of missing config property :" + TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI, testUrl != null); @@ -61,10 +61,6 @@ public class ABFSContractTestBinding extends AbstractAbfsIntegrationTest { } } - public Configuration getConfiguration() { - return super.getConfiguration(); - } - public boolean isSecureMode() { return this.getAuthType() == AuthType.SharedKey ? false : true; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java index 8a955bc6062..59df4f0deb8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java @@ -45,7 +45,7 @@ public class ITestAbfsFileSystemContractAppend extends AbstractContractAppendTes @Override protected Configuration createConfiguration() { - return binding.getConfiguration(); + return binding.getRawConfiguration(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java index 383528b75a8..c67e2bc5144 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java @@ -41,7 +41,7 @@ public class ITestAbfsFileSystemContractConcat extends AbstractContractConcatTes @Override protected Configuration createConfiguration() { - return binding.getConfiguration(); + return binding.getRawConfiguration(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java index 3c3e9490365..11d01641ead 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java @@ -42,7 +42,7 @@ public class ITestAbfsFileSystemContractCreate extends AbstractContractCreateTes @Override protected Configuration createConfiguration() { - return binding.getConfiguration(); + return binding.getRawConfiguration(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java index 1d1136c3538..9d77829c6fb 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java @@ -42,7 +42,7 @@ public class ITestAbfsFileSystemContractDelete extends AbstractContractDeleteTes @Override protected Configuration createConfiguration() { - return binding.getConfiguration(); + return binding.getRawConfiguration(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java index 08b7eefe198..f64abf9cb37 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java @@ -41,7 +41,7 @@ public class ITestAbfsFileSystemContractGetFileStatus extends AbstractContractGe @Override protected Configuration createConfiguration() { - return this.binding.getConfiguration(); + return this.binding.getRawConfiguration(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java index 7b785753f2c..99959ed2d02 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java @@ -42,7 +42,7 @@ public class ITestAbfsFileSystemContractMkdir extends AbstractContractMkdirTest @Override protected Configuration createConfiguration() { - return binding.getConfiguration(); + return binding.getRawConfiguration(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java index 41f691d512f..43552e50b7a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java @@ -42,7 +42,7 @@ public class ITestAbfsFileSystemContractOpen extends AbstractContractOpenTest { @Override protected Configuration createConfiguration() { - return binding.getConfiguration(); + return binding.getRawConfiguration(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java index 82f104a44bf..b92bef68a09 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java @@ -42,7 +42,7 @@ public class ITestAbfsFileSystemContractRename extends AbstractContractRenameTes @Override protected Configuration createConfiguration() { - return binding.getConfiguration(); + return binding.getRawConfiguration(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java index 5b5493fdcd0..5da2c76907e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java @@ -42,7 +42,7 @@ public class ITestAbfsFileSystemContractRootDirectory extends AbstractContractRo @Override protected Configuration createConfiguration() { - return binding.getConfiguration(); + return binding.getRawConfiguration(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java index 4529e752b01..35a5e1733d0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java @@ -42,7 +42,7 @@ public class ITestAbfsFileSystemContractSeek extends AbstractContractSeekTest{ @Override protected Configuration createConfiguration() { - return binding.getConfiguration(); + return binding.getRawConfiguration(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java index 6c4f9badc0f..40434842eb9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java @@ -41,7 +41,7 @@ public class ITestAbfsFileSystemContractSetTimes extends AbstractContractSetTime @Override protected Configuration createConfiguration() { - return binding.getConfiguration(); + return binding.getRawConfiguration(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index a2fdd095ac5..6a92bb2b4e3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -35,6 +35,8 @@ import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; */ public final class TestAbfsClient { + private final String accountName = "bogusAccountName"; + private void validateUserAgent(String expectedPattern, URL baseUrl, AbfsConfiguration config, @@ -55,7 +57,7 @@ public final class TestAbfsClient { String expectedUserAgentPattern = "Azure Blob FS\\/1.0 \\(JavaJRE ([^\\)]+)\\)"; final Configuration configuration = new Configuration(); configuration.unset(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY); - AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration); + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, accountName); validateUserAgent(expectedUserAgentPattern, new URL("http://azure.com"), abfsConfiguration, false); } @@ -65,7 +67,7 @@ public final class TestAbfsClient { String expectedUserAgentPattern = "Azure Blob FS\\/1.0 \\(JavaJRE ([^\\)]+)\\) Partner Service"; final Configuration configuration = new Configuration(); configuration.set(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, "Partner Service"); - AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration); + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, accountName); validateUserAgent(expectedUserAgentPattern, new URL("http://azure.com"), abfsConfiguration, false); } @@ -77,8 +79,8 @@ public final class TestAbfsClient { configuration.set(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, "Partner Service"); configuration.set(ConfigurationKeys.FS_AZURE_SSL_CHANNEL_MODE_KEY, SSLSocketFactoryEx.SSLChannelMode.Default_JSSE.name()); - AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration); + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, accountName); validateUserAgent(expectedUserAgentPattern, new URL("https://azure.com"), abfsConfiguration, true); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java index d17e767724c..5ef4eab7a00 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java @@ -54,6 +54,7 @@ public class TestShellDecryptionKeyProvider { String key = "key"; conf.set(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME + account, key); + try { provider.getStorageAccountKey(account, conf); Assert diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AbfsTestUtils.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AbfsTestUtils.java index 9c636a2ce4f..7e6336d1ae5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AbfsTestUtils.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AbfsTestUtils.java @@ -37,6 +37,11 @@ import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST public final class AbfsTestUtils extends AbstractAbfsIntegrationTest{ private static final Logger LOG = LoggerFactory.getLogger(AbfsTestUtils.class); + + public AbfsTestUtils() throws Exception { + super(); + } + /** * If unit tests were interrupted and crushed accidentally, the test containers won't be deleted. * In that case, dev can use this tool to list and delete all test containers. From 26211019c80e6180297dd94abcefe718b70e8cd9 Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Fri, 14 Sep 2018 16:50:26 +0000 Subject: [PATCH 20/29] HADOOP-15753. ABFS: support path "abfs://mycluster/file/path" Contributed by Da Zhou. --- .../fs/azurebfs/AzureBlobFileSystem.java | 23 ++++++++++++++++++ .../ITestAzureBlobFileSystemFileStatus.java | 24 +++++++++++++++++++ 2 files changed, 47 insertions(+) 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 7cbf4d7d3f7..2e8de782888 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 @@ -366,6 +366,29 @@ public class AzureBlobFileSystem extends FileSystem { } } + /** + * Qualify a path to one which uses this FileSystem and, if relative, + * made absolute. + * @param path to qualify. + * @return this path if it contains a scheme and authority and is absolute, or + * a new path that includes a path and authority and is fully qualified + * @see Path#makeQualified(URI, Path) + * @throws IllegalArgumentException if the path has a schema/URI different + * from this FileSystem. + */ + @Override + public Path makeQualified(Path path) { + // To support format: abfs://{dfs.nameservices}/file/path, + // path need to be first converted to URI, then get the raw path string, + // during which {dfs.nameservices} will be omitted. + if (path != null ) { + String uriPath = path.toUri().getPath(); + path = uriPath.isEmpty() ? path : new Path(uriPath); + } + return super.makeQualified(path); + } + + @Override public Path getWorkingDirectory() { return this.workingDir; 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 b08b9202dd2..02f938f19f4 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 @@ -98,4 +98,28 @@ public class ITestAzureBlobFileSystemFileStatus extends validateStatus(fs, TEST_FOLDER, true); } + @Test + public void testAbfsPathWithHost() throws IOException { + AzureBlobFileSystem fs = this.getFileSystem(); + Path pathWithHost1 = new Path("abfs://mycluster/abfs/file1.txt"); + Path pathwithouthost1 = new Path("/abfs/file1.txt"); + + Path pathWithHost2 = new Path("abfs://mycluster/abfs/file2.txt"); + Path pathwithouthost2 = new Path("/abfs/file2.txt"); + + // verify compatibility of this path format + fs.create(pathWithHost1); + assertTrue(fs.exists(pathwithouthost1)); + + fs.create(pathwithouthost2); + assertTrue(fs.exists(pathWithHost2)); + + // verify get + FileStatus fileStatus1 = fs.getFileStatus(pathWithHost1); + assertEquals(pathwithouthost1.getName(), fileStatus1.getPath().getName()); + + FileStatus fileStatus2 = fs.getFileStatus(pathwithouthost2); + assertEquals(pathWithHost2.getName(), fileStatus2.getPath().getName()); + } + } From b4c23043d38325aad8af4488caca115eae789f07 Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Fri, 14 Sep 2018 22:34:19 +0000 Subject: [PATCH 21/29] HADOOP-15757. ABFS: remove dependency on common-codec Base64. Contributed by Da Zhou. --- .../src/config/checkstyle-suppressions.xml | 2 + .../fs/azurebfs/AzureBlobFileSystemStore.java | 6 +- ...se64StringConfigurationBasicValidator.java | 4 +- .../services/SharedKeyCredentials.java | 9 +- .../hadoop/fs/azurebfs/utils/Base64.java | 329 ++++++++++++++++++ ...TestAbfsConfigurationFieldsValidation.java | 7 +- .../TestConfigurationValidators.java | 7 +- 7 files changed, 344 insertions(+), 20 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Base64.java diff --git a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml index 751a22773d7..10cf77e0c2b 100644 --- a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml +++ b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml @@ -44,4 +44,6 @@ + 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 e3bea3251d0..d16cf3622bb 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 @@ -41,7 +41,6 @@ import java.util.Hashtable; import java.util.List; import java.util.Map; import java.util.Set; -import javax.xml.bind.DatatypeConverter; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; @@ -77,6 +76,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy; import org.apache.hadoop.fs.azurebfs.services.SharedKeyCredentials; +import org.apache.hadoop.fs.azurebfs.utils.Base64; import org.apache.hadoop.fs.azurebfs.utils.UriUtils; import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclStatus; @@ -865,7 +865,7 @@ public class AzureBlobFileSystemStore { throw new CharacterCodingException(); } - String encodedPropertyValue = DatatypeConverter.printBase64Binary(encoder.encode(CharBuffer.wrap(value)).array()); + String encodedPropertyValue = Base64.encode(encoder.encode(CharBuffer.wrap(value)).array()); commaSeparatedProperties.append(key) .append(AbfsHttpConstants.EQUAL) .append(encodedPropertyValue); @@ -903,7 +903,7 @@ public class AzureBlobFileSystemStore { throw new InvalidFileSystemPropertyException(xMsProperties); } - byte[] decodedValue = DatatypeConverter.parseBase64Binary(nameValue[1]); + byte[] decodedValue = Base64.decode(nameValue[1]); final String value; try { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java index 6bb997bbc79..fc7d713cb41 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java @@ -23,7 +23,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; -import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.fs.azurebfs.utils.Base64; /** * String Base64 configuration value Validator. @@ -42,7 +42,7 @@ public class Base64StringConfigurationBasicValidator extends ConfigurationBasicV return result; } - if (!Base64.isBase64(configValue)) { + if (!Base64.validateIsBase64String(configValue)) { throw new InvalidConfigurationValueException(getConfigKey()); } return configValue; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java index 105a1a2dbf6..9ab9e504506 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java @@ -39,10 +39,9 @@ import java.util.TimeZone; import java.util.regex.Matcher; import java.util.regex.Pattern; -import org.apache.commons.codec.Charsets; -import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.utils.Base64; /** * Represents the shared key credentials used to access an Azure Storage @@ -52,8 +51,6 @@ public class SharedKeyCredentials { private static final int EXPECTED_BLOB_QUEUE_CANONICALIZED_STRING_LENGTH = 300; private static final Pattern CRLF = Pattern.compile("\r\n", Pattern.LITERAL); private static final String HMAC_SHA256 = "HmacSHA256"; - private static final Base64 BASE_64 = new Base64(); - /** * Stores a reference to the RFC1123 date/time pattern. */ @@ -73,7 +70,7 @@ public class SharedKeyCredentials { throw new IllegalArgumentException("Invalid account key."); } this.accountName = accountName; - this.accountKey = BASE_64.decode(accountKey); + this.accountKey = Base64.decode(accountKey); initializeMac(); } @@ -100,7 +97,7 @@ public class SharedKeyCredentials { synchronized (this) { hmac = hmacSha256.doFinal(utf8Bytes); } - return new String(BASE_64.encode(hmac), Charsets.UTF_8); + return Base64.encode(hmac); } /** diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Base64.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Base64.java new file mode 100644 index 00000000000..c1910060420 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Base64.java @@ -0,0 +1,329 @@ +/** + * 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; + +/** + * Base64 + */ +public final class Base64 { + /** + * The Base 64 Characters. + */ + private static final String BASE_64_CHARS = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/"; + + /** + * Decoded values, -1 is invalid character, -2 is = pad character. + */ + private static final byte[] DECODE_64 = { + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, // 0-15 + + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, /* + * 16- 31 + */ + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 62, -1, -1, -1, 63, /* + * 32- 47 + */ + 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, -1, -1, -1, -2, -1, -1, /* + * 48- 63 + */ + -1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, /* 64-79 */ + 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, -1, -1, -1, -1, -1, /* + * 80- 95 + */ + -1, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, /* + * 96- 111 + */ + 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, -1, -1, -1, -1, -1 /* + * 112- 127 + */ + }; + + /** + * Decodes a given Base64 string into its corresponding byte array. + * + * @param data + * the Base64 string, as a String object, to decode + * + * @return the corresponding decoded byte array + * @throws IllegalArgumentException + * If the string is not a valid base64 encoded string + */ + public static byte[] decode(final String data) { + if (data == null) { + throw new IllegalArgumentException("The data parameter is not a valid base64-encoded string."); + } + + int byteArrayLength = 3 * data.length() / 4; + + if (data.endsWith("==")) { + byteArrayLength -= 2; + } + else if (data.endsWith("=")) { + byteArrayLength -= 1; + } + + final byte[] retArray = new byte[byteArrayLength]; + int byteDex = 0; + int charDex = 0; + + for (; charDex < data.length(); charDex += 4) { + // get 4 chars, convert to 3 bytes + final int char1 = DECODE_64[(byte) data.charAt(charDex)]; + final int char2 = DECODE_64[(byte) data.charAt(charDex + 1)]; + final int char3 = DECODE_64[(byte) data.charAt(charDex + 2)]; + final int char4 = DECODE_64[(byte) data.charAt(charDex + 3)]; + + if (char1 < 0 || char2 < 0 || char3 == -1 || char4 == -1) { + // invalid character(-1), or bad padding (-2) + throw new IllegalArgumentException("The data parameter is not a valid base64-encoded string."); + } + + int tVal = char1 << 18; + tVal += char2 << 12; + tVal += (char3 & 0xff) << 6; + tVal += char4 & 0xff; + + if (char3 == -2) { + // two "==" pad chars, check bits 12-24 + tVal &= 0x00FFF000; + retArray[byteDex++] = (byte) (tVal >> 16 & 0xFF); + } + else if (char4 == -2) { + // one pad char "=" , check bits 6-24. + tVal &= 0x00FFFFC0; + retArray[byteDex++] = (byte) (tVal >> 16 & 0xFF); + retArray[byteDex++] = (byte) (tVal >> 8 & 0xFF); + + } + else { + // No pads take all 3 bytes, bits 0-24 + retArray[byteDex++] = (byte) (tVal >> 16 & 0xFF); + retArray[byteDex++] = (byte) (tVal >> 8 & 0xFF); + retArray[byteDex++] = (byte) (tVal & 0xFF); + } + } + return retArray; + } + + /** + * Decodes a given Base64 string into its corresponding byte array. + * + * @param data + * the Base64 string, as a String object, to decode + * + * @return the corresponding decoded byte array + * @throws IllegalArgumentException + * If the string is not a valid base64 encoded string + */ + public static Byte[] decodeAsByteObjectArray(final String data) { + int byteArrayLength = 3 * data.length() / 4; + + if (data.endsWith("==")) { + byteArrayLength -= 2; + } + else if (data.endsWith("=")) { + byteArrayLength -= 1; + } + + final Byte[] retArray = new Byte[byteArrayLength]; + int byteDex = 0; + int charDex = 0; + + for (; charDex < data.length(); charDex += 4) { + // get 4 chars, convert to 3 bytes + final int char1 = DECODE_64[(byte) data.charAt(charDex)]; + final int char2 = DECODE_64[(byte) data.charAt(charDex + 1)]; + final int char3 = DECODE_64[(byte) data.charAt(charDex + 2)]; + final int char4 = DECODE_64[(byte) data.charAt(charDex + 3)]; + + if (char1 < 0 || char2 < 0 || char3 == -1 || char4 == -1) { + // invalid character(-1), or bad padding (-2) + throw new IllegalArgumentException("The data parameter is not a valid base64-encoded string."); + } + + int tVal = char1 << 18; + tVal += char2 << 12; + tVal += (char3 & 0xff) << 6; + tVal += char4 & 0xff; + + if (char3 == -2) { + // two "==" pad chars, check bits 12-24 + tVal &= 0x00FFF000; + retArray[byteDex++] = (byte) (tVal >> 16 & 0xFF); + } + else if (char4 == -2) { + // one pad char "=" , check bits 6-24. + tVal &= 0x00FFFFC0; + retArray[byteDex++] = (byte) (tVal >> 16 & 0xFF); + retArray[byteDex++] = (byte) (tVal >> 8 & 0xFF); + + } + else { + // No pads take all 3 bytes, bits 0-24 + retArray[byteDex++] = (byte) (tVal >> 16 & 0xFF); + retArray[byteDex++] = (byte) (tVal >> 8 & 0xFF); + retArray[byteDex++] = (byte) (tVal & 0xFF); + } + } + return retArray; + } + + /** + * Encodes a byte array as a Base64 string. + * + * @param data + * the byte array to encode + * @return the Base64-encoded string, as a String object + */ + public static String encode(final byte[] data) { + final StringBuilder builder = new StringBuilder(); + final int dataRemainder = data.length % 3; + + int j = 0; + int n = 0; + for (; j < data.length; j += 3) { + + if (j < data.length - dataRemainder) { + n = ((data[j] & 0xFF) << 16) + ((data[j + 1] & 0xFF) << 8) + (data[j + 2] & 0xFF); + } + else { + if (dataRemainder == 1) { + n = (data[j] & 0xFF) << 16; + } + else if (dataRemainder == 2) { + n = ((data[j] & 0xFF) << 16) + ((data[j + 1] & 0xFF) << 8); + } + } + + // Left here for readability + // byte char1 = (byte) ((n >>> 18) & 0x3F); + // byte char2 = (byte) ((n >>> 12) & 0x3F); + // byte char3 = (byte) ((n >>> 6) & 0x3F); + // byte char4 = (byte) (n & 0x3F); + builder.append(BASE_64_CHARS.charAt((byte) ((n >>> 18) & 0x3F))); + builder.append(BASE_64_CHARS.charAt((byte) ((n >>> 12) & 0x3F))); + builder.append(BASE_64_CHARS.charAt((byte) ((n >>> 6) & 0x3F))); + builder.append(BASE_64_CHARS.charAt((byte) (n & 0x3F))); + } + + final int bLength = builder.length(); + + // append '=' to pad + if (data.length % 3 == 1) { + builder.replace(bLength - 2, bLength, "=="); + } + else if (data.length % 3 == 2) { + builder.replace(bLength - 1, bLength, "="); + } + + return builder.toString(); + } + + /** + * Encodes a byte array as a Base64 string. + * + * @param data + * the byte array to encode + * @return the Base64-encoded string, as a String object + */ + public static String encode(final Byte[] data) { + final StringBuilder builder = new StringBuilder(); + final int dataRemainder = data.length % 3; + + int j = 0; + int n = 0; + for (; j < data.length; j += 3) { + + if (j < data.length - dataRemainder) { + n = ((data[j] & 0xFF) << 16) + ((data[j + 1] & 0xFF) << 8) + (data[j + 2] & 0xFF); + } + else { + if (dataRemainder == 1) { + n = (data[j] & 0xFF) << 16; + } + else if (dataRemainder == 2) { + n = ((data[j] & 0xFF) << 16) + ((data[j + 1] & 0xFF) << 8); + } + } + + // Left here for readability + // byte char1 = (byte) ((n >>> 18) & 0x3F); + // byte char2 = (byte) ((n >>> 12) & 0x3F); + // byte char3 = (byte) ((n >>> 6) & 0x3F); + // byte char4 = (byte) (n & 0x3F); + builder.append(BASE_64_CHARS.charAt((byte) ((n >>> 18) & 0x3F))); + builder.append(BASE_64_CHARS.charAt((byte) ((n >>> 12) & 0x3F))); + builder.append(BASE_64_CHARS.charAt((byte) ((n >>> 6) & 0x3F))); + builder.append(BASE_64_CHARS.charAt((byte) (n & 0x3F))); + } + + final int bLength = builder.length(); + + // append '=' to pad + if (data.length % 3 == 1) { + builder.replace(bLength - 2, bLength, "=="); + } + else if (data.length % 3 == 2) { + builder.replace(bLength - 1, bLength, "="); + } + + return builder.toString(); + } + + /** + * Determines whether the given string contains only Base64 characters. + * + * @param data + * the string, as a String object, to validate + * @return true if data is a valid Base64 string, otherwise false + */ + public static boolean validateIsBase64String(final String data) { + + if (data == null || data.length() % 4 != 0) { + return false; + } + + for (int m = 0; m < data.length(); m++) { + final byte charByte = (byte) data.charAt(m); + + // pad char detected + if (DECODE_64[charByte] == -2) { + if (m < data.length() - 2) { + return false; + } + else if (m == data.length() - 2 && DECODE_64[(byte) data.charAt(m + 1)] != -2) { + return false; + } + } + + if (charByte < 0 || DECODE_64[charByte] == -1) { + return false; + } + } + + return true; + } + + /** + * Private Default Ctor. + */ + private Base64() { + // No op + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java index 50f7bd98bf3..eeed6cec872 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAbfsConfigurationFieldsValidation.java @@ -31,6 +31,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidati import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation; import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException; +import org.apache.hadoop.fs.azurebfs.utils.Base64; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SSL_CHANNEL_MODE_KEY; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE; @@ -42,7 +43,6 @@ import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.D import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_AZURE_BLOCK_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.AZURE_BLOCK_LOCATION_HOST_DEFAULT; -import org.apache.commons.codec.binary.Base64; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; @@ -97,10 +97,9 @@ public class TestAbfsConfigurationFieldsValidation { public TestAbfsConfigurationFieldsValidation() throws Exception { super(); - Base64 base64 = new Base64(); this.accountName = "testaccount1.blob.core.windows.net"; - this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8); - this.encodedAccountKey = new String(base64.encode("someAccountKey".getBytes(Charsets.UTF_8)), Charsets.UTF_8); + this.encodedString = Base64.encode("base64Value".getBytes(Charsets.UTF_8)); + this.encodedAccountKey = Base64.encode("someAccountKey".getBytes(Charsets.UTF_8)); Configuration configuration = new Configuration(); configuration.addResource(TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME); configuration.set(INT_KEY, "1234565"); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java index a9556ab1c0f..f02eadc9a04 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java @@ -18,14 +18,11 @@ package org.apache.hadoop.fs.azurebfs.diagnostics; -import org.apache.commons.codec.Charsets; - import org.junit.Assert; import org.junit.Test; - import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; -import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.fs.azurebfs.utils.Base64; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_BUFFER_SIZE; @@ -109,7 +106,7 @@ public class TestConfigurationValidators extends Assert { @Test public void testBase64StringConfigValidator() throws Exception { - String encodedVal = new String(new Base64().encode("someValue".getBytes()), Charsets.UTF_8); + String encodedVal = Base64.encode("someValue".getBytes()); Base64StringConfigurationBasicValidator base64StringConfigurationValidator = new Base64StringConfigurationBasicValidator(FAKE_KEY, "", false); assertEquals("", base64StringConfigurationValidator.validate(null)); From 1cf38a38da060ed06377099057a31f17d570c3c1 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 18 Sep 2018 10:56:56 +0100 Subject: [PATCH 22/29] HADOOP-15744. AbstractContractAppendTest fails against HDFS on HADOOP-15407 branch. Contributed by Steve Loughran. --- .../apache/hadoop/fs/RawLocalFileSystem.java | 2 +- .../contract/AbstractContractAppendTest.java | 57 ++++++------------- .../ITestAzureNativeContractAppend.java | 23 -------- 3 files changed, 19 insertions(+), 63 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java index 6e9d433a8e3..bd003ae90ab 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java @@ -291,7 +291,7 @@ public class RawLocalFileSystem extends FileSystem { Progressable progress) throws IOException { FileStatus status = getFileStatus(f); if (status.isDirectory()) { - throw new FileAlreadyExistsException("Cannot append to a directory: " + f); + throw new IOException("Cannot append to a diretory (=" + f + " )"); } return new FSDataOutputStream(new BufferedOutputStream( createOutputStreamWithMode(f, true, null), bufferSize), statistics, diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractAppendTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractAppendTest.java index 0be220e6511..d61b6354498 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractAppendTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractAppendTest.java @@ -18,12 +18,7 @@ package org.apache.hadoop.fs.contract; -import java.io.FileNotFoundException; -import java.io.IOException; - import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileAlreadyExistsException; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.junit.Test; import org.slf4j.Logger; @@ -32,7 +27,6 @@ import org.slf4j.LoggerFactory; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test append -if supported @@ -81,10 +75,15 @@ public abstract class AbstractContractAppendTest extends AbstractFSContractTestB @Test public void testAppendNonexistentFile() throws Throwable { - //expected - handleExpectedException( - intercept(Exception.class, - () -> getFileSystem().append(target).close())); + try { + FSDataOutputStream out = getFileSystem().append(target); + //got here: trouble + out.close(); + fail("expected a failure"); + } catch (Exception e) { + //expected + handleExpectedException(e); + } } @Test @@ -117,9 +116,15 @@ public abstract class AbstractContractAppendTest extends AbstractFSContractTestB @Test public void testAppendMissingTarget() throws Throwable { - handleExpectedException( - intercept(Exception.class, - () -> getFileSystem().append(target).close())); + try { + FSDataOutputStream out = getFileSystem().append(target); + //got here: trouble + out.close(); + fail("expected a failure"); + } catch (Exception e) { + //expected + handleExpectedException(e); + } } @Test @@ -144,30 +149,4 @@ public abstract class AbstractContractAppendTest extends AbstractFSContractTestB dataset.length); ContractTestUtils.compareByteArrays(dataset, bytes, dataset.length); } - - @Test - public void testAppendFileAfterDelete() throws Exception { - final FileSystem fs = getFileSystem(); - final Path filePath = target; - fs.create(filePath); - fs.delete(filePath, false); - intercept(FileNotFoundException.class, - () -> fs.append(filePath)); - } - - @Test - public void testAppendDirectory() throws Exception { - final FileSystem fs = getFileSystem(); - - final Path folderPath = target; - fs.mkdirs(folderPath); - IOException ex = intercept(IOException.class, - () -> fs.append(folderPath)); - if (ex instanceof FileAlreadyExistsException) { - handleExpectedException(ex); - } else { - handleRelaxedException("Append to a directory", - "FileAlreadyExistsException", ex); - } - } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractAppend.java index db4a843287a..fd21bd20b2e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractAppend.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractAppend.java @@ -18,19 +18,10 @@ package org.apache.hadoop.fs.azure.contract; -import java.io.FileNotFoundException; -import java.io.IOException; - -import org.junit.Test; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileAlreadyExistsException; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.AbstractContractAppendTest; import org.apache.hadoop.fs.contract.AbstractFSContract; import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Append test, skipping one of them. @@ -47,18 +38,4 @@ public class ITestAzureNativeContractAppend extends AbstractContractAppendTest { public void testRenameFileBeingAppended() throws Throwable { skip("Skipping as renaming an opened file is not supported"); } - - /** - * Wasb returns a different exception, so change the intercept logic here. - */ - @Override - @Test - public void testAppendDirectory() throws Exception { - final FileSystem fs = getFileSystem(); - - final Path folderPath = path("testAppendDirectory"); - fs.mkdirs(folderPath); - intercept(FileNotFoundException.class, - () -> fs.append(folderPath)); - } } From 524776625d14f6938d89edb923ed05683df819ff Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 18 Sep 2018 11:48:46 +0100 Subject: [PATCH 23/29] HADOOP-15715. ITestAzureBlobFileSystemE2E timing out with non-scale timeout of 10 min. Contributed by Da Zhou --- .../azurebfs/ITestAzureBlobFileSystemE2E.java | 44 +++++++++++-------- .../constants/TestConfigurationKeys.java | 2 +- 2 files changed, 26 insertions(+), 20 deletions(-) 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 6021e5d51be..2e994911acc 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 @@ -38,7 +38,6 @@ import static org.junit.Assert.assertArrayEquals; * Test end to end between ABFS client and ABFS server. */ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest { - private static final Path TEST_FILE = new Path("testfile"); private static final int TEST_BYTE = 100; private static final int TEST_OFFSET = 100; private static final int TEST_DEFAULT_BUFFER_SIZE = 4 * 1024 * 1024; @@ -52,21 +51,16 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest { @Test public void testWriteOneByteToFile() throws Exception { - final AzureBlobFileSystem fs = getFileSystem(); - - try(FSDataOutputStream stream = fs.create(TEST_FILE)) { - stream.write(TEST_BYTE); - } - - FileStatus fileStatus = fs.getFileStatus(TEST_FILE); - assertEquals(1, fileStatus.getLen()); + final Path testFilePath = new Path(methodName.getMethodName()); + testWriteOneByteToFile(testFilePath); } @Test public void testReadWriteBytesToFile() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); - testWriteOneByteToFile(); - try(FSDataInputStream inputStream = fs.open(TEST_FILE, + final Path testFilePath = new Path(methodName.getMethodName()); + testWriteOneByteToFile(testFilePath); + try(FSDataInputStream inputStream = fs.open(testFilePath, TEST_DEFAULT_BUFFER_SIZE)) { assertEquals(TEST_BYTE, inputStream.read()); } @@ -81,17 +75,17 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest { final byte[] b = new byte[2 * readBufferSize]; new Random().nextBytes(b); - - try(FSDataOutputStream writeStream = fs.create(TEST_FILE)) { + final Path testFilePath = new Path(methodName.getMethodName()); + try(FSDataOutputStream writeStream = fs.create(testFilePath)) { writeStream.write(b); writeStream.flush(); } - try (FSDataInputStream readStream = fs.open(TEST_FILE)) { + try (FSDataInputStream readStream = fs.open(testFilePath)) { assertEquals(readBufferSize, readStream.read(bytesToRead, 0, readBufferSize)); - try (FSDataOutputStream writeStream = fs.create(TEST_FILE)) { + try (FSDataOutputStream writeStream = fs.create(testFilePath)) { writeStream.write(b); writeStream.flush(); } @@ -104,15 +98,16 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest { @Test public void testWriteWithBufferOffset() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); + final Path testFilePath = new Path(methodName.getMethodName()); final byte[] b = new byte[1024 * 1000]; new Random().nextBytes(b); - try (FSDataOutputStream stream = fs.create(TEST_FILE)) { + try (FSDataOutputStream stream = fs.create(testFilePath)) { stream.write(b, TEST_OFFSET, b.length - TEST_OFFSET); } final byte[] r = new byte[TEST_DEFAULT_READ_BUFFER_SIZE]; - FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE); + FSDataInputStream inputStream = fs.open(testFilePath, TEST_DEFAULT_BUFFER_SIZE); int result = inputStream.read(r); assertNotEquals(-1, result); @@ -124,13 +119,14 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest { @Test public void testReadWriteHeavyBytesToFileWithSmallerChunks() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); + final Path testFilePath = new Path(methodName.getMethodName()); final byte[] writeBuffer = new byte[5 * 1000 * 1024]; new Random().nextBytes(writeBuffer); - write(TEST_FILE, writeBuffer); + write(testFilePath, writeBuffer); final byte[] readBuffer = new byte[5 * 1000 * 1024]; - FSDataInputStream inputStream = fs.open(TEST_FILE, TEST_DEFAULT_BUFFER_SIZE); + FSDataInputStream inputStream = fs.open(testFilePath, TEST_DEFAULT_BUFFER_SIZE); int offset = 0; while (inputStream.read(readBuffer, offset, TEST_OFFSET) > 0) { offset += TEST_OFFSET; @@ -139,4 +135,14 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest { assertArrayEquals(readBuffer, writeBuffer); inputStream.close(); } + + private void testWriteOneByteToFile(Path testFilePath) throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + try(FSDataOutputStream stream = fs.create(testFilePath)) { + stream.write(TEST_BYTE); + } + + FileStatus fileStatus = fs.getFileStatus(testFilePath); + assertEquals(1, fileStatus.getLen()); + } } 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 49a0e2ee761..5565a4920e4 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 @@ -35,7 +35,7 @@ public final class TestConfigurationKeys { public static final String TEST_CONFIGURATION_FILE_NAME = "azure-test.xml"; public static final String TEST_CONTAINER_PREFIX = "abfs-testcontainer-"; - public static final int TEST_TIMEOUT = 10 * 60 * 1000; + public static final int TEST_TIMEOUT = 15 * 60 * 1000; private TestConfigurationKeys() {} } From 51d368982b01ee390c7e5e60fedb8cde27c0016a Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 18 Sep 2018 12:09:25 +0100 Subject: [PATCH 24/29] HADOOP-15714. Tune abfs/wasb parallel and sequential test execution. Contributed by Da Zhou. --- hadoop-tools/hadoop-azure/pom.xml | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml index 42f4d05d404..52f0bae1008 100644 --- a/hadoop-tools/hadoop-azure/pom.xml +++ b/hadoop-tools/hadoop-azure/pom.xml @@ -256,7 +256,8 @@ parallel-tests-wasb - parallel-tests-wasb + parallel-tests + wasb @@ -438,7 +439,8 @@ parallel-tests-abfs - parallel-tests-abfs + parallel-tests + abfs @@ -602,6 +604,7 @@ parallel-tests + both @@ -812,7 +815,7 @@ sequential-tests - sequential-tests + !parallel-tests From df2166a6435b4b62011a363ea24dfcc9bc44181c Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 18 Sep 2018 12:20:52 +0100 Subject: [PATCH 25/29] HADOOP-15719. Fail-fast when using OAuth over http. Contributed by Da Zhou. --- .../fs/azurebfs/AzureBlobFileSystem.java | 13 ++--- .../fs/azurebfs/AzureBlobFileSystemStore.java | 7 +++ ...onfigurationPropertyNotFoundException.java | 2 +- .../services/TestOauthFailOverHttp.java | 55 +++++++++++++++++++ 4 files changed, 69 insertions(+), 8 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestOauthFailOverHttp.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 2e8de782888..5605e362ea5 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 @@ -101,12 +101,11 @@ public class AzureBlobFileSystem extends FileSystem { this.userGroupInformation = UserGroupInformation.getCurrentUser(); this.user = userGroupInformation.getUserName(); this.abfsStore = new AzureBlobFileSystemStore(uri, this.isSecure(), configuration, userGroupInformation); - - LOG.debug("Initializing NativeAzureFileSystem for {}", uri); + final AbfsConfiguration abfsConfiguration = abfsStore.getAbfsConfiguration(); this.setWorkingDirectory(this.getHomeDirectory()); - if (abfsStore.getAbfsConfiguration().getCreateRemoteFileSystemDuringInitialization()) { + if (abfsConfiguration.getCreateRemoteFileSystemDuringInitialization()) { if (!this.fileSystemExists()) { try { this.createFileSystem(); @@ -116,7 +115,7 @@ public class AzureBlobFileSystem extends FileSystem { } } - if (!abfsStore.getAbfsConfiguration().getSkipUserGroupMetadataDuringInitialization()) { + if (!abfsConfiguration.getSkipUserGroupMetadataDuringInitialization()) { this.primaryUserGroup = userGroupInformation.getPrimaryGroupName(); } else { //Provide a default group name @@ -124,15 +123,15 @@ public class AzureBlobFileSystem extends FileSystem { } if (UserGroupInformation.isSecurityEnabled()) { - this.delegationTokenEnabled = abfsStore.getAbfsConfiguration().isDelegationTokenManagerEnabled(); + this.delegationTokenEnabled = abfsConfiguration.isDelegationTokenManagerEnabled(); if (this.delegationTokenEnabled) { LOG.debug("Initializing DelegationTokenManager for {}", uri); - this.delegationTokenManager = abfsStore.getAbfsConfiguration().getDelegationTokenManager(); + this.delegationTokenManager = abfsConfiguration.getDelegationTokenManager(); } } - AbfsClientThrottlingIntercept.initializeSingleton(abfsStore.getAbfsConfiguration().isAutoThrottlingEnabled()); + AbfsClientThrottlingIntercept.initializeSingleton(abfsConfiguration.isAutoThrottlingEnabled()); } @Override 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 d16cf3622bb..cf7387b6e6e 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 @@ -88,6 +88,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_ABFS_ENDPOINT; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME; import static org.apache.hadoop.util.Time.now; /** @@ -130,6 +131,12 @@ public class AzureBlobFileSystemStore { this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList( abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA))); + if (AuthType.OAuth == abfsConfiguration.getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey) + && !FileSystemUriSchemes.ABFS_SECURE_SCHEME.equals(uri.getScheme())) { + throw new IllegalArgumentException( + String.format("Incorrect URI %s, URI scheme must be abfss when authenticating using Oauth.", uri)); + } + initializeClient(uri, fileSystemName, accountName, isSecure); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ConfigurationPropertyNotFoundException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ConfigurationPropertyNotFoundException.java index bf3b2f34d6e..43a71ab43cb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ConfigurationPropertyNotFoundException.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ConfigurationPropertyNotFoundException.java @@ -27,6 +27,6 @@ import org.apache.hadoop.classification.InterfaceStability; @InterfaceStability.Evolving public class ConfigurationPropertyNotFoundException extends AzureBlobFileSystemException { public ConfigurationPropertyNotFoundException(String property) { - super("Configuration property " + property + "not found."); + super("Configuration property " + property + " not found."); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestOauthFailOverHttp.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestOauthFailOverHttp.java new file mode 100644 index 00000000000..de07c4b2b91 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestOauthFailOverHttp.java @@ -0,0 +1,55 @@ +/** + * 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.net.URI; + +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.azurebfs.constants.FileSystemUriSchemes; + +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Test Oauth fail fast when uri scheme is incorrect. + */ +public class TestOauthFailOverHttp { + + @Test + public void testOauthFailWithSchemeAbfs() throws Exception { + Configuration conf = new Configuration(); + final String account = "fakeaccount.dfs.core.windows.net"; + conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, account); + conf.setEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.OAuth); + URI defaultUri = new URI(FileSystemUriSchemes.ABFS_SCHEME, + "fakecontainer@" + account, + null, + null, + null); + conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString()); + // IllegalArgumentException is expected + // when authenticating using Oauth and scheme is not abfss + intercept(IllegalArgumentException.class, "Incorrect URI", + () -> FileSystem.get(conf)); + } +} From a55d26b23eb7d912daf0dd4237a852fa2f6b7780 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 19 Sep 2018 13:57:39 +0100 Subject: [PATCH 26/29] HADOOP-15769. ABFS: distcp tests are always skipped. Contributed by Steve Loughran --- .../azurebfs/contract/ITestAbfsFileSystemContractDistCp.java | 5 +++++ .../contract/ITestAbfsFileSystemContractSecureDistCp.java | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java index 544bbbfbdb1..529fe831e2b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java @@ -37,6 +37,11 @@ public class ITestAbfsFileSystemContractDistCp extends AbstractContractDistCpTes super.setup(); } + @Override + protected Configuration createConfiguration() { + return binding.getRawConfiguration(); + } + @Override protected AbfsFileSystemContract createContract(Configuration conf) { return new AbfsFileSystemContract(conf, false); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java index fc235e36c61..fa77c2e649c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java @@ -37,6 +37,11 @@ public class ITestAbfsFileSystemContractSecureDistCp extends AbstractContractDis super.setup(); } + @Override + protected Configuration createConfiguration() { + return binding.getRawConfiguration(); + } + @Override protected AbfsFileSystemContract createContract(Configuration conf) { return new AbfsFileSystemContract(conf, true); From 8e831ba458ab5ebe9c919c76d66615bbef63f82e Mon Sep 17 00:00:00 2001 From: Sean Mackrory Date: Wed, 19 Sep 2018 10:14:49 -0600 Subject: [PATCH 27/29] HADOOP-15773. Fixing checkstyle and other issues raised by Yetus. --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 2 - .../fs/azurebfs/AzureBlobFileSystem.java | 28 +-- .../CustomTokenProviderAdaptee.java | 4 +- .../src/site/markdown/testing_azure.md | 16 +- .../ITestNativeFileSystemStatistics.java | 2 +- .../ITestAzureBlobFileSystemPermission.java | 3 +- .../azurebfs/ITestAzureBlobFilesystemAcl.java | 186 ++++++++++-------- .../fs/azurebfs/TestAccountConfiguration.java | 12 ++ .../TestShellDecryptionKeyProvider.java | 4 +- .../fs/azurebfs/utils/AbfsTestUtils.java | 6 +- 10 files changed, 144 insertions(+), 119 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 927a315be98..f0088ff1279 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 @@ -249,7 +249,6 @@ public class AbfsConfiguration{ * @param defaultValue Class returned if none is configured * @param xface Interface shared by all possible values * @return Highest-precedence Class object that was found - * @throws IOException */ public Class getClass(String name, Class defaultValue, Class xface) { return rawConfig.getClass(accountConf(name), @@ -263,7 +262,6 @@ public class AbfsConfiguration{ * @param name Account-agnostic configuration key * @param defaultValue Value returned if none is configured * @return value in String form if one exists, else null - * @throws IOException */ public > T getEnum(String name, T defaultValue) { return rawConfig.getEnum(accountConf(name), 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 5605e362ea5..200f3e77e0b 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 @@ -130,7 +130,7 @@ public class AzureBlobFileSystem extends FileSystem { this.delegationTokenManager = abfsConfiguration.getDelegationTokenManager(); } } - + AbfsClientThrottlingIntercept.initializeSingleton(abfsConfiguration.isAutoThrottlingEnabled()); } @@ -380,7 +380,7 @@ public class AzureBlobFileSystem extends FileSystem { // To support format: abfs://{dfs.nameservices}/file/path, // path need to be first converted to URI, then get the raw path string, // during which {dfs.nameservices} will be omitted. - if (path != null ) { + if (path != null) { String uriPath = path.toUri().getPath(); path = uriPath.isEmpty() ? path : new Path(uriPath); } @@ -581,8 +581,8 @@ public class AzureBlobFileSystem extends FileSystem { if (!getIsNamespaceEnabeld()) { throw new UnsupportedOperationException( - "modifyAclEntries is only supported by storage accounts " + - "with the hierarchical namespace enabled."); + "modifyAclEntries is only supported by storage accounts with the " + + "hierarchical namespace enabled."); } if (aclSpec == null || aclSpec.isEmpty()) { @@ -612,8 +612,8 @@ public class AzureBlobFileSystem extends FileSystem { if (!getIsNamespaceEnabeld()) { throw new UnsupportedOperationException( - "removeAclEntries is only supported by storage accounts " + - "with the hierarchical namespace enabled."); + "removeAclEntries is only supported by storage accounts with the " + + "hierarchical namespace enabled."); } if (aclSpec == null || aclSpec.isEmpty()) { @@ -639,8 +639,8 @@ public class AzureBlobFileSystem extends FileSystem { if (!getIsNamespaceEnabeld()) { throw new UnsupportedOperationException( - "removeDefaultAcl is only supported by storage accounts" + - " with the hierarchical namespace enabled."); + "removeDefaultAcl is only supported by storage accounts with the " + + "hierarchical namespace enabled."); } try { @@ -664,8 +664,8 @@ public class AzureBlobFileSystem extends FileSystem { if (!getIsNamespaceEnabeld()) { throw new UnsupportedOperationException( - "removeAcl is only supported by storage accounts" + - " with the hierarchical namespace enabled."); + "removeAcl is only supported by storage accounts with the " + + "hierarchical namespace enabled."); } try { @@ -692,8 +692,8 @@ public class AzureBlobFileSystem extends FileSystem { if (!getIsNamespaceEnabeld()) { throw new UnsupportedOperationException( - "setAcl is only supported by storage accounts" + - " with the hierarchical namespace enabled."); + "setAcl is only supported by storage accounts with the hierarchical " + + "namespace enabled."); } if (aclSpec == null || aclSpec.size() == 0) { @@ -720,8 +720,8 @@ public class AzureBlobFileSystem extends FileSystem { if (!getIsNamespaceEnabeld()) { throw new UnsupportedOperationException( - "getAclStatus is only supported by storage accounts" + - " with the hierarchical namespace enabled."); + "getAclStatus is only supported by storage account with the " + + "hierarchical namespace enabled."); } try { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomTokenProviderAdaptee.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomTokenProviderAdaptee.java index 889d506c4a5..d8bda700712 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomTokenProviderAdaptee.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomTokenProviderAdaptee.java @@ -47,8 +47,8 @@ public interface CustomTokenProviderAdaptee { * @param accountName Account Name * @throws IOException if instance can not be configured. */ - void initialize(Configuration configuration, final String accountName) - throws IOException; + void initialize(Configuration configuration, String accountName) + throws IOException; /** * Obtain the access token that should be added to https connection's header. 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 a4d920afde4..c2afe74e22e 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md @@ -578,15 +578,15 @@ provide the details and summary of the operation. ## Testing the Azure ABFS Client -Azure Data Lake Storage Gen 2 (ADLS Gen 2) is a set of capabilities dedicated to -big data analytics, built on top of Azure Blob Storage. The ABFS and ABFSS +Azure Data Lake Storage Gen 2 (ADLS Gen 2) is a set of capabilities dedicated to +big data analytics, built on top of Azure Blob Storage. The ABFS and ABFSS schemes target the ADLS Gen 2 REST API, and the WASB and WASBS schemes target the Azure Blob Storage REST API. ADLS Gen 2 offers better performance and -scalability. ADLS Gen 2 also offers authentication and authorization compatible -with the Hadoop Distributed File System permissions model when hierarchical +scalability. ADLS Gen 2 also offers authentication and authorization compatible +with the Hadoop Distributed File System permissions model when hierarchical namespace is enabled for the storage account. Furthermore, the metadata and data produced by ADLS Gen 2 REST API can be consumed by Blob REST API, and vice versa. - + 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 @@ -610,7 +610,7 @@ credentials. fs.azure.wasb.account.name {ACCOUNT_NAME}.blob.core.windows.net - + fs.azure.account.key.{ACCOUNT_NAME}.blob.core.windows.net {ACCOUNT_ACCESS_KEY} @@ -641,7 +641,7 @@ hierarchical namespace enabled, and set the following configuration settings: fs.azure.account.auth.type.{YOUR_ABFS_ACCOUNT_NAME} {AUTH TYPE} - The authorization type can be SharedKey, OAuth, or Custom. The + The authorization type can be SharedKey, OAuth, or Custom. The default is SharedKey. @@ -744,4 +744,4 @@ http[s]://[account][domain-suffix]/[filesystem], please use the following: fs.azure.abfs.endpoint {IP}:{PORT} -``` \ No newline at end of file +``` diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeFileSystemStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeFileSystemStatistics.java index cbb09ddff8b..447f65f2bd1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeFileSystemStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeFileSystemStatistics.java @@ -35,7 +35,7 @@ import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.writeStringT /** * Because FileSystem.Statistics is per FileSystem, so statistics can not be ran in * parallel, hence in this test file, force them to run in sequential. - * */ + */ public class ITestNativeFileSystemStatistics extends AbstractWasbTestWithTimeout{ @Test diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java index e0e0758cf05..bbb2e240bee 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemPermission.java @@ -23,7 +23,6 @@ import java.util.Collection; import java.util.UUID; import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -106,4 +105,4 @@ public class ITestAzureBlobFileSystemPermission extends AbstractAbfsIntegrationT FileStatus status = fs.getFileStatus(path); Assert.assertEquals(permission.applyUMask(DEFAULT_UMASK_PERMISSION), status.getPermission()); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java index 67da26babcb..acafe03be0d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFilesystemAcl.java @@ -56,6 +56,20 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { private static final FsAction READ = FsAction.READ; private static final FsAction READ_EXECUTE = FsAction.READ_EXECUTE; private static final FsAction READ_WRITE = FsAction.READ_WRITE; + + private static final short RW = 0600; + private static final short RWX = 0700; + private static final short RWX_R = 0740; + private static final short RWX_RW = 0760; + private static final short RWX_RWX = 0770; + private static final short RWX_RX = 0750; + private static final short RWX_RX_RX = 0755; + private static final short RW_R = 0640; + private static final short RW_RW = 0660; + private static final short RW_RWX = 0670; + private static final short RW_R_R = 0644; + private static final short STICKY_RWX_RWX = 01770; + private static Path testRoot = new Path("/test"); private Path path; @@ -68,7 +82,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - fs.mkdirs(path, FsPermission.createImmutable((short) 0750)); + fs.mkdirs(path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, ALL), @@ -93,7 +107,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { aclEntry(DEFAULT, GROUP, READ_EXECUTE), aclEntry(DEFAULT, MASK, READ_EXECUTE), aclEntry(DEFAULT, OTHER, NONE) }, returned); - assertPermission(fs, (short) 0750); + assertPermission(fs, (short) RWX_RX); } @Test @@ -102,7 +116,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); - fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, ALL), aclEntry(ACCESS, USER, "foo", ALL), @@ -117,7 +131,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assertArrayEquals(new AclEntry[] { aclEntry(ACCESS, USER, "foo", READ_EXECUTE), aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned); - assertPermission(fs, (short) 0750); + assertPermission(fs, (short) RWX_RX); } @Test @@ -125,7 +139,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, "foo", ALL)); fs.setAcl(path, aclSpec); @@ -140,7 +154,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { aclEntry(DEFAULT, GROUP, READ_EXECUTE), aclEntry(DEFAULT, MASK, READ_EXECUTE), aclEntry(DEFAULT, OTHER, NONE) }, returned); - assertPermission(fs, (short) 0750); + assertPermission(fs, (short) RWX_RX); } @Test @@ -149,7 +163,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); - fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, "foo", READ_WRITE)); fs.modifyAclEntries(path, aclSpec); @@ -158,7 +172,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assertArrayEquals(new AclEntry[] { aclEntry(ACCESS, USER, "foo", READ_WRITE), aclEntry(ACCESS, GROUP, READ) }, returned); - assertPermission(fs, (short) 0660); + assertPermission(fs, (short) RW_RW); } @Test @@ -166,7 +180,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, ALL), aclEntry(DEFAULT, GROUP, READ_EXECUTE), @@ -178,7 +192,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { aclEntry(DEFAULT, USER, ALL), aclEntry(DEFAULT, GROUP, READ_EXECUTE), aclEntry(DEFAULT, OTHER, NONE) }, returned); - assertPermission(fs, (short) 0750); + assertPermission(fs, (short) RWX_RX); } @Test @@ -187,7 +201,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); - fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, "foo", ALL), aclEntry(ACCESS, MASK, NONE)); @@ -197,7 +211,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assertArrayEquals(new AclEntry[] { aclEntry(ACCESS, USER, "foo", ALL), aclEntry(ACCESS, GROUP, READ) }, returned); - assertPermission(fs, (short) 0600); + assertPermission(fs, (short) RW); } @Test @@ -250,7 +264,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); - fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, "foo", ALL)); fs.modifyAclEntries(path, aclSpec); @@ -261,7 +275,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, ALL), aclEntry(ACCESS, USER, "foo", ALL), @@ -281,7 +295,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { aclEntry(DEFAULT, GROUP, READ_EXECUTE), aclEntry(DEFAULT, MASK, READ_EXECUTE), aclEntry(DEFAULT, OTHER, NONE) }, returned); - assertPermission(fs, (short) 0750); + assertPermission(fs, (short) RWX_RX); } @Test @@ -290,7 +304,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); - fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, ALL), aclEntry(ACCESS, USER, "foo", ALL), @@ -306,7 +320,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assertArrayEquals(new AclEntry[] { aclEntry(ACCESS, USER, "bar", READ_WRITE), aclEntry(ACCESS, GROUP, READ_WRITE) }, returned); - assertPermission(fs, (short) 0760); + assertPermission(fs, (short) RWX_RW); } @Test @@ -314,7 +328,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, ALL), aclEntry(ACCESS, GROUP, READ_EXECUTE), @@ -333,7 +347,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { aclEntry(DEFAULT, GROUP, READ_EXECUTE), aclEntry(DEFAULT, MASK, READ_EXECUTE), aclEntry(DEFAULT, OTHER, NONE) }, returned); - assertPermission(fs, (short) 0750); + assertPermission(fs, (short) RWX_RX); } @Test @@ -342,7 +356,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); - fs.setPermission(path, FsPermission.createImmutable((short) 0760)); + fs.setPermission(path, FsPermission.createImmutable((short) RWX_RW)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, ALL), aclEntry(ACCESS, USER, "foo", ALL), @@ -356,7 +370,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { AclStatus s = fs.getAclStatus(path); AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(new AclEntry[] { }, returned); - assertPermission(fs, (short) 0760); + assertPermission(fs, (short) RWX_RW); } @Test @@ -364,7 +378,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, ALL), aclEntry(ACCESS, USER, "foo", ALL), @@ -384,7 +398,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { aclEntry(DEFAULT, USER, ALL), aclEntry(DEFAULT, GROUP, READ_EXECUTE), aclEntry(DEFAULT, OTHER, NONE) }, returned); - assertPermission(fs, (short) 0750); + assertPermission(fs, (short) RWX_RX); } @Test @@ -431,7 +445,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, ALL), aclEntry(ACCESS, USER, "foo", ALL), @@ -445,7 +459,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assertArrayEquals(new AclEntry[] { aclEntry(ACCESS, USER, "foo", ALL), aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned); - assertPermission(fs, (short) 0770); + assertPermission(fs, (short) RWX_RWX); } @Test @@ -454,7 +468,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); - fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, ALL), aclEntry(ACCESS, USER, "foo", ALL), @@ -467,7 +481,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assertArrayEquals(new AclEntry[] { aclEntry(ACCESS, USER, "foo", ALL), aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned); - assertPermission(fs, (short) 0770); + assertPermission(fs, (short) RWX_RWX); } @Test @@ -475,7 +489,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, "foo", ALL)); fs.setAcl(path, aclSpec); @@ -483,7 +497,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { AclStatus s = fs.getAclStatus(path); AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(new AclEntry[] { }, returned); - assertPermission(fs, (short) 0750); + assertPermission(fs, (short) RWX_RX); } @Test @@ -491,12 +505,12 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); fs.removeDefaultAcl(path); AclStatus s = fs.getAclStatus(path); AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(new AclEntry[] { }, returned); - assertPermission(fs, (short) 0750); + assertPermission(fs, (short) RWX_RX); } @Test @@ -518,7 +532,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assertArrayEquals(new AclEntry[] { aclEntry(ACCESS, USER, "foo", ALL), aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned); - assertPermission(fs, (short) 01770); + assertPermission(fs, (short) STICKY_RWX_RWX); } @Test(expected=FileNotFoundException.class) @@ -535,7 +549,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, ALL), aclEntry(ACCESS, USER, "foo", ALL), @@ -549,7 +563,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { AclStatus s = fs.getAclStatus(path); AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(new AclEntry[] { }, returned); - assertPermission(fs, (short) 0750); + assertPermission(fs, (short) RWX_RX); } @Test @@ -558,12 +572,12 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); - fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); fs.removeAcl(path); AclStatus s = fs.getAclStatus(path); AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(new AclEntry[] { }, returned); - assertPermission(fs, (short) 0640); + assertPermission(fs, (short) RW_R); } @Test @@ -591,7 +605,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, ALL), aclEntry(ACCESS, GROUP, READ_EXECUTE), @@ -602,7 +616,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { AclStatus s = fs.getAclStatus(path); AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(new AclEntry[] { }, returned); - assertPermission(fs, (short) 0750); + assertPermission(fs, (short) RWX_RX); } @Test(expected=FileNotFoundException.class) @@ -619,7 +633,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, ALL), aclEntry(ACCESS, USER, "foo", ALL), @@ -637,7 +651,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { aclEntry(DEFAULT, GROUP, READ_EXECUTE), aclEntry(DEFAULT, MASK, ALL), aclEntry(DEFAULT, OTHER, NONE) }, returned); - assertPermission(fs, (short) 0770); + assertPermission(fs, (short) RWX_RWX); } @Test @@ -646,7 +660,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); - fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, READ_WRITE), aclEntry(ACCESS, USER, "foo", READ), @@ -658,7 +672,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assertArrayEquals(new AclEntry[] { aclEntry(ACCESS, USER, "foo", READ), aclEntry(ACCESS, GROUP, READ) }, returned); - assertPermission(fs, (short) 0640); + assertPermission(fs, (short) RW_R); } @Test @@ -666,7 +680,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, "foo", ALL)); fs.setAcl(path, aclSpec); @@ -678,7 +692,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { aclEntry(DEFAULT, GROUP, READ_EXECUTE), aclEntry(DEFAULT, MASK, ALL), aclEntry(DEFAULT, OTHER, NONE) }, returned); - assertPermission(fs, (short) 0750); + assertPermission(fs, (short) RWX_RX); } @Test @@ -687,7 +701,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); - fs.setPermission(path, FsPermission.createImmutable((short) 0644)); + fs.setPermission(path, FsPermission.createImmutable((short) RW_R_R)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, READ_WRITE), aclEntry(ACCESS, USER, "foo", READ), @@ -702,7 +716,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { AclStatus s = fs.getAclStatus(path); AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(new AclEntry[] { }, returned); - assertPermission(fs, (short) 0640); + assertPermission(fs, (short) RW_R); } @Test @@ -710,7 +724,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, ALL), aclEntry(DEFAULT, GROUP, READ_EXECUTE), @@ -722,7 +736,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { aclEntry(DEFAULT, USER, ALL), aclEntry(DEFAULT, GROUP, READ_EXECUTE), aclEntry(DEFAULT, OTHER, NONE) }, returned); - assertPermission(fs, (short) 0750); + assertPermission(fs, (short) RWX_RX); } @Test @@ -731,7 +745,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); - fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, READ_WRITE), aclEntry(ACCESS, USER, "foo", READ), @@ -744,7 +758,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assertArrayEquals(new AclEntry[] { aclEntry(ACCESS, USER, "foo", READ), aclEntry(ACCESS, GROUP, READ) }, returned); - assertPermission(fs, (short) 0670); + assertPermission(fs, (short) RW_RWX); } @Test @@ -770,7 +784,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { aclEntry(DEFAULT, GROUP, READ_EXECUTE), aclEntry(DEFAULT, MASK, ALL), aclEntry(DEFAULT, OTHER, NONE) }, returned); - assertPermission(fs, (short) 01770); + assertPermission(fs, (short) STICKY_RWX_RWX); } @Test(expected=FileNotFoundException.class) @@ -793,7 +807,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); - fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, "foo", ALL)); fs.setAcl(path, aclSpec); @@ -804,7 +818,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, ALL), aclEntry(ACCESS, USER, "foo", ALL), @@ -812,7 +826,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { aclEntry(ACCESS, OTHER, NONE), aclEntry(DEFAULT, USER, "foo", ALL)); fs.setAcl(path, aclSpec); - fs.setPermission(path, FsPermission.createImmutable((short) 0700)); + fs.setPermission(path, FsPermission.createImmutable((short) RWX)); AclStatus s = fs.getAclStatus(path); AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(new AclEntry[] { @@ -823,7 +837,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { aclEntry(DEFAULT, GROUP, READ_EXECUTE), aclEntry(DEFAULT, MASK, ALL), aclEntry(DEFAULT, OTHER, NONE) }, returned); - assertPermission(fs, (short) 0700); + assertPermission(fs, (short) RWX); } @Test @@ -832,20 +846,20 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); fs.create(path).close(); - fs.setPermission(path, FsPermission.createImmutable((short) 0640)); + fs.setPermission(path, FsPermission.createImmutable((short) RW_R)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, READ_WRITE), aclEntry(ACCESS, USER, "foo", READ), aclEntry(ACCESS, GROUP, READ), aclEntry(ACCESS, OTHER, NONE)); fs.setAcl(path, aclSpec); - fs.setPermission(path, FsPermission.createImmutable((short) 0600)); + fs.setPermission(path, FsPermission.createImmutable((short) RW)); AclStatus s = fs.getAclStatus(path); AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(new AclEntry[] { aclEntry(ACCESS, USER, "foo", READ), aclEntry(ACCESS, GROUP, READ) }, returned); - assertPermission(fs, (short) 0600); + assertPermission(fs, (short) RW); } @Test @@ -853,14 +867,14 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, ALL), aclEntry(ACCESS, GROUP, READ_EXECUTE), aclEntry(ACCESS, OTHER, NONE), aclEntry(DEFAULT, USER, "foo", ALL)); fs.setAcl(path, aclSpec); - fs.setPermission(path, FsPermission.createImmutable((short) 0700)); + fs.setPermission(path, FsPermission.createImmutable((short) RWX)); AclStatus s = fs.getAclStatus(path); AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(new AclEntry[] { @@ -869,7 +883,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { aclEntry(DEFAULT, GROUP, READ_EXECUTE), aclEntry(DEFAULT, MASK, ALL), aclEntry(DEFAULT, OTHER, NONE) }, returned); - assertPermission(fs, (short) 0700); + assertPermission(fs, (short) RWX); } @Test @@ -877,7 +891,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, "foo", ALL)); fs.setAcl(path, aclSpec); @@ -888,7 +902,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assertArrayEquals(new AclEntry[] { aclEntry(ACCESS, USER, "foo", ALL), aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned); - assertPermission(fs, filePath, (short) 0640); + assertPermission(fs, filePath, (short) RW_R); } @Test @@ -897,7 +911,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, "foo", ALL)); fs.modifyAclEntries(path, aclSpec); @@ -906,7 +920,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { AclStatus s = fs.getAclStatus(filePath); AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(new AclEntry[] { }, returned); - assertPermission(fs, filePath, (short) 0644); + assertPermission(fs, filePath, (short) RW_R_R); } @Test @@ -915,7 +929,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, ALL), aclEntry(DEFAULT, GROUP, READ_EXECUTE), @@ -926,7 +940,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { AclStatus s = fs.getAclStatus(filePath); AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(new AclEntry[] { }, returned); - assertPermission(fs, filePath, (short) 0640); + assertPermission(fs, filePath, (short) RW_R); } @Test @@ -934,7 +948,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, "foo", ALL)); fs.setAcl(path, aclSpec); @@ -952,7 +966,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { aclEntry(DEFAULT, GROUP, READ_EXECUTE), aclEntry(DEFAULT, MASK, ALL), aclEntry(DEFAULT, OTHER, NONE) }, returned); - assertPermission(fs, dirPath, (short) 0770); + assertPermission(fs, dirPath, (short) RWX_RWX); } @Test @@ -961,7 +975,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(ACCESS, USER, "foo", ALL)); fs.modifyAclEntries(path, aclSpec); @@ -970,7 +984,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { AclStatus s = fs.getAclStatus(dirPath); AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(new AclEntry[] { }, returned); - assertPermission(fs, dirPath, (short) 0755); + assertPermission(fs, dirPath, (short) RWX_RX_RX); } @Test @@ -979,7 +993,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, ALL), aclEntry(DEFAULT, GROUP, READ_EXECUTE), @@ -993,7 +1007,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { aclEntry(DEFAULT, USER, ALL), aclEntry(DEFAULT, GROUP, READ_EXECUTE), aclEntry(DEFAULT, OTHER, NONE) }, returned); - assertPermission(fs, dirPath, (short) 0750); + assertPermission(fs, dirPath, (short) RWX_RX); } @Test @@ -1001,13 +1015,13 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0755)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX_RX)); List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, "foo", ALL)); fs.setAcl(path, aclSpec); Path filePath = new Path(path, "file1"); int bufferSize = 4 * 1024 * 1024; - fs.create(filePath, new FsPermission((short) 0740), false, bufferSize, + fs.create(filePath, new FsPermission((short) RWX_R), false, bufferSize, fs.getDefaultReplication(filePath), fs.getDefaultBlockSize(path), null) .close(); AclStatus s = fs.getAclStatus(filePath); @@ -1015,7 +1029,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assertArrayEquals(new AclEntry[] { aclEntry(ACCESS, USER, "foo", ALL), aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned); - assertPermission(fs, filePath, (short) 0740); + assertPermission(fs, filePath, (short) RWX_R); } @Test @@ -1023,12 +1037,12 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { final AzureBlobFileSystem fs = this.getFileSystem(); assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); - FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0755)); + FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) RWX_RX_RX)); List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, "foo", ALL)); fs.setAcl(path, aclSpec); Path dirPath = new Path(path, "dir1"); - fs.mkdirs(dirPath, new FsPermission((short) 0740)); + fs.mkdirs(dirPath, new FsPermission((short) RWX_R)); AclStatus s = fs.getAclStatus(dirPath); AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(new AclEntry[] { @@ -1039,7 +1053,7 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { aclEntry(DEFAULT, GROUP, READ_EXECUTE), aclEntry(DEFAULT, MASK, ALL), aclEntry(DEFAULT, OTHER, READ_EXECUTE) }, returned); - assertPermission(fs, dirPath, (short) 0740); + assertPermission(fs, dirPath, (short) RWX_R); } @Test @@ -1048,20 +1062,20 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); Path dirPath = new Path(path, "dir"); - FileSystem.mkdirs(fs, dirPath, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, dirPath, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, "foo", ALL)); fs.setAcl(dirPath, aclSpec); Path filePath = new Path(path, "file1"); fs.create(filePath).close(); - fs.setPermission(filePath, FsPermission.createImmutable((short) 0640)); + fs.setPermission(filePath, FsPermission.createImmutable((short) RW_R)); Path renamedFilePath = new Path(dirPath, "file1"); fs.rename(filePath, renamedFilePath); AclEntry[] expected = new AclEntry[] { }; AclStatus s = fs.getAclStatus(renamedFilePath); AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(expected, returned); - assertPermission(fs, renamedFilePath, (short) 0640); + assertPermission(fs, renamedFilePath, (short) RW_R); } @Test @@ -1070,19 +1084,19 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { assumeTrue(fs.getIsNamespaceEnabeld()); path = new Path(testRoot, UUID.randomUUID().toString()); Path dirPath = new Path(path, "dir"); - FileSystem.mkdirs(fs, dirPath, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, dirPath, FsPermission.createImmutable((short) RWX_RX)); List aclSpec = Lists.newArrayList( aclEntry(DEFAULT, USER, "foo", ALL)); fs.setAcl(dirPath, aclSpec); Path subdirPath = new Path(path, "subdir"); - FileSystem.mkdirs(fs, subdirPath, FsPermission.createImmutable((short) 0750)); + FileSystem.mkdirs(fs, subdirPath, FsPermission.createImmutable((short) RWX_RX)); Path renamedSubdirPath = new Path(dirPath, "subdir"); fs.rename(subdirPath, renamedSubdirPath); AclEntry[] expected = new AclEntry[] { }; AclStatus s = fs.getAclStatus(renamedSubdirPath); AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]); assertArrayEquals(expected, returned); - assertPermission(fs, renamedSubdirPath, (short) 0750); + assertPermission(fs, renamedSubdirPath, (short) RWX_RX); } @Test @@ -1247,4 +1261,4 @@ public class ITestAzureBlobFilesystemAcl extends AbstractAbfsIntegrationTest { throws Exception { AclTestHelpers.assertPermission(fs, pathToCheck, perm); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java index 425485cb3ac..a790cf21487 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestAccountConfiguration.java @@ -191,6 +191,9 @@ public class TestAccountConfiguration { abfsConf.getLong(globalKey, 0), 1); } + /** + * Dummy type used for testing handling of enums in configuration. + */ public enum GetEnumType { TRUE, FALSE } @@ -226,12 +229,21 @@ public class TestAccountConfiguration { abfsConf.getEnum(globalKey, GetEnumType.FALSE), GetEnumType.TRUE); } + /** + * Dummy type used for testing handling of classes in configuration. + */ interface GetClassInterface { } + /** + * Dummy type used for testing handling of classes in configuration. + */ private class GetClassImpl0 implements GetClassInterface { } + /** + * Dummy type used for testing handling of classes in configuration. + */ private class GetClassImpl1 implements GetClassInterface { } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java index 5ef4eab7a00..b8df38eed0a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestShellDecryptionKeyProvider.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.azurebfs.services; import java.io.File; +import java.nio.charset.Charset; import org.junit.Assert; import org.junit.Test; @@ -74,7 +75,8 @@ public class TestShellDecryptionKeyProvider { // Create a simple script which echoes the given key plus the given // expected result (so that we validate both script input and output) File scriptFile = new File(TEST_ROOT_DIR, "testScript.cmd"); - FileUtils.writeStringToFile(scriptFile, "@echo %1 " + expectedResult); + FileUtils.writeStringToFile(scriptFile, "@echo %1 " + expectedResult, + Charset.forName("UTF-8")); ShellDecryptionKeyProvider provider = new ShellDecryptionKeyProvider(); Configuration conf = new Configuration(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AbfsTestUtils.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AbfsTestUtils.java index 7e6336d1ae5..21edfe800e1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AbfsTestUtils.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/AbfsTestUtils.java @@ -31,9 +31,9 @@ import org.apache.hadoop.fs.azurebfs.services.AuthType; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONTAINER_PREFIX; - /* - * Some Utils for ABFS tests. - * */ +/** + * Some Utils for ABFS tests. + */ public final class AbfsTestUtils extends AbstractAbfsIntegrationTest{ private static final Logger LOG = LoggerFactory.getLogger(AbfsTestUtils.class); From a5692c2da5fda41d104c1213a67f014aaf8e4434 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 20 Sep 2018 17:35:09 +0100 Subject: [PATCH 28/29] HADOOP-15704. Mark ABFS extension package and interfaces as LimitedPrivate/Unstable. Contributed by Steve Loughran. --- .../extensions/CustomDelegationTokenManager.java | 4 ++++ .../extensions/CustomTokenProviderAdaptee.java | 4 ++-- .../hadoop/fs/azurebfs/extensions/package-info.java | 13 ++++++++++++- 3 files changed, 18 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomDelegationTokenManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomDelegationTokenManager.java index f5716e089c0..422f8c25110 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomDelegationTokenManager.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomDelegationTokenManager.java @@ -21,6 +21,8 @@ package org.apache.hadoop.fs.azurebfs.extensions; import java.io.IOException; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier; import org.apache.hadoop.security.token.Token; @@ -28,6 +30,8 @@ import org.apache.hadoop.security.token.Token; /** * Interface for Managing the Delegation tokens. */ +@InterfaceAudience.LimitedPrivate("authorization-subsystems") +@InterfaceStability.Unstable public interface CustomDelegationTokenManager { /** diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomTokenProviderAdaptee.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomTokenProviderAdaptee.java index d8bda700712..d57eef6548b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomTokenProviderAdaptee.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomTokenProviderAdaptee.java @@ -35,8 +35,8 @@ import org.apache.hadoop.conf.Configuration; * This class will be dynamically loaded, initialized, and invoked to provide * AAD Access Tokens and their Expiry. */ -@InterfaceAudience.Public -@InterfaceStability.Evolving +@InterfaceAudience.LimitedPrivate("authorization-subsystems") +@InterfaceStability.Unstable public interface CustomTokenProviderAdaptee { /** diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/package-info.java index 667b7b2e07c..caf4bdae6a6 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/package-info.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/package-info.java @@ -16,6 +16,17 @@ * limitations under the License. */ -@InterfaceAudience.Public +/** + * This package is for extension points under ABFS; + * There are no stability guarantees as these extension points are + * deep inside the ABFS implementation code. + * + * Note, however: this is how the ABFS client integrates with + * authorization services and other aspects of Azure's infrastructure. + * Do not change these APIs without good reason or detailed discussion. + */ +@InterfaceAudience.LimitedPrivate("authorization-subsystems") +@InterfaceStability.Unstable package org.apache.hadoop.fs.azurebfs.extensions; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; From d0b4624c88fc48932a7c2800185ed48bb1c5e0fe Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 21 Sep 2018 11:06:24 +0100 Subject: [PATCH 29/29] HADOOP-15778. ABFS: Fix client side throttling for read. Contributed by Sneha Varma. --- .../AbfsClientThrottlingIntercept.java | 22 +++++++++++++++++-- .../azurebfs/services/AbfsRestOperation.java | 3 ++- 2 files changed, 22 insertions(+), 3 deletions(-) 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 97ea2a64861..1c6ce17a38c 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,9 +19,12 @@ package org.apache.hadoop.fs.azurebfs.services; import java.net.HttpURLConnection; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; + /** * Throttles Azure Blob File System read and write operations to achieve maximum * throughput by minimizing errors. The errors occur when the account ingress @@ -37,6 +40,7 @@ import org.slf4j.LoggerFactory; public final class AbfsClientThrottlingIntercept { 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; @@ -82,7 +86,8 @@ public final class AbfsClientThrottlingIntercept { } break; case ReadFile: - contentLength = abfsHttpOperation.getBytesReceived(); + String range = abfsHttpOperation.getConnection().getRequestProperty(HttpHeaderConfigurations.RANGE); + contentLength = getContentLengthIfKnown(range); if (contentLength > 0) { singleton.readThrottler.addBytesTransferred(contentLength, isFailedOperation); @@ -114,4 +119,17 @@ public final class AbfsClientThrottlingIntercept { break; } } -} \ No newline at end of file + + private static long getContentLengthIfKnown(String range) { + long contentLength = 0; + // Format is "bytes=%d-%d" + if (range != null && range.startsWith(RANGE_PREFIX)) { + String[] offsets = range.substring(RANGE_PREFIX.length()).split("-"); + if (offsets.length == 2) { + contentLength = Long.parseLong(offsets[1]) - Long.parseLong(offsets[0]) + + 1; + } + } + return contentLength; + } +} 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 9a7187996de..3f5717ee7e1 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 @@ -156,9 +156,10 @@ public class AbfsRestOperation { client.getAccessToken()); } + AbfsClientThrottlingIntercept.sendingRequest(operationType); + if (hasRequestBody) { // HttpUrlConnection requires - AbfsClientThrottlingIntercept.sendingRequest(operationType); httpOperation.sendRequest(buffer, bufferOffset, bufferLength); }