From bde36699bb5b96becc83cb4f4ea56167576eaefe Mon Sep 17 00:00:00 2001 From: Jinglun Date: Sun, 26 Jan 2025 09:27:19 +0800 Subject: [PATCH] HADOOP-19236. Incorporate VolcanoEngine Cloud TOS File System Implementation. Contributed by: ZhengHu, SunXin, XianyinXin, Rascal Wu, FangBo, Yuanzhihuan. --- .../hadoop-cloud-storage/pom.xml | 5 + .../dev-support/findbugs-exclude.xml | 89 + .../hadoop-tos/pom.xml | 183 ++ .../org/apache/hadoop/fs/tosfs/RawFS.java | 47 + .../apache/hadoop/fs/tosfs/RawFSUtils.java | 67 + .../apache/hadoop/fs/tosfs/RawFileStatus.java | 74 + .../apache/hadoop/fs/tosfs/RawFileSystem.java | 770 +++++++++ .../hadoop/fs/tosfs/RawLocatedFileStatus.java | 30 + .../apache/hadoop/fs/tosfs/TosChecksum.java | 67 + .../org/apache/hadoop/fs/tosfs/TosFS.java | 47 + .../apache/hadoop/fs/tosfs/TosFileSystem.java | 43 + .../hadoop/fs/tosfs/commit/CommitContext.java | 45 + .../hadoop/fs/tosfs/commit/CommitUtils.java | 370 ++++ .../hadoop/fs/tosfs/commit/Committer.java | 510 ++++++ .../fs/tosfs/commit/CommitterFactory.java | 33 + .../fs/tosfs/commit/MagicOutputStream.java | 151 ++ .../hadoop/fs/tosfs/commit/Pending.java | 179 ++ .../hadoop/fs/tosfs/commit/PendingSet.java | 123 ++ .../hadoop/fs/tosfs/commit/SuccessData.java | 233 +++ .../fs/tosfs/commit/mapred/Committer.java | 189 +++ .../fs/tosfs/commit/mapred/package-info.java | 26 + .../fs/tosfs/commit/ops/PendingOps.java | 43 + .../tosfs/commit/ops/PendingOpsFactory.java | 40 + .../fs/tosfs/commit/ops/RawPendingOps.java | 57 + .../fs/tosfs/commit/ops/package-info.java | 26 + .../hadoop/fs/tosfs/commit/package-info.java | 26 + .../apache/hadoop/fs/tosfs/common/Bytes.java | 103 ++ .../apache/hadoop/fs/tosfs/common/Chain.java | 148 ++ .../apache/hadoop/fs/tosfs/common/Tasks.java | 592 +++++++ .../hadoop/fs/tosfs/common/ThreadPools.java | 140 ++ .../hadoop/fs/tosfs/common/package-info.java | 26 + .../hadoop/fs/tosfs/conf/ArgumentKey.java | 32 + .../apache/hadoop/fs/tosfs/conf/ConfKeys.java | 156 ++ .../hadoop/fs/tosfs/conf/FileStoreKeys.java | 41 + .../apache/hadoop/fs/tosfs/conf/TosKeys.java | 276 +++ .../hadoop/fs/tosfs/conf/package-info.java | 26 + .../hadoop/fs/tosfs/object/BucketInfo.java | 91 + .../hadoop/fs/tosfs/object/ChecksumInfo.java | 37 + .../hadoop/fs/tosfs/object/ChecksumType.java | 41 + .../hadoop/fs/tosfs/object/Constants.java | 32 + .../fs/tosfs/object/DirectoryStorage.java | 50 + .../hadoop/fs/tosfs/object/FileStore.java | 648 +++++++ .../fs/tosfs/object/InputStreamProvider.java | 34 + .../fs/tosfs/object/MultipartUpload.java | 102 ++ .../fs/tosfs/object/ObjectConstants.java | 27 + .../hadoop/fs/tosfs/object/ObjectContent.java | 52 + .../hadoop/fs/tosfs/object/ObjectInfo.java | 117 ++ .../object/ObjectMultiRangeInputStream.java | 233 +++ .../fs/tosfs/object/ObjectOutputStream.java | 343 ++++ .../tosfs/object/ObjectRangeInputStream.java | 199 +++ .../hadoop/fs/tosfs/object/ObjectStorage.java | 372 ++++ .../fs/tosfs/object/ObjectStorageFactory.java | 73 + .../hadoop/fs/tosfs/object/ObjectUtils.java | 92 + .../apache/hadoop/fs/tosfs/object/Part.java | 78 + .../hadoop/fs/tosfs/object/PrefixStorage.java | 254 +++ .../exceptions/ChecksumMismatchException.java | 34 + .../exceptions/InvalidObjectKeyException.java | 31 + .../exceptions/NotAppendableException.java | 25 + .../tosfs/object/exceptions/package-info.java | 26 + .../hadoop/fs/tosfs/object/package-info.java | 26 + .../object/request/ListObjectsRequest.java | 85 + .../fs/tosfs/object/request/package-info.java | 26 + .../object/response/ListObjectsResponse.java | 43 + .../tosfs/object/response/package-info.java | 26 + .../tosfs/object/staging/FileStagingPart.java | 177 ++ .../fs/tosfs/object/staging/StagingPart.java | 79 + .../hadoop/fs/tosfs/object/staging/State.java | 23 + .../fs/tosfs/object/staging/package-info.java | 26 + .../tosfs/object/tos/ChainTOSInputStream.java | 137 ++ .../fs/tosfs/object/tos/DelegationClient.java | 1255 ++++++++++++++ .../object/tos/DelegationClientBuilder.java | 185 ++ .../fs/tosfs/object/tos/GetObjectOutput.java | 60 + .../hadoop/fs/tosfs/object/tos/TOS.java | 1038 ++++++++++++ .../fs/tosfs/object/tos/TOSErrorCodes.java | 53 + .../fs/tosfs/object/tos/TOSInputStream.java | 121 ++ .../hadoop/fs/tosfs/object/tos/TOSUtils.java | 119 ++ .../fs/tosfs/object/tos/TosObjectInfo.java | 78 + .../tos/auth/AbstractCredentialsProvider.java | 79 + .../object/tos/auth/CredentialsProvider.java | 35 + .../auth/DefaultCredentialsProviderChain.java | 101 ++ .../auth/EnvironmentCredentialsProvider.java | 34 + .../object/tos/auth/ExpireableCredential.java | 67 + .../tos/auth/SimpleCredentialsProvider.java | 58 + .../tosfs/object/tos/auth/package-info.java | 26 + .../fs/tosfs/object/tos/package-info.java | 26 + .../hadoop/fs/tosfs/ops/DefaultFsOps.java | 189 +++ .../hadoop/fs/tosfs/ops/DirectoryFsOps.java | 109 ++ .../org/apache/hadoop/fs/tosfs/ops/FsOps.java | 91 + .../apache/hadoop/fs/tosfs/ops/RenameOp.java | 230 +++ .../hadoop/fs/tosfs/ops/package-info.java | 26 + .../apache/hadoop/fs/tosfs/package-info.java | 26 + .../hadoop/fs/tosfs/util/CommonUtils.java | 46 + .../apache/hadoop/fs/tosfs/util/FSUtils.java | 67 + .../hadoop/fs/tosfs/util/FuseUtils.java | 28 + .../hadoop/fs/tosfs/util/Iterables.java | 128 ++ .../hadoop/fs/tosfs/util/JsonCodec.java | 46 + .../hadoop/fs/tosfs/util/LazyReload.java | 35 + .../hadoop/fs/tosfs/util/LazyReloadIter.java | 69 + .../hadoop/fs/tosfs/util/ParseUtils.java | 65 + .../apache/hadoop/fs/tosfs/util/Range.java | 104 ++ .../apache/hadoop/fs/tosfs/util/Reload.java | 29 + .../hadoop/fs/tosfs/util/RemoteIterators.java | 123 ++ .../hadoop/fs/tosfs/util/RetryableUtils.java | 54 + .../hadoop/fs/tosfs/util/Serializer.java | 25 + .../fs/tosfs/util/TOSClientContextUtils.java | 59 + .../hadoop/fs/tosfs/util/UUIDUtils.java | 31 + .../hadoop/fs/tosfs/util/package-info.java | 26 + .../src/site/markdown/cloud-storage/index.md | 430 +++++ .../org/apache/hadoop/fs/tosfs/TestEnv.java | 36 + .../hadoop/fs/tosfs/TestRawFSUtils.java | 42 + .../hadoop/fs/tosfs/TestRawFileSystem.java | 55 + .../hadoop/fs/tosfs/TestTosChecksum.java | 126 ++ .../hadoop/fs/tosfs/TestTosFileSystem.java | 53 + .../hadoop/fs/tosfs/commit/BaseJobSuite.java | 268 +++ .../fs/tosfs/commit/CommitterTestBase.java | 433 +++++ .../hadoop/fs/tosfs/commit/JobSuite.java | 228 +++ .../hadoop/fs/tosfs/commit/MRJobTestBase.java | 241 +++ .../hadoop/fs/tosfs/commit/TestCommitter.java | 29 + .../hadoop/fs/tosfs/commit/TestMRJob.java | 49 + .../tosfs/commit/TestMagicOutputStream.java | 203 +++ .../commit/mapred/CommitterTestBase.java | 381 +++++ .../fs/tosfs/commit/mapred/JobSuite.java | 227 +++ .../fs/tosfs/commit/mapred/TestCommitter.java | 29 + .../fs/tosfs/contract/TestChecksum.java | 137 ++ .../hadoop/fs/tosfs/contract/TestCreate.java | 40 + .../hadoop/fs/tosfs/contract/TestDelete.java | 69 + .../hadoop/fs/tosfs/contract/TestDistCp.java | 46 + .../fs/tosfs/contract/TestGetFileStatus.java | 169 ++ .../hadoop/fs/tosfs/contract/TestMkdir.java | 39 + .../hadoop/fs/tosfs/contract/TestOpen.java | 75 + .../hadoop/fs/tosfs/contract/TestRename.java | 281 ++++ .../hadoop/fs/tosfs/contract/TestRootDir.java | 37 + .../hadoop/fs/tosfs/contract/TestSeek.java | 37 + .../fs/tosfs/contract/TestUnbuffer.java | 37 + .../hadoop/fs/tosfs/contract/TestXAttr.java | 177 ++ .../hadoop/fs/tosfs/contract/TosContract.java | 61 + .../tosfs/object/ObjectStorageTestBase.java | 108 ++ .../fs/tosfs/object/ObjectTestUtils.java | 127 ++ .../fs/tosfs/object/TestDirectoryStorage.java | 224 +++ .../TestObjectMultiRangeInputStream.java | 451 +++++ .../tosfs/object/TestObjectOutputStream.java | 423 +++++ .../object/TestObjectRangeInputStream.java | 144 ++ .../fs/tosfs/object/TestObjectStorage.java | 1495 +++++++++++++++++ .../object/tos/TestChainTOSInputStream.java | 245 +++ .../tos/TestDelegationClientBuilder.java | 462 +++++ .../tosfs/object/tos/TestTOSInputStream.java | 153 ++ .../object/tos/TestTOSObjectStorage.java | 302 ++++ .../tosfs/object/tos/TestTOSRetryPolicy.java | 210 +++ .../auth/TestAbstractCredentialsProvider.java | 60 + .../TestDefaultCredentialsProviderChain.java | 209 +++ .../TestEnvironmentCredentialsProvider.java | 67 + .../auth/TestSimpleCredentialsProvider.java | 76 + .../hadoop/fs/tosfs/ops/TestBaseFsOps.java | 306 ++++ .../hadoop/fs/tosfs/ops/TestBaseOps.java | 78 + .../hadoop/fs/tosfs/ops/TestDefaultFsOps.java | 104 ++ .../fs/tosfs/ops/TestDirectoryFsOps.java | 83 + .../hadoop/fs/tosfs/ops/TestRenameOp.java | 198 +++ .../hadoop/fs/tosfs/util/TempFiles.java | 98 ++ .../hadoop/fs/tosfs/util/TestFSUtils.java | 66 + .../hadoop/fs/tosfs/util/TestIterables.java | 165 ++ .../hadoop/fs/tosfs/util/TestLazyReload.java | 95 ++ .../hadoop/fs/tosfs/util/TestRange.java | 87 + .../hadoop/fs/tosfs/util/TestUtility.java | 168 ++ .../src/test/resources/contract/tos.xml | 113 ++ .../src/test/resources/core-site.xml | 59 + .../src/test/resources/log4j.properties | 23 + hadoop-cloud-storage-project/pom.xml | 1 + hadoop-project/pom.xml | 6 + hadoop-project/src/site/site.xml | 1 + 169 files changed, 24035 insertions(+) create mode 100644 hadoop-cloud-storage-project/hadoop-tos/dev-support/findbugs-exclude.xml create mode 100644 hadoop-cloud-storage-project/hadoop-tos/pom.xml create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFS.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFSUtils.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFileStatus.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFileSystem.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawLocatedFileStatus.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/TosChecksum.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/TosFS.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/TosFileSystem.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/CommitContext.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/CommitUtils.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/Committer.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/CommitterFactory.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/MagicOutputStream.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/Pending.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/PendingSet.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/SuccessData.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/mapred/Committer.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/mapred/package-info.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/PendingOps.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/PendingOpsFactory.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/RawPendingOps.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/package-info.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/package-info.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/Bytes.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/Chain.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/Tasks.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/ThreadPools.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/package-info.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/ArgumentKey.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/ConfKeys.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/FileStoreKeys.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/TosKeys.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/package-info.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/BucketInfo.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ChecksumInfo.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ChecksumType.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/Constants.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/DirectoryStorage.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/FileStore.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/InputStreamProvider.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/MultipartUpload.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectConstants.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectContent.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectInfo.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectMultiRangeInputStream.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectOutputStream.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectRangeInputStream.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorage.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorageFactory.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectUtils.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/Part.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/PrefixStorage.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/ChecksumMismatchException.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/InvalidObjectKeyException.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/NotAppendableException.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/package-info.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/package-info.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/ListObjectsRequest.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/package-info.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/ListObjectsResponse.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/package-info.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/FileStagingPart.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/StagingPart.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/State.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/package-info.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/ChainTOSInputStream.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClient.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClientBuilder.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/GetObjectOutput.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOS.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOSErrorCodes.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOSInputStream.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOSUtils.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TosObjectInfo.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/AbstractCredentialsProvider.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/CredentialsProvider.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/DefaultCredentialsProviderChain.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/EnvironmentCredentialsProvider.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/ExpireableCredential.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/SimpleCredentialsProvider.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/package-info.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/package-info.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/DefaultFsOps.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/DirectoryFsOps.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/FsOps.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/RenameOp.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/package-info.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/package-info.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/CommonUtils.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/FSUtils.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/FuseUtils.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Iterables.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/JsonCodec.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/LazyReload.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/LazyReloadIter.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/ParseUtils.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Range.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Reload.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/RemoteIterators.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/RetryableUtils.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Serializer.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/TOSClientContextUtils.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/UUIDUtils.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/package-info.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/site/markdown/cloud-storage/index.md create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestEnv.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestRawFSUtils.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestRawFileSystem.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestTosChecksum.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestTosFileSystem.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/BaseJobSuite.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/CommitterTestBase.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/JobSuite.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/MRJobTestBase.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/TestCommitter.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/TestMRJob.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/TestMagicOutputStream.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/mapred/CommitterTestBase.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/mapred/JobSuite.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/mapred/TestCommitter.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestChecksum.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestCreate.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestDelete.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestDistCp.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestGetFileStatus.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestMkdir.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestOpen.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestRename.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestRootDir.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestSeek.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestUnbuffer.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestXAttr.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TosContract.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/ObjectStorageTestBase.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/ObjectTestUtils.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestDirectoryStorage.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectMultiRangeInputStream.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectOutputStream.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectRangeInputStream.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectStorage.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestChainTOSInputStream.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestDelegationClientBuilder.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestTOSInputStream.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestTOSObjectStorage.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestTOSRetryPolicy.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/auth/TestAbstractCredentialsProvider.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/auth/TestDefaultCredentialsProviderChain.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/auth/TestEnvironmentCredentialsProvider.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/auth/TestSimpleCredentialsProvider.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestBaseFsOps.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestBaseOps.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestDefaultFsOps.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestDirectoryFsOps.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestRenameOp.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TempFiles.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestFSUtils.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestIterables.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestLazyReload.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestRange.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestUtility.java create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/resources/contract/tos.xml create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/resources/core-site.xml create mode 100644 hadoop-cloud-storage-project/hadoop-tos/src/test/resources/log4j.properties diff --git a/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml b/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml index eb5352ae556c8..7f77826beaf31 100644 --- a/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml +++ b/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml @@ -125,5 +125,10 @@ hadoop-huaweicloud compile + + org.apache.hadoop + hadoop-tos + compile + diff --git a/hadoop-cloud-storage-project/hadoop-tos/dev-support/findbugs-exclude.xml b/hadoop-cloud-storage-project/hadoop-tos/dev-support/findbugs-exclude.xml new file mode 100644 index 0000000000000..3bfb3c3ff1bd8 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/dev-support/findbugs-exclude.xml @@ -0,0 +1,89 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/hadoop-cloud-storage-project/hadoop-tos/pom.xml b/hadoop-cloud-storage-project/hadoop-tos/pom.xml new file mode 100644 index 0000000000000..559b7241fb64e --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/pom.xml @@ -0,0 +1,183 @@ + + + + 4.0.0 + + org.apache.hadoop + hadoop-project + 3.5.0-SNAPSHOT + ../../hadoop-project + + + hadoop-tos + 3.5.0-SNAPSHOT + Apache Hadoop Volcano Engine Services support + + This module contains code to support integration with Volcano Engine TOS. + It also declares the dependencies needed to work with Volcano Engine services. + + jar + + + UTF-8 + 2.8.9 + + + + + org.apache.hadoop + hadoop-common + provided + + + org.apache.hadoop + hadoop-mapreduce-client-core + provided + + + com.volcengine + ve-tos-java-sdk-hadoop + ${ve-tos-java-sdk.version} + + + org.slf4j + slf4j-api + + + org.jetbrains + annotations + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-core + + + + + + + + org.apache.hadoop + hadoop-common + test + test-jar + + + + org.apache.hadoop + hadoop-mapreduce-examples + test + + + org.apache.hadoop + hadoop-minicluster + test + + + org.apache.hadoop + hadoop-distcp + ${hadoop.version} + test + + + org.apache.hadoop + hadoop-distcp + ${hadoop.version} + test + test-jar + + + + org.assertj + assertj-core + test + + + junit + junit + test + + + org.mockito + mockito-core + 4.11.0 + test + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + classes + 1 + true + 2 + 8 + -Xmx2048m + + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + true + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + com.github.spotbugs + spotbugs-maven-plugin + + true + ${basedir}/dev-support/findbugs-exclude.xml + + Max + + + + + diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFS.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFS.java new file mode 100644 index 0000000000000..6fba99b3d61f1 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFS.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.tosfs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.DelegateToFileSystem; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; + +/** + * The implementation class of the raw AbstractFileSystem. If you want to use object storage as + * YARN’s resource storage dir via the fs.defaultFS configuration property in Hadoop’s + * core-site.xml, you should add this configuration to Hadoop's core-site.xml. + *
+ *  fs.AbstractFileSystem.{scheme}.impl=io.proton.fs.RawFS.
+ * 
+ */ +public class RawFS extends DelegateToFileSystem { + private static final int TOS_DEFAULT_PORT = -1; + + public RawFS(URI uri, Configuration conf) throws IOException, URISyntaxException { + super(uri, new RawFileSystem(), conf, uri.getScheme(), false); + } + + @Override + public int getUriDefaultPort() { + return TOS_DEFAULT_PORT; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFSUtils.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFSUtils.java new file mode 100644 index 0000000000000..45b7694aa2714 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFSUtils.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.tosfs; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.util.Preconditions; + +import java.util.Objects; + +public final class RawFSUtils { + private RawFSUtils() { + } + + /** + * Check whether root is the parent of p. + * + * @param root the root path. + * @param p the path to check. + * @return true means the node is included in the subtree which has the root node. + */ + public static boolean inSubtree(String root, String p) { + return inSubtree(new Path(root), new Path(p)); + } + + /** + * Check whether root is the parent of node. + * + * @param root the root path. + * @param node the path to check. + * @return true means the node is included in the subtree which has the root node. + */ + public static boolean inSubtree(Path root, Path node) { + Preconditions.checkNotNull(root, "Root cannot be null"); + Preconditions.checkNotNull(node, "Node cannot be null"); + if (root.isRoot()) { + return true; + } + + if (Objects.equals(root, node)) { + return true; + } + + while (!node.isRoot()) { + if (Objects.equals(root, node)) { + return true; + } + node = node.getParent(); + } + return false; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFileStatus.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFileStatus.java new file mode 100644 index 0000000000000..5e1a841f5682f --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFileStatus.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.tosfs; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import java.util.Arrays; +import java.util.Objects; + +public class RawFileStatus extends FileStatus { + private final byte[] checksum; + + /** + * File status of directory. + * + * @param length the length of the file, 0 if it is a directory. + * @param isdir whether it is a directory. + * @param blocksize the size of the block. + * @param modificationTime the last modified time. + * @param path the file status path. + * @param owner the owner. + * @param checksum the checksum of the file. + */ + public RawFileStatus( + long length, boolean isdir, long blocksize, + long modificationTime, Path path, String owner, byte[] checksum) { + super(length, isdir, 1, blocksize, modificationTime, path); + setOwner(owner); + setGroup(owner); + this.checksum = checksum; + } + + public byte[] checksum() { + return checksum; + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), Arrays.hashCode(checksum)); + } + + @Override + public boolean equals(Object o) { + if (!(o instanceof RawFileStatus)) { + return false; + } + if (this == o) { + return true; + } + if (!super.equals(o)) { + return false; + } + + RawFileStatus other = (RawFileStatus)o; + return Arrays.equals(checksum, other.checksum); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFileSystem.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFileSystem.java new file mode 100644 index 0000000000000..7b9f76b42ff78 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFileSystem.java @@ -0,0 +1,770 @@ +/* + * 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.tosfs; + +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.FileChecksum; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FsServerDefaults; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.ParentNotDirectoryException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.XAttrSetFlag; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.tosfs.commit.MagicOutputStream; +import org.apache.hadoop.fs.tosfs.common.Bytes; +import org.apache.hadoop.fs.tosfs.common.ThreadPools; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.object.ChecksumInfo; +import org.apache.hadoop.fs.tosfs.object.Constants; +import org.apache.hadoop.fs.tosfs.object.DirectoryStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectInfo; +import org.apache.hadoop.fs.tosfs.object.ObjectMultiRangeInputStream; +import org.apache.hadoop.fs.tosfs.object.ObjectOutputStream; +import org.apache.hadoop.fs.tosfs.object.ObjectRangeInputStream; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectStorageFactory; +import org.apache.hadoop.fs.tosfs.object.ObjectUtils; +import org.apache.hadoop.fs.tosfs.object.exceptions.InvalidObjectKeyException; +import org.apache.hadoop.fs.tosfs.ops.DefaultFsOps; +import org.apache.hadoop.fs.tosfs.ops.DirectoryFsOps; +import org.apache.hadoop.fs.tosfs.ops.FsOps; +import org.apache.hadoop.fs.tosfs.util.FSUtils; +import org.apache.hadoop.fs.tosfs.util.FuseUtils; +import org.apache.hadoop.fs.tosfs.util.Range; +import org.apache.hadoop.fs.tosfs.util.RemoteIterators; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.thirdparty.com.google.common.collect.Iterators; +import org.apache.hadoop.util.DataChecksum; +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.util.Preconditions; +import org.apache.hadoop.util.Progressable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URI; +import java.util.Collections; +import java.util.Date; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import static org.apache.hadoop.fs.XAttrSetFlag.CREATE; +import static org.apache.hadoop.fs.XAttrSetFlag.REPLACE; + +public class RawFileSystem extends FileSystem { + private static final Logger LOG = LoggerFactory.getLogger(RawFileSystem.class); + private static final String MULTIPART_THREAD_POOL_PREFIX = "rawfs-multipart-thread-pool"; + private static final String TASK_THREAD_POOL_PREFIX = "rawfs-task-thread-pool"; + // This is the same as HdfsClientConfigKeys.DFS_BLOCK_SIZE_KEY, we do not + // use that directly because we don't want to introduce the hdfs client library. + private static final String DFS_BLOCK_SIZE_KEY = "dfs.blocksize"; + private static final long DFS_BLOCK_SIZE_DEFAULT = 128 << 20; + + private String scheme; + private String username; + private Path workingDir; + private URI uri; + private String bucket; + private ObjectStorage storage; + // Use for task parallel execution, such as parallel to copy multiple files. + private ExecutorService taskThreadPool; + // Use for file multipart upload only. + private ExecutorService uploadThreadPool; + private FsOps fsOps; + + @Override + public URI getUri() { + return uri; + } + + @Override + public String getScheme() { + return scheme; + } + + @VisibleForTesting + String bucket() { + return bucket; + } + + @Override + public void setConf(Configuration conf) { + super.setConf(conf); + } + + @Override + public FSDataInputStream open(Path path, int bufferSize) throws IOException { + LOG.debug("Opening '{}' for reading.", path); + RawFileStatus status = innerFileStatus(path); + if (status.isDirectory()) { + throw new FileNotFoundException( + String.format("Can't open %s because it is a directory", path)); + } + + // Parse the range size from the hadoop conf. + long rangeSize = getConf().getLong( + ConfKeys.FS_OBJECT_STREAM_RANGE_SIZE, + ConfKeys.FS_OBJECT_STREAM_RANGE_SIZE_DEFAULT); + Preconditions.checkArgument(rangeSize > 0, "Object storage range size must be positive."); + + FSInputStream fsIn = new ObjectMultiRangeInputStream(taskThreadPool, storage, path, + status.getLen(), rangeSize, status.checksum()); + return new FSDataInputStream(fsIn); + } + + public FSDataInputStream open(Path path, byte[] expectedChecksum, Range range) { + return new FSDataInputStream( + new ObjectRangeInputStream(storage, path, range, expectedChecksum)); + } + + @Override + public FSDataOutputStream create(Path path, FsPermission permission, boolean overwrite, + int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { + FileStatus fileStatus = getFileStatusOrNull(path); + if (fileStatus != null) { + if (fileStatus.isDirectory()) { + throw new FileAlreadyExistsException(path + " is a directory"); + } + + if (!overwrite) { + throw new FileAlreadyExistsException(path + " already exists"); + } + LOG.debug("Overwriting file {}", path); + } + + if (MagicOutputStream.isMagic(path)) { + return new FSDataOutputStream( + new MagicOutputStream(this, storage, uploadThreadPool, getConf(), makeQualified(path)), + null); + } else { + ObjectOutputStream out = + new ObjectOutputStream(storage, uploadThreadPool, getConf(), makeQualified(path), true); + + if (fileStatus == null && FuseUtils.fuseEnabled()) { + // The fuse requires the file to be visible when accessing getFileStatus once we created + // the file, so here we close and commit the file to be visible explicitly for fuse, and + // then reopen the file output stream for further data bytes writing. + out.close(); + out = + new ObjectOutputStream(storage, uploadThreadPool, getConf(), makeQualified(path), true); + } + + return new FSDataOutputStream(out, null); + } + } + + @Override + public RemoteIterator listFiles(Path f, boolean recursive) throws IOException { + Path path = makeQualified(f); + LOG.debug("listFiles({}, {})", path, recursive); + + // assume the path is a dir at first, and list sub files + RemoteIterator subFiles = RemoteIterators.fromIterable( + fsOps.listDir(path, recursive, key -> !ObjectInfo.isDir(key)), this::toLocatedFileStatus); + if (!subFiles.hasNext()) { + final RawFileStatus fileStatus = innerFileStatus(path); + if (fileStatus.isFile()) { + return RemoteIterators.fromSingleton(toLocatedFileStatus(fileStatus)); + } + } + return subFiles; + } + + private RawLocatedFileStatus toLocatedFileStatus(RawFileStatus status) throws IOException { + return new RawLocatedFileStatus(status, + status.isFile() ? getFileBlockLocations(status, 0, status.getLen()) : null); + } + + @Override + public FSDataOutputStream createNonRecursive( + Path path, + FsPermission permission, + EnumSet flag, + int bufferSize, + short replication, + long blockSize, + Progressable progress) throws IOException { + Path qualified = makeQualified(path); + return create(qualified, permission, flag.contains(CreateFlag.OVERWRITE), + bufferSize, replication, blockSize, progress); + } + + @Override + public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) + throws IOException { + throw new IOException("Not supported"); + } + + /** + * Rename src path to dest path, if dest path is an existed dir, + * then FS will rename the src path under the dst dir. + * E.g. rename('/a/b', '/a/c') and dest 'c' is an existed dir, + * then the source path '/a/b' will be renamed with dest path '/a/b/c' internally. + * + *
    + *
  • Return false if src doesn't exist
  • + *
  • Return false if src is root
  • + *
  • Return false if dst path is under src path, e.g. rename('/a/b', '/a/b/c')
  • + *
  • Return false if dst path already exists
  • + *
  • Return true if rename('/a/b', '/a/b') and 'b' is an existed file
  • + *
  • Return true if rename('/a/b', '/a') and 'a' is an existed dir, + * fs will rename '/a/b' to '/a/b' internally
  • + *
  • Return false if rename('/a/b', '/a/b') and 'b' is an existed dir, + * because fs will try to rename '/a/b' to '/a/b/b', which is under '/a/b', this behavior + * is forbidden.
  • + *
+ * + * @param src path to be renamed + * @param dst path after rename + * @return true if rename is successful + * @throws IOException on failure + */ + @Override + public boolean rename(Path src, Path dst) throws IOException { + LOG.debug("Rename source path {} to dest path {}", src, dst); + + // 1. Check source and destination path + Future srcStatusFuture = taskThreadPool.submit(() -> checkAndGetSrcStatus(src)); + Future destPathFuture = taskThreadPool.submit(() -> checkAndGetDstPath(src, dst)); + + FileStatus srcStatus; + Path dstPath; + try { + srcStatus = srcStatusFuture.get(); + dstPath = destPathFuture.get(); + + if (src.equals(dstPath)) { + return true; + } + } catch (InterruptedException | ExecutionException e) { + LOG.error("Failed to rename path, src: {}, dst: {}", src, dst, e); + return false; + } + + // 2. Start copy source to destination + if (srcStatus.isDirectory()) { + fsOps.renameDir(srcStatus.getPath(), dstPath); + } else { + fsOps.renameFile(srcStatus.getPath(), dstPath, srcStatus.getLen()); + } + + return true; + } + + private Path checkAndGetDstPath(Path src, Path dest) throws IOException { + FileStatus destStatus = getFileStatusOrNull(dest); + // 1. Rebuilding the destination path + Path finalDstPath = dest; + if (destStatus != null && destStatus.isDirectory()) { + finalDstPath = new Path(dest, src.getName()); + } + + // 2. No need to check the dest path because renaming itself is allowed. + if (src.equals(finalDstPath)) { + return finalDstPath; + } + + // 3. Ensure the source path cannot be the ancestor of destination path. + if (RawFSUtils.inSubtree(src, finalDstPath)) { + throw new IOException(String.format("Failed to rename since it is prohibited to " + + "rename dest path %s under src path %s", finalDstPath, src)); + } + + // 4. Ensure the destination path doesn't exist. + FileStatus finalDstStatus = destStatus; + if (destStatus != null && destStatus.isDirectory()) { + finalDstStatus = getFileStatusOrNull(finalDstPath); + } + if (finalDstStatus != null) { + throw new FileAlreadyExistsException( + String.format("Failed to rename since the dest path %s already exists.", finalDstPath)); + } else { + return finalDstPath; + } + } + + private FileStatus checkAndGetSrcStatus(Path src) throws IOException { + // throw FileNotFoundException if src not found. + FileStatus srcStatus = innerFileStatus(src); + + if (src.isRoot()) { + throw new IOException(String.format("Cannot rename the root directory %s to another name", + src)); + } + return srcStatus; + } + + @Override + public boolean delete(Path f, boolean recursive) throws IOException { + LOG.debug("Delete path {} - recursive {}", f, recursive); + try { + FileStatus fileStatus = getFileStatus(f); + Path path = fileStatus.getPath(); + + if (path.isRoot()) { + return deleteRoot(path, recursive); + } else { + if (fileStatus.isDirectory()) { + fsOps.deleteDir(path, recursive); + } else { + fsOps.deleteFile(path); + } + return true; + } + } catch (FileNotFoundException e) { + LOG.debug("Couldn't delete {} - does not exist", f); + return false; + } + } + + /** + * Reject deleting root directory and implement the specific logic to compatible with + * AbstractContractRootDirectoryTest rm test cases. + * + * @param root the root path. + * @param recursive indicate whether delete directory recursively + * @return true if root directory is empty, false if trying to delete a non-empty dir recursively. + * @throws IOException if trying to delete the non-empty root dir non-recursively. + */ + private boolean deleteRoot(Path root, boolean recursive) throws IOException { + LOG.info("Delete the {} root directory of {}", bucket, recursive); + boolean isEmptyDir = fsOps.isEmptyDirectory(root); + if (isEmptyDir) { + return true; + } + if (recursive) { + // AbstractContractRootDirectoryTest#testRmRootRecursive doesn't expect any exception if + // trying to delete a non-empty root directory recursively, so we have to return false here + // instead of throwing a IOException. + return false; + } else { + // AbstractContractRootDirectoryTest#testRmNonEmptyRootDirNonRecursive expect a exception if + // trying to delete a non-empty root directory non-recursively, so we have to throw a + // IOException instead of returning false. + throw new PathIOException(bucket, "Cannot delete root path"); + } + } + + @Override + public RawFileStatus[] listStatus(Path f) throws IOException { + LOG.debug("List status for path: {}", f); + return Iterators.toArray(listStatus(f, false), RawFileStatus.class); + } + + public Iterator listStatus(Path f, boolean recursive) throws IOException { + Path path = makeQualified(f); + // Assuming path is a dir at first. + Iterator iterator = fsOps.listDir(path, recursive, key -> true).iterator(); + if (iterator.hasNext()) { + return iterator; + } else { + RawFileStatus fileStatus = innerFileStatus(path); + if (fileStatus.isFile()) { + return Collections.singletonList(fileStatus).iterator(); + } else { + // The path is an empty dir. + return Collections.emptyIterator(); + } + } + } + + @Override + public RemoteIterator listStatusIterator(Path p) throws IOException { + // We expect throw FileNotFoundException if the path doesn't exist during creating the + // RemoteIterator instead of throwing FileNotFoundException during call hasNext method. + + // The follow RemoteIterator is as same as {@link FileSystem#DirListingIterator} above + // hadoop 3.2.2, but below 3.2.2, the DirListingIterator fetches the directory entries during + // call hasNext method instead of create the DirListingIterator instance. + return new RemoteIterator() { + private DirectoryEntries entries = listStatusBatch(p, null); + private int index = 0; + + @Override + public boolean hasNext() { + return index < entries.getEntries().length || entries.hasMore(); + } + + private void fetchMore() throws IOException { + byte[] token = entries.getToken(); + entries = listStatusBatch(p, token); + index = 0; + } + + @Override + public FileStatus next() throws IOException { + if (!hasNext()) { + throw new NoSuchElementException("No more items in iterator"); + } else { + if (index == entries.getEntries().length) { + fetchMore(); + if (!hasNext()) { + throw new NoSuchElementException("No more items in iterator"); + } + } + + return entries.getEntries()[index++]; + } + } + }; + } + + public static long dateToLong(final Date date) { + return date == null ? 0L : date.getTime(); + } + + @Override + public Path getWorkingDirectory() { + return workingDir; + } + + @Override + public void setWorkingDirectory(Path newDir) { + this.workingDir = newDir; + } + + @Override + public boolean mkdirs(Path path, FsPermission permission) throws IOException { + try { + FileStatus fileStatus = innerFileStatus(path); + if (fileStatus.isDirectory()) { + return true; + } else { + throw new FileAlreadyExistsException("Path is a file: " + path); + } + } catch (FileNotFoundException e) { + Path dir = makeQualified(path); + validatePath(dir); + fsOps.mkdirs(dir); + } + return true; + } + + private void validatePath(Path path) throws IOException { + Path parent = path.getParent(); + do { + try { + FileStatus fileStatus = innerFileStatus(parent); + if (fileStatus.isDirectory()) { + // If path exists and a directory, exit + break; + } else { + throw new FileAlreadyExistsException(String.format("Can't make directory for path '%s'," + + " it is a file.", parent)); + } + } catch (FileNotFoundException ignored) { + } + parent = parent.getParent(); + } while (parent != null); + } + + @Override + public FileStatus getFileStatus(Path path) throws IOException { + try { + return innerFileStatus(path); + } catch (ParentNotDirectoryException e) { + // Treat ParentNotDirectoryException as FileNotFoundException for the case that check whether + // path exist or not. + throw new FileNotFoundException(e.getMessage()); + } + } + + + /** + * Get the file status of given path. + * + * @param f the path + * @return {@link RawFileStatus} describe file status info. + * @throws FileNotFoundException if the path doesn't exist. + * @throws ParentNotDirectoryException if the path is locating under an existing file, which is + * not allowed in directory bucket case. + */ + RawFileStatus innerFileStatus(Path f) throws ParentNotDirectoryException, FileNotFoundException { + Path qualifiedPath = f.makeQualified(uri, workingDir); + RawFileStatus fileStatus = getFileStatusOrNull(qualifiedPath); + if (fileStatus == null) { + throw new FileNotFoundException( + String.format("No such file or directory: %s", qualifiedPath)); + } + return fileStatus; + } + + /** + * The different with {@link RawFileSystem#getFileStatus(Path)} is that: + * 1. throw {@link ParentNotDirectoryException} if the path is locating under an existing file in + * directory bucket case, but {@link RawFileSystem#getFileStatus(Path)} will ignore whether the + * invalid path and throw {@link FileNotFoundException} + * 2. return null if the path doesn't exist instead of throwing {@link FileNotFoundException}. + * + * @param path the object path. + * @return null if the path doesn't exist. + * @throws ParentNotDirectoryException if the path is locating under an existing file, which is + * not allowed in directory bucket case. + */ + public RawFileStatus getFileStatusOrNull(final Path path) throws ParentNotDirectoryException { + Path qualifiedPath = path.makeQualified(uri, workingDir); + String key = ObjectUtils.pathToKey(qualifiedPath); + + // Root directory always exists + if (key.isEmpty()) { + return new RawFileStatus(0, true, 0, 0, qualifiedPath, username, Constants.MAGIC_CHECKSUM); + } + + try { + ObjectInfo obj = storage.objectStatus(key); + if (obj == null) { + return null; + } else { + return objectToFileStatus(obj); + } + } catch (InvalidObjectKeyException e) { + String msg = + String.format("The object key %s is a invalid key, detail: %s", key, e.getMessage()); + throw new ParentNotDirectoryException(msg); + } + } + + private RawFileStatus objectToFileStatus(ObjectInfo obj) { + Path keyPath = makeQualified(ObjectUtils.keyToPath(obj.key())); + long blockSize = obj.isDir() ? 0 : getDefaultBlockSize(keyPath); + long modificationTime = dateToLong(obj.mtime()); + return new RawFileStatus(obj.size(), obj.isDir(), blockSize, modificationTime, keyPath, + username, obj.checksum()); + } + + @Override + @Deprecated + public long getDefaultBlockSize() { + return getConf().getLongBytes(DFS_BLOCK_SIZE_KEY, DFS_BLOCK_SIZE_DEFAULT); + } + + @Override + public FsServerDefaults getServerDefaults(Path p) { + Configuration config = getConf(); + // CRC32 is chosen as default as it is available in all + // releases that support checksum. + // The client trash configuration is ignored. + return new FsServerDefaults(getDefaultBlockSize(), + config.getInt("dfs.bytes-per-checksum", 512), + 64 * 1024, + getDefaultReplication(), + config.getInt(CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY, + CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT), + false, + CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT, + DataChecksum.Type.CRC32, + ""); + } + + private void stopAllServices() { + ThreadPools.shutdown(uploadThreadPool, 30, TimeUnit.SECONDS); + ThreadPools.shutdown(taskThreadPool, 30, TimeUnit.SECONDS); + } + + @Override + public void initialize(URI name, Configuration conf) throws IOException { + super.initialize(name, conf); + setConf(conf); + this.scheme = FSUtils.scheme(conf, name); + + // Username is the current user at the time the FS was instantiated. + this.username = UserGroupInformation.getCurrentUser().getShortUserName(); + this.workingDir = new Path("/user", username).makeQualified(name, null); + this.uri = URI.create(scheme + "://" + name.getAuthority()); + this.bucket = this.uri.getAuthority(); + this.storage = ObjectStorageFactory.create(scheme, bucket, getConf()); + if (storage.bucket() == null) { + throw new FileNotFoundException(String.format("Bucket: %s not found.", name.getAuthority())); + } + + int taskThreadPoolSize = + getConf().getInt(ConfKeys.FS_TASK_THREAD_POOL_SIZE.key(storage.scheme()), + ConfKeys.FS_TASK_THREAD_POOL_SIZE_DEFAULT); + this.taskThreadPool = ThreadPools.newWorkerPool(TASK_THREAD_POOL_PREFIX, taskThreadPoolSize); + + int uploadThreadPoolSize = + getConf().getInt(ConfKeys.FS_MULTIPART_THREAD_POOL_SIZE.key(storage.scheme()), + ConfKeys.FS_MULTIPART_THREAD_POOL_SIZE_DEFAULT); + this.uploadThreadPool = + ThreadPools.newWorkerPool(MULTIPART_THREAD_POOL_PREFIX, uploadThreadPoolSize); + + if (storage.bucket().isDirectory()) { + fsOps = new DirectoryFsOps((DirectoryStorage) storage, this::objectToFileStatus); + } else { + fsOps = new DefaultFsOps(storage, getConf(), taskThreadPool, this::objectToFileStatus); + } + } + + @Override + public void close() throws IOException { + try { + super.close(); + storage.close(); + } finally { + stopAllServices(); + } + } + + public ObjectStorage storage() { + return storage; + } + + public ExecutorService uploadThreadPool() { + return uploadThreadPool; + } + + /** + * @return null if checksum is not supported. + */ + @Override + public FileChecksum getFileChecksum(Path f, long length) throws IOException { + Preconditions.checkArgument(length >= 0); + + RawFileStatus fileStatus = innerFileStatus(f); + if (fileStatus.isDirectory()) { + // Compatible with HDFS + throw new FileNotFoundException(String.format("Path is not a file, %s", f)); + } + if (!getConf().getBoolean(ConfKeys.FS_CHECKSUM_ENABLED.key(storage.scheme()), + ConfKeys.FS_CHECKSUM_ENABLED_DEFAULT)) { + return null; + } + + ChecksumInfo csInfo = storage.checksumInfo(); + return new TosChecksum(csInfo.algorithm(), fileStatus.checksum()); + } + + @Override + public String getCanonicalServiceName() { + return null; + } + + @Override + public void setXAttr(Path path, String name, byte[] value, EnumSet flag) + throws IOException { + Preconditions.checkNotNull(name, "xAttr name must not be null."); + Preconditions.checkArgument(!name.isEmpty(), "xAttr name must not be empty."); + Preconditions.checkNotNull(value, "xAttr value must not be null."); + + if (getFileStatus(path).isFile()) { + Path qualifiedPath = path.makeQualified(uri, workingDir); + String key = ObjectUtils.pathToKey(qualifiedPath); + + Map existedTags = storage.getTags(key); + validateXAttrFlag(name, existedTags.containsKey(name), flag); + + String newValue = Bytes.toString(value); + String previousValue = existedTags.put(name, newValue); + if (!newValue.equals(previousValue)) { + storage.putTags(key, existedTags); + } + } + } + + @Override + public Map getXAttrs(Path path) throws IOException { + if (getFileStatus(path).isDirectory()) { + return new HashMap<>(); + } else { + Path qualifiedPath = path.makeQualified(uri, workingDir); + String key = ObjectUtils.pathToKey(qualifiedPath); + + Map tags = storage.getTags(key); + return tags.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, t -> Bytes.toBytes(t.getValue()))); + } + } + + @Override + public byte[] getXAttr(Path path, String name) throws IOException { + Map xAttrs = getXAttrs(path); + if (xAttrs.containsKey(name)) { + return xAttrs.get(name); + } else { + throw new IOException("Attribute with name " + name + " is not found."); + } + } + + @Override + public Map getXAttrs(Path path, List names) throws IOException { + Map xAttrs = getXAttrs(path); + xAttrs.keySet().retainAll(names); + if (xAttrs.size() == names.size()) { + return xAttrs; + } else { + List badNames = names.stream().filter(n -> !xAttrs.containsKey(n)).collect( + Collectors.toList()); + throw new IOException("Attributes with name " + badNames + " are not found."); + } + } + + @Override + public List listXAttrs(Path path) throws IOException { + return Lists.newArrayList(getXAttrs(path).keySet()); + } + + @Override + public void removeXAttr(Path path, String name) throws IOException { + if (getFileStatus(path).isFile()) { + Path qualifiedPath = path.makeQualified(uri, workingDir); + String key = ObjectUtils.pathToKey(qualifiedPath); + + Map existedTags = storage.getTags(key); + if (existedTags.remove(name) != null) { + storage.putTags(key, existedTags); + } + } + } + + private void validateXAttrFlag(String xAttrName, boolean xAttrExists, EnumSet flag) + throws IOException { + if (xAttrExists) { + if (!flag.contains(REPLACE)) { + throw new IOException("XAttr: " + xAttrName + " already exists. The REPLACE flag must be" + + " specified."); + } + } else { + if (!flag.contains(CREATE)) { + throw new IOException("XAttr: " + xAttrName + " does not exist. The CREATE flag must be" + + " specified."); + } + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawLocatedFileStatus.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawLocatedFileStatus.java new file mode 100644 index 0000000000000..562289455d317 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawLocatedFileStatus.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.tosfs; + +import org.apache.hadoop.fs.BlockLocation; +import org.apache.hadoop.fs.LocatedFileStatus; + +import static org.apache.hadoop.util.Preconditions.checkNotNull; + +public class RawLocatedFileStatus extends LocatedFileStatus { + public RawLocatedFileStatus(RawFileStatus status, BlockLocation[] locations) { + super(checkNotNull(status), locations); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/TosChecksum.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/TosChecksum.java new file mode 100644 index 0000000000000..f0bedc60c4bc6 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/TosChecksum.java @@ -0,0 +1,67 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs; + +import org.apache.hadoop.fs.FileChecksum; +import org.apache.hadoop.fs.tosfs.common.Bytes; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +public class TosChecksum extends FileChecksum { + private String algorithm; + private byte[] checksum; + + public TosChecksum(String algorithm, byte[] checksum) { + this.algorithm = algorithm; + this.checksum = checksum; + } + + @Override + public String getAlgorithmName() { + return algorithm; + } + + @Override + public int getLength() { + return checksum.length; + } + + @Override + public byte[] getBytes() { + return checksum; + } + + @Override + public void write(DataOutput out) throws IOException { + byte[] algorithmBytes = Bytes.toBytes(algorithm); + out.write(algorithmBytes.length); + out.write(algorithmBytes); + out.write(checksum.length); + out.write(checksum); + } + + @Override + public void readFields(DataInput in) throws IOException { + byte[] algorithmBytes = new byte[in.readInt()]; + in.readFully(algorithmBytes); + algorithm = Bytes.toString(algorithmBytes); + checksum = new byte[in.readInt()]; + in.readFully(checksum); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/TosFS.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/TosFS.java new file mode 100644 index 0000000000000..3146e2d6cb7a5 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/TosFS.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.tosfs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.tosfs.object.tos.TOS; +import org.apache.hadoop.util.Preconditions; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; + +public class TosFS extends RawFS { + public TosFS(URI uri, Configuration conf) throws IOException, URISyntaxException { + super(verifyURI(uri, conf), conf); + } + + private static URI verifyURI(URI uri, Configuration conf) { + Preconditions.checkNotNull(uri); + + String scheme = uri.getScheme(); + if (scheme == null || scheme.isEmpty()) { + scheme = FileSystem.getDefaultUri(conf).getScheme(); + } + Preconditions.checkArgument(scheme.equals(TOS.TOS_SCHEME), + "Unsupported scheme %s, expected scheme is %s.", scheme, TOS.TOS_SCHEME); + + return uri; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/TosFileSystem.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/TosFileSystem.java new file mode 100644 index 0000000000000..dc68b6733a89f --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/TosFileSystem.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.tosfs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.tosfs.object.tos.TOS; +import org.apache.hadoop.util.Preconditions; + +import java.io.IOException; +import java.net.URI; + +public class TosFileSystem extends RawFileSystem { + @Override + public void initialize(URI name, Configuration conf) throws IOException { + Preconditions.checkNotNull(name); + + String scheme = name.getScheme(); + if (scheme == null || scheme.isEmpty()) { + scheme = FileSystem.getDefaultUri(conf).getScheme(); + } + Preconditions.checkArgument(scheme.equals(TOS.TOS_SCHEME), + "Unsupported scheme %s, expected scheme is %s.", scheme, TOS.TOS_SCHEME); + + super.initialize(name, conf); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/CommitContext.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/CommitContext.java new file mode 100644 index 0000000000000..40d371bcf882f --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/CommitContext.java @@ -0,0 +1,45 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.util.Lists; + +import java.util.List; + +public class CommitContext { + private final List pendingSets; + // It will be accessed in multi-threads, please access it in a thread-safe context. + private final List destKeys; + + public CommitContext(List pendingSets) { + this.pendingSets = pendingSets; + this.destKeys = Lists.newArrayList(); + } + + public List pendingSets() { + return pendingSets; + } + + public synchronized void addDestKey(String destKey) { + destKeys.add(destKey); + } + + public synchronized List destKeys() { + return destKeys; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/CommitUtils.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/CommitUtils.java new file mode 100644 index 0000000000000..e592fc43d0b11 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/CommitUtils.java @@ -0,0 +1,370 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit; + +import org.apache.commons.io.IOUtils; +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.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.tosfs.commit.mapred.Committer; +import org.apache.hadoop.fs.tosfs.util.Serializer; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.util.Preconditions; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.function.Supplier; + +public final class CommitUtils { + private CommitUtils() { + } + + public static final String COMMITTER_NAME = Committer.class.getName(); + + /** + * Support scheme for tos committer. + */ + public static final String FS_STORAGE_OBJECT_SCHEME = "fs.object-storage.scheme"; + public static final String DEFAULT_FS_STORAGE_OBJECT_SCHEME = "tos,oss,s3,s3a,s3n,obs,filestore"; + + /** + * Path for "magic" writes: path and {@link #PENDING_SUFFIX} files: {@value}. + */ + public static final String MAGIC = "__magic"; + + /** + * Marker of the start of a directory tree for calculating the final path names: {@value}. + */ + public static final String BASE = "__base"; + + /** + * Suffix applied to pending commit metadata: {@value}. + */ + public static final String PENDING_SUFFIX = ".pending"; + + /** + * Suffix applied to multiple pending commit metadata: {@value}. + */ + public static final String PENDINGSET_SUFFIX = ".pendingset"; + + /** + * Marker file to create on success: {@value}. + */ + public static final String SUCCESS = "_SUCCESS"; + + /** + * Format string used to build a summary file from a Job ID. + */ + public static final String SUMMARY_FILENAME_FORMAT = "summary-%s.json"; + + /** + * Extra Data key for task attempt in pendingset files. + */ + public static final String TASK_ATTEMPT_ID = "task.attempt.id"; + + /** + * The UUID for jobs: {@value}. + * This was historically created in Spark 1.x's SQL queries, see SPARK-33230. + */ + public static final String SPARK_WRITE_UUID = "spark.sql.sources.writeJobUUID"; + + /** + * Get the magic location for the output path. + * Format: ${out}/__magic + * + * @param out the base output directory. + * @return the location of magic job attempts. + */ + public static Path magicPath(Path out) { + return new Path(out, MAGIC); + } + + /** + * Compute the "magic" path for a job.
+ * Format: ${jobOutput}/__magic/${jobId} + * + * @param jobId unique Job ID. + * @param jobOutput the final output directory. + * @return the path to store job attempt data. + */ + public static Path magicJobPath(String jobId, Path jobOutput) { + return new Path(magicPath(jobOutput), jobId); + } + + /** + * Get the Application Attempt ID for this job. + * + * @param context the context to look in + * @return the Application Attempt ID for a given job, or 0 + */ + public static int appAttemptId(JobContext context) { + return context.getConfiguration().getInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 0); + } + + /** + * Compute the "magic" path for a job attempt.
+ * Format: ${jobOutput}/__magic/${jobId}/${appAttemptId} + * + * @param jobId unique Job ID. + * @param appAttemptId the ID of the application attempt for this job. + * @param jobOutput the final output directory. + * @return the path to store job attempt data. + */ + public static Path magicJobAttemptPath(String jobId, int appAttemptId, Path jobOutput) { + return new Path(magicPath(jobOutput), formatAppAttemptDir(jobId, appAttemptId)); + } + + /** + * Compute the "magic" path for a job attempt.
+ * Format: ${jobOutput}/__magic/${jobId}/${appAttemptId} + * + * @param context the context of the job. + * @param jobOutput the final output directory. + * @return the path to store job attempt data. + */ + public static Path magicJobAttemptPath(JobContext context, Path jobOutput) { + String jobId = buildJobId(context); + return magicJobAttemptPath(jobId, appAttemptId(context), jobOutput); + } + + private static String formatAppAttemptDir(String jobId, int appAttemptId) { + return String.format("%s/%02d", jobId, appAttemptId); + } + + /** + * Compute the path where the output of magic task attempts are stored.
+ * Format: ${jobOutput}/__magic/${jobId}/${appAttemptId}/tasks + * + * @param jobId unique Job ID. + * @param jobOutput The output path to commit work into. + * @param appAttemptId the ID of the application attempt for this job. + * @return the path where the output of magic task attempts are stored. + */ + public static Path magicTaskAttemptsPath(String jobId, Path jobOutput, int appAttemptId) { + return new Path(magicJobAttemptPath(jobId, appAttemptId, jobOutput), "tasks"); + } + + /** + * Compute the path where the output of a task attempt is stored until that task is committed. + * This path is marked as a base path for relocations, so subdirectory information is preserved. + * Format: ${jobOutput}/__magic/${jobId}/${appAttemptId}/tasks/${taskAttemptId}/__base + * + * @param context the context of the task attempt. + * @param jobId unique Job ID. + * @param jobOutput The output path to commit work into. + * @return the path where a task attempt should be stored. + */ + public static Path magicTaskAttemptBasePath(TaskAttemptContext context, String jobId, + Path jobOutput) { + return new Path(magicTaskAttemptPath(context, jobId, jobOutput), BASE); + } + + /** + * Compute the path where the output of a task attempt is stored until that task is committed. + * This path is marked as a base path for relocations, so subdirectory information is preserved. + * Format: ${jobOutput}/__magic/${jobId}/${appAttemptId}/tasks/${taskAttemptId}/__base + * + * @param context the context of the task attempt. + * @param jobOutput The output path to commit work into. + * @return the path where a task attempt should be stored. + */ + public static Path magicTaskAttemptBasePath(TaskAttemptContext context, Path jobOutput) { + String jobId = buildJobId(context); + return magicTaskAttemptBasePath(context, jobId, jobOutput); + } + + /** + * Get the magic task attempt path, without any annotations to mark relative references. + * If there is an app attempt property in the context configuration, that is included. + * Format: ${jobOutput}/__magic/${jobId}/${appAttemptId}/tasks/${taskAttemptId} + * + * @param context the context of the task attempt. + * @param jobId unique Job ID. + * @param jobOutput The output path to commit work into. + * @return the path under which all attempts go. + */ + public static Path magicTaskAttemptPath(TaskAttemptContext context, String jobId, + Path jobOutput) { + return new Path(magicTaskAttemptsPath(jobId, jobOutput, appAttemptId(context)), + String.valueOf(context.getTaskAttemptID())); + } + + /** + * Get the magic task attempt path, without any annotations to mark relative references. + * If there is an app attempt property in the context configuration, that is included. + * Format: ${jobOutput}/__magic/${jobId}/${appAttemptId}/tasks/${taskAttemptId} + * + * @param context the context of the task attempt. + * @param jobOutput The output path to commit work into. + * @return the path under which all attempts go. + */ + public static Path magicTaskAttemptPath(TaskAttemptContext context, Path jobOutput) { + String jobId = buildJobId(context); + return magicTaskAttemptPath(context, jobId, jobOutput); + } + + /** + * Get the magic task pendingset path. + * Format: ${jobOutput}/__magic/${jobId}/${appAttemptId}/${taskId}.pendingset + * + * @param context the context of the task attempt. + * @param jobOutput The output path to commit work into. + * @return the magic pending set path. + */ + public static Path magicTaskPendingSetPath(TaskAttemptContext context, Path jobOutput) { + String taskId = String.valueOf(context.getTaskAttemptID().getTaskID()); + return new Path(magicJobAttemptPath(context, jobOutput), + String.format("%s%s", taskId, PENDINGSET_SUFFIX)); + } + + public static String buildJobId(Configuration conf, JobID jobId) { + String jobUUID = conf.getTrimmed(SPARK_WRITE_UUID, ""); + if (!jobUUID.isEmpty()) { + if (jobUUID.startsWith(JobID.JOB)) { + return jobUUID; + } else { + return String.format("%s_%s", JobID.JOB, jobUUID); + } + } + + // if no other option was supplied, return the job ID. + // This is exactly what MR jobs expect, but is not what + // Spark jobs can do as there is a risk of jobID collision. + return jobId != null ? jobId.toString() : "NULL_JOB_ID"; + } + + public static String buildJobId(JobContext context) { + return buildJobId(context.getConfiguration(), context.getJobID()); + } + + /** + * Get a job name; returns meaningful text if there is no name. + * + * @param context job context + * @return a string for logs + */ + public static String jobName(JobContext context) { + String name = context.getJobName(); + return (name != null && !name.isEmpty()) ? name : "(anonymous)"; + } + + /** + * Format: ${output}/_SUCCESS. + * + * @param output the output path. + * @return the success marker file path. + */ + public static Path successMarker(Path output) { + return new Path(output, SUCCESS); + } + + /** + * Format: ${reportDir}/summary-xxxxx.json. + * + * @param reportDir the report directory. + * @param jobId the job id. + * @return the summary report file path. + */ + public static Path summaryReport(Path reportDir, String jobId) { + return new Path(reportDir, String.format(SUMMARY_FILENAME_FORMAT, jobId)); + } + + public static void save(FileSystem fs, Path path, byte[] data) throws IOException { + // By default, fs.create(path) will create parent folder recursively, and overwrite + // it if it's already exist. + try (FSDataOutputStream out = fs.create(path)) { + IOUtils.copy(new ByteArrayInputStream(data), out); + } + } + + public static void save(FileSystem fs, Path path, Serializer instance) throws IOException { + save(fs, path, instance.serialize()); + } + + public static byte[] load(FileSystem fs, Path path) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + try (FSDataInputStream in = fs.open(path)) { + IOUtils.copy(in, out); + } + return out.toByteArray(); + } + + public static List listPendingFiles(FileSystem fs, Path dir) throws IOException { + List pendingFiles = Lists.newArrayList(); + CommitUtils.listFiles(fs, dir, true, f -> { + if (f.getPath().toString().endsWith(CommitUtils.PENDING_SUFFIX)) { + pendingFiles.add(f); + } + }); + return pendingFiles; + } + + public static void listFiles(FileSystem fs, Path dir, boolean recursive, FileVisitor visitor) + throws IOException { + RemoteIterator iter = fs.listFiles(dir, recursive); + while (iter.hasNext()) { + FileStatus f = iter.next(); + visitor.visit(f); + } + } + + public interface FileVisitor { + void visit(FileStatus f); + } + + public static boolean supportObjectStorageCommit(Configuration conf, Path outputPath) { + return supportSchemes(conf).contains(outputPath.toUri().getScheme()); + } + + private static List supportSchemes(Configuration conf) { + String schemes = conf.get(FS_STORAGE_OBJECT_SCHEME, DEFAULT_FS_STORAGE_OBJECT_SCHEME); + Preconditions.checkNotNull(schemes, "%s cannot be null", FS_STORAGE_OBJECT_SCHEME); + return Arrays.asList(schemes.split(",")); + } + + private static Set errorStage = new HashSet<>(); + private static boolean testMode = false; + + public static void injectError(String stage) { + errorStage.add(stage); + testMode = true; + } + + public static void removeError(String stage) { + errorStage.remove(stage); + } + + public static void triggerError(Supplier error, String stage) throws T { + if (testMode && errorStage.contains(stage)) { + throw error.get(); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/Committer.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/Committer.java new file mode 100644 index 0000000000000..fbea4c59f40ab --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/Committer.java @@ -0,0 +1,510 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.commit.ops.PendingOps; +import org.apache.hadoop.fs.tosfs.commit.ops.PendingOpsFactory; +import org.apache.hadoop.fs.tosfs.common.Tasks; +import org.apache.hadoop.fs.tosfs.common.ThreadPools; +import org.apache.hadoop.fs.tosfs.object.MultipartUpload; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectStorageFactory; +import org.apache.hadoop.fs.tosfs.object.ObjectUtils; +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.JobStatus; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.thirdparty.com.google.common.base.MoreObjects; +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ExecutorService; + +public class Committer extends PathOutputCommitter { + private static final Logger LOG = LoggerFactory.getLogger(Committer.class); + + public static final String COMMITTER_THREADS = "fs.job.committer.threads"; + public static final String COMMITTER_SUMMARY_REPORT_DIR = + "fs.job.committer.summary.report.directory"; + public static final int DEFAULT_COMMITTER_THREADS = Runtime.getRuntime().availableProcessors(); + public static final String THREADS_PREFIX = "job-committer-thread-pool"; + + private final String jobId; + private final Path outputPath; + // This is the directory for all intermediate work, where the output format will write data. + // This may not be on the final file system + private Path workPath; + private final String role; + private final Configuration conf; + private final FileSystem destFs; + private final ObjectStorage storage; + private final PendingOps ops; + + public Committer(Path outputPath, TaskAttemptContext context) throws IOException { + this(outputPath, context, String.format("Task committer %s", context.getTaskAttemptID())); + this.workPath = CommitUtils.magicTaskAttemptBasePath(context, outputPath); + LOG.info("Task attempt {} has work path {}", context.getTaskAttemptID(), getWorkPath()); + } + + public Committer(Path outputPath, JobContext context) throws IOException { + this(outputPath, context, String.format("Job committer %s", context.getJobID())); + } + + private Committer(Path outputPath, JobContext context, String role) throws IOException { + super(outputPath, context); + this.jobId = CommitUtils.buildJobId(context); + this.outputPath = outputPath; + this.role = role; + this.conf = context.getConfiguration(); + this.destFs = outputPath.getFileSystem(conf); + LOG.info("{} instantiated for job '{}' ID {} with destination {}", + role, + CommitUtils.jobName(context), + jobId, outputPath); + // Initialize the object storage. + this.storage = ObjectStorageFactory.create(outputPath.toUri().getScheme(), + outputPath.toUri().getAuthority(), conf); + this.ops = PendingOpsFactory.create(destFs, storage); + } + + @Override + public Path getOutputPath() { + return outputPath; + } + + @Override + public Path getWorkPath() { + return workPath; + } + + @Override + public void setupJob(JobContext context) throws IOException { + checkJobId(context); + LOG.info("Setup Job {}", jobId); + Path jobOutput = getOutputPath(); + + // delete the success marker if exists. + destFs.delete(CommitUtils.successMarker(jobOutput), false); + + // create the destination directory. + destFs.mkdirs(jobOutput); + + logUncompletedMPUIfPresent(jobOutput); + + // Reset the job path, and create the job path with job attempt sub path. + Path jobPath = CommitUtils.magicJobPath(jobId, outputPath); + Path jobAttemptPath = CommitUtils.magicJobAttemptPath(context, outputPath); + destFs.delete(jobPath, true); + destFs.mkdirs(jobAttemptPath); + } + + private void logUncompletedMPUIfPresent(Path jobOutput) { + // do a scan and add warn log message for active uploads. + int nums = 0; + for (MultipartUpload upload : storage.listUploads(ObjectUtils.pathToKey(jobOutput, true))) { + if (nums++ > 10) { + LOG.warn("There are more than 10 uncompleted multipart uploads under path {}.", jobOutput); + break; + } + LOG.warn("Uncompleted multipart upload {} is under path {}, either jobs are running" + + " concurrently or failed jobs are not being cleaned up.", upload, jobOutput); + } + } + + @Override + public void commitJob(JobContext context) throws IOException { + checkJobId(context); + LOG.info("{}: committing job {}", role, jobId); + String stage = null; + Exception failure = null; + SuccessData successData = null; + + ExecutorService threadPool = ThreadPools.newWorkerPool(THREADS_PREFIX, commitThreads()); + List pendingSets = Lists.newArrayList(); + try { + // Step.1 List active pending commits. + stage = "preparing"; + CommitUtils.listFiles(destFs, CommitUtils.magicJobAttemptPath(context, outputPath), true, + f -> { + if (f.getPath().toString().endsWith(CommitUtils.PENDINGSET_SUFFIX)) { + pendingSets.add(f); + } + }); + + // Step.2 Load and commit those active pending commits. + stage = "commit"; + CommitContext commitCtxt = new CommitContext(pendingSets); + loadAndCommitPendingSets(threadPool, commitCtxt); + + // Step.3 Save the success marker. + stage = "marker"; + successData = createSuccessData(commitCtxt.destKeys()); + CommitUtils.triggerError(() -> new IOException("Mock error of success marker."), stage); + CommitUtils.save(destFs, CommitUtils.successMarker(outputPath), successData); + + // Step.4 Abort those orphan multipart uploads and cleanup the staging dir. + stage = "clean"; + cleanup(threadPool, true); + } catch (Exception e) { + failure = e; + LOG.warn("Commit failure for job {} stage {}", CommitUtils.buildJobId(context), stage, e); + + // Revert all pending sets when marker step fails. + if (stage.equals("marker")) { + CommonUtils.runQuietly( + () -> loadAndRevertPendingSets(threadPool, new CommitContext(pendingSets))); + } + CommonUtils.runQuietly(() -> cleanup(threadPool, true)); + throw e; + } finally { + saveSummaryReportQuietly(stage, context, successData, failure); + CommonUtils.runQuietly(threadPool::shutdown); + + cleanupResources(); + } + } + + private SuccessData createSuccessData(Iterable filenames) { + SuccessData data = SuccessData.builder() + .setName(SuccessData.class.getName()) + .setCommitter(CommitUtils.COMMITTER_NAME) + .setTimestamp(System.currentTimeMillis()) + .setHostname(NetUtils.getHostname()) + .setDescription(role) + .setJobId(jobId) + .addFileNames(filenames) + .build(); + + data.addDiagnosticInfo(COMMITTER_THREADS, Integer.toString(commitThreads())); + return data; + } + + private void saveSummaryReportQuietly(String activeStage, JobContext context, SuccessData report, + Throwable thrown) { + Configuration jobConf = context.getConfiguration(); + String reportDir = jobConf.get(COMMITTER_SUMMARY_REPORT_DIR, ""); + if (reportDir.isEmpty()) { + LOG.debug("Summary directory conf: {} is not set", COMMITTER_SUMMARY_REPORT_DIR); + return; + } + + Path path = CommitUtils.summaryReport(new Path(reportDir), jobId); + LOG.debug("Summary report path is {}", path); + + try { + if (report == null) { + report = createSuccessData(null); + } + if (thrown != null) { + report.recordJobFailure(thrown); + } + report.addDiagnosticInfo("stage", activeStage); + + CommitUtils.save(path.getFileSystem(jobConf), path, report); + LOG.info("Job summary saved to {}", path); + } catch (Exception e) { + LOG.warn("Failed to save summary to {}", path, e); + } + } + + private void loadAndCommitPendingSets(ExecutorService outerPool, CommitContext commitContext) { + ExecutorService innerPool = + ThreadPools.newWorkerPool("commit-pending-files-pool", commitThreads()); + try { + Tasks.foreach(commitContext.pendingSets()) + .stopOnFailure() + .throwFailureWhenFinished() + .executeWith(outerPool) + .abortWith(pendingSet -> loadAndAbort(innerPool, pendingSet)) + .revertWith(pendingSet -> loadAndRevert(innerPool, pendingSet)) + .run(pendingSet -> loadAndCommit(commitContext, innerPool, pendingSet)); + } finally { + CommonUtils.runQuietly(innerPool::shutdown); + } + } + + private void loadAndRevertPendingSets(ExecutorService outerPool, CommitContext commitContext) { + Tasks.foreach(commitContext.pendingSets()) + .throwFailureWhenFinished() + .executeWith(outerPool) + .run(pendingSet -> loadAndRevert(outerPool, pendingSet)); + } + + /** + * Load {@link PendingSet} from file and abort those {@link Pending} commits. + */ + private void loadAndAbort(ExecutorService pool, FileStatus pendingSetFile) { + PendingSet pendingSet = PendingSet.deserialize(destFs, pendingSetFile); + Tasks.foreach(pendingSet.commits()) + .suppressFailureWhenFinished() + .executeWith(pool) + .run(ops::abort); + } + + /** + * Load {@link PendingSet} from file and revert those {@link Pending} commits. + */ + private void loadAndRevert(ExecutorService pool, FileStatus pendingSetFile) { + PendingSet pendingSet = PendingSet.deserialize(destFs, pendingSetFile); + Tasks.foreach(pendingSet.commits()) + .suppressFailureWhenFinished() + .executeWith(pool) + .run(ops::revert); + } + + /** + * Load {@link PendingSet} from file and commit those {@link Pending} commits. + */ + private void loadAndCommit(CommitContext commitCtxt, ExecutorService pool, + FileStatus pendingSetFile) { + PendingSet pendingSet = PendingSet.deserialize(destFs, pendingSetFile); + // Verify that whether the job id is matched. + String jobID = pendingSet.jobId(); + if (!StringUtils.isNoneEmpty(jobID) && !Objects.equals(jobID, jobId())) { + throw new IllegalStateException( + String.format("Mismatch in Job ID (%s) and commit job ID (%s)", jobId(), jobID)); + } + + Tasks.foreach(pendingSet.commits()) + .stopOnFailure() + .throwFailureWhenFinished() + .executeWith(pool) + .onFailure((pending, exception) -> ops.abort(pending)) + .abortWith(ops::abort) + .revertWith(ops::revert) + .run(pending -> { + ops.commit(pending); + commitCtxt.addDestKey(pending.destKey()); + }); + } + + @Override + public void abortJob(JobContext context, JobStatus.State state) { + checkJobId(context); + LOG.info("{}: aborting job {} in state {}", role, jobId, state); + ExecutorService service = ThreadPools.newWorkerPool(THREADS_PREFIX, commitThreads()); + try { + cleanup(service, false); + } finally { + service.shutdown(); + + cleanupResources(); + } + } + + @Override + public void setupTask(TaskAttemptContext context) throws IOException { + checkJobId(context); + LOG.info("Setup Task {}", context.getTaskAttemptID()); + Path taskAttemptBasePath = CommitUtils.magicTaskAttemptBasePath(context, outputPath); + // Delete the task attempt path if somehow it was there. + destFs.delete(taskAttemptBasePath, true); + // Make an empty directory. + destFs.mkdirs(taskAttemptBasePath); + } + + @Override + public boolean needsTaskCommit(TaskAttemptContext taskContext) { + return true; + } + + @Override + public void commitTask(TaskAttemptContext context) throws IOException { + checkJobId(context); + LOG.info("Commit task {}", context); + ExecutorService pool = ThreadPools.newWorkerPool(THREADS_PREFIX, commitThreads()); + try { + PendingSet commits = innerCommitTask(pool, context); + LOG.info("Task {} committed {} files", context.getTaskAttemptID(), commits.size()); + } catch (IOException e) { + LOG.error("Failed to commit task {}", context.getTaskAttemptID(), e); + throw e; + } finally { + // Shutdown the thread pool quietly. + CommonUtils.runQuietly(pool::shutdown); + + // Delete the task attempt path quietly. + Path taskAttemptPath = CommitUtils.magicTaskAttemptPath(context, outputPath); + LOG.info("Delete task attempt path {}", taskAttemptPath); + CommonUtils.runQuietly(() -> destFs.delete(taskAttemptPath, true)); + } + } + + private PendingSet innerCommitTask(ExecutorService pool, TaskAttemptContext context) + throws IOException { + Path taskAttemptBasePath = CommitUtils.magicTaskAttemptBasePath(context, outputPath); + PendingSet pendingSet = new PendingSet(jobId); + try { + // Load the pending files and fill them into the pending set. + List pendingFiles = CommitUtils.listPendingFiles(destFs, taskAttemptBasePath); + // Use the thread-safe collection to collect the pending list. + List pendings = Collections.synchronizedList(Lists.newArrayList()); + Tasks.foreach(pendingFiles) + .throwFailureWhenFinished() + .executeWith(pool) + .run(f -> { + try { + byte[] data = CommitUtils.load(destFs, f.getPath()); + pendings.add(Pending.deserialize(data)); + } catch (IOException e) { + LOG.warn("Failed to load .pending file {}", f.getPath(), e); + throw new UncheckedIOException(e); + } + }); + pendingSet.addAll(pendings); + + // Add the extra task attempt id property. + String taskId = String.valueOf(context.getTaskAttemptID()); + pendingSet.addExtraData(CommitUtils.TASK_ATTEMPT_ID, taskId); + + // Save the pending set to file system. + Path taskOutput = CommitUtils.magicTaskPendingSetPath(context, outputPath); + LOG.info("Saving work of {} to {}", taskId, taskOutput); + CommitUtils.save(destFs, taskOutput, pendingSet.serialize()); + + } catch (Exception e) { + LOG.error("Encounter error when loading pending set from {}", taskAttemptBasePath, e); + if (!pendingSet.commits().isEmpty()) { + Tasks.foreach(pendingSet.commits()) + .executeWith(pool) + .suppressFailureWhenFinished() + .run(ops::abort); + } + throw e; + } + + return pendingSet; + } + + @Override + public void abortTask(TaskAttemptContext context) throws IOException { + checkJobId(context); + Path taskAttemptBasePath = CommitUtils.magicTaskAttemptBasePath(context, outputPath); + try { + // Load the pending files from the underlying filesystem. + List pendingFiles = CommitUtils.listPendingFiles(destFs, taskAttemptBasePath); + Tasks.foreach(pendingFiles) + .throwFailureWhenFinished() + .run(f -> { + try { + byte[] serializedData = CommitUtils.load(destFs, f.getPath()); + ops.abort(Pending.deserialize(serializedData)); + } catch (FileNotFoundException e) { + LOG.debug("Listed file already deleted: {}", f); + } catch (IOException e) { + throw new UncheckedIOException(e); + } finally { + final FileStatus pendingFile = f; + CommonUtils.runQuietly(() -> destFs.delete(pendingFile.getPath(), false)); + } + }); + } finally { + CommonUtils.runQuietly(() -> destFs.delete(taskAttemptBasePath, true)); + } + } + + @Override + public void recoverTask(TaskAttemptContext context) { + checkJobId(context); + String taskId = context.getTaskAttemptID().toString(); + throw new UnsupportedOperationException( + String.format("Unable to recover task %s, output: %s", taskId, outputPath)); + } + + private int commitThreads() { + return conf.getInt(COMMITTER_THREADS, DEFAULT_COMMITTER_THREADS); + } + + private void cleanup(ExecutorService pool, boolean suppress) { + LOG.info("Cleanup the job by abort the multipart uploads and clean staging dir, suppress {}", + suppress); + try { + Path jobOutput = getOutputPath(); + Iterable pending = storage.listUploads( + ObjectUtils.pathToKey(CommitUtils.magicJobPath(jobId, jobOutput), true)); + Tasks.foreach(pending) + .executeWith(pool) + .suppressFailureWhenFinished() + .run(u -> storage.abortMultipartUpload(u.key(), u.uploadId())); + } catch (Exception e) { + if (suppress) { + LOG.error("The following exception has been suppressed when cleanup job", e); + } else { + throw e; + } + } finally { + CommonUtils.runQuietly(this::cleanupStagingDir); + } + } + + private void cleanupStagingDir() throws IOException { + // Note: different jobs share the same __magic folder, like, + // tos://bucket/path/to/table/__magic/job-A/..., and + // tos://bucket/path/to/table/__magic/job-B/... + // Job should only delete its own job folder to avoid the failure of other jobs, + // and, folder __magic should be deleted by the last job. + // This design does not assure the security of two jobs that one job founds there + // isn't another job be running, however, when it is deleting __magic but another + // job will visit it at the same time. We think the probability is low and we don't + // deal with it. + destFs.delete(CommitUtils.magicJobPath(jobId, outputPath), true); + Path magicPath = CommitUtils.magicPath(outputPath); + if (destFs.listStatus(magicPath).length == 0) { + destFs.delete(magicPath, true); + } + } + + public String jobId() { + return jobId; + } + + private void checkJobId(JobContext context) { + String jobIdInContext = CommitUtils.buildJobId(context); + Preconditions.checkArgument(Objects.equals(jobId, jobIdInContext), String.format( + "JobId set in the context: %s is not consistent with the initial jobId of the committer:" + + " %s, please check you settings in your taskAttemptContext.", + jobIdInContext, jobId)); + } + + private void cleanupResources() { + CommonUtils.runQuietly(storage::close); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("role", role) + .add("jobId", jobId) + .add("outputPath", outputPath) + .toString(); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/CommitterFactory.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/CommitterFactory.java new file mode 100644 index 0000000000000..d85b1b5c9afd4 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/CommitterFactory.java @@ -0,0 +1,33 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter; +import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory; + +import java.io.IOException; + +public class CommitterFactory extends PathOutputCommitterFactory { + + @Override + public PathOutputCommitter createOutputCommitter(Path outputPath, + TaskAttemptContext context) throws IOException { + return new Committer(outputPath, context); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/MagicOutputStream.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/MagicOutputStream.java new file mode 100644 index 0000000000000..1bb22efc3beaa --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/MagicOutputStream.java @@ -0,0 +1,151 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.object.ObjectOutputStream; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectStorageFactory; +import org.apache.hadoop.fs.tosfs.object.ObjectUtils; +import org.apache.hadoop.fs.tosfs.object.Part; +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.util.Preconditions; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.stream.Collectors; + +public class MagicOutputStream extends ObjectOutputStream { + + private final FileSystem fs; + private final Path pendingPath; + private boolean closeStorage = false; + + public MagicOutputStream(FileSystem fs, ExecutorService threadPool, Configuration conf, + Path magic) { + this(fs, + ObjectStorageFactory.create(magic.toUri().getScheme(), magic.toUri().getHost(), conf), + threadPool, + conf, + magic); + closeStorage = true; + } + + public MagicOutputStream(FileSystem fs, ObjectStorage storage, ExecutorService threadPool, + Configuration conf, Path magic) { + super(storage, threadPool, conf, magic, false); + this.fs = fs; + this.pendingPath = createPendingPath(magic); + } + + static String toDestKey(Path magicPath) { + Preconditions.checkArgument(isMagic(magicPath), "Destination path is not magic %s", magicPath); + String magicKey = ObjectUtils.pathToKey(magicPath); + List splits = Lists.newArrayList(magicKey.split("/")); + + // Break the full splits list into three collections: , __magic, + int magicIndex = splits.indexOf(CommitUtils.MAGIC); + Preconditions.checkArgument(magicIndex >= 0, "Cannot locate %s in path %s", CommitUtils.MAGIC, + magicPath); + List parentSplits = splits.subList(0, magicIndex); + List childrenSplits = splits.subList(magicIndex + 1, splits.size()); + Preconditions.checkArgument(!childrenSplits.isEmpty(), + "No path found under %s for path %s", CommitUtils.MAGIC, magicPath); + + // Generate the destination splits which will be joined into the destination object key. + List destSplits = Lists.newArrayList(parentSplits); + if (childrenSplits.contains(CommitUtils.BASE)) { + // Break the into three collections: , __base, + // , and add all into the destination splits. + int baseIndex = childrenSplits.indexOf(CommitUtils.BASE); + Preconditions.checkArgument(baseIndex >= 0, "Cannot locate %s in path %s", CommitUtils.BASE, + magicPath); + List baseChildrenSplits = + childrenSplits.subList(baseIndex + 1, childrenSplits.size()); + Preconditions.checkArgument(!baseChildrenSplits.isEmpty(), + "No path found under %s for magic path %s", CommitUtils.BASE, magicPath); + destSplits.addAll(baseChildrenSplits); + } else { + // Just add the last elements of the into the destination splits. + String filename = childrenSplits.get(childrenSplits.size() - 1); + destSplits.add(filename); + } + + return StringUtils.join(destSplits, "/"); + } + + @Override + protected String createDestKey(Path magicPath) { + return toDestKey(magicPath); + } + + @Override + protected void finishUpload(String destKey, String uploadId, List parts) + throws IOException { + Pending pending = Pending.builder() + .setBucket(storage().bucket().name()) + .setUploadId(uploadId) + .setLength(parts.stream().mapToLong(Part::size).sum()) + .setDestKey(destKey) + .setCreatedTimestamp(System.currentTimeMillis()) + .addParts(parts) + .build(); + + persist(pendingPath, pending.serialize()); + } + + @Override + public synchronized void close() throws IOException { + super.close(); + if (closeStorage) { + storage().close(); + } + } + + protected void persist(Path p, byte[] data) throws IOException { + CommitUtils.save(fs, p, data); + } + + public String pendingKey() { + return ObjectUtils.pathToKey(pendingPath); + } + + private static Path createPendingPath(Path magic) { + return new Path(magic.getParent(), + String.format("%s%s", magic.getName(), CommitUtils.PENDING_SUFFIX)); + } + + // .pending and .pendingset files are not typical magic files. + private static boolean isInternalFile(Path p) { + return p.toString().endsWith(CommitUtils.PENDINGSET_SUFFIX) || p.toString() + .endsWith(CommitUtils.PENDING_SUFFIX); + } + + public static boolean isMagic(Path p) { + Preconditions.checkNotNull(p, "path cannot be null."); + String path = p.toUri().getPath(); + List splits = Arrays.stream(path.split("/")) + .filter(StringUtils::isNoneEmpty) + .collect(Collectors.toList()); + return splits.contains(CommitUtils.MAGIC) && !isInternalFile(p); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/Pending.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/Pending.java new file mode 100644 index 0000000000000..408b6fbe4811c --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/Pending.java @@ -0,0 +1,179 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.tosfs.object.Part; +import org.apache.hadoop.fs.tosfs.util.JsonCodec; +import org.apache.hadoop.fs.tosfs.util.Serializer; +import org.apache.hadoop.thirdparty.com.google.common.base.MoreObjects; +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.util.Preconditions; + +import java.io.IOException; +import java.util.List; +import java.util.Objects; + +/** + * Metadata that will be serialized as json and be saved in the .pending files. + */ +public class Pending implements Serializer { + private static final JsonCodec CODEC = new JsonCodec<>(Pending.class); + + private String bucket; + private String destKey; + private String uploadId; + private long length; + private long createdTimestamp; + private List parts; + + // No-arg constructor for json serializer, don't use. + public Pending() { + } + + public Pending( + String bucket, String destKey, + String uploadId, long length, + long createdTimestamp, List parts) { + this.bucket = bucket; + this.destKey = destKey; + this.uploadId = uploadId; + this.length = length; + this.createdTimestamp = createdTimestamp; + this.parts = parts; + } + + public String bucket() { + return bucket; + } + + public String destKey() { + return destKey; + } + + public String uploadId() { + return uploadId; + } + + public long length() { + return length; + } + + public long createdTimestamp() { + return createdTimestamp; + } + + public List parts() { + return parts; + } + + @Override + public byte[] serialize() throws IOException { + return CODEC.toBytes(this); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("bucket", bucket) + .add("destKey", destKey) + .add("uploadId", uploadId) + .add("length", length) + .add("createdTimestamp", createdTimestamp) + .add("uploadParts", StringUtils.join(parts, ",")) + .toString(); + } + + public static Pending deserialize(byte[] data) throws IOException { + return CODEC.fromBytes(data); + } + + @Override + public int hashCode() { + return Objects.hash(bucket, destKey, uploadId, length, createdTimestamp, parts); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof Pending)) { + return false; + } + Pending that = (Pending) o; + return Objects.equals(bucket, that.bucket) + && Objects.equals(destKey, that.destKey) + && Objects.equals(uploadId, that.uploadId) + && Objects.equals(length, that.length) + && Objects.equals(createdTimestamp, that.createdTimestamp) + && Objects.equals(parts, that.parts); + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + private String bucket; + private String destKey; + private String uploadId; + private long length; + private long createdTimestamp; + private final List parts = Lists.newArrayList(); + + public Builder setBucket(String bucketInput) { + this.bucket = bucketInput; + return this; + } + + public Builder setDestKey(String destKeyInput) { + this.destKey = destKeyInput; + return this; + } + + public Builder setUploadId(String uploadIdInput) { + this.uploadId = uploadIdInput; + return this; + } + + public Builder setLength(long lengthInput) { + this.length = lengthInput; + return this; + } + + public Builder setCreatedTimestamp(long createdTimestampInput) { + this.createdTimestamp = createdTimestampInput; + return this; + } + + public Builder addParts(List partsInput) { + this.parts.addAll(partsInput); + return this; + } + + public Pending build() { + Preconditions.checkArgument(StringUtils.isNoneEmpty(bucket), "Empty bucket"); + Preconditions.checkArgument(StringUtils.isNoneEmpty(destKey), "Empty object destination key"); + Preconditions.checkArgument(StringUtils.isNoneEmpty(uploadId), "Empty uploadId"); + Preconditions.checkArgument(length >= 0, "Invalid length: %s", length); + parts.forEach( + part -> Preconditions.checkArgument(StringUtils.isNoneEmpty(part.eTag(), "Empty etag"))); + + return new Pending(bucket, destKey, uploadId, length, createdTimestamp, parts); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/PendingSet.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/PendingSet.java new file mode 100644 index 0000000000000..c68db9718acda --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/PendingSet.java @@ -0,0 +1,123 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.tosfs.util.JsonCodec; +import org.apache.hadoop.fs.tosfs.util.Serializer; +import org.apache.hadoop.thirdparty.com.google.common.collect.Iterables; +import org.apache.hadoop.thirdparty.com.google.common.collect.Maps; +import org.apache.hadoop.util.Lists; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class PendingSet implements Serializer { + private static final JsonCodec CODEC = new JsonCodec<>(PendingSet.class); + + private String jobId; + private List pendings; + private Map extraData; + + // No-arg constructor for json serializer, don't use. + public PendingSet() { + } + + public PendingSet(String jobId) { + this(jobId, Lists.newArrayList()); + } + + public PendingSet(String jobId, List pendings) { + this.jobId = jobId; + this.pendings = Lists.newArrayList(pendings); + this.extraData = Maps.newHashMap(); + } + + public PendingSet addAll(Iterable items) { + Iterables.addAll(pendings, items); + return this; + } + + public PendingSet add(Pending pending) { + pendings.add(pending); + return this; + } + + public PendingSet addExtraData(String key, String val) { + extraData.put(key, val); + return this; + } + + public String jobId() { + return jobId; + } + + public List commits() { + return pendings; + } + + public Map extraData() { + return extraData; + } + + public int size() { + return pendings.size(); + } + + @Override + public byte[] serialize() throws IOException { + return CODEC.toBytes(this); + } + + public static PendingSet deserialize(byte[] data) { + try { + return CODEC.fromBytes(data); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + public static PendingSet deserialize(FileSystem fs, FileStatus f) { + try { + return deserialize(CommitUtils.load(fs, f.getPath())); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public int hashCode() { + return Objects.hash(jobId, pendings, extraData); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof PendingSet)) { + return false; + } + PendingSet that = (PendingSet) o; + return Objects.equals(jobId, that.jobId) + && Objects.equals(pendings, that.pendings) + && Objects.equals(extraData, that.extraData); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/SuccessData.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/SuccessData.java new file mode 100644 index 0000000000000..c28dc1f9f4395 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/SuccessData.java @@ -0,0 +1,233 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit; + +import org.apache.hadoop.fs.tosfs.util.JsonCodec; +import org.apache.hadoop.fs.tosfs.util.Serializer; +import org.apache.hadoop.thirdparty.com.google.common.base.MoreObjects; +import org.apache.hadoop.thirdparty.com.google.common.base.Throwables; +import org.apache.hadoop.thirdparty.com.google.common.collect.Iterables; +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.util.StringUtils; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class SuccessData implements Serializer { + private static final JsonCodec CODEC = new JsonCodec<>(SuccessData.class); + + private String name; + private boolean success = true; + private long timestamp; + private String date; + private String hostname; + private String committer; + private String description; + private String jobId; + // Filenames in the commit. + private final List filenames = new ArrayList<>(); + + // Diagnostics information. + private final Map diagnostics = new HashMap<>(); + + // No-arg constructor for json serializer, Don't use. + public SuccessData() { + } + + public SuccessData(Builder builder) { + this.name = builder.name; + this.success = builder.success; + this.timestamp = builder.timestamp; + this.date = builder.date; + this.hostname = builder.hostname; + this.committer = builder.committer; + this.description = builder.description; + this.jobId = builder.jobId; + this.filenames.addAll(builder.filenames); + } + + public String name() { + return name; + } + + public boolean success() { + return success; + } + + public long timestamp() { + return timestamp; + } + + public String date() { + return date; + } + + public String hostname() { + return hostname; + } + + public String committer() { + return committer; + } + + public String description() { + return description; + } + + public String jobId() { + return jobId; + } + + public Map diagnostics() { + return diagnostics; + } + + public List filenames() { + return filenames; + } + + public void recordJobFailure(Throwable thrown) { + this.success = false; + String stacktrace = Throwables.getStackTraceAsString(thrown); + addDiagnosticInfo("exception", thrown.toString()); + addDiagnosticInfo("stacktrace", stacktrace); + } + + public void addDiagnosticInfo(String key, String value) { + diagnostics.put(key, value); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("name", name) + .add("success", success) + .add("timestamp", timestamp) + .add("date", date) + .add("hostname", hostname) + .add("committer", committer) + .add("description", description) + .add("jobId", jobId) + .add("filenames", StringUtils.join(",", filenames)) + .toString(); + } + + public static Builder builder() { + return new Builder(); + } + + @Override + public byte[] serialize() throws IOException { + return CODEC.toBytes(this); + } + + public static SuccessData deserialize(byte[] data) throws IOException { + return CODEC.fromBytes(data); + } + + @Override + public int hashCode() { + return Objects.hash(name, success, timestamp, date, hostname, committer, description, jobId, + filenames); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof SuccessData)) { + return false; + } + SuccessData that = (SuccessData) o; + return Objects.equals(name, that.name) + && Objects.equals(success, that.success) + && Objects.equals(timestamp, that.timestamp) + && Objects.equals(date, that.date) + && Objects.equals(hostname, that.hostname) + && Objects.equals(committer, that.committer) + && Objects.equals(description, that.description) + && Objects.equals(jobId, that.jobId) + && Objects.equals(filenames, that.filenames); + } + + public static class Builder { + private String name = SuccessData.class.getName(); + private boolean success = true; + private long timestamp; + private String date; + private String hostname; + private String committer; + private String description; + private String jobId; + private final List filenames = Lists.newArrayList(); + + public Builder setName(String nameInput) { + this.name = nameInput; + return this; + } + + public Builder setSuccess(boolean successInput) { + this.success = successInput; + return this; + } + + public Builder setTimestamp(long timestampInput) { + this.timestamp = timestampInput; + return this; + } + + public Builder setDate(String dateInput) { + this.date = dateInput; + return this; + } + + public Builder setHostname(String hostnameInput) { + this.hostname = hostnameInput; + return this; + } + + public Builder setCommitter(String committerInput) { + this.committer = committerInput; + return this; + } + + public Builder setDescription(String descriptionInput) { + this.description = descriptionInput; + return this; + } + + public Builder setJobId(String jobIdInput) { + this.jobId = jobIdInput; + return this; + } + + public Builder addFileNames(Iterable newFileNamesInput) { + if (newFileNamesInput != null) { + Iterables.addAll(this.filenames, newFileNamesInput); + } + return this; + } + + public SuccessData build() { + return new SuccessData(this); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/mapred/Committer.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/mapred/Committer.java new file mode 100644 index 0000000000000..ce380145e8118 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/mapred/Committer.java @@ -0,0 +1,189 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit.mapred; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.commit.CommitUtils; +import org.apache.hadoop.mapred.FileOutputCommitter; +import org.apache.hadoop.mapred.FileOutputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.JobContext; +import org.apache.hadoop.mapred.JobStatus; +import org.apache.hadoop.mapred.TaskAttemptContext; +import org.apache.hadoop.mapreduce.OutputCommitter; +import org.apache.hadoop.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +public class Committer extends FileOutputCommitter { + private static final Logger LOG = LoggerFactory.getLogger(Committer.class); + private OutputCommitter wrapped = null; + + private static Path getOutputPath(JobContext context) { + JobConf conf = context.getJobConf(); + return FileOutputFormat.getOutputPath(conf); + } + + private static Path getOutputPath(TaskAttemptContext context) { + JobConf conf = context.getJobConf(); + return FileOutputFormat.getOutputPath(conf); + } + + private OutputCommitter getWrapped(JobContext context) throws IOException { + if (wrapped == null) { + wrapped = CommitUtils.supportObjectStorageCommit(context.getConfiguration(), + getOutputPath(context)) ? + new org.apache.hadoop.fs.tosfs.commit.Committer(getOutputPath(context), context) : + new FileOutputCommitter(); + LOG.debug("Using OutputCommitter implementation {}", wrapped.getClass().getName()); + } + return wrapped; + } + + @InterfaceAudience.Private + @Override + public Path getTaskAttemptPath(TaskAttemptContext context) throws IOException { + Path out = getOutputPath(context); + return out == null ? null : getTaskAttemptPath(context, out); + } + + private OutputCommitter getWrapped(TaskAttemptContext context) throws IOException { + if (wrapped == null) { + wrapped = CommitUtils.supportObjectStorageCommit(context.getConfiguration(), + getOutputPath(context)) ? + new org.apache.hadoop.fs.tosfs.commit.Committer(getOutputPath(context), context) : + new FileOutputCommitter(); + } + return wrapped; + } + + @Override + public Path getWorkPath(TaskAttemptContext context, Path outputPath) + throws IOException { + if (getWrapped(context) instanceof org.apache.hadoop.fs.tosfs.commit.Committer) { + return ((org.apache.hadoop.fs.tosfs.commit.Committer) getWrapped(context)).getWorkPath(); + } + return super.getWorkPath(context, outputPath); + } + + private Path getTaskAttemptPath(TaskAttemptContext context, Path out) throws IOException { + Path workPath = FileOutputFormat.getWorkOutputPath(context.getJobConf()); + if(workPath == null && out != null) { + if (getWrapped(context) instanceof org.apache.hadoop.fs.tosfs.commit.Committer) { + return CommitUtils.magicTaskAttemptPath(context, getOutputPath(context)); + } else { + return org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter + .getTaskAttemptPath(context, out); + } + } + return workPath; + } + + @Override + public void setupJob(JobContext context) throws IOException { + getWrapped(context).setupJob(context); + } + + @Override + public void commitJob(JobContext context) throws IOException { + getWrapped(context).commitJob(context); + } + + @Override + @Deprecated + public void cleanupJob(JobContext context) throws IOException { + getWrapped(context).cleanupJob(context); + } + + @Override + public void abortJob(JobContext context, int runState) + throws IOException { + JobStatus.State state; + if(runState == JobStatus.State.RUNNING.getValue()) { + state = JobStatus.State.RUNNING; + } else if(runState == JobStatus.State.SUCCEEDED.getValue()) { + state = JobStatus.State.SUCCEEDED; + } else if(runState == JobStatus.State.FAILED.getValue()) { + state = JobStatus.State.FAILED; + } else if(runState == JobStatus.State.PREP.getValue()) { + state = JobStatus.State.PREP; + } else if(runState == JobStatus.State.KILLED.getValue()) { + state = JobStatus.State.KILLED; + } else { + throw new IllegalArgumentException(runState+" is not a valid runState."); + } + getWrapped(context).abortJob(context, state); + } + + @Override + public void setupTask(TaskAttemptContext context) throws IOException { + getWrapped(context).setupTask(context); + } + + @Override + public void commitTask(TaskAttemptContext context) throws IOException { + getWrapped(context).commitTask(context); + } + + @Override + public void abortTask(TaskAttemptContext context) throws IOException { + getWrapped(context).abortTask(context); + } + + @Override + public boolean needsTaskCommit(TaskAttemptContext context) + throws IOException { + return getWrapped(context).needsTaskCommit(context); + } + + @Override + @Deprecated + public boolean isRecoverySupported() { + return false; + } + + @Override + public boolean isCommitJobRepeatable(JobContext context) throws IOException { + return getWrapped(context).isCommitJobRepeatable(context); + } + + @Override + public boolean isRecoverySupported(JobContext context) throws IOException { + return getWrapped(context).isRecoverySupported(context); + } + + @Override + public void recoverTask(TaskAttemptContext context) + throws IOException { + getWrapped(context).recoverTask(context); + } + + public String jobId() { + Preconditions.checkNotNull(wrapped, "Encountered uninitialized job committer."); + return wrapped instanceof org.apache.hadoop.fs.tosfs.commit.Committer ? + ((org.apache.hadoop.fs.tosfs.commit.Committer) wrapped).jobId() : null; + } + + public Path getWorkPath() { + Preconditions.checkNotNull(wrapped, "Encountered uninitialized job committer."); + return wrapped instanceof org.apache.hadoop.fs.tosfs.commit.Committer ? + ((org.apache.hadoop.fs.tosfs.commit.Committer) wrapped).getWorkPath() : null; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/mapred/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/mapred/package-info.java new file mode 100644 index 0000000000000..761997146ce4f --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/mapred/package-info.java @@ -0,0 +1,26 @@ +/* + * 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. + */ + +/** + * Classes for hadoop-tos job committer. + */ +@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"}) +@InterfaceStability.Evolving +package org.apache.hadoop.fs.tosfs.commit.mapred; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/PendingOps.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/PendingOps.java new file mode 100644 index 0000000000000..7012b63babaed --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/PendingOps.java @@ -0,0 +1,43 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit.ops; + +import org.apache.hadoop.fs.tosfs.commit.Pending; + +public interface PendingOps { + /** + * Revert the committed {@link Pending}, usually we need to remove or delete the committed files. + * + * @param commit to revert. + */ + void revert(Pending commit); + + /** + * Abort the uncommitted {@link Pending}, to prevent any further committing. + * + * @param commit to abort. + */ + void abort(Pending commit); + + /** + * Commit the {@link Pending} files to be visible. If we want to revert this completed result, + * please just use {@link PendingOps#revert(Pending)} to revert this commit. + * + * @param commit to be visible. + */ + void commit(Pending commit); +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/PendingOpsFactory.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/PendingOpsFactory.java new file mode 100644 index 0000000000000..40411d02c1e96 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/PendingOpsFactory.java @@ -0,0 +1,40 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit.ops; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; + +public final class PendingOpsFactory { + public static final String PENDING_OPS_IMPL = "pending.ops.impl"; + public static final String DEFAULT_PENDING_OPS_IMPL = RawPendingOps.class.getName(); + + private PendingOpsFactory() { + } + + public static PendingOps create(FileSystem fs, ObjectStorage storage) { + try { + String opsImpl = fs.getConf().get(PENDING_OPS_IMPL, DEFAULT_PENDING_OPS_IMPL); + Class clazz = Class.forName(opsImpl); + return (PendingOps) clazz + .getDeclaredConstructor(FileSystem.class, ObjectStorage.class) + .newInstance(fs, storage); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/RawPendingOps.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/RawPendingOps.java new file mode 100644 index 0000000000000..fe8b453d759c5 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/RawPendingOps.java @@ -0,0 +1,57 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit.ops; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.tosfs.commit.Pending; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * PendingOps will revert, abort or commit the given {@link Pending} commit. + */ +public class RawPendingOps implements PendingOps { + private static final Logger LOG = LoggerFactory.getLogger(RawPendingOps.class); + + private final ObjectStorage storage; + + /** + * Constructor for {@link PendingOpsFactory} to reflect a new instance. + * + * @param fs the file system. + * @param storage the object storage. + */ + public RawPendingOps(FileSystem fs, ObjectStorage storage) { + this.storage = storage; + } + + public void revert(Pending commit) { + LOG.info("Revert the commit by deleting the object key - {}", commit); + storage.delete(commit.destKey()); + } + + public void abort(Pending commit) { + LOG.info("Abort the commit by aborting multipart upload - {}", commit); + storage.abortMultipartUpload(commit.destKey(), commit.uploadId()); + } + + public void commit(Pending commit) { + LOG.info("Commit by completing the multipart uploads - {}", commit); + storage.completeUpload(commit.destKey(), commit.uploadId(), commit.parts()); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/package-info.java new file mode 100644 index 0000000000000..d89c05382f894 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/package-info.java @@ -0,0 +1,26 @@ +/* + * 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. + */ + +/** + * Classes for hadoop-tos job committer. + */ +@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"}) +@InterfaceStability.Evolving +package org.apache.hadoop.fs.tosfs.commit.ops; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/package-info.java new file mode 100644 index 0000000000000..350a379725f1e --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/package-info.java @@ -0,0 +1,26 @@ +/* + * 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. + */ + +/** + * Classes for hadoop-tos job committer. + */ +@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"}) +@InterfaceStability.Evolving +package org.apache.hadoop.fs.tosfs.commit; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/Bytes.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/Bytes.java new file mode 100644 index 0000000000000..810551b4c4511 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/Bytes.java @@ -0,0 +1,103 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.common; + +import org.apache.hadoop.util.Preconditions; + +import java.nio.charset.StandardCharsets; + +public final class Bytes { + private Bytes() { + } + + // Encode basic Java types into big-endian binaries. + + public static byte[] toBytes(boolean b) { + return new byte[] {b ? (byte) -1 : (byte) 0}; + } + + public static byte[] toBytes(byte b) { + return new byte[] {b}; + } + + public static byte[] toBytes(short val) { + byte[] b = new byte[2]; + for (int i = 1; i >= 0; i--) { + b[i] = (byte) val; + val >>>= 8; + } + return b; + } + + public static byte[] toBytes(int val) { + byte[] b = new byte[4]; + for (int i = 3; i >= 0; i--) { + b[i] = (byte) val; + val >>>= 8; + } + return b; + } + + public static byte[] toBytes(long val) { + byte[] b = new byte[8]; + for (int i = 7; i >= 0; i--) { + b[i] = (byte) val; + val >>>= 8; + } + return b; + } + + public static byte[] toBytes(String s) { + return s.getBytes(StandardCharsets.UTF_8); + } + + // Decode big-endian binaries into basic Java types. + + public static long toLong(byte[] b) { + return toLong(b, 0, 8); + } + + public static long toLong(byte[] b, int off, int len) { + Preconditions.checkArgument(len == 8, "Invalid len: %s", len); + Preconditions.checkArgument(off >= 0 && off + len <= b.length, + "Invalid off: %s, len: %s, array size: %s", off, len, b.length); + long l = 0; + for (int i = off; i < off + len; i++) { + l <<= 8; + l ^= b[i] & 0xFF; + } + return l; + } + + public static byte[] toBytes(byte[] b, int off, int len) { + Preconditions.checkArgument(off >= 0, "off %s must be >=0", off); + Preconditions.checkArgument(len >= 0, "len %s must be >= 0", len); + Preconditions.checkArgument(off + len <= b.length, "off (%s) + len (%s) must be <= %s", off, + len, b.length); + byte[] data = new byte[len]; + System.arraycopy(b, off, data, 0, len); + return data; + } + + public static String toString(byte[] b) { + return new String(b, StandardCharsets.UTF_8); + } + + public static String toString(byte[] b, int off, int len) { + return new String(b, off, len, StandardCharsets.UTF_8); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/Chain.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/Chain.java new file mode 100644 index 0000000000000..b77c625b56336 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/Chain.java @@ -0,0 +1,148 @@ +/* + * 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.tosfs.common; + +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.thirdparty.com.google.common.collect.Queues; +import org.apache.hadoop.util.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Deque; +import java.util.List; +import java.util.Queue; +import java.util.function.Predicate; + +public final class Chain implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(Chain.class); + + private final List suppressed = Lists.newArrayList(); + private final Queue> queue; + private final Predicate shouldContinue; + private T curItem; + + private Chain(Deque> queue, Predicate shouldContinue) { + this.queue = queue; + this.shouldContinue = shouldContinue; + this.curItem = null; + } + + public R run(Task task) throws IOException { + while (true) { + if (curItem == null && !nextItem()) { + IOException ex = new IOException("Failed to run task after attempt all items"); + suppressed.forEach(ex::addSuppressed); + throw ex; + } + + try { + return task.run(curItem); + } catch (IOException e) { + LOG.debug("Encounter exception while running task with item {}", curItem, e); + // Resetting the current caller to be null, for triggering the next round election. + if (curItem != null) { + CommonUtils.runQuietly(curItem::close); + curItem = null; + } + suppressed.add(e); + + if (shouldContinue != null && !shouldContinue.test(e)) { + IOException ex = + new IOException("Failed to run the chain since the encountered error not retryable."); + suppressed.forEach(ex::addSuppressed); + throw ex; + } + } + } + } + + public T curItem() { + return curItem; + } + + private boolean nextItem() { + if (curItem != null) { + CommonUtils.runQuietly(curItem::close); + curItem = null; + } + + while (!queue.isEmpty()) { + ItemFactory nextFactory = queue.poll(); + try { + curItem = nextFactory.newItem(); + return true; + } catch (IOException e) { + curItem = null; + LOG.debug("Failed to create new item", e); + suppressed.add(e); + } + } + + return false; + } + + @Override + public void close() throws IOException { + if (curItem != null) { + curItem.close(); + } + } + + public interface ItemFactory { + T newItem() throws IOException; + } + + public interface Task { + R run(T call) throws IOException; + } + + public static class Builder { + private final Deque> factories = Queues.newArrayDeque(); + private Predicate shouldContinue; + + public Builder addFirst(ItemFactory factory) { + factories.addFirst(factory); + return this; + } + + public Builder addLast(ItemFactory factory) { + factories.addLast(factory); + return this; + } + + public Builder shouldContinue(Predicate continueCondition) { + this.shouldContinue = continueCondition; + return this; + } + + public Chain build() throws IOException { + Chain chain = new Chain<>(factories, shouldContinue); + + // Do nothing in the chain task to initialize the first item. + chain.run(item -> null); + return chain; + } + } + + public static Builder builder() { + return new Builder<>(); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/Tasks.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/Tasks.java new file mode 100644 index 0000000000000..71c2a444d7d03 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/Tasks.java @@ -0,0 +1,592 @@ +/* + * 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.tosfs.common; + +import org.apache.hadoop.util.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Queue; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Predicate; +import java.util.stream.Stream; + +/** + * Copied from Apache Iceberg. + */ +public final class Tasks { + + private static final Logger LOG = LoggerFactory.getLogger(Tasks.class); + + private Tasks() { + } + + public static class UnrecoverableException extends RuntimeException { + public UnrecoverableException(String message) { + super(message); + } + + public UnrecoverableException(String message, Throwable cause) { + super(message, cause); + } + + public UnrecoverableException(Throwable cause) { + super(cause); + } + } + + public interface FailureTask { + void run(I item, Exception exception) throws E; + } + + public interface Task { + void run(I item) throws E; + } + + public static class Builder { + private final Iterable items; + private ExecutorService service = null; + private FailureTask onFailure = null; + private boolean stopOnFailure = false; + private boolean throwFailureWhenFinished = true; + private Task revertTask = null; + private boolean stopRevertsOnFailure = false; + private Task abortTask = null; + private boolean stopAbortsOnFailure = false; + + // retry settings + private List> stopRetryExceptions = + Lists.newArrayList(UnrecoverableException.class); + private List> onlyRetryExceptions = null; + private Predicate shouldRetryPredicate = null; + private int maxAttempts = 1; // not all operations can be retried + private long minSleepTimeMs = 1000; // 1 second + private long maxSleepTimeMs = 600000; // 10 minutes + private long maxDurationMs = 600000; // 10 minutes + private double scaleFactor = 2.0; // exponential + + public Builder(Iterable items) { + this.items = items; + } + + public Builder executeWith(ExecutorService svc) { + this.service = svc; + return this; + } + + public Builder onFailure(FailureTask task) { + this.onFailure = task; + return this; + } + + public Builder stopOnFailure() { + this.stopOnFailure = true; + return this; + } + + public Builder throwFailureWhenFinished() { + this.throwFailureWhenFinished = true; + return this; + } + + public Builder throwFailureWhenFinished(boolean throwWhenFinished) { + this.throwFailureWhenFinished = throwWhenFinished; + return this; + } + + public Builder suppressFailureWhenFinished() { + this.throwFailureWhenFinished = false; + return this; + } + + public Builder revertWith(Task task) { + this.revertTask = task; + return this; + } + + public Builder stopRevertsOnFailure() { + this.stopRevertsOnFailure = true; + return this; + } + + public Builder abortWith(Task task) { + this.abortTask = task; + return this; + } + + public Builder stopAbortsOnFailure() { + this.stopAbortsOnFailure = true; + return this; + } + + @SafeVarargs public final Builder stopRetryOn(Class... exceptions) { + stopRetryExceptions.addAll(Arrays.asList(exceptions)); + return this; + } + + public Builder shouldRetryTest(Predicate shouldRetry) { + this.shouldRetryPredicate = shouldRetry; + return this; + } + + public Builder noRetry() { + this.maxAttempts = 1; + return this; + } + + public Builder retry(int nTimes) { + this.maxAttempts = nTimes + 1; + return this; + } + + public Builder onlyRetryOn(Class exception) { + this.onlyRetryExceptions = Collections.singletonList(exception); + return this; + } + + @SafeVarargs public final Builder onlyRetryOn(Class... exceptions) { + this.onlyRetryExceptions = Lists.newArrayList(exceptions); + return this; + } + + public Builder exponentialBackoff(long backoffMinSleepTimeMs, long backoffMaxSleepTimeMs, + long backoffMaxRetryTimeMs, double backoffScaleFactor) { + this.minSleepTimeMs = backoffMinSleepTimeMs; + this.maxSleepTimeMs = backoffMaxSleepTimeMs; + this.maxDurationMs = backoffMaxRetryTimeMs; + this.scaleFactor = backoffScaleFactor; + return this; + } + + public boolean run(Task task) { + return run(task, RuntimeException.class); + } + + public boolean run(Task task, Class exceptionClass) throws E { + if (service != null) { + return runParallel(task, exceptionClass); + } else { + return runSingleThreaded(task, exceptionClass); + } + } + + private boolean runSingleThreaded( + Task task, Class exceptionClass) throws E { + List succeeded = Lists.newArrayList(); + List exceptions = Lists.newArrayList(); + + Iterator iterator = items.iterator(); + boolean threw = true; + try { + while (iterator.hasNext()) { + I item = iterator.next(); + try { + runTaskWithRetry(task, item); + succeeded.add(item); + } catch (Exception e) { + exceptions.add(e); + + if (onFailure != null) { + tryRunOnFailure(item, e); + } + + if (stopOnFailure) { + break; + } + } + } + + threw = false; + + } finally { + // threw handles exceptions that were *not* caught by the catch block, + // and exceptions that were caught and possibly handled by onFailure + // are kept in exceptions. + if (threw || !exceptions.isEmpty()) { + if (revertTask != null) { + boolean failed = false; + for (I item : succeeded) { + try { + revertTask.run(item); + } catch (Exception e) { + failed = true; + LOG.error("Failed to revert task", e); + // keep going + } + if (stopRevertsOnFailure && failed) { + break; + } + } + } + + if (abortTask != null) { + boolean failed = false; + while (iterator.hasNext()) { + try { + abortTask.run(iterator.next()); + } catch (Exception e) { + failed = true; + LOG.error("Failed to abort task", e); + // keep going + } + if (stopAbortsOnFailure && failed) { + break; + } + } + } + } + } + + if (throwFailureWhenFinished && !exceptions.isEmpty()) { + Tasks.throwOne(exceptions, exceptionClass); + } else if (throwFailureWhenFinished && threw) { + throw new RuntimeException("Task set failed with an uncaught throwable"); + } + + return !threw; + } + + private void tryRunOnFailure(I item, Exception failure) { + try { + onFailure.run(item, failure); + } catch (Exception failException) { + failure.addSuppressed(failException); + LOG.error("Failed to clean up on failure", failException); + // keep going + } + } + + private boolean runParallel( + final Task task, Class exceptionClass) throws E { + final Queue succeeded = new ConcurrentLinkedQueue<>(); + final Queue exceptions = new ConcurrentLinkedQueue<>(); + final AtomicBoolean taskFailed = new AtomicBoolean(false); + final AtomicBoolean abortFailed = new AtomicBoolean(false); + final AtomicBoolean revertFailed = new AtomicBoolean(false); + + List> futures = Lists.newArrayList(); + + for (final I item : items) { + // submit a task for each item that will either run or abort the task + futures.add(service.submit(() -> { + if (!(stopOnFailure && taskFailed.get())) { + // run the task with retries + boolean threw = true; + try { + runTaskWithRetry(task, item); + + succeeded.add(item); + + threw = false; + + } catch (Exception e) { + taskFailed.set(true); + exceptions.add(e); + + if (onFailure != null) { + tryRunOnFailure(item, e); + } + } finally { + if (threw) { + taskFailed.set(true); + } + } + + } else if (abortTask != null) { + // abort the task instead of running it + if (stopAbortsOnFailure && abortFailed.get()) { + return; + } + + boolean failed = true; + try { + abortTask.run(item); + failed = false; + } catch (Exception e) { + LOG.error("Failed to abort task", e); + // swallow the exception + } finally { + if (failed) { + abortFailed.set(true); + } + } + } + })); + } + + // let the above tasks complete (or abort) + exceptions.addAll(waitFor(futures)); + futures.clear(); + + if (taskFailed.get() && revertTask != null) { + // at least one task failed, revert any that succeeded + for (final I item : succeeded) { + futures.add(service.submit(() -> { + if (stopRevertsOnFailure && revertFailed.get()) { + return; + } + + boolean failed = true; + try { + revertTask.run(item); + failed = false; + } catch (Exception e) { + LOG.error("Failed to revert task", e); + // swallow the exception + } finally { + if (failed) { + revertFailed.set(true); + } + } + })); + } + + // let the revert tasks complete + exceptions.addAll(waitFor(futures)); + } + + if (throwFailureWhenFinished && !exceptions.isEmpty()) { + Tasks.throwOne(exceptions, exceptionClass); + } else if (throwFailureWhenFinished && taskFailed.get()) { + throw new RuntimeException("Task set failed with an uncaught throwable"); + } + + return !taskFailed.get(); + } + + private void runTaskWithRetry( + Task task, I item) throws E { + long start = System.currentTimeMillis(); + int attempt = 0; + while (true) { + attempt += 1; + try { + task.run(item); + break; + + } catch (Exception e) { + long durationMs = System.currentTimeMillis() - start; + if (attempt >= maxAttempts || (durationMs > maxDurationMs && attempt > 1)) { + if (durationMs > maxDurationMs) { + LOG.info("Stopping retries after {} ms", durationMs); + } + throw e; + } + + if (shouldRetryPredicate != null) { + if (!shouldRetryPredicate.test(e)) { + throw e; + } + + } else if (onlyRetryExceptions != null) { + // if onlyRetryExceptions are present, then this retries if one is found + boolean matchedRetryException = false; + for (Class exClass : onlyRetryExceptions) { + if (exClass.isInstance(e)) { + matchedRetryException = true; + break; + } + } + if (!matchedRetryException) { + throw e; + } + + } else { + // otherwise, always retry unless one of the stop exceptions is found + for (Class exClass : stopRetryExceptions) { + if (exClass.isInstance(e)) { + throw e; + } + } + } + + int delayMs = + (int) Math.min(minSleepTimeMs * Math.pow(scaleFactor, attempt - 1), maxSleepTimeMs); + int jitter = ThreadLocalRandom.current().nextInt(Math.max(1, (int) (delayMs * 0.1))); + + LOG.warn("Retrying task after failure: {}", e.getMessage(), e); + + try { + TimeUnit.MILLISECONDS.sleep(delayMs + jitter); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new RuntimeException(ie); + } + } + } + } + } + + private static Collection waitFor( + Collection> futures) { + while (true) { + int numFinished = 0; + for (Future future : futures) { + if (future.isDone()) { + numFinished += 1; + } + } + + if (numFinished == futures.size()) { + List uncaught = Lists.newArrayList(); + // all of the futures are done, get any uncaught exceptions + for (Future future : futures) { + try { + future.get(); + + } catch (InterruptedException e) { + LOG.warn("Interrupted while getting future results", e); + for (Throwable t : uncaught) { + e.addSuppressed(t); + } + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + + } catch (CancellationException e) { + // ignore cancellations + + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + if (Error.class.isInstance(cause)) { + for (Throwable t : uncaught) { + cause.addSuppressed(t); + } + throw (Error) cause; + } + + if (cause != null) { + uncaught.add(e); + } + + LOG.warn("Task threw uncaught exception", cause); + } + } + + return uncaught; + + } else { + try { + Thread.sleep(10); + } catch (InterruptedException e) { + LOG.warn("Interrupted while waiting for tasks to finish", e); + + for (Future future : futures) { + future.cancel(true); + } + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + } + } + + /** + * A range, [ 0, size ). + */ + private static class Range implements Iterable { + private final int size; + + Range(int size) { + this.size = size; + } + + @Override + public Iterator iterator() { + return new Iterator() { + private int current = 0; + + @Override + public boolean hasNext() { + return current < size; + } + + @Override + public Integer next() { + if (!hasNext()) { + throw new NoSuchElementException("No more items."); + } + int ret = current; + current += 1; + return ret; + } + }; + } + } + + public static Builder range(int upTo) { + return new Builder<>(new Range(upTo)); + } + + public static Builder foreach(Iterator items) { + return new Builder<>(() -> items); + } + + public static Builder foreach(Iterable items) { + return new Builder<>(items); + } + + @SafeVarargs public static Builder foreach(I... items) { + return new Builder<>(Arrays.asList(items)); + } + + public static Builder foreach(Stream items) { + return new Builder<>(items::iterator); + } + + private static void throwOne(Collection exceptions, + Class allowedException) throws E { + Iterator iter = exceptions.iterator(); + Throwable exception = iter.next(); + Class exceptionClass = exception.getClass(); + + while (iter.hasNext()) { + Throwable other = iter.next(); + if (!exceptionClass.isInstance(other)) { + exception.addSuppressed(other); + } + } + + castAndThrow(exception, allowedException); + } + + public static void castAndThrow( + Throwable exception, Class exceptionClass) throws E { + if (exception instanceof RuntimeException) { + throw (RuntimeException) exception; + } else if (exception instanceof Error) { + throw (Error) exception; + } else if (exceptionClass.isInstance(exception)) { + throw (E) exception; + } + throw new RuntimeException(exception); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/ThreadPools.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/ThreadPools.java new file mode 100644 index 0000000000000..da2beb54d24bf --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/ThreadPools.java @@ -0,0 +1,140 @@ +/* + * 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.tosfs.common; + +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.hadoop.util.concurrent.HadoopExecutors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.Thread.UncaughtExceptionHandler; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; + +/** + * Copied from Apache Iceberg. + */ +public final class ThreadPools { + + private static final Logger LOG = LoggerFactory.getLogger(ThreadPools.class); + + private ThreadPools() { + } + + public static final String WORKER_THREAD_POOL_SIZE_PROP = "tos.worker.num-threads"; + + public static final int WORKER_THREAD_POOL_SIZE = + poolSize(Math.max(2, Runtime.getRuntime().availableProcessors())); + + private static final ExecutorService WORKER_POOL = newWorkerPool("tos-default-worker-pool"); + + public static ExecutorService defaultWorkerPool() { + return WORKER_POOL; + } + + public static ExecutorService newWorkerPool(String namePrefix) { + return newWorkerPool(namePrefix, WORKER_THREAD_POOL_SIZE); + } + + public static ExecutorService newWorkerPool(String namePrefix, int poolSize) { + return Executors.newFixedThreadPool(poolSize, newDaemonThreadFactory(namePrefix)); + } + + public static ScheduledExecutorService newScheduleWorkerPool(String namePrefix, int poolSize) { + return Executors.newScheduledThreadPool(poolSize, newDaemonThreadFactory(namePrefix)); + } + + /** + * Helper routine to shutdown a {@link ExecutorService}. Will wait up to a + * certain timeout for the ExecutorService to gracefully shutdown. If the + * ExecutorService did not shutdown and there are still tasks unfinished after + * the timeout period, the ExecutorService will be notified to forcibly shut + * down. Another timeout period will be waited before giving up. So, at most, + * a shutdown will be allowed to wait up to twice the timeout value before + * giving up. + *

+ * This method is copied from + * {@link HadoopExecutors#shutdown(ExecutorService, Logger, long, TimeUnit)}. + * + * @param executorService ExecutorService to shutdown + * @param timeout the maximum time to wait + * @param unit the time unit of the timeout argument + */ + public static void shutdown(ExecutorService executorService, long timeout, TimeUnit unit) { + if (executorService == null) { + return; + } + + try { + executorService.shutdown(); + LOG.debug("Gracefully shutting down executor service. Waiting max {} {}", timeout, unit); + + if (!executorService.awaitTermination(timeout, unit)) { + LOG.debug("Executor service has not shutdown yet. Forcing. Will wait up to an additional" + + " {} {} for shutdown", timeout, unit); + executorService.shutdownNow(); + } + + if (executorService.awaitTermination(timeout, unit)) { + LOG.debug("Succesfully shutdown executor service"); + } else { + LOG.error("Unable to shutdown executor service after timeout {} {}", (2 * timeout), unit); + } + } catch (InterruptedException e) { + LOG.error("Interrupted while attempting to shutdown", e); + executorService.shutdownNow(); + } catch (Exception e) { + LOG.warn("Exception closing executor service {}", e.getMessage()); + LOG.debug("Exception closing executor service", e); + throw e; + } + } + + private static int poolSize(int defaultSize) { + String value = System.getProperty(WORKER_THREAD_POOL_SIZE_PROP); + if (value != null) { + try { + return Integer.parseUnsignedInt(value); + } catch (NumberFormatException e) { + // will return the default + } + } + return defaultSize; + } + + public static ThreadFactory newDaemonThreadFactory(String namePrefix) { + return new ThreadFactoryBuilder().setDaemon(true).setNameFormat(namePrefix + "-%d") + .setUncaughtExceptionHandler( + (t, e) -> LOG.error("Thread {} encounter uncaught exception", t, e)).build(); + } + + public static Thread newDaemonThread(String name, Runnable runnable, + UncaughtExceptionHandler handler) { + Thread t = new Thread(runnable); + t.setName(name); + t.setDaemon(true); + if (handler != null) { + t.setUncaughtExceptionHandler(handler); + } + return t; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/package-info.java new file mode 100644 index 0000000000000..3d54d536c15e5 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/package-info.java @@ -0,0 +1,26 @@ +/* + * 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. + */ + +/** + * Classes for hadoop-tos common. + */ +@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"}) +@InterfaceStability.Evolving +package org.apache.hadoop.fs.tosfs.common; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/ArgumentKey.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/ArgumentKey.java new file mode 100644 index 0000000000000..7c7821bcff419 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/ArgumentKey.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.tosfs.conf; + +public class ArgumentKey { + + private final String template; + + public ArgumentKey(String template) { + this.template = template; + } + + public String key(Object... arguments) { + return String.format(template, arguments); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/ConfKeys.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/ConfKeys.java new file mode 100644 index 0000000000000..2142bba86932a --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/ConfKeys.java @@ -0,0 +1,156 @@ +/* + * 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.tosfs.conf; + +import org.apache.hadoop.util.Preconditions; + +public final class ConfKeys { + + private ConfKeys() {} + + /** + * Object storage endpoint to connect to, which should include both region and object domain name. + * e.g. 'fs.tos.endpoint'='tos-cn-beijing.volces.com'. + */ + public static final ArgumentKey FS_OBJECT_STORAGE_ENDPOINT = new ArgumentKey("fs.%s.endpoint"); + + /** + * The region of the object storage, e.g. fs.tos.region. Parsing template "fs.%s.endpoint" to + * know the region. + */ + public static final ArgumentKey FS_OBJECT_STORAGE_REGION = new ArgumentKey("fs.%s.region"); + + /** + * The object storage implementation for the defined scheme. For example, we can delegate the + * scheme 'abc' to TOS (or other object storage),and access the TOS object storage as + * 'abc://bucket/path/to/key' + */ + public static final ArgumentKey FS_OBJECT_STORAGE_IMPL = + new ArgumentKey("fs.objectstorage.%s.impl"); + + /** + * The batch size of deleting multiple objects per request for the given object storage. + * e.g. fs.tos.delete.batch-size + */ + public static final ArgumentKey FS_BATCH_DELETE_SIZE = new ArgumentKey("fs.%s.delete.batch-size"); + public static final int FS_BATCH_DELETE_SIZE_DEFAULT = 250; + + /** + * The multipart upload part size of the given object storage, e.g. fs.tos.multipart.size. + */ + public static final ArgumentKey FS_MULTIPART_SIZE = new ArgumentKey("fs.%s.multipart.size"); + public static final long FS_MULTIPART_SIZE_DEFAULT = 8L << 20; + + /** + * The threshold (larger than this value) to enable multipart upload during copying objects + * in the given object storage. If the copied data size is less than threshold, will copy data via + * executing copyObject instead of uploadPartCopy. E.g. fs.tos.multipart.copy-threshold + */ + public static final ArgumentKey FS_MULTIPART_COPY_THRESHOLD = + new ArgumentKey("fs.%s.multipart.copy-threshold"); + public static final long FS_MULTIPART_COPY_THRESHOLD_DEFAULT = 5L << 20; + + /** + * The threshold which control whether enable multipart upload during writing data to the given + * object storage, if the write data size is less than threshold, will write data via simple put + * instead of multipart upload. E.g. fs.tos.multipart.threshold. + */ + public static final ArgumentKey FS_MULTIPART_THRESHOLD = + new ArgumentKey("fs.%s.multipart.threshold"); + public static final long FS_MULTIPART_THRESHOLD_DEFAULT = 10 << 20; + + /** + * The max byte size which will buffer the staging data in-memory before flushing to the staging + * file. It will decrease the random write in local staging disk dramatically if writing plenty of + * small files. + */ + public static final ArgumentKey FS_MULTIPART_STAGING_BUFFER_SIZE = + new ArgumentKey("fs.%s.multipart.staging-buffer-size"); + public static final int FS_MULTIPART_STAGING_BUFFER_SIZE_DEFAULT = 4 << 10; + + /** + * The multipart upload part staging dir(s) of the given object storage. + * e.g. fs.tos.multipart.staging-dir. + * Separate the staging dirs with comma if there are many staging dir paths. + */ + public static final ArgumentKey FS_MULTIPART_STAGING_DIR = + new ArgumentKey("fs.%s.multipart.staging-dir"); + public static final String FS_MULTIPART_STAGING_DIR_DEFAULT = defaultDir("multipart-staging-dir"); + + /** + * True to create the missed parent dir asynchronously during deleting or renaming a file or dir. + */ + public static final ArgumentKey FS_ASYNC_CREATE_MISSED_PARENT = + new ArgumentKey("fs.%s.missed.parent.dir.async-create"); + public static final boolean FS_ASYNC_CREATE_MISSED_PARENT_DEFAULT = true; + + /** + * Whether using rename semantic of object storage during rename files, otherwise using + * copy + delete. + * Please ensure that the object storage support and enable rename semantic and before enable it, + * and also ensure grant rename permission to the requester. + * If you are using TOS, you have to send putBucketRename request before sending rename request, + * otherwise MethodNotAllowed exception will be thrown. + */ + public static final ArgumentKey FS_OBJECT_RENAME_ENABLED = + new ArgumentKey("fs.%s.rename.enabled"); + public static final boolean FS_OBJECT_RENAME_ENABLED_DEFAULT = false; + + /** + * The range size when open object storage input stream. Value must be positive. + */ + public static final String FS_OBJECT_STREAM_RANGE_SIZE = "fs.objectstorage.stream.range-size"; + public static final long FS_OBJECT_STREAM_RANGE_SIZE_DEFAULT = Long.MAX_VALUE; + + /** + * The size of thread pool used for running tasks in parallel for the given object fs, + * e.g. delete objects, copy files. the key example: fs.tos.task.thread-pool-size. + */ + public static final ArgumentKey FS_TASK_THREAD_POOL_SIZE = + new ArgumentKey("fs.%s.task.thread-pool-size"); + public static final int FS_TASK_THREAD_POOL_SIZE_DEFAULT = + Math.max(2, Runtime.getRuntime().availableProcessors()); + + /** + * The size of thread pool used for uploading multipart in parallel for the given object storage, + * e.g. fs.tos.multipart.thread-pool-size + */ + public static final ArgumentKey FS_MULTIPART_THREAD_POOL_SIZE = + new ArgumentKey("fs.%s.multipart.thread-pool-size"); + public static final int FS_MULTIPART_THREAD_POOL_SIZE_DEFAULT = + Math.max(2, Runtime.getRuntime().availableProcessors()); + + /** + * The toggle indicates whether enable checksum during getting file status for the given object. + * E.g. fs.tos.checksum.enabled + */ + public static final ArgumentKey FS_CHECKSUM_ENABLED = new ArgumentKey("fs.%s.checksum.enabled"); + public static final boolean FS_CHECKSUM_ENABLED_DEFAULT = true; + + public static String defaultDir(String basename) { + String tmpdir = System.getProperty("java.io.tmpdir"); + Preconditions.checkNotNull(tmpdir, "System property 'java.io.tmpdir' cannot be null"); + + if (tmpdir.endsWith("/")) { + return String.format("%s%s", tmpdir, basename); + } else { + return String.format("%s/%s", tmpdir, basename); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/FileStoreKeys.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/FileStoreKeys.java new file mode 100644 index 0000000000000..9ef098e2bbda5 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/FileStoreKeys.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.tosfs.conf; + +import org.apache.hadoop.fs.tosfs.object.ChecksumType; + +public final class FileStoreKeys { + + private FileStoreKeys() {} + + /** + * The key indicates the name of the filestore checksum algorithm. Specify the algorithm name to + * satisfy different storage systems. For example, the hdfs style name is COMPOSITE-CRC32 and + * COMPOSITE-CRC32C. + */ + public static final String FS_FILESTORE_CHECKSUM_ALGORITHM = "fs.filestore.checksum-algorithm"; + public static final String FS_FILESTORE_CHECKSUM_ALGORITHM_DEFAULT = "TOS-CHECKSUM"; + + /** + * The key indicates how to retrieve file checksum from filestore, error will be thrown if the + * configured checksum type is not supported. The supported checksum type is: MD5. + */ + public static final String FS_FILESTORE_CHECKSUM_TYPE = "fs.filestore.checksum-type"; + public static final String FS_FILESTORE_CHECKSUM_TYPE_DEFAULT = ChecksumType.MD5.name(); +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/TosKeys.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/TosKeys.java new file mode 100644 index 0000000000000..3266fc5f252b4 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/TosKeys.java @@ -0,0 +1,276 @@ +/* + * 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.tosfs.conf; + +import org.apache.hadoop.fs.tosfs.object.ChecksumType; +import org.apache.hadoop.fs.tosfs.object.tos.TOSErrorCodes; + +public final class TosKeys { + + private TosKeys() {} + + /** + * The accessKey key to access the tos object storage. + */ + public static final String FS_TOS_ACCESS_KEY_ID = "fs.tos.access-key-id"; + + /** + * The secret access key to access the object storage. + */ + public static final String FS_TOS_SECRET_ACCESS_KEY = "fs.tos.secret-access-key"; + + /** + * The session token to access the object storage. + */ + public static final String FS_TOS_SESSION_TOKEN = "fs.tos.session-token"; + + /** + * The access key to access the object storage for the configured bucket, where %s is the bucket + * name. + */ + public static final ArgumentKey FS_TOS_BUCKET_ACCESS_KEY_ID = + new ArgumentKey("fs.tos.bucket.%s.access-key-id"); + + /** + * The secret access key to access the object storage for the configured bucket, where %s is the + * bucket name. + */ + public static final ArgumentKey FS_TOS_BUCKET_SECRET_ACCESS_KEY = + new ArgumentKey("fs.tos.bucket.%s.secret-access-key"); + + /** + * The session token to access the object storage for the configured bucket, where %s is the + * bucket name. + */ + public static final ArgumentKey FS_TOS_BUCKET_SESSION_TOKEN = + new ArgumentKey("fs.tos.bucket.%s.session-token"); + + // Credential + /** + * Default credentials provider chain that looks for credentials in this order: + * SimpleCredentialsProvider,EnvironmentCredentialsProvider. + */ + public static final String FS_TOS_CREDENTIALS_PROVIDER = "fs.tos.credentials.provider"; + public static final String FS_TOS_CREDENTIALS_PROVIDER_DEFAULT = + "org.apache.hadoop.fs.tosfs.object.tos.auth.DefaultCredentialsProviderChain"; + + /** + * User customized credential provider classes, separate provider class name with comma if there + * are multiple providers. + */ + public static final String FS_TOS_CUSTOM_CREDENTIAL_PROVIDER_CLASSES = + "fs.tos.credential.provider.custom.classes"; + + public static final String[] FS_TOS_CUSTOM_CREDENTIAL_PROVIDER_CLASSES_DEFAULT = + new String[] {"org.apache.hadoop.fs.tosfs.object.tos.auth.EnvironmentCredentialsProvider", + "org.apache.hadoop.fs.tosfs.object.tos.auth.SimpleCredentialsProvider"}; + + /** + * Set a canned ACL for newly created and copied objects. Value may be 'private', 'public-read', + * 'public-read-write', 'authenticated-read', 'bucket-owner-read', 'bucket-owner-full-control', + * 'bucket-owner-entrusted'. If set, caller IAM role must have 'tos:PutObjectAcl' permission on + * the bucket. + */ + public static final String FS_TOS_ACL_DEFAULT = "fs.tos.acl.default"; + + // TOS http client. + /** + * The maximum number of connections to the TOS service that a client can create. + */ + public static final String FS_TOS_HTTP_MAX_CONNECTIONS = "fs.tos.http.maxConnections"; + public static final int FS_TOS_HTTP_MAX_CONNECTIONS_DEFAULT = 1024; + + /** + * The time that a connection thread can be in idle state, larger than which the thread will be + * terminated. + */ + public static final String FS_TOS_HTTP_IDLE_CONNECTION_TIME_MILLS = + "fs.tos.http.idleConnectionTimeMills"; + public static final int FS_TOS_HTTP_IDLE_CONNECTION_TIME_MILLS_DEFAULT = 60000; + + /** + * The connect timeout that the tos client tries to connect to the TOS service. + */ + public static final String FS_TOS_HTTP_CONNECT_TIMEOUT_MILLS = "fs.tos.http.connectTimeoutMills"; + public static final int FS_TOS_HTTP_CONNECT_TIMEOUT_MILLS_DEFAULT = 10000; + + /** + * The reading timeout when reading data from tos. Note that it is configured for the tos client + * sdk, not hadoop-tos. + */ + public static final String FS_TOS_HTTP_READ_TIMEOUT_MILLS = "fs.tos.http.readTimeoutMills"; + public static final int FS_TOS_HTTP_READ_TIMEOUT_MILLS_DEFAULT = 30000; + + /** + * The writing timeout when uploading data to tos. Note that it is configured for the tos client + * sdk, not hadoop-tos. + */ + public static final String FS_TOS_HTTP_WRITE_TIMEOUT_MILLS = "fs.tos.http.writeTimeoutMills"; + public static final int FS_TOS_HTTP_WRITE_TIMEOUT_MILLS_DEFAULT = 30000; + + /** + * Enables SSL connections to TOS or not. + */ + public static final String FS_TOS_HTTP_ENABLE_VERIFY_SSL = "fs.tos.http.enableVerifySSL"; + public static final boolean FS_TOS_HTTP_ENABLE_VERIFY_SSL_DEFAULT = true; + + /** + * The timeout (in minutes) of the dns cache used in tos client. + */ + public static final String FS_TOS_HTTP_DNS_CACHE_TIME_MINUTES = "fs.tos.http.dnsCacheTimeMinutes"; + public static final int FS_TOS_HTTP_DNS_CACHE_TIME_MINUTES_DEFAULT = 0; + + /** + * Used for directory bucket, whether enable recursive delete capability in TOS server, which will + * atomic delete all objects under given dir(inclusive), otherwise the client will list all sub + * objects, and then send batch delete request to TOS to delete dir. + */ + public static final String FS_TOS_RMR_SERVER_ENABLED = "fs.tos.rmr.server.enabled"; + public static final boolean FS_FS_TOS_RMR_SERVER_ENABLED_DEFAULT = false; + + /** + * If fs.tos.rmr.client.enabled is true, client will list all objects under the given dir and + * delete them by batch. Set value with true will use the recursive delete capability of TOS SDK, + * otherwise will delete object one by one via preorder tree walk. + */ + public static final String FS_TOS_RMR_CLIENT_ENABLE = "fs.tos.rmr.client.enabled"; + public static final boolean FS_TOS_RMR_CLIENT_ENABLE_DEFAULT = true; + + /** + * The prefix will be used as the product name in TOS SDK. The final user agent pattern is + * '{prefix}/TOS_FS/{hadoop tos version}'. + */ + public static final String FS_TOS_USER_AGENT_PREFIX = "fs.tos.user.agent.prefix"; + public static final String FS_TOS_USER_AGENT_PREFIX_DEFAULT = "HADOOP-TOS"; + + // TOS common keys. + /** + * The threshold indicates whether reuse the socket connection to optimize read performance during + * closing tos object inputstream of get object. If the remaining bytes is less than max drain + * bytes during closing the inputstream, will just skip the bytes instead of closing the socket + * connection. + */ + public static final String FS_TOS_MAX_DRAIN_BYTES = "fs.tos.max-drain-bytes"; + public static final long FS_TOS_MAX_DRAIN_BYTES_DEFAULT = 1024 * 1024L; + + /** + * Whether disable the tos http client cache in the current JVM. + */ + public static final String FS_TOS_DISABLE_CLIENT_CACHE = "fs.tos.client.disable.cache"; + public static final boolean FS_TOS_DISABLE_CLIENT_CACHE_DEFAULT = false; + + /** + * The batch size when deleting the objects in batches. + */ + public static final String FS_TOS_DELETE_OBJECTS_COUNT = "fs.tos.batch.delete.objects-count"; + public static final int FS_TOS_DELETE_OBJECTS_COUNT_DEFAULT = 1000; + + /** + * The maximum retry times when deleting objects in batches failed. + */ + public static final String FS_TOS_BATCH_DELETE_MAX_RETRIES = "fs.tos.batch.delete.max-retries"; + public static final int FS_TOS_BATCH_DELETE_MAX_RETRIES_DEFAULT = 20; + + /** + * The codes from TOS deleteMultiObjects response, client will resend the batch delete request to + * delete the failed keys again if the response only contains these codes, otherwise won't send + * request anymore. + */ + public static final String FS_TOS_BATCH_DELETE_RETRY_CODES = "fs.tos.batch.delete.retry-codes"; + public static final String[] FS_TOS_BATCH_DELETE_RETRY_CODES_DEFAULT = + new String[] {"ExceedAccountQPSLimit", "ExceedAccountRateLimit", "ExceedBucketQPSLimit", + "ExceedBucketRateLimit", "InternalError", "ServiceUnavailable", "SlowDown", + "TooManyRequests"}; + + /** + * The retry interval (in milliseconds) when deleting objects in batches failed. + */ + public static final String FS_TOS_BATCH_DELETE_RETRY_INTERVAL = + "fs.tos.batch.delete.retry.interval"; + public static final long FS_TOS_BATCH_DELETE_RETRY_INTERVAL_DEFAULT = 1000L; + + /** + * The batch size of listing object per request for the given object storage, such as listing a + * directory, searching for all objects whose path starts with the directory path, and returning + * them as a list. + */ + public static final String FS_TOS_LIST_OBJECTS_COUNT = "fs.tos.list.objects-count"; + public static final int FS_TOS_LIST_OBJECTS_COUNT_DEFAULT = 1000; + + /** + * The maximum retry times of sending request via TOS client, client will resend the request if + * got retryable exceptions, e.g. SocketException, UnknownHostException, SSLException, + * InterruptedException, SocketTimeoutException, or got TOO_MANY_REQUESTS, INTERNAL_SERVER_ERROR + * http codes. + */ + public static final String FS_TOS_REQUEST_MAX_RETRY_TIMES = "fs.tos.request.max.retry.times"; + public static final int FS_TOS_REQUEST_MAX_RETRY_TIMES_DEFAULT = 20; + + /** + * The fast-fail error codes means the error cannot be solved by retrying the request. TOS client + * won't retry the request if receiving a 409 http status code and if the error code is in the + * configured non-retryable error code list. + */ + public static final String FS_TOS_FAST_FAILURE_409_ERROR_CODES = + "fs.tos.fast-fail-409-error-codes"; + public static final String FS_TOS_FAST_FAILURE_409_ERROR_CODES_DEFAULT = + TOSErrorCodes.FAST_FAILURE_CONFLICT_ERROR_CODES; + + /** + * The maximum retry times of reading object content via TOS client, client will resend the + * request to create a new input stream if getting unexpected end of stream error during reading + * the input stream. + */ + public static final String FS_TOS_MAX_READ_OBJECT_RETRIES = "fs.tos.inputstream.max.retry.times"; + public static final int FS_TOS_MAX_READ_OBJECT_RETRIES_DEFAULT = 5; + + /** + * Enable the crc check when uploading files to tos or not. + */ + public static final String FS_TOS_CRC_CHECK_ENABLED = "fs.tos.crc.check.enable"; + public static final boolean FS_TOS_CRC_CHECK_ENABLED_DEFAULT = true; + + /** + * Whether enable tos getFileStatus API or not, which returns the object info directly in one RPC + * request, otherwise, might need to send three RPC requests to get object info. + * For example, there is a key 'a/b/c' exists in TOS, and we want to get object status of 'a/b', + * the GetFileStatus('a/b') will return the prefix 'a/b/' as a directory object directly. If this + * property is disabled, we need to head('a/b') at first, and then head('a/b/'), and last call + * list('a/b/', limit=1) to get object info. Using GetFileStatus API can reduce the RPC call + * times. + */ + public static final String FS_TOS_GET_FILE_STATUS_ENABLED = "fs.tos.get-file-status.enabled"; + public static final boolean FS_TOS_GET_FILE_STATUS_ENABLED_DEFAULT = true; + + /** + * The key indicates the name of the tos checksum algorithm. Specify the algorithm name to compare + * checksums between different storage systems. For example to compare checksums between hdfs and + * tos, we need to configure the algorithm name to COMPOSITE-CRC32C. + */ + public static final String FS_TOS_CHECKSUM_ALGORITHM = "fs.tos.checksum-algorithm"; + public static final String FS_TOS_CHECKSUM_ALGORITHM_DEFAULT = "TOS-CHECKSUM"; + + /** + * The key indicates how to retrieve file checksum from tos, error will be thrown if the + * configured checksum type is not supported by tos. The supported checksum types are: + * CRC32C, CRC64ECMA. + */ + public static final String FS_TOS_CHECKSUM_TYPE = "fs.tos.checksum-type"; + public static final String FS_TOS_CHECKSUM_TYPE_DEFAULT = ChecksumType.CRC64ECMA.name(); +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/package-info.java new file mode 100644 index 0000000000000..95b0f6c044778 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/package-info.java @@ -0,0 +1,26 @@ +/* + * 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. + */ + +/** + * Classes for hadoop-tos conf. + */ +@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"}) +@InterfaceStability.Evolving +package org.apache.hadoop.fs.tosfs.conf; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/BucketInfo.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/BucketInfo.java new file mode 100644 index 0000000000000..8c9a9d6e15216 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/BucketInfo.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.tosfs.object; + +import org.apache.hadoop.thirdparty.com.google.common.base.MoreObjects; + +import java.util.Objects; + +/** + * There are two kinds of bucket types: general purpose bucket(common bucket) and directory bucket. + * Directory bucket organize data hierarchically into directories as opposed to the flat storage + * structure of general purpose buckets. + *

+ * Only a few object storages support directory bucket, e.g. S3, OBS, TOS. Generally, directory + * bucket supports rename or delete dir with constant time complexity O(1), but these object + * storages have slight differences on these APIs. E.g. S3 doesn't provide rename API. S3 will + * recursively delete any empty parent directories in the object path during delete an object in a + * directory bucket, but TOS won't delete any empty parent directories. + *

+ * And also there are some general difference between general purpose bucket and directory bucket. + *

    + *
  • Directory bucket treats the object end with '/' as the directory during creating object. + *
  • + *
  • TOS directory bucket will create missed parent dir during create object automatically, + * but general purpose bucket only create one object.
  • + *
  • Directory bucket doesn't allow create any object under a file, but general purpose bucket + * haven't this constraint.
  • + *
  • If a object 'a/b/' exists in directory bucket, both head('a/b') and head('a/b/') will get + * the object meta, but only head('a/b/') can get the object meta from general purpose bucket
  • + *
  • TOS directory bucket provides atomic rename/delete dir abilities
  • + *
+ */ +public class BucketInfo { + private final String name; + private final boolean directory; + + public BucketInfo(String name, boolean directory) { + this.name = name; + this.directory = directory; + } + + public String name() { + return name; + } + + public boolean isDirectory() { + return directory; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof BucketInfo)) { + return false; + } + + BucketInfo that = (BucketInfo) o; + return Objects.equals(name, that.name) + && Objects.equals(directory, that.directory); + } + + @Override + public int hashCode() { + return Objects.hash(name, directory); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("name", name) + .add("directory", directory) + .toString(); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ChecksumInfo.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ChecksumInfo.java new file mode 100644 index 0000000000000..b7e529d3c1147 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ChecksumInfo.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.tosfs.object; + +public class ChecksumInfo { + private final String algorithm; + private final ChecksumType checksumType; + + public ChecksumInfo(String algorithm, ChecksumType checksumType) { + this.algorithm = algorithm; + this.checksumType = checksumType; + } + + public String algorithm() { + return algorithm; + } + + public ChecksumType checksumType() { + return checksumType; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ChecksumType.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ChecksumType.java new file mode 100644 index 0000000000000..b0159f3fb5897 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ChecksumType.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.tosfs.object; + +public enum ChecksumType { + CRC32C((byte) 2, 4), + CRC64ECMA((byte) 3, 8), + MD5((byte) 4, 128); + + private final byte value; + private final int bytes; + + ChecksumType(byte value, int bytes) { + this.value = value; + this.bytes = bytes; + } + + public byte value() { + return value; + } + + public int bytes() { + return bytes; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/Constants.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/Constants.java new file mode 100644 index 0000000000000..163c99c2eb2ee --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/Constants.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.tosfs.object; + +public final class Constants { + private Constants() { + } + + public static final String TOS_FS = "TOS_FS"; + + // Magic checksum means doesn't support checksum, if the file type is dir or the filesystem/object + // storage doesn't implement checksum algorithm will use magic checksum as the file checksum. + public static final byte[] MAGIC_CHECKSUM = new byte[] {'M'}; + + public static final String SLASH = "/"; +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/DirectoryStorage.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/DirectoryStorage.java new file mode 100644 index 0000000000000..d840c9842a0c5 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/DirectoryStorage.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.tosfs.object; + +public interface DirectoryStorage extends ObjectStorage { + + /** + * List the objects under the given dir key. Does not guarantee to return the list of objects in a + * sorted order. + * + * @param key the dir key. + * @param recursive indicate whether list dir recursively or not. + * @return the sub objects of the given dir key, not include the dir key. + */ + Iterable listDir(String key, boolean recursive); + + /** + * The given key could be a file or a directory, if a directory, it can an empty or non-empty + * directory. + * Deleting a non-empty dir is only allowed if recursive is true. + * + * @param key the dir key. + * @param recursive indicate whether delete dir recursively or not. + */ + void deleteDir(String key, boolean recursive); + + /** + * Whether the directory is empty. + * + * @param key the dir key. + * @return true if the dir is empty or doesn't exist. + */ + boolean isEmptyDir(String key); +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/FileStore.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/FileStore.java new file mode 100644 index 0000000000000..afd3b81c5a88a --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/FileStore.java @@ -0,0 +1,648 @@ +/* + * 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.tosfs.object; + +import org.apache.commons.codec.digest.DigestUtils; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.conf.FileStoreKeys; +import org.apache.hadoop.fs.tosfs.object.exceptions.NotAppendableException; +import org.apache.hadoop.fs.tosfs.object.request.ListObjectsRequest; +import org.apache.hadoop.fs.tosfs.object.response.ListObjectsResponse; +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.fs.tosfs.util.Range; +import org.apache.hadoop.fs.tosfs.util.UUIDUtils; +import org.apache.hadoop.thirdparty.com.google.common.base.Strings; +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.UnsupportedEncodingException; +import java.net.URLDecoder; +import java.net.URLEncoder; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Date; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.TreeSet; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class FileStore implements ObjectStorage { + + private static final Logger LOG = LoggerFactory.getLogger(FileStore.class); + + private static final String NAME = "filestore"; + private static final String STAGING_DIR = "__STAGING__"; + private static final String SLASH = "/"; + + public static final String DEFAULT_BUCKET = "dummy-bucket"; + public static final String ENV_FILE_STORAGE_ROOT = "FILE_STORAGE_ROOT"; + + private static final int MAX_DELETE_OBJECTS_COUNT = 1000; + private static final int MIN_PART_SIZE = 5 * 1024 * 1024; + private static final int MAX_PART_COUNT = 10000; + + private String bucket; + private String root; + private Configuration conf; + private ChecksumInfo checksumInfo; + + @Override + public String scheme() { + return NAME; + } + + @Override + public BucketInfo bucket() { + return new BucketInfo(bucket, false); + } + + @Override + public void initialize(Configuration config, String bucketName) { + this.bucket = bucketName; + this.conf = config; + String endpoint = config.get(ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key(NAME)); + if (endpoint == null || endpoint.isEmpty()) { + endpoint = System.getenv(ENV_FILE_STORAGE_ROOT); + } + Preconditions.checkNotNull(endpoint, "%s cannot be null", + ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key(NAME)); + + if (endpoint.endsWith(SLASH)) { + this.root = endpoint; + } else { + this.root = endpoint + SLASH; + } + LOG.debug("the root path is: {}", this.root); + + String algorithm = config.get(FileStoreKeys.FS_FILESTORE_CHECKSUM_ALGORITHM, + FileStoreKeys.FS_FILESTORE_CHECKSUM_ALGORITHM_DEFAULT); + ChecksumType checksumType = ChecksumType.valueOf( + config.get(FileStoreKeys.FS_FILESTORE_CHECKSUM_TYPE, + FileStoreKeys.FS_FILESTORE_CHECKSUM_TYPE_DEFAULT).toUpperCase()); + Preconditions.checkArgument(checksumType == ChecksumType.MD5, + "Checksum type %s is not supported by FileStore.", checksumType.name()); + checksumInfo = new ChecksumInfo(algorithm, checksumType); + + File rootDir = new File(root); + if (!rootDir.mkdirs() && !rootDir.exists()) { + throw new IllegalArgumentException("Failed to create root dir. " + root); + } else { + LOG.info("Create root dir successfully. {}", root); + } + } + + @Override + public Configuration conf() { + return conf; + } + + private static String encode(String key) { + try { + return URLEncoder.encode(key, "UTF-8"); + } catch (UnsupportedEncodingException e) { + LOG.warn("failed to encode key: {}", key); + return key; + } + } + + private static String decode(String key) { + try { + return URLDecoder.decode(key, "UTF-8"); + } catch (UnsupportedEncodingException e) { + LOG.warn("failed to decode key: {}", key); + return key; + } + } + + @Override + public ObjectContent get(String key, long offset, long limit) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(key), "Key should not be empty."); + File file = path(encode(key)).toFile(); + if (!file.exists()) { + throw new RuntimeException(String.format("File not found %s", file.getAbsolutePath())); + } + + Range range = ObjectUtils.calculateRange(offset, limit, file.length()); + try (FileInputStream in = new FileInputStream(file)) { + in.skip(range.off()); + byte[] bs = new byte[(int) range.len()]; + in.read(bs); + + byte[] fileChecksum = getFileChecksum(file.toPath()); + return new ObjectContent(fileChecksum, new ByteArrayInputStream(bs)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public byte[] put(String key, InputStreamProvider streamProvider, long contentLength) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(key), "Key should not be empty."); + File destFile = path(encode(key)).toFile(); + copyInputStreamToFile(streamProvider.newStream(), destFile, contentLength); + + return ObjectInfo.isDir(key) ? Constants.MAGIC_CHECKSUM : getFileChecksum(destFile.toPath()); + } + + @Override + public byte[] append(String key, InputStreamProvider streamProvider, long contentLength) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(key), "Key should not be empty."); + File destFile = path(encode(key)).toFile(); + if (!destFile.exists()) { + if (contentLength == 0) { + throw new NotAppendableException(String.format( + "%s is not appendable because append non-existed object with " + + "zero byte is not supported.", key)); + } + return put(key, streamProvider, contentLength); + } else { + appendInputStreamToFile(streamProvider.newStream(), destFile, contentLength); + return ObjectInfo.isDir(key) ? Constants.MAGIC_CHECKSUM : getFileChecksum(destFile.toPath()); + } + } + + private static File createTmpFile(File destFile) { + String tmpFilename = ".tmp." + UUIDUtils.random(); + File file = new File(destFile.getParentFile(), tmpFilename); + + try { + if (!file.exists() && !file.createNewFile()) { + throw new RuntimeException("failed to create tmp file"); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + return file; + } + + @Override + public void delete(String key) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(key), "Key should not be empty."); + File file = path(encode(key)).toFile(); + if (file.exists()) { + try { + if (file.isDirectory()) { + FileUtils.deleteDirectory(file); + } else { + Files.delete(file.toPath()); + } + } catch (IOException ex) { + throw new RuntimeException(ex); + } + } + } + + @Override + public List batchDelete(List keys) { + Preconditions.checkArgument(keys.size() <= MAX_DELETE_OBJECTS_COUNT, + "The batch delete object count should <= %s", MAX_DELETE_OBJECTS_COUNT); + List failedKeys = Lists.newArrayList(); + for (String key : keys) { + try { + delete(key); + } catch (Exception e) { + LOG.error("Failed to delete key {}", key, e); + failedKeys.add(key); + } + } + return failedKeys; + } + + @Override + public void deleteAll(String prefix) { + Iterable objects = listAll(prefix, ""); + ObjectUtils.deleteAllObjects(this, objects, conf.getInt( + ConfKeys.FS_BATCH_DELETE_SIZE.key(NAME), + ConfKeys.FS_BATCH_DELETE_SIZE_DEFAULT)); + } + + @Override + public ObjectInfo head(String key) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(key), "Key should not be empty."); + File file = path(encode(key)).toFile(); + if (file.exists()) { + return toObjectInfo(file.toPath()); + } else { + return null; + } + } + + @Override + public Iterable list(ListObjectsRequest request) { + try (Stream stream = Files.walk(Paths.get(root))) { + List allObjects = list(stream, request.prefix(), request.startAfter()) + .collect(Collectors.toList()); + int maxKeys = request.maxKeys() < 0 ? allObjects.size() : request.maxKeys(); + return Collections.singletonList( + splitObjects(request.prefix(), request.delimiter(), maxKeys, request.startAfter(), + allObjects)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private ListObjectsResponse splitObjects( + String prefix, + String delimiter, + int limit, + String startAfter, + List objects) { + int retSize = Math.min(limit, objects.size()); + + if (Strings.isNullOrEmpty(delimiter)) { + // the response only contains objects + List retObjs = objects.subList(0, retSize); + return new ListObjectsResponse(retObjs, Collections.emptyList()); + } else { + // the response only contains objects and common prefixes + Set commonPrefixes = new TreeSet<>(); + List objectInfos = new ArrayList<>(); + + for (ObjectInfo obj : objects) { + String suffixKey = obj.key().substring(prefix.length()); + String[] tokens = suffixKey.split(delimiter, 2); + if (tokens.length == 2) { + String key = prefix + tokens[0] + delimiter; + // the origin key is bigger than startAfter, + // but after the new key after split might equal to startAfter, need to exclude. + if (!key.equals(startAfter)) { + commonPrefixes.add(key); + + // why don't break the loop before add new key to common prefixes list + // is that the new key might be an existed common prefix, but we still want to continue + // visited new object since the new object might also be an existed common prefix until + // the total size is out of limit. + if (commonPrefixes.size() + objectInfos.size() > retSize) { + commonPrefixes.remove(key); + break; + } + } + } else { + if (commonPrefixes.size() + objectInfos.size() >= retSize) { + break; + } + + objectInfos.add(obj); + } + } + return new ListObjectsResponse(objectInfos, new ArrayList<>(commonPrefixes)); + } + } + + @Override + public MultipartUpload createMultipartUpload(String key) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(key), "Key should not be empty."); + + String uploadId = UUIDUtils.random(); + Path uploadDir = uploadPath(key, uploadId); + if (uploadDir.toFile().mkdirs()) { + return new MultipartUpload(key, uploadId, MIN_PART_SIZE, MAX_PART_COUNT); + } else { + throw new RuntimeException("Failed to create MultipartUpload with key: " + key); + } + } + + private Path uploadPath(String key, String uploadId) { + return Paths.get(root, STAGING_DIR, encode(key), uploadId); + } + + @Override + public Part uploadPart( + String key, String uploadId, int partNum, + InputStreamProvider streamProvider, long contentLength) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(key), "Key should not be empty."); + + File uploadDir = uploadPath(key, uploadId).toFile(); + if (!uploadDir.exists()) { + throw new RuntimeException("cannot locate the upload id: " + uploadId); + } + + File partFile = new File(uploadDir, String.valueOf(partNum)); + copyInputStreamToFile(streamProvider.newStream(), partFile, contentLength); + + try { + byte[] data = Files.readAllBytes(partFile.toPath()); + return new Part(partNum, data.length, DigestUtils.md5Hex(data)); + } catch (IOException e) { + LOG.error("failed to locate the part file: {}", partFile.getAbsolutePath()); + throw new RuntimeException(e); + } + } + + private static void appendInputStreamToFile(InputStream in, File partFile, long contentLength) { + try (FileOutputStream out = new FileOutputStream(partFile, true)) { + long copiedBytes = IOUtils.copyLarge(in, out, 0, contentLength); + + if (copiedBytes < contentLength) { + throw new IOException(String.format("Unexpect end of stream, expected to write length:%s," + + " actual written:%s", contentLength, copiedBytes)); + } + } catch (IOException e) { + throw new RuntimeException(e); + } finally { + CommonUtils.runQuietly(in::close); + } + } + + private static void copyInputStreamToFile(InputStream in, File partFile, long contentLength) { + File tmpFile = createTmpFile(partFile); + try (FileOutputStream out = new FileOutputStream(tmpFile)) { + long copiedBytes = IOUtils.copyLarge(in, out, 0, contentLength); + + if (copiedBytes < contentLength) { + throw new IOException( + String.format("Unexpect end of stream, expected length:%s, actual:%s", contentLength, + tmpFile.length())); + } + } catch (IOException e) { + CommonUtils.runQuietly(() -> FileUtils.delete(tmpFile)); + throw new RuntimeException(e); + } finally { + CommonUtils.runQuietly(in::close); + } + + if (!tmpFile.renameTo(partFile)) { + throw new RuntimeException("failed to put file since rename fail."); + } + } + + @Override + public byte[] completeUpload(String key, String uploadId, List uploadParts) { + Preconditions.checkArgument(uploadParts != null && uploadParts.size() > 0, + "upload parts cannot be null or empty."); + File uploadDir = uploadPath(key, uploadId).toFile(); + if (!uploadDir.exists()) { + throw new RuntimeException("cannot locate the upload id: " + uploadId); + } + + List partNums = listPartNums(uploadDir); + if (partNums.size() != uploadParts.size()) { + throw new RuntimeException(String.format("parts length mismatched: %d != %d", + partNums.size(), uploadParts.size())); + } + + Collections.sort(partNums); + uploadParts.sort(Comparator.comparingInt(Part::num)); + + Path keyPath = path(encode(key)); + File tmpFile = createTmpFile(keyPath.toFile()); + try (FileOutputStream outputStream = new FileOutputStream(tmpFile); + FileChannel outputChannel = outputStream.getChannel()) { + int offset = 0; + for (int i = 0; i < partNums.size(); i++) { + Part part = uploadParts.get(i); + if (part.num() != partNums.get(i)) { + throw new RuntimeException( + String.format("part num mismatched: %d != %d", part.num(), partNums.get(i))); + } + + File partFile = new File(uploadDir, String.valueOf(part.num())); + checkPartFile(part, partFile); + + try (FileInputStream inputStream = new FileInputStream(partFile); + FileChannel inputChannel = inputStream.getChannel()) { + outputChannel.transferFrom(inputChannel, offset, partFile.length()); + offset += partFile.length(); + } + } + } catch (IOException e) { + throw new RuntimeException(e); + } + + if (!tmpFile.renameTo(keyPath.toFile())) { + throw new RuntimeException("rename file failed"); + } else { + try { + FileUtils.deleteDirectory(uploadDir); + } catch (IOException e) { + LOG.warn("failed to clean upload directory."); + } + } + + return getFileChecksum(keyPath); + } + + private byte[] getFileChecksum(Path keyPath) { + return getFileMD5(keyPath); + } + + private static byte[] getFileMD5(Path keyPath) { + try { + return DigestUtils.md5(Files.readAllBytes(keyPath)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private static void checkPartFile(Part part, File partFile) throws IOException { + if (part.size() != partFile.length()) { + throw new RuntimeException(String.format("part size mismatched: %d != %d", + part.size(), partFile.length())); + } + + try (FileInputStream inputStream = new FileInputStream(partFile)) { + String md5Hex = DigestUtils.md5Hex(inputStream); + if (!Objects.equals(part.eTag(), md5Hex)) { + throw new RuntimeException(String.format("part etag mismatched: %s != %s", + part.eTag(), md5Hex)); + } + } + } + + private List listPartNums(File uploadDir) { + try (Stream stream = Files.list(uploadDir.toPath())) { + return stream + .map(f -> Integer.valueOf(f.toFile().getName())) + .collect(Collectors.toList()); + } catch (IOException e) { + LOG.error("failed to list part files."); + throw new RuntimeException(e); + } + } + + @Override + public void abortMultipartUpload(String key, String uploadId) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(key), "Key should not be empty."); + Path uploadDir = uploadPath(key, uploadId); + if (uploadDir.toFile().exists()) { + try { + FileUtils.deleteDirectory(uploadDir.toFile()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + @Override + public Iterable listUploads(String prefix) { + Path stagingDir = Paths.get(root, STAGING_DIR); + if (!Files.exists(stagingDir)) { + return Collections.emptyList(); + } + try (Stream encodedKeyStream = Files.list(stagingDir)) { + return encodedKeyStream + .filter(key -> Objects.equals(prefix, "") || + key.toFile().getName().startsWith(encode(prefix))) + .flatMap(key -> { + try { + return Files.list(key) + .map(id -> new MultipartUpload(decode(key.toFile().getName()), + id.toFile().getName(), MIN_PART_SIZE, MAX_PART_COUNT)); + } catch (IOException e) { + throw new RuntimeException(e); + } + }) + .sorted() + .collect(Collectors.toList()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public Part uploadPartCopy( + String srcKey, String dstKey, String uploadId, int partNum, long copySourceRangeStart, + long copySourceRangeEnd) { + File uploadDir = uploadPath(dstKey, uploadId).toFile(); + if (!uploadDir.exists()) { + throw new RuntimeException(String.format("Upload directory %s already exits", uploadDir)); + } + File partFile = new File(uploadDir, String.valueOf(partNum)); + int fileSize = (int) (copySourceRangeEnd - copySourceRangeStart + 1); + try (InputStream is = get(srcKey, copySourceRangeStart, fileSize).stream(); + FileOutputStream fos = new FileOutputStream(partFile)) { + byte[] data = new byte[fileSize]; + IOUtils.readFully(is, data); + fos.write(data); + return new Part(partNum, fileSize, DigestUtils.md5Hex(data)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public void copy(String srcKey, String dstKey) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(srcKey), "Src key should not be empty."); + File file = path(encode(srcKey)).toFile(); + if (!file.exists()) { + throw new RuntimeException(String.format("File not found %s", file.getAbsolutePath())); + } + + put(dstKey, () -> get(srcKey).stream(), file.length()); + } + + @Override + public void rename(String srcKey, String dstKey) { + Preconditions.checkArgument(!Objects.equals(srcKey, dstKey), + "Cannot rename to the same object"); + Preconditions.checkNotNull(head(srcKey), "Source key %s doesn't exist", srcKey); + + File srcFile = path(encode(srcKey)).toFile(); + File dstFile = path(encode(dstKey)).toFile(); + boolean ret = srcFile.renameTo(dstFile); + if (!ret) { + throw new RuntimeException(String.format("Failed to rename %s to %s", srcKey, dstKey)); + } + } + + @Override + public ObjectInfo objectStatus(String key) { + ObjectInfo obj = head(key); + if (obj == null && !ObjectInfo.isDir(key)) { + key = key + '/'; + obj = head(key); + } + + if (obj == null) { + Iterable objs = list(key, null, 1); + if (objs.iterator().hasNext()) { + obj = new ObjectInfo(key, 0, new Date(0), Constants.MAGIC_CHECKSUM); + } + } + + return obj; + } + + @Override + public ChecksumInfo checksumInfo() { + return checksumInfo; + } + + private Stream list(Stream stream, String prefix, String startAfter) { + return stream + .filter(p -> { + String absolutePath = p.toFile().getAbsolutePath(); + return !Objects.equals(key(absolutePath), "") && + decode(key(absolutePath)).startsWith(prefix) + && !absolutePath.contains(STAGING_DIR) + && filter(decode(key(absolutePath)), startAfter); + }) + .map(this::toObjectInfo) + .sorted(Comparator.comparing(ObjectInfo::key)); + } + + private boolean filter(String key, String startAfter) { + if (Strings.isNullOrEmpty(startAfter)) { + return true; + } else { + return key.compareTo(startAfter) > 0; + } + } + + private ObjectInfo toObjectInfo(Path path) { + File file = path.toFile(); + String key = decode(key(file.getAbsolutePath())); + return new ObjectInfo(key, file.length(), new Date(file.lastModified()), + getFileChecksum(path)); + } + + private Path path(String key) { + return Paths.get(root, key); + } + + private String key(String path) { + if (path.length() < root.length()) { + // root = path + "/" + return ""; + } + return path.substring(root.length()); + } + + @Override + public void close() throws IOException { + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/InputStreamProvider.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/InputStreamProvider.java new file mode 100644 index 0000000000000..98e654f99b5e0 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/InputStreamProvider.java @@ -0,0 +1,34 @@ +/* + * 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.tosfs.object; + +import java.io.InputStream; + +/** + * Provides the content stream of a request. + *

+ * Each call to the {@link #newStream()} method must result in a stream + * whose position is at the beginning of the content. + * Implementations may return a new stream or the same stream for each call. + * If returning a new stream, the implementation must ensure to {@code close()} + * and free any resources acquired by the previous stream. + */ +public interface InputStreamProvider { + InputStream newStream(); +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/MultipartUpload.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/MultipartUpload.java new file mode 100644 index 0000000000000..e7c89f1f8279b --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/MultipartUpload.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.tosfs.object; + +import org.apache.hadoop.thirdparty.com.google.common.base.MoreObjects; + +import java.util.Objects; + +public class MultipartUpload implements Comparable { + private final String key; + private final String uploadId; + private final int minPartSize; + private final int maxPartCount; + + public MultipartUpload(String key, String uploadId, int minPartSize, int maxPartCount) { + this.key = key; + this.uploadId = uploadId; + this.minPartSize = minPartSize; + this.maxPartCount = maxPartCount; + } + + public String key() { + return key; + } + + public String uploadId() { + return uploadId; + } + + public int minPartSize() { + return minPartSize; + } + + public int maxPartCount() { + return maxPartCount; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof MultipartUpload)) { + return false; + } + + MultipartUpload that = (MultipartUpload) o; + if (!Objects.equals(key, that.key)) { + return false; + } + if (!Objects.equals(uploadId, that.uploadId)) { + return false; + } + if (!Objects.equals(minPartSize, that.minPartSize)) { + return false; + } + return Objects.equals(maxPartCount, that.maxPartCount); + } + + @Override + public int hashCode() { + return Objects.hash(key, uploadId, minPartSize, maxPartCount); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("key", key) + .add("uploadId", uploadId) + .add("minPartSize", minPartSize) + .add("maxPartCount", maxPartCount) + .toString(); + } + + @Override + public int compareTo(MultipartUpload o) { + if (this == o) { + return 0; + } else if (o == null) { + return 1; + } else if (this.key.compareTo(o.key) == 0) { + return this.uploadId.compareTo(o.uploadId); + } else { + return this.key.compareTo(o.key); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectConstants.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectConstants.java new file mode 100644 index 0000000000000..fb42b0091bc63 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectConstants.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.tosfs.object; + +public final class ObjectConstants { + public static final int MIN_PART_SIZE = 5 * 1024 * 1024; + public static final int MAX_PART_COUNT = 10000; + + private ObjectConstants() { + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectContent.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectContent.java new file mode 100644 index 0000000000000..6961fd83bb7bb --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectContent.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.tosfs.object; + +import org.apache.hadoop.fs.tosfs.object.exceptions.ChecksumMismatchException; +import org.apache.hadoop.fs.tosfs.util.CommonUtils; + +import java.io.InputStream; +import java.util.Arrays; + +public class ObjectContent { + private final byte[] checksum; + private final InputStream stream; + + public ObjectContent(byte[] checksum, InputStream stream) { + this.checksum = checksum; + this.stream = stream; + } + + public InputStream stream() { + return stream; + } + + public InputStream verifiedStream(byte[] expectedChecksum) throws ChecksumMismatchException { + if (!Arrays.equals(expectedChecksum, checksum)) { + CommonUtils.runQuietly(stream::close); + throw new ChecksumMismatchException(expectedChecksum, checksum); + } + + return stream; + } + + public byte[] checksum() { + return checksum; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectInfo.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectInfo.java new file mode 100644 index 0000000000000..08f38ecbdd961 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectInfo.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.tosfs.object; + +import org.apache.hadoop.thirdparty.com.google.common.base.MoreObjects; +import org.apache.hadoop.util.StringUtils; + +import java.util.Arrays; +import java.util.Date; +import java.util.Objects; + +import static org.apache.hadoop.util.Preconditions.checkArgument; + +public class ObjectInfo { + private final String key; + private final long size; + private final Date mtime; + private final boolean isDir; + private final byte[] checksum; + + public ObjectInfo(String key, long size, Date mtime, byte[] checksum) { + this(key, size, mtime, checksum, ObjectInfo.isDir(key)); + } + + public ObjectInfo(String key, long size, Date mtime, byte[] checksum, boolean isDir) { + checkArgument(key != null, "Key is null"); + checkArgument(size >= 0, "The size of key(%s) is negative", key); + checkArgument(mtime != null, "The modified time of key(%s) null.", key); + this.key = key; + this.size = size; + this.mtime = mtime; + this.isDir = isDir; + // checksum can be null since some object storage might not support checksum. + this.checksum = checksum == null || isDir ? Constants.MAGIC_CHECKSUM : checksum; + } + + public String key() { + return key; + } + + /** + * The size of directory object is 0. + * + * @return the size of object. + */ + public long size() { + return isDir ? 0 : size; + } + + public Date mtime() { + return mtime; + } + + /** + * @return {@link Constants#MAGIC_CHECKSUM} if the object is a dir or the object storage + * doesn't support the given checksum type. + */ + public byte[] checksum() { + return checksum; + } + + public boolean isDir() { + return isDir; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof ObjectInfo)) { + return false; + } + + ObjectInfo that = (ObjectInfo) o; + return Objects.equals(key, that.key) + && Objects.equals(size, that.size) + && Objects.equals(mtime, that.mtime) + && Arrays.equals(checksum, that.checksum) + && Objects.equals(isDir, that.isDir); + } + + @Override + public int hashCode() { + return Objects.hash(key, size, mtime, Arrays.hashCode(checksum), isDir); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("key", key) + .add("size", size) + .add("mtime", mtime) + .add("checksum", StringUtils.byteToHexString(checksum)) + .add("isDir", isDir) + .toString(); + } + + public static boolean isDir(String key) { + return key.endsWith(Constants.SLASH); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectMultiRangeInputStream.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectMultiRangeInputStream.java new file mode 100644 index 0000000000000..4a9357e46bf75 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectMultiRangeInputStream.java @@ -0,0 +1,233 @@ +/* + * 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.tosfs.object; + +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.fs.tosfs.util.FSUtils; +import org.apache.hadoop.fs.tosfs.util.Range; +import org.apache.hadoop.thirdparty.com.google.common.primitives.Ints; +import org.apache.hadoop.util.Preconditions; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; + +public class ObjectMultiRangeInputStream extends FSInputStream { + private final AtomicBoolean closed = new AtomicBoolean(false); + private final ExecutorService threadPool; + private final ObjectStorage storage; + private final String objectKey; + private final long contentLength; + private final long rangeSize; + + private volatile ObjectRangeInputStream stream; + private volatile long nextPos = 0; + private volatile long currPos = 0; + // All range streams should have same checksum. + private final byte[] checksum; + + public ObjectMultiRangeInputStream( + ExecutorService threadPool, + ObjectStorage storage, + Path path, + long contentLength, + long rangeSize, + byte[] checksum) { + this(threadPool, storage, ObjectUtils.pathToKey(path), contentLength, rangeSize, checksum); + } + + public ObjectMultiRangeInputStream( + ExecutorService threadPool, + ObjectStorage storage, + String objectKey, + long contentLength, + long rangeSize, + byte[] checksum) { + this.threadPool = threadPool; + this.storage = storage; + this.objectKey = objectKey; + this.contentLength = contentLength; + this.rangeSize = rangeSize; + this.checksum = checksum; + + Preconditions.checkNotNull(checksum, "Checksum should not be null."); + } + + @Override + public synchronized void seek(long pos) throws IOException { + if (pos < 0) { + throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK + " " + pos); + } + + if (contentLength <= 0) { + return; + } + + nextPos = pos; + } + + @Override + public synchronized long getPos() { + return nextPos; + } + + @Override + public synchronized boolean seekToNewSource(long targetPos) throws IOException { + checkNotClosed(); + return false; + } + + @Override + public synchronized int read() throws IOException { + byte[] buf = new byte[1]; + int n = read(buf, 0, buf.length); + if (n < 0) { + return -1; + } else { + return buf[0] & 0xFF; + } + } + + @Override + public synchronized int read(byte[] buffer, int offset, int length) throws IOException { + checkNotClosed(); + FSUtils.checkReadParameters(buffer, offset, length); + if (length == 0) { + return 0; + } + + int total = 0; + while (total < length) { + if (contentLength == 0 || nextPos >= contentLength) { + return total == 0 ? -1 : total; + } + + seekStream(); + int n = stream.read(buffer, offset, length - total); + if (n < 0) { + return total == 0 ? -1 : total; + } + + total += n; + offset += n; + currPos += n; + nextPos += n; + } + + return total; + } + + @Override + public int read(long position, byte[] buffer, int offset, int length) throws IOException { + checkNotClosed(); + // Check the arguments, according to the HDFS contract. + if (position < 0) { + throw new EOFException("position is negative"); + } + FSUtils.checkReadParameters(buffer, offset, length); + if (length == 0) { + return 0; + } + + if (contentLength == 0 || position >= contentLength) { + return -1; + } + + long remaining = contentLength - position; + int limit = (remaining >= length) ? length : (int) remaining; + + try (InputStream in = storage.get(objectKey, position, limit).verifiedStream(checksum)) { + return in.read(buffer, offset, limit); + } + } + + private void seekStream() throws IOException { + if (stream != null && stream.include(nextPos)) { + // Seek to a random position which is still located in the current range of stream. + if (nextPos != currPos) { + stream.seek(nextPos); + currPos = nextPos; + } + return; + } + + // Seek to a position which is located in another range of new stream. + currPos = nextPos; + openStream(); + } + + private void openStream() throws IOException { + closeStream(true); + + long off = (nextPos / rangeSize) * rangeSize; + Range range = Range.of(off, Math.min(contentLength - off, rangeSize)); + if (nextPos < range.end()) { + stream = new ObjectRangeInputStream(storage, objectKey, range, checksum); + stream.seek(nextPos); + } + } + + private void closeStream(boolean asyncClose) throws IOException { + if (stream != null) { + if (asyncClose) { + final ObjectRangeInputStream streamToClose = stream; + threadPool.submit(() -> CommonUtils.runQuietly(streamToClose::close)); + } else { + stream.close(); + } + stream = null; + } + } + + private void checkNotClosed() throws IOException { + if (closed.get()) { + throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + } + } + + @Override + public synchronized void close() throws IOException { + super.close(); + if (closed.compareAndSet(false, true)) { + closeStream(false); + } + } + + // for test + public long nextExpectPos() { + return currPos; + } + + @Override + public synchronized int available() throws IOException { + checkNotClosed(); + return Ints.saturatedCast(contentLength - nextPos); + } + + @VisibleForTesting + ObjectRangeInputStream stream() { + return stream; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectOutputStream.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectOutputStream.java new file mode 100644 index 0000000000000..8f86321e2fabc --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectOutputStream.java @@ -0,0 +1,343 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.object; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.object.staging.FileStagingPart; +import org.apache.hadoop.fs.tosfs.object.staging.StagingPart; +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.fs.tosfs.util.UUIDUtils; +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.File; +import java.io.IOException; +import java.io.OutputStream; +import java.io.SequenceInputStream; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +public class ObjectOutputStream extends OutputStream { + private static final Logger LOG = LoggerFactory.getLogger(ObjectOutputStream.class); + + private final ObjectStorage storage; + private final ExecutorService uploadPool; + private long totalWroteSize; + private final String destKey; + private final String destScheme; + private final long multiUploadThreshold; + private final long byteSizePerPart; + private final int stagingBufferSize; + private final boolean allowPut; + private final List stagingDirs; + private final List stagingParts = Lists.newArrayList(); + + // For multipart uploads. + private final AtomicInteger partNumGetter = new AtomicInteger(0); + private MultipartUpload multipartUpload = null; + private final List> results = Lists.newArrayList(); + + private StagingPart curPart; + private final AtomicBoolean closed = new AtomicBoolean(false); + + public ObjectOutputStream(ObjectStorage storage, ExecutorService threadPool, Configuration conf, + Path dest, boolean allowPut) { + this.storage = storage; + this.uploadPool = threadPool; + this.destScheme = dest.toUri().getScheme(); + this.totalWroteSize = 0; + this.destKey = createDestKey(dest); + this.multiUploadThreshold = conf.getLong(ConfKeys.FS_MULTIPART_THRESHOLD.key(destScheme), + ConfKeys.FS_MULTIPART_THRESHOLD_DEFAULT); + this.byteSizePerPart = conf.getLong(ConfKeys.FS_MULTIPART_SIZE.key(destScheme), + ConfKeys.FS_MULTIPART_SIZE_DEFAULT); + this.stagingBufferSize = conf.getInt(ConfKeys.FS_MULTIPART_STAGING_BUFFER_SIZE.key(destScheme), + ConfKeys.FS_MULTIPART_STAGING_BUFFER_SIZE_DEFAULT); + this.allowPut = allowPut; + this.stagingDirs = createStagingDirs(conf, destScheme); + + if (!allowPut) { + this.multipartUpload = storage.createMultipartUpload(destKey); + } + } + + private static List createStagingDirs(Configuration conf, String scheme) { + String[] dirs = conf.getStrings(ConfKeys.FS_MULTIPART_STAGING_DIR.key(scheme), + ConfKeys.FS_MULTIPART_STAGING_DIR_DEFAULT); + Preconditions.checkArgument(dirs != null && dirs.length > 0, "'%s' cannot be an empty list", + ConfKeys.FS_MULTIPART_STAGING_DIR.key(scheme)); + + List stagingDirs = new ArrayList<>(); + for (String dir : dirs) { + // Create the directory if not exist. + File stagingDir = new File(dir); + if (!stagingDir.exists() && stagingDir.mkdirs()) { + Preconditions.checkArgument(stagingDir.setWritable(true, false), + "Failed to change staging dir permission to writable, please check %s with value %s", + ConfKeys.FS_MULTIPART_STAGING_DIR.key(scheme), dir); + Preconditions.checkArgument(stagingDir.setReadable(true, false), + "Failed to change staging dir permission to readable, please check %s with value %s", + ConfKeys.FS_MULTIPART_STAGING_DIR.key(scheme), dir); + } else { + Preconditions.checkArgument(stagingDir.exists(), + "Failed to create staging dir, please check %s with value %s", + ConfKeys.FS_MULTIPART_STAGING_DIR.key(scheme), dir); + Preconditions.checkArgument(stagingDir.isDirectory(), + "Staging dir should be a directory, please check %s with value %s", + ConfKeys.FS_MULTIPART_STAGING_DIR.key(scheme), dir); + } + stagingDirs.add(stagingDir); + } + return stagingDirs; + } + + private File chooseStagingDir() { + // Choose a random directory from the staging dirs as the candidate staging dir. + return stagingDirs.get(ThreadLocalRandom.current().nextInt(stagingDirs.size())); + } + + @Override + public void write(int b) throws IOException { + write(new byte[]{(byte) b}, 0, 1); + } + + protected String createDestKey(Path dest) { + return ObjectUtils.pathToKey(dest); + } + + @Override + public synchronized void write(byte[] buf, int off, int len) throws IOException { + if (len == 0) { + return; + } + Preconditions.checkArgument(off >= 0 && off < buf.length, + "Invalid offset - off: %s, len: %s, bufferSize: %s", off, len, buf.length); + Preconditions.checkArgument(len >= 0 && off + len <= buf.length, + "Invalid length - off: %s, len: %s, bufferSize: %s", off, len, buf.length); + Preconditions.checkState(!closed.get(), "OutputStream is closed."); + + while (len > 0) { + if (curPart == null) { + curPart = newStagingPart(); + } + + Preconditions.checkArgument(curPart.size() <= byteSizePerPart, + "Invalid staging size (%s) which is greater than part size (%s)", curPart.size(), + byteSizePerPart); + + // size is the remaining length to fill a complete upload part. + int size = (int) Math.min(byteSizePerPart - curPart.size(), len); + curPart.write(buf, off, size); + + off += size; + len -= size; + totalWroteSize += size; + + // Switch to the next staging part if current staging part is full. + if (curPart.size() >= byteSizePerPart) { + curPart.complete(); + + // Upload this part if multipart upload was triggered. + if (multipartUpload != null) { + CompletableFuture result = + asyncUploadPart(curPart, partNumGetter.incrementAndGet()); + results.add(result); + } + + // Reset the stagingOut + curPart = null; + } + + // Trigger the multipart upload when reach the configured threshold. + if (multipartUpload == null && totalWroteSize >= multiUploadThreshold) { + multipartUpload = storage.createMultipartUpload(destKey); + Preconditions.checkState(byteSizePerPart >= multipartUpload.minPartSize(), + "Configured upload part size %s must be greater than or equals to the minimal" + + " part size %s, please check configure key %s.", byteSizePerPart, + multipartUpload.minPartSize(), ConfKeys.FS_MULTIPART_THRESHOLD.key(destScheme)); + + // Upload the accumulated staging files whose length >= byteSizePerPart. + for (StagingPart stagingPart : stagingParts) { + if (stagingPart.size() >= byteSizePerPart) { + CompletableFuture result = + asyncUploadPart(stagingPart, partNumGetter.incrementAndGet()); + results.add(result); + } + } + } + } + } + + private CompletableFuture asyncUploadPart(final StagingPart stagingPart, + final int partNum) { + final MultipartUpload immutableUpload = multipartUpload; + return CompletableFuture.supplyAsync(() -> uploadPart(stagingPart, partNum), uploadPool) + .whenComplete((part, err) -> { + stagingPart.cleanup(); + if (err != null) { + LOG.error("Failed to upload part, multipartUpload: {}, partNum: {}, stagingPart: {}", + immutableUpload, partNum, stagingPart, err); + } + }); + } + + private CompletableFuture asyncUploadEmptyPart(final int partNum) { + final MultipartUpload immutableUpload = multipartUpload; + return CompletableFuture.supplyAsync( + () -> storage.uploadPart( + immutableUpload.key(), + immutableUpload.uploadId(), + partNum, + () -> new ByteArrayInputStream(new byte[0]), + 0), + uploadPool) + .whenComplete((part, err) -> { + if (err != null) { + LOG.error("Failed to upload empty part, multipartUpload: {}, partNum: {}", + immutableUpload, partNum, err); + } + }); + } + + private Part uploadPart(StagingPart stagingPart, int partNum) { + Preconditions.checkNotNull(storage, "Object storage cannot be null."); + Preconditions.checkNotNull(multipartUpload, "Multipart upload is not initialized."); + return storage.uploadPart(multipartUpload.key(), multipartUpload.uploadId(), + partNum, stagingPart::newIn, stagingPart.size()); + } + + protected void finishUpload(String key, String uploadId, List parts) throws IOException { + storage.completeUpload(key, uploadId, parts); + } + + private void simplePut() throws IOException { + if (curPart != null) { + curPart.complete(); + } + storage.put( + destKey, + () -> stagingParts() + .stream() + .map(StagingPart::newIn) + .reduce(SequenceInputStream::new) + .orElseGet(() -> new ByteArrayInputStream(new byte[0])), + stagingParts().stream().mapToLong(StagingPart::size).sum()); + // Reset the staging output stream. + curPart = null; + } + + synchronized List waitForPartsUpload() { + Preconditions.checkArgument(multipartUpload != null, "Multipart upload cannot be null"); + Preconditions.checkArgument(!results.isEmpty(), "Upload parts cannot be empty"); + // Waiting for all the upload parts to be finished. + return results.stream() + .map(CompletableFuture::join) + .sorted(Comparator.comparing(Part::num)) + .collect(Collectors.toList()); + } + + @Override + public synchronized void close() throws IOException { + if (!closed.compareAndSet(false, true)) { + return; + } + + try { + // Use the simple PUT API if wrote bytes is not reached the multipart threshold. + if (multipartUpload == null && allowPut) { + simplePut(); + return; + } + Preconditions.checkNotNull(multipartUpload, + "MultipartUpload cannot be null since allowPut was disabled."); + + // Use multipart upload API to upload those parts. + if (totalWroteSize <= 0) { + // Write an empty part for this zero-byte file. + CompletableFuture result = asyncUploadEmptyPart(partNumGetter.incrementAndGet()); + results.add(result); + } else if (curPart != null) { + curPart.complete(); + // Submit the last part to upload thread pool. + CompletableFuture result = asyncUploadPart(curPart, partNumGetter.incrementAndGet()); + results.add(result); + // Reset the staging output stream. + curPart = null; + } + + // Finish the multipart uploads. + finishUpload(multipartUpload.key(), multipartUpload.uploadId(), waitForPartsUpload()); + + } catch (Exception e) { + LOG.error("Encountering error when closing output stream", e); + if (multipartUpload != null) { + CommonUtils.runQuietly( + () -> storage.abortMultipartUpload(multipartUpload.key(), multipartUpload.uploadId())); + } + throw e; + } finally { + // Clear all the staging part. + deleteStagingPart(stagingParts); + } + } + + public long totalWroteSize() { + return totalWroteSize; + } + + public ObjectStorage storage() { + return storage; + } + + public List stagingParts() { + return stagingParts; + } + + public String destKey() { + return destKey; + } + + public MultipartUpload upload() { + return multipartUpload; + } + + private void deleteStagingPart(List parts) { + for (StagingPart part : parts) { + part.cleanup(); + } + } + + private StagingPart newStagingPart() { + String stagingPath = String.format("%s/staging-%s.tmp", chooseStagingDir(), + UUIDUtils.random()); + StagingPart part = new FileStagingPart(stagingPath, stagingBufferSize); + stagingParts.add(part); + return part; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectRangeInputStream.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectRangeInputStream.java new file mode 100644 index 0000000000000..8e78dbb03e485 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectRangeInputStream.java @@ -0,0 +1,199 @@ +/* + * 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.tosfs.object; + +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.util.FSUtils; +import org.apache.hadoop.fs.tosfs.util.Range; +import org.apache.hadoop.thirdparty.com.google.common.io.ByteStreams; +import org.apache.hadoop.thirdparty.com.google.common.primitives.Ints; +import org.apache.hadoop.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; + +public class ObjectRangeInputStream extends FSInputStream { + private static final Logger LOG = LoggerFactory.getLogger(ObjectRangeInputStream.class); + private static final int MAX_SKIP_SIZE = 1024 * 1024; + + private final ObjectStorage storage; + private final String objectKey; + private final Range range; + private final byte[] checksum; + + private InputStream stream; + private long nextPos; + private long currPos; + private boolean closed = false; + + public ObjectRangeInputStream(ObjectStorage storage, Path path, Range range, byte[] checksum) { + this(storage, ObjectUtils.pathToKey(path), range, checksum); + } + + public ObjectRangeInputStream( + ObjectStorage storage, String objectKey, Range range, byte[] checksum) { + this.storage = storage; + this.objectKey = objectKey; + this.range = range; + this.checksum = checksum; + + this.stream = null; + this.nextPos = range.off(); + this.currPos = nextPos; + + Preconditions.checkNotNull(checksum, "Checksum should not be null."); + } + + @Override + public int read() throws IOException { + byte[] buf = new byte[1]; + int n = read(buf, 0, buf.length); + if (n < 0) { + return -1; + } else { + return buf[0] & 0xFF; + } + } + + @Override + public int read(byte[] buffer, int offset, int length) throws IOException { + checkNotClosed(); + FSUtils.checkReadParameters(buffer, offset, length); + + if (length == 0) { + return 0; + } + + if (!range.include(nextPos)) { + return -1; + } + + seekStream(); + + int toRead = Math.min(length, Ints.saturatedCast(range.end() - nextPos)); + int readLen = stream.read(buffer, offset, toRead); + if (readLen > 0) { + nextPos += readLen; + currPos += readLen; + } + return readLen; + } + + @Override + public void close() throws IOException { + super.close(); + closeStream(); + closed = true; + } + + @Override + public int read(long position, byte[] buffer, int offset, int length) throws IOException { + checkNotClosed(); + + FSUtils.checkReadParameters(buffer, offset, length); + if (!range.include(position)) { + return -1; + } + + int toRead = Math.min(length, Ints.saturatedCast(range.end() - position)); + if (toRead == 0) { + return 0; + } + + try (InputStream in = openStream(position, toRead)) { + return in.read(buffer, offset, toRead); + } + } + + @Override + public void seek(long pos) throws IOException { + checkNotClosed(); + Preconditions.checkArgument(range.include(pos), "Position %s must be in range %s", pos, range); + this.nextPos = pos; + } + + @Override + public long getPos() throws IOException { + checkNotClosed(); + return nextPos; + } + + @Override + public boolean seekToNewSource(long targetPos) throws IOException { + checkNotClosed(); + return false; + } + + private void seekStream() throws IOException { + // sequential read + if (stream != null && nextPos == currPos) { + return; + } + + // random read + if (stream != null && nextPos > currPos) { + long skip = nextPos - currPos; + // It is not worth skipping because the skip size is too big, or it can't read any bytes + // after skip. + if (skip < MAX_SKIP_SIZE) { + try { + ByteStreams.skipFully(stream, skip); + currPos = nextPos; + return; + } catch (IOException ignored) { + LOG.warn("Failed to skip {} bytes in stream, will try to reopen the stream", skip); + } + } + } + + currPos = nextPos; + + closeStream(); + stream = openStream(nextPos, range.end() - nextPos); + } + + private InputStream openStream(long offset, long limit) throws IOException { + return storage.get(objectKey, offset, limit).verifiedStream(checksum); + } + + private void closeStream() throws IOException { + if (stream != null) { + stream.close(); + } + stream = null; + } + + private void checkNotClosed() throws IOException { + if (closed) { + throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + } + } + + public boolean include(long pos) { + return range.include(pos); + } + + public Range range() { + return range; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorage.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorage.java new file mode 100644 index 0000000000000..07187bcd237fc --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorage.java @@ -0,0 +1,372 @@ +/* + * 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.tosfs.object; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.object.exceptions.InvalidObjectKeyException; +import org.apache.hadoop.fs.tosfs.util.LazyReload; +import org.apache.hadoop.fs.tosfs.object.exceptions.NotAppendableException; +import org.apache.hadoop.fs.tosfs.object.request.ListObjectsRequest; +import org.apache.hadoop.fs.tosfs.object.response.ListObjectsResponse; + +import java.io.ByteArrayInputStream; +import java.io.Closeable; +import java.io.InputStream; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +public interface ObjectStorage extends Closeable { + String EMPTY_DELIMITER = ""; + + /** + * @return Scheme of the object storage. + */ + String scheme(); + + /** + * @return null if bucket doesn't exist. + */ + BucketInfo bucket(); + + /** + * Initialize the Object storage, according to the properties. + * + * @param conf to initialize the {@link ObjectStorage} + * @param bucket the corresponding bucket name, each object store has one bucket. + */ + void initialize(Configuration conf, String bucket); + + /** + * @return storage conf + */ + Configuration conf(); + + default ObjectContent get(String key) { + return get(key, 0, -1); + } + + /** + * Get the data for the given object specified by key. + * Throw {@link RuntimeException} if object key doesn't exist. + * Throw {@link RuntimeException} if object key is null or empty. + * + * @param key the object key. + * @param offset the offset to start read. + * @param limit the max length to read. + * @return {@link InputStream} to read the object content. + */ + ObjectContent get(String key, long offset, long limit); + + default byte[] put(String key, byte[] data) { + return put(key, data, 0, data.length); + } + + default byte[] put(String key, byte[] data, int off, int len) { + return put(key, () -> new ByteArrayInputStream(data, off, len), len); + } + + /** + * Put data read from a reader to an object specified by key. The implementation must ensure to + * close the stream created by stream provider after finishing stream operation. + * Throw {@link RuntimeException} if object key is null or empty. + * + * @param key for the object. + * @param streamProvider the binary input stream provider that create input stream to write. + * @param contentLength the content length, if the actual data is bigger than content length, the + * object can be created, but the object data will be truncated to the given + * content length, if the actual data is smaller than content length, will + * create object failed with unexpect end of IOException. + * @return the checksum of uploaded object + */ + byte[] put(String key, InputStreamProvider streamProvider, long contentLength); + + default byte[] append(String key, byte[] data) { + return append(key, data, 0, data.length); + } + + default byte[] append(String key, byte[] data, int off, int len) { + return append(key, () -> new ByteArrayInputStream(data, off, len), len); + } + + /** + * Append data read from a reader to an object specified by key. If the object exists, data will + * be appended to the tail. Otherwise, the object will be created and data will be written to it. + * Content length could be zero if object exists. If the object doesn't exist and content length + * is zero, a {@link NotAppendableException} will be thrown. + *

+ * The first one wins if there are concurrent appends. + *

+ * The implementation must ensure to close the stream created by stream provider after finishing + * stream operation. + * Throw {@link RuntimeException} if object key is null or empty. + * + * @param key for the object. + * @param streamProvider the binary input stream provider that create input stream to write. + * @param contentLength the appended content length. If the actual appended data is bigger than + * content length, the object can be appended but the data to append will be + * truncated to the given content length. If the actual data is smaller than + * content length, append object will fail with unexpect end IOException. + * @return the checksum of appended object. + * @throws NotAppendableException if the object already exists and is not appendable, or the + * object doesn't exist and content length is zero. + */ + byte[] append(String key, InputStreamProvider streamProvider, long contentLength); + + /** + * Delete an object. + * No exception thrown if the object key doesn't exist. + * Throw {@link RuntimeException} if object key is null or empty. + * + * @param key the given object key to be deleted. + */ + void delete(String key); + + /** + * Delete multiple keys. If one key doesn't exist, it will be treated as delete succeed, won't be + * included in response list. + * + * @param keys the given object keys to be deleted + * @return the keys delete failed + */ + List batchDelete(List keys); + + /** + * Delete all objects with the given prefix(include the prefix if the corresponding object + * exists). + * + * @param prefix the prefix key. + */ + void deleteAll(String prefix); + + /** + * Head returns some information about the object or a null if not found. + * Throw {@link RuntimeException} if object key is null or empty. + * There are some differences between directory bucket and general purpose bucket: + *

    + *
  • Assume an file object 'a/b' exists, only head("a/b") will get the meta of object 'a/b' + * for both general purpose bucket and directory bucket
  • + *
  • Assume an dir object 'a/b/' exists, regarding general purpose bucket, only head("a/b/") + * will get the meta of object 'a/b/', but for directory bucket, both head("a/b") and + * head("a/b/") will get the meta of object 'a/b/'
  • + *
+ * + * @param key for the specified object. + * @return {@link ObjectInfo}, null if the object does not exist. + * @throws InvalidObjectKeyException if the object is locating under an existing file in directory + * bucket, which is not allowed. + */ + ObjectInfo head(String key); + + /** + * List objects according to the given {@link ListObjectsRequest}. + * + * @param request {@link ListObjectsRequest} + * @return the iterable of {@link ListObjectsResponse} which contains objects and common prefixes + */ + Iterable list(ListObjectsRequest request); + + /** + * List limited objects in a given bucket. + * + * @param prefix Limits the response to keys that begin with the specified prefix. + * @param startAfter StartAfter is where you want the object storage to start listing from. + * object storage starts listing after this specified key. + * StartAfter can be any key in the bucket. + * @param limit Limit the maximum number of response objects. + * @return {@link ObjectInfo} the object list with matched prefix key + */ + default Iterable list(String prefix, String startAfter, int limit) { + ListObjectsRequest request = ListObjectsRequest.builder() + .prefix(prefix) + .startAfter(startAfter) + .maxKeys(limit) + .delimiter(EMPTY_DELIMITER) + .build(); + + return new LazyReload<>(() -> { + Iterator iterator = list(request).iterator(); + return buf -> { + if (!iterator.hasNext()) { + return true; + } + buf.addAll(iterator.next().objects()); + + return !iterator.hasNext(); + }; + }); + } + + /** + * List all objects in a given bucket. + * + * @param prefix Limits the response to keys that begin with the specified prefix. + * @param startAfter StartAfter is where you want the object storage to start listing from. + * object storage starts listing after this specified key. + * StartAfter can be any key in the bucket. + * @return {@link ObjectInfo} Iterable to iterate over the objects with matched prefix key + * and StartAfter + */ + default Iterable listAll(String prefix, String startAfter) { + return list(prefix, startAfter, -1); + } + + /** + * CreateMultipartUpload starts to upload a large object part by part. + * + * @param key for the specified object. + * @return {@link MultipartUpload}. + */ + MultipartUpload createMultipartUpload(String key); + + /** + * UploadPart upload a part of an object. The implementation must ensure to close the stream + * created by stream provider after finishing stream operation. + * + * @param key for the specified object. + * @param uploadId for the multipart upload id. + * @param partNum upload part number. + * @param streamProvider the stream provider to provider part stream + * @param contentLength the content length, if the actual data is bigger than content length, the + * object can be created, but the object data will be truncated to the given + * content length, if the actual data is smaller than content length, will + * create object failed with unexpect end of IOException. + * @return the uploaded part. + */ + Part uploadPart(String key, String uploadId, int partNum, InputStreamProvider streamProvider, + long contentLength); + + /** + * Complete the multipart uploads with given object key and upload id. + * + * @param key for the specified object. + * @param uploadId id of the multipart upload. + * @param uploadParts parts to upload. + * @return the checksum of uploaded object + */ + byte[] completeUpload(String key, String uploadId, List uploadParts); + + /** + * Abort a multipart upload. + * + * @param key object key. + * @param uploadId multipart upload Id. + */ + void abortMultipartUpload(String key, String uploadId); + + /** + * List multipart uploads under a path. + * + * @param prefix for uploads to abort. + * @return Iterable to iterate over multipart unloads. + */ + Iterable listUploads(String prefix); + + /** + * upload part copy with mutipart upload id. + * + * @param srcKey source object key + * @param dstKey dest object key + * @param uploadId id of the multipart upload copy + * @param partNum part num of the multipart upload copy + * @param copySourceRangeStart copy source range start of source object + * @param copySourceRangeEnd copy source range end of source object + * @return {@link Part}. + */ + Part uploadPartCopy( + String srcKey, String dstKey, String uploadId, int partNum, long copySourceRangeStart, + long copySourceRangeEnd); + + /** + * Copy binary content from one object to another object. + * + * @param srcKey source object key + * @param dstKey dest object key + */ + void copy(String srcKey, String dstKey); + + /** + * Atomic rename source object to dest object without any data copying. + * Will overwrite dest object if dest object exists. + * + * @param srcKey source object key + * @param dstKey dest object key + * @throws RuntimeException if rename failed,e.g. srcKey is equal to dstKey or the source object + * doesn't exist. + */ + void rename(String srcKey, String dstKey); + + /** + * Attach tags to specified object. This method will overwrite all existed tags with the new tags. + * Remove all existed tags if the new tags are empty. The maximum tags number is 10. + * + * @param key the key of the object key. + * @param newTags the new tags to put. + * @throws RuntimeException if key doesn't exist. + */ + default void putTags(String key, Map newTags) { + throw new UnsupportedOperationException( + this.getClass().getSimpleName() + " doesn't support putObjectTagging."); + } + + /** + * Get all attached tags of the object. + * + * @param key the key of the object. + * @return map containing all tags. + * @throws RuntimeException if key doesn't exist. + */ + default Map getTags(String key) { + throw new UnsupportedOperationException( + this.getClass().getSimpleName() + " doesn't support getObjectTagging."); + } + + /** + * Gets the object status for the given key. + * It's different from {@link ObjectStorage#head(String)}, it returns object info if the key + * exists or the prefix with value key exists. + *

+ * There are three kinds of implementations: + *

    + *
  • Uses the headObject API if the object storage support directory bucket and the requested + * bucket is a directory bucket, the object storage will return object directly if the file or + * dir exists, otherwise return null
  • + *
  • Uses getFileStatus API if the object storage support it, e.g. TOS. The object storage + * will return the object directly if the key or prefix exists, otherwise return null.
  • + *
  • If the object storage doesn't support above all cases, you have to try to headObject(key) + * at first, if the object doesn't exist, and then headObject(key + "/") later if the key + * doesn't end with '/', and if neither the new key doesn't exist, and then use listObjects API + * to check whether the prefix/key exist.
  • + *
+ * + * @param key the object + * @return object info if the key or prefix exists, otherwise return null. + * @throws InvalidObjectKeyException if the object is locating under an existing file in directory + * bucket, which is not allowed. + */ + ObjectInfo objectStatus(String key); + + /** + * Get the object storage checksum information, including checksum algorithm name, + * checksum type, etc. + * + * @return checksum information of this storage. + */ + ChecksumInfo checksumInfo(); +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorageFactory.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorageFactory.java new file mode 100644 index 0000000000000..09c8a72ab743f --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorageFactory.java @@ -0,0 +1,73 @@ +/* + * 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.tosfs.object; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.object.tos.TOS; +import org.apache.hadoop.util.Preconditions; + +import java.lang.reflect.InvocationTargetException; + +import static org.apache.hadoop.fs.tosfs.conf.ConfKeys.FS_OBJECT_STORAGE_IMPL; + +public final class ObjectStorageFactory { + + private static final Configuration DEFAULT_IMPLS = new Configuration(); + + static { + // Setup default object storage impl for scheme "tos" and "filestore". + DEFAULT_IMPLS.set(ConfKeys.FS_OBJECT_STORAGE_IMPL.key("tos"), TOS.class.getName()); + DEFAULT_IMPLS.set(ConfKeys.FS_OBJECT_STORAGE_IMPL.key("filestore"), FileStore.class.getName()); + } + + private ObjectStorageFactory() { + } + + public static ObjectStorage createWithPrefix(String prefix, String scheme, String bucket, + Configuration conf) { + ObjectStorage storage = create(scheme, bucket, conf); + return new PrefixStorage(storage, prefix); + } + + public static ObjectStorage create(String scheme, String bucket, Configuration conf) { + Preconditions.checkArgument(StringUtils.isNotEmpty(scheme), "Scheme is null or empty."); + Preconditions.checkArgument(StringUtils.isNotEmpty(bucket), "Bucket is null or empty."); + Preconditions.checkNotNull(conf, "Conf is null."); + + try { + String confKey = FS_OBJECT_STORAGE_IMPL.key(scheme); + String impl = conf.get(confKey, DEFAULT_IMPLS.get(confKey)); + + Preconditions.checkArgument(StringUtils.isNotEmpty(impl), + "Cannot locate the ObjectStorage implementation for scheme '%s'", scheme); + ObjectStorage store = + (ObjectStorage) Class.forName(impl).getDeclaredConstructor().newInstance(); + store.initialize(conf, bucket); + return store; + } catch (ClassNotFoundException | + InvocationTargetException | + InstantiationException | + IllegalAccessException | + NoSuchMethodException e) { + throw new RuntimeException(e); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectUtils.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectUtils.java new file mode 100644 index 0000000000000..a0b00f366d963 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectUtils.java @@ -0,0 +1,92 @@ +/* + * 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.tosfs.object; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.util.Range; +import org.apache.hadoop.thirdparty.com.google.common.base.Joiner; +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.util.Preconditions; + +import java.util.List; + +public final class ObjectUtils { + public static final String SLASH = "/"; + + private ObjectUtils() { + } + + public static Path keyToPath(String key) { + return new Path(SLASH + key); + } + + public static String path(String key) { + return key.startsWith(SLASH) ? key : SLASH + key; + } + + public static String pathToKey(Path p) { + return pathToKey(p, false); + } + + public static String pathToKey(Path p, Boolean isDir) { + Preconditions.checkArgument(p != null, "Null path"); + if (p.toUri().getScheme() != null && p.toUri().getPath().isEmpty()) { + return ""; + } + String key = p.toUri().getPath().substring(1); + if (isDir && !key.isEmpty()) { + return key.endsWith(SLASH) ? key : key + SLASH; + } + return key; + } + + public static void deleteAllObjects(ObjectStorage storage, Iterable objects, + int batchSize) { + List keysToDelete = Lists.newArrayList(); + for (ObjectInfo obj : objects) { + keysToDelete.add(obj.key()); + + if (keysToDelete.size() == batchSize) { + batchDelete(storage, keysToDelete); + keysToDelete.clear(); + } + } + + if (!keysToDelete.isEmpty()) { + batchDelete(storage, keysToDelete); + } + } + + private static void batchDelete(ObjectStorage storage, List keys) { + List failedKeys = storage.batchDelete(keys); + if (!failedKeys.isEmpty()) { + throw new RuntimeException(String.format("Failed to delete %s objects, detail: %s", + failedKeys.size(), Joiner.on(",").join(failedKeys))); + } + } + + public static Range calculateRange(final long offset, final long limit, final long objSize) { + Preconditions.checkArgument(offset >= 0, + String.format("offset is a negative number: %s", offset)); + Preconditions.checkArgument(offset <= objSize, + String.format("offset: %s is bigger than object size: %s", offset, objSize)); + long len = limit < 0 ? objSize - offset : Math.min(objSize - offset, limit); + return Range.of(offset, len); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/Part.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/Part.java new file mode 100644 index 0000000000000..ace7acaf28a5d --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/Part.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.tosfs.object; + +import org.apache.hadoop.thirdparty.com.google.common.base.MoreObjects; + +import java.util.Objects; + +public class Part { + private int num; + private long size; + private String eTag; + + // No-arg constructor for json serializer, don't use. + public Part() { + } + + public Part(int num, long size, String eTag) { + this.num = num; + this.size = size; + this.eTag = eTag; + } + + public int num() { + return num; + } + + public long size() { + return size; + } + + public String eTag() { + return eTag; + } + + @Override + public int hashCode() { + return Objects.hash(num, size, eTag); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof Part)) { + return false; + } + Part that = (Part) o; + return Objects.equals(num, that.num) + && Objects.equals(size, that.size) + && Objects.equals(eTag, that.eTag); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("PartNum", num) + .add("PartSize", size) + .add("ETag", eTag) + .toString(); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/PrefixStorage.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/PrefixStorage.java new file mode 100644 index 0000000000000..18185bd1efd14 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/PrefixStorage.java @@ -0,0 +1,254 @@ +/* + * 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.tosfs.object; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.object.request.ListObjectsRequest; +import org.apache.hadoop.fs.tosfs.object.response.ListObjectsResponse; +import org.apache.hadoop.thirdparty.com.google.common.base.Strings; +import org.apache.hadoop.thirdparty.com.google.common.collect.Iterables; +import org.apache.hadoop.util.Preconditions; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class PrefixStorage implements DirectoryStorage { + private final ObjectStorage storage; + private final String prefix; + + public PrefixStorage(ObjectStorage storage, String prefix) { + this.storage = storage; + this.prefix = prefix; + } + + @Override + public String scheme() { + return storage.scheme(); + } + + @Override + public BucketInfo bucket() { + return storage.bucket(); + } + + @Override + public void initialize(Configuration conf, String bucket) { + storage.initialize(conf, bucket); + } + + @Override + public Configuration conf() { + return storage.conf(); + } + + @Override + public ObjectContent get(String key, long offset, long limit) { + Preconditions.checkArgument(key != null && key.length() > 0, + "Object key cannot be null or empty."); + return storage.get(prefix + key, offset, limit); + } + + @Override + public byte[] put(String key, InputStreamProvider streamProvider, long contentLength) { + Preconditions.checkArgument(key != null && key.length() > 0, + "Object key cannot be null or empty."); + return storage.put(prefix + key, streamProvider, contentLength); + } + + @Override + public byte[] append(String key, InputStreamProvider streamProvider, long contentLength) { + Preconditions.checkArgument(key != null && key.length() > 0, + "Object key cannot be null or empty."); + return storage.append(prefix + key, streamProvider, contentLength); + } + + @Override + public void delete(String key) { + Preconditions.checkArgument(key != null, "Object key cannot be null or empty."); + storage.delete(prefix + key); + } + + @Override + public List batchDelete(List keys) { + return storage.batchDelete(keys.stream().map(key -> prefix + key).collect(Collectors.toList())); + } + + @Override + public void deleteAll(String prefixToDelete) { + storage.deleteAll(this.prefix + prefixToDelete); + } + + @Override + public ObjectInfo head(String key) { + Preconditions.checkArgument(key != null && key.length() > 0, + "Object key cannot be null or empty."); + return removePrefix(storage.head(prefix + key)); + } + + private ListObjectsResponse removePrefix(ListObjectsResponse response) { + List objects = response.objects().stream() + .map(this::removePrefix) + .collect(Collectors.toList()); + List commonPrefixKeys = response.commonPrefixes().stream() + .map(this::removePrefix) + .collect(Collectors.toList()); + return new ListObjectsResponse(objects, commonPrefixKeys); + } + + @Override + public Iterable list(ListObjectsRequest request) { + String startAfter = Strings.isNullOrEmpty(request.startAfter()) ? + request.startAfter() : prefix + request.startAfter(); + + ListObjectsRequest newReq = ListObjectsRequest.builder() + .prefix(prefix + request.prefix()) + .startAfter(startAfter) + .maxKeys(request.maxKeys()) + .delimiter(request.delimiter()) + .build(); + + return Iterables.transform(storage.list(newReq), this::removePrefix); + } + + @Override + public MultipartUpload createMultipartUpload(String key) { + Preconditions.checkArgument(key != null && key.length() > 0, + "Object key cannot be null or empty."); + return removePrefix(storage.createMultipartUpload(prefix + key)); + } + + @Override + public Part uploadPart( + String key, String uploadId, int partNum, + InputStreamProvider streamProvider, long contentLength) { + Preconditions.checkArgument(key != null && key.length() > 0, + "Object key cannot be null or empty."); + return storage.uploadPart(prefix + key, uploadId, partNum, streamProvider, contentLength); + } + + @Override + public byte[] completeUpload(String key, String uploadId, List uploadParts) { + Preconditions.checkArgument(key != null && key.length() > 0, + "Object key cannot be null or empty."); + return storage.completeUpload(prefix + key, uploadId, uploadParts); + } + + @Override + public void abortMultipartUpload(String key, String uploadId) { + Preconditions.checkArgument(key != null && key.length() > 0, + "Object key cannot be null or empty."); + storage.abortMultipartUpload(prefix + key, uploadId); + } + + @Override + public Iterable listUploads(String keyPrefix) { + return Iterables.transform(storage.listUploads(prefix + keyPrefix), this::removePrefix); + } + + @Override + public Part uploadPartCopy( + String srcKey, String dstKey, String uploadId, int partNum, long copySourceRangeStart, + long copySourceRangeEnd) { + return storage.uploadPartCopy(prefix + srcKey, prefix + dstKey, uploadId, partNum, + copySourceRangeStart, copySourceRangeEnd); + } + + @Override + public void copy(String srcKey, String dstKey) { + storage.copy(prefix + srcKey, prefix + dstKey); + } + + @Override + public void rename(String srcKey, String dstKey) { + storage.rename(prefix + srcKey, prefix + dstKey); + } + + private ObjectInfo removePrefix(ObjectInfo o) { + if (o == null) { + return null; + } + return new ObjectInfo(removePrefix(o.key()), o.size(), o.mtime(), o.checksum(), o.isDir()); + } + + private MultipartUpload removePrefix(MultipartUpload u) { + if (u == null) { + return null; + } + return new MultipartUpload(removePrefix(u.key()), u.uploadId(), u.minPartSize(), + u.maxPartCount()); + } + + private String removePrefix(String key) { + if (key == null) { + return null; + } else if (key.startsWith(prefix)) { + return key.substring(prefix.length()); + } else { + return key; + } + } + + @Override + public void putTags(String key, Map newTags) { + storage.putTags(prefix + key, newTags); + } + + @Override + public Map getTags(String key) { + return storage.getTags(prefix + key); + } + + @Override + public ObjectInfo objectStatus(String key) { + Preconditions.checkArgument(key != null && !key.isEmpty(), + "Object key cannot be null or empty."); + return removePrefix(storage.objectStatus(prefix + key)); + } + + @Override + public ChecksumInfo checksumInfo() { + return storage.checksumInfo(); + } + + @Override + public void close() throws IOException { + storage.close(); + } + + @Override + public Iterable listDir(String key, boolean recursive) { + Preconditions.checkArgument(storage instanceof DirectoryStorage); + return Iterables.transform(((DirectoryStorage) storage).listDir(prefix + key, recursive), + this::removePrefix); + } + + @Override + public void deleteDir(String key, boolean recursive) { + Preconditions.checkArgument(storage instanceof DirectoryStorage); + ((DirectoryStorage) storage).deleteDir(prefix + key, recursive); + } + + @Override + public boolean isEmptyDir(String key) { + Preconditions.checkArgument(storage instanceof DirectoryStorage); + return ((DirectoryStorage) storage).isEmptyDir(prefix + key); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/ChecksumMismatchException.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/ChecksumMismatchException.java new file mode 100644 index 0000000000000..309eebe0fcad6 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/ChecksumMismatchException.java @@ -0,0 +1,34 @@ +/* + * 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.tosfs.object.exceptions; + +import org.apache.hadoop.util.StringUtils; + +import java.io.IOException; + +public class ChecksumMismatchException extends IOException { + public ChecksumMismatchException(String message) { + super(message); + } + + public ChecksumMismatchException(byte[] expected, byte[] actual) { + this(String.format("Expected checksum is %s while actual checksum is %s", + StringUtils.byteToHexString(expected), StringUtils.byteToHexString(actual))); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/InvalidObjectKeyException.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/InvalidObjectKeyException.java new file mode 100644 index 0000000000000..f9eaf021e28e0 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/InvalidObjectKeyException.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.tosfs.object.exceptions; + +/** + * Regarding accessing an object in directory bucket, if the object is locating under an existing + * file in directory bucket, the {@link InvalidObjectKeyException} will be thrown. E.g. there is a + * file object 'a/b/file' exists in directory bucket, the {@link InvalidObjectKeyException} will be + * thrown if head object 'a/b/file/c' no matter whether 'c' exists or not. + */ +public class InvalidObjectKeyException extends RuntimeException { + public InvalidObjectKeyException(Throwable cause) { + super(cause); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/NotAppendableException.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/NotAppendableException.java new file mode 100644 index 0000000000000..2a7def1176390 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/NotAppendableException.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.tosfs.object.exceptions; + +public class NotAppendableException extends RuntimeException { + public NotAppendableException(String msg) { + super(msg); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/package-info.java new file mode 100644 index 0000000000000..c87e1a7eaa031 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/package-info.java @@ -0,0 +1,26 @@ +/* + * 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. + */ + +/** + * Classes for hadoop-tos object. + */ +@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"}) +@InterfaceStability.Evolving +package org.apache.hadoop.fs.tosfs.object.exceptions; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/package-info.java new file mode 100644 index 0000000000000..7819ecd6c8af0 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/package-info.java @@ -0,0 +1,26 @@ +/* + * 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. + */ + +/** + * Classes for hadoop-tos object. + */ +@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"}) +@InterfaceStability.Evolving +package org.apache.hadoop.fs.tosfs.object; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/ListObjectsRequest.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/ListObjectsRequest.java new file mode 100644 index 0000000000000..170426acb07c2 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/ListObjectsRequest.java @@ -0,0 +1,85 @@ +/* + * 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.tosfs.object.request; + +public final class ListObjectsRequest { + private final String prefix; + private final String startAfter; + private final int maxKeys; + private final String delimiter; + + private ListObjectsRequest(String prefix, String startAfter, int maxKeys, String delimiter) { + this.prefix = prefix; + this.startAfter = startAfter; + this.maxKeys = maxKeys; + this.delimiter = delimiter; + } + + public String prefix() { + return prefix; + } + + public String startAfter() { + return startAfter; + } + + public int maxKeys() { + return maxKeys; + } + + public String delimiter() { + return delimiter; + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + private String prefix; + private String startAfter; + // -1 means list all object keys + private int maxKeys = -1; + private String delimiter; + + public Builder prefix(String prefixInput) { + this.prefix = prefixInput; + return this; + } + + public Builder startAfter(String startAfterInput) { + this.startAfter = startAfterInput; + return this; + } + + public Builder maxKeys(int maxKeysInput) { + this.maxKeys = maxKeysInput; + return this; + } + + public Builder delimiter(String delimiterInput) { + this.delimiter = delimiterInput; + return this; + } + + public ListObjectsRequest build() { + return new ListObjectsRequest(prefix, startAfter, maxKeys, delimiter); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/package-info.java new file mode 100644 index 0000000000000..6e7e52f403950 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/package-info.java @@ -0,0 +1,26 @@ +/* + * 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. + */ + +/** + * Classes for hadoop-tos object request. + */ +@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"}) +@InterfaceStability.Evolving +package org.apache.hadoop.fs.tosfs.object.request; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/ListObjectsResponse.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/ListObjectsResponse.java new file mode 100644 index 0000000000000..eb3adf8522ae8 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/ListObjectsResponse.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.tosfs.object.response; + +import org.apache.hadoop.fs.tosfs.object.ObjectInfo; + +import java.util.List; + +public class ListObjectsResponse { + private final List objects; + private final List commonPrefixes; + + public ListObjectsResponse( + List objects, + List commonPrefixes) { + this.objects = objects; + this.commonPrefixes = commonPrefixes; + } + + public List objects() { + return objects; + } + + public List commonPrefixes() { + return commonPrefixes; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/package-info.java new file mode 100644 index 0000000000000..6678617057243 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/package-info.java @@ -0,0 +1,26 @@ +/* + * 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. + */ + +/** + * Classes for hadoop-tos object response. + */ +@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"}) +@InterfaceStability.Evolving +package org.apache.hadoop.fs.tosfs.object.response; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/FileStagingPart.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/FileStagingPart.java new file mode 100644 index 0000000000000..fb39e949febf6 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/FileStagingPart.java @@ -0,0 +1,177 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.object.staging; + +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.thirdparty.com.google.common.base.MoreObjects; +import org.apache.hadoop.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.UncheckedIOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; + +public class FileStagingPart implements StagingPart { + private static final Logger LOG = LoggerFactory.getLogger(FileStagingPart.class); + + private final Path path; + private final int stagingBufferSize; + private final StagingFileOutputStream out; + private State state = State.WRITABLE; + + public FileStagingPart(String filePath, int stagingBufferSize) { + this.path = Paths.get(filePath); + this.stagingBufferSize = stagingBufferSize; + this.out = new StagingFileOutputStream(path, stagingBufferSize); + } + + @Override + public synchronized void write(byte[] b, int off, int len) throws IOException { + Preconditions.checkState(state == State.WRITABLE, + "Cannot write the part since it's not writable now, state: %s", state); + out.write(b, off, len); + } + + @Override + public synchronized void complete() throws IOException { + Preconditions.checkState(state == State.WRITABLE, + "Cannot complete the part since it's not writable now, state: %s", state); + out.close(); + state = State.READABLE; + } + + @Override + public synchronized InputStream newIn() { + Preconditions.checkState(state == State.READABLE, + "Cannot read the part since it's not readable now, state: %s.", state); + return out.newIn(); + } + + @Override + public synchronized long size() { + return out.size(); + } + + @Override + public synchronized State state() { + return state; + } + + @Override + public synchronized void cleanup() { + if (state != State.CLEANED) { + try { + // Close the stream quietly. + CommonUtils.runQuietly(out::close, false); + + // Delete the staging file if exists. + Files.deleteIfExists(path); + } catch (Exception e) { + LOG.error("Failed to delete staging file, stagingFile: {}", path, e); + } finally { + state = State.CLEANED; + } + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("path", path) + .add("stagingBufferSize", stagingBufferSize) + .add("wroteByteSize", size()) + .toString(); + } + + private final static class StagingFileOutputStream extends OutputStream { + private final Path path; + private byte[] buffer; + private boolean memBuffered; + private int writePos; + private OutputStream out; + + private StagingFileOutputStream(Path path, int stagingBufferSize) { + this.path = path; + this.buffer = new byte[stagingBufferSize]; + this.memBuffered = true; + this.writePos = 0; + } + + private int size() { + return writePos; + } + + public InputStream newIn() { + // Just wrap it as a byte array input stream if the staging bytes are still in the in-memory + // buffer. + if (memBuffered) { + return new ByteArrayInputStream(buffer, 0, writePos); + } + + // Create a buffered file input stream. + try { + return new BufferedInputStream(Files.newInputStream(path)); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public void write(int b) throws IOException { + write(new byte[]{(byte) b}, 0, 1); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + if (memBuffered && writePos + len > buffer.length) { + flushMemToFile(); + } + + if (memBuffered) { + System.arraycopy(b, off, buffer, writePos, len); + } else { + out.write(b, off, len); + } + + writePos += len; + } + + @Override + public void close() throws IOException { + if (out != null) { + out.close(); + out = null; + } + } + + private void flushMemToFile() throws IOException { + // Flush the buffered data to the new file OutputStream. + out = new BufferedOutputStream(Files.newOutputStream(path)); + out.write(buffer, 0, writePos); + memBuffered = false; + buffer = null; + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/StagingPart.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/StagingPart.java new file mode 100644 index 0000000000000..b4fa812397285 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/StagingPart.java @@ -0,0 +1,79 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.object.staging; + +import java.io.IOException; +import java.io.InputStream; + +public interface StagingPart { + + /** + * Write bytes into the staging part. + * + * @param b the buffer to write. + * @throws IOException if any IO error. + */ + default void write(byte[] b) throws IOException { + write(b, 0, b.length); + } + + /** + * Write the bytes into the staging part. + * + * @param b the buffer to write. + * @param off the start offset in buffer. + * @param len the length. + * @throws IOException if any IO error. + */ + void write(byte[] b, int off, int len) throws IOException; + + /** + * Complete the writing process and cannot write more bytes once we've completed this part. + * + * @throws IOException if any IO error. + */ + void complete() throws IOException; + + /** + * The wrote size of staging part. + * + * @return the staging part size. + */ + long size(); + + /** + * Access the {@link State} of this part. + * + * @return the {@link State}. + */ + State state(); + + /** + * Create a separate new {@link InputStream} to read the staging part data once we've completed + * the writing by calling {@link StagingPart#complete()} . Call this method several times will + * return many {@link InputStream}s, and remember to close the newly created stream. + * + * @return a totally new {@link InputStream}. + */ + InputStream newIn(); + + /** + * Clean all the {@link StagingPart}'s resources, such as removing temporary file, free the + * buffered data etc. it should be idempotent and quiet (without throwing IO error). + */ + void cleanup(); +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/State.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/State.java new file mode 100644 index 0000000000000..418baa6d9b13a --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/State.java @@ -0,0 +1,23 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.object.staging; + +public enum State { + WRITABLE, + READABLE, + CLEANED +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/package-info.java new file mode 100644 index 0000000000000..c9c2a9033c40e --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/package-info.java @@ -0,0 +1,26 @@ +/* + * 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. + */ + +/** + * Classes for hadoop-tos object staging part. + */ +@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"}) +@InterfaceStability.Evolving +package org.apache.hadoop.fs.tosfs.object.staging; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/ChainTOSInputStream.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/ChainTOSInputStream.java new file mode 100644 index 0000000000000..7790e7c54a33d --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/ChainTOSInputStream.java @@ -0,0 +1,137 @@ +/* + * 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.tosfs.object.tos; + +import org.apache.hadoop.fs.tosfs.common.Chain; +import org.apache.hadoop.util.Preconditions; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.UncheckedIOException; +import java.util.concurrent.atomic.AtomicBoolean; + +public class ChainTOSInputStream extends InputStream { + private final Chain chain; + private final TOS.GetObjectFactory factory; + private final String key; + private long curOff; + private final long endOff; // range end offset (inclusive) + private final long maxDrainByteSize; + private final int maxInputStreamRetries; + + private int readBytes; + private long skipped; + private byte[] objChecksum = null; + private final AtomicBoolean closed = new AtomicBoolean(false); + + public ChainTOSInputStream( + TOS.GetObjectFactory factory, + String key, + long startOff, + long endOff, + long maxDrainByteSize, + int maxInputStreamRetries) { + this.factory = factory; + this.key = key; + this.curOff = startOff; + this.endOff = endOff; + this.maxDrainByteSize = maxDrainByteSize; + this.maxInputStreamRetries = maxInputStreamRetries; + this.chain = createChain(); + Preconditions.checkNotNull(objChecksum, "Checksum should not be null."); + } + + private Chain createChain() { + Chain.Builder builder = Chain.builder() + .shouldContinue(e -> !(e instanceof EOFException)); + + for (int i = 0; i <= maxInputStreamRetries; i++) { + builder.addLast(() -> { + GetObjectOutput output = factory.create(key, curOff, endOff); + + // Note: If there are some IO errors occur, the ChainTOSInputStream will create a new + // stream in the chain to continue reading object data, we need to record the checksum + // during first open object stream, and ensure the checksum of object stream won't be + // changed if opening object many times within the lifecycle of the chained stream in case + // the underlying object is changed. + if (objChecksum == null) { + // Init the stream checksum. + objChecksum = output.checksum(); + } + return new TOSInputStream(output, curOff, endOff, maxDrainByteSize, objChecksum); + }); + } + + try { + return builder.build(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public long skip(long n) throws IOException { + skipped = 0; + return chain.run(stream -> { + long skip = stream.skip(n - skipped); + + curOff += skip; + skipped += skip; + return skipped; + }); + } + + @Override + public int read() throws IOException { + return chain.run(stream -> { + int ret = stream.read(); + curOff++; + return ret; + }); + } + + @Override + public int available() throws IOException { + return chain.run(InputStream::available); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + readBytes = 0; + return chain.run(in -> { + int read = in.read(b, off + readBytes, len - readBytes); + + readBytes += read; + curOff += read; + return readBytes; + }); + } + + @Override + public void close() throws IOException { + if (closed.compareAndSet(false, true)) { + chain.close(); + } + } + + public byte[] checksum() { + return objChecksum; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClient.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClient.java new file mode 100644 index 0000000000000..a1b9f9ce4cb16 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClient.java @@ -0,0 +1,1255 @@ +/* + * 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.tosfs.object.tos; + +import com.volcengine.tos.TOSClientConfiguration; +import com.volcengine.tos.TOSV2ClientBuilder; +import com.volcengine.tos.TosClientException; +import com.volcengine.tos.TosException; +import com.volcengine.tos.TosServerException; +import com.volcengine.tos.auth.Credential; +import com.volcengine.tos.auth.Credentials; +import com.volcengine.tos.TOSV2; +import com.volcengine.tos.comm.HttpStatus; +import com.volcengine.tos.comm.common.ACLType; +import com.volcengine.tos.internal.RequestOptionsBuilder; +import com.volcengine.tos.model.acl.GetObjectAclOutput; +import com.volcengine.tos.model.acl.PutObjectAclInput; +import com.volcengine.tos.model.acl.PutObjectAclOutput; +import com.volcengine.tos.model.bucket.CreateBucketInput; +import com.volcengine.tos.model.bucket.CreateBucketOutput; +import com.volcengine.tos.model.bucket.CreateBucketV2Input; +import com.volcengine.tos.model.bucket.CreateBucketV2Output; +import com.volcengine.tos.model.bucket.DeleteBucketCORSInput; +import com.volcengine.tos.model.bucket.DeleteBucketCORSOutput; +import com.volcengine.tos.model.bucket.DeleteBucketCustomDomainInput; +import com.volcengine.tos.model.bucket.DeleteBucketCustomDomainOutput; +import com.volcengine.tos.model.bucket.DeleteBucketEncryptionInput; +import com.volcengine.tos.model.bucket.DeleteBucketEncryptionOutput; +import com.volcengine.tos.model.bucket.DeleteBucketInput; +import com.volcengine.tos.model.bucket.DeleteBucketInventoryInput; +import com.volcengine.tos.model.bucket.DeleteBucketInventoryOutput; +import com.volcengine.tos.model.bucket.DeleteBucketLifecycleInput; +import com.volcengine.tos.model.bucket.DeleteBucketLifecycleOutput; +import com.volcengine.tos.model.bucket.DeleteBucketMirrorBackInput; +import com.volcengine.tos.model.bucket.DeleteBucketMirrorBackOutput; +import com.volcengine.tos.model.bucket.DeleteBucketOutput; +import com.volcengine.tos.model.bucket.DeleteBucketPolicyInput; +import com.volcengine.tos.model.bucket.DeleteBucketPolicyOutput; +import com.volcengine.tos.model.bucket.DeleteBucketRealTimeLogInput; +import com.volcengine.tos.model.bucket.DeleteBucketRealTimeLogOutput; +import com.volcengine.tos.model.bucket.DeleteBucketRenameInput; +import com.volcengine.tos.model.bucket.DeleteBucketRenameOutput; +import com.volcengine.tos.model.bucket.DeleteBucketReplicationInput; +import com.volcengine.tos.model.bucket.DeleteBucketReplicationOutput; +import com.volcengine.tos.model.bucket.DeleteBucketTaggingInput; +import com.volcengine.tos.model.bucket.DeleteBucketTaggingOutput; +import com.volcengine.tos.model.bucket.DeleteBucketWebsiteInput; +import com.volcengine.tos.model.bucket.DeleteBucketWebsiteOutput; +import com.volcengine.tos.model.bucket.GetBucketACLInput; +import com.volcengine.tos.model.bucket.GetBucketACLOutput; +import com.volcengine.tos.model.bucket.GetBucketCORSInput; +import com.volcengine.tos.model.bucket.GetBucketCORSOutput; +import com.volcengine.tos.model.bucket.GetBucketEncryptionInput; +import com.volcengine.tos.model.bucket.GetBucketEncryptionOutput; +import com.volcengine.tos.model.bucket.GetBucketInventoryInput; +import com.volcengine.tos.model.bucket.GetBucketInventoryOutput; +import com.volcengine.tos.model.bucket.GetBucketLifecycleInput; +import com.volcengine.tos.model.bucket.GetBucketLifecycleOutput; +import com.volcengine.tos.model.bucket.GetBucketLocationInput; +import com.volcengine.tos.model.bucket.GetBucketLocationOutput; +import com.volcengine.tos.model.bucket.GetBucketMirrorBackInput; +import com.volcengine.tos.model.bucket.GetBucketMirrorBackOutput; +import com.volcengine.tos.model.bucket.GetBucketNotificationInput; +import com.volcengine.tos.model.bucket.GetBucketNotificationOutput; +import com.volcengine.tos.model.bucket.GetBucketNotificationType2Input; +import com.volcengine.tos.model.bucket.GetBucketNotificationType2Output; +import com.volcengine.tos.model.bucket.GetBucketPolicyInput; +import com.volcengine.tos.model.bucket.GetBucketPolicyOutput; +import com.volcengine.tos.model.bucket.GetBucketRealTimeLogInput; +import com.volcengine.tos.model.bucket.GetBucketRealTimeLogOutput; +import com.volcengine.tos.model.bucket.GetBucketRenameInput; +import com.volcengine.tos.model.bucket.GetBucketRenameOutput; +import com.volcengine.tos.model.bucket.GetBucketReplicationInput; +import com.volcengine.tos.model.bucket.GetBucketReplicationOutput; +import com.volcengine.tos.model.bucket.GetBucketTaggingInput; +import com.volcengine.tos.model.bucket.GetBucketTaggingOutput; +import com.volcengine.tos.model.bucket.GetBucketVersioningInput; +import com.volcengine.tos.model.bucket.GetBucketVersioningOutput; +import com.volcengine.tos.model.bucket.GetBucketWebsiteInput; +import com.volcengine.tos.model.bucket.GetBucketWebsiteOutput; +import com.volcengine.tos.model.bucket.HeadBucketOutput; +import com.volcengine.tos.model.bucket.HeadBucketV2Input; +import com.volcengine.tos.model.bucket.HeadBucketV2Output; +import com.volcengine.tos.model.bucket.ListBucketCustomDomainInput; +import com.volcengine.tos.model.bucket.ListBucketCustomDomainOutput; +import com.volcengine.tos.model.bucket.ListBucketInventoryInput; +import com.volcengine.tos.model.bucket.ListBucketInventoryOutput; +import com.volcengine.tos.model.bucket.ListBucketsInput; +import com.volcengine.tos.model.bucket.ListBucketsOutput; +import com.volcengine.tos.model.bucket.ListBucketsV2Input; +import com.volcengine.tos.model.bucket.ListBucketsV2Output; +import com.volcengine.tos.model.bucket.PutBucketACLInput; +import com.volcengine.tos.model.bucket.PutBucketACLOutput; +import com.volcengine.tos.model.bucket.PutBucketCORSInput; +import com.volcengine.tos.model.bucket.PutBucketCORSOutput; +import com.volcengine.tos.model.bucket.PutBucketCustomDomainInput; +import com.volcengine.tos.model.bucket.PutBucketCustomDomainOutput; +import com.volcengine.tos.model.bucket.PutBucketEncryptionInput; +import com.volcengine.tos.model.bucket.PutBucketEncryptionOutput; +import com.volcengine.tos.model.bucket.PutBucketInventoryInput; +import com.volcengine.tos.model.bucket.PutBucketInventoryOutput; +import com.volcengine.tos.model.bucket.PutBucketLifecycleInput; +import com.volcengine.tos.model.bucket.PutBucketLifecycleOutput; +import com.volcengine.tos.model.bucket.PutBucketMirrorBackInput; +import com.volcengine.tos.model.bucket.PutBucketMirrorBackOutput; +import com.volcengine.tos.model.bucket.PutBucketNotificationInput; +import com.volcengine.tos.model.bucket.PutBucketNotificationOutput; +import com.volcengine.tos.model.bucket.PutBucketNotificationType2Input; +import com.volcengine.tos.model.bucket.PutBucketNotificationType2Output; +import com.volcengine.tos.model.bucket.PutBucketPolicyInput; +import com.volcengine.tos.model.bucket.PutBucketPolicyOutput; +import com.volcengine.tos.model.bucket.PutBucketRealTimeLogInput; +import com.volcengine.tos.model.bucket.PutBucketRealTimeLogOutput; +import com.volcengine.tos.model.bucket.PutBucketRenameInput; +import com.volcengine.tos.model.bucket.PutBucketRenameOutput; +import com.volcengine.tos.model.bucket.PutBucketReplicationInput; +import com.volcengine.tos.model.bucket.PutBucketReplicationOutput; +import com.volcengine.tos.model.bucket.PutBucketStorageClassInput; +import com.volcengine.tos.model.bucket.PutBucketStorageClassOutput; +import com.volcengine.tos.model.bucket.PutBucketTaggingInput; +import com.volcengine.tos.model.bucket.PutBucketTaggingOutput; +import com.volcengine.tos.model.bucket.PutBucketVersioningInput; +import com.volcengine.tos.model.bucket.PutBucketVersioningOutput; +import com.volcengine.tos.model.bucket.PutBucketWebsiteInput; +import com.volcengine.tos.model.bucket.PutBucketWebsiteOutput; +import com.volcengine.tos.model.object.AbortMultipartUploadInput; +import com.volcengine.tos.model.object.AbortMultipartUploadOutput; +import com.volcengine.tos.model.object.AppendObjectInput; +import com.volcengine.tos.model.object.AppendObjectOutput; +import com.volcengine.tos.model.object.CompleteMultipartUploadInput; +import com.volcengine.tos.model.object.CompleteMultipartUploadOutput; +import com.volcengine.tos.model.object.CompleteMultipartUploadV2Input; +import com.volcengine.tos.model.object.CompleteMultipartUploadV2Output; +import com.volcengine.tos.model.object.CopyObjectOutput; +import com.volcengine.tos.model.object.CopyObjectV2Input; +import com.volcengine.tos.model.object.CopyObjectV2Output; +import com.volcengine.tos.model.object.CreateMultipartUploadInput; +import com.volcengine.tos.model.object.CreateMultipartUploadOutput; +import com.volcengine.tos.model.object.DeleteMultiObjectsInput; +import com.volcengine.tos.model.object.DeleteMultiObjectsOutput; +import com.volcengine.tos.model.object.DeleteMultiObjectsV2Input; +import com.volcengine.tos.model.object.DeleteMultiObjectsV2Output; +import com.volcengine.tos.model.object.DeleteObjectInput; +import com.volcengine.tos.model.object.DeleteObjectOutput; +import com.volcengine.tos.model.object.DeleteObjectTaggingInput; +import com.volcengine.tos.model.object.DeleteObjectTaggingOutput; +import com.volcengine.tos.model.object.DownloadFileInput; +import com.volcengine.tos.model.object.DownloadFileOutput; +import com.volcengine.tos.model.object.FetchObjectInput; +import com.volcengine.tos.model.object.FetchObjectOutput; +import com.volcengine.tos.model.object.GetFetchTaskInput; +import com.volcengine.tos.model.object.GetFetchTaskOutput; +import com.volcengine.tos.model.object.GetFileStatusInput; +import com.volcengine.tos.model.object.GetFileStatusOutput; +import com.volcengine.tos.model.object.GetObjectACLV2Input; +import com.volcengine.tos.model.object.GetObjectACLV2Output; +import com.volcengine.tos.model.object.GetObjectOutput; +import com.volcengine.tos.model.object.GetObjectTaggingInput; +import com.volcengine.tos.model.object.GetObjectTaggingOutput; +import com.volcengine.tos.model.object.GetObjectToFileInput; +import com.volcengine.tos.model.object.GetObjectToFileOutput; +import com.volcengine.tos.model.object.GetObjectV2Input; +import com.volcengine.tos.model.object.GetObjectV2Output; +import com.volcengine.tos.model.object.GetSymlinkInput; +import com.volcengine.tos.model.object.GetSymlinkOutput; +import com.volcengine.tos.model.object.HeadObjectOutput; +import com.volcengine.tos.model.object.HeadObjectV2Input; +import com.volcengine.tos.model.object.HeadObjectV2Output; +import com.volcengine.tos.model.object.ListMultipartUploadsInput; +import com.volcengine.tos.model.object.ListMultipartUploadsOutput; +import com.volcengine.tos.model.object.ListMultipartUploadsV2Input; +import com.volcengine.tos.model.object.ListMultipartUploadsV2Output; +import com.volcengine.tos.model.object.ListObjectVersionsInput; +import com.volcengine.tos.model.object.ListObjectVersionsOutput; +import com.volcengine.tos.model.object.ListObjectVersionsV2Input; +import com.volcengine.tos.model.object.ListObjectVersionsV2Output; +import com.volcengine.tos.model.object.ListObjectsInput; +import com.volcengine.tos.model.object.ListObjectsOutput; +import com.volcengine.tos.model.object.ListObjectsType2Input; +import com.volcengine.tos.model.object.ListObjectsType2Output; +import com.volcengine.tos.model.object.ListObjectsV2Input; +import com.volcengine.tos.model.object.ListObjectsV2Output; +import com.volcengine.tos.model.object.ListPartsInput; +import com.volcengine.tos.model.object.ListPartsOutput; +import com.volcengine.tos.model.object.ListUploadedPartsInput; +import com.volcengine.tos.model.object.ListUploadedPartsOutput; +import com.volcengine.tos.model.object.ObjectMetaRequestOptions; +import com.volcengine.tos.model.object.PreSignedPolicyURLInput; +import com.volcengine.tos.model.object.PreSignedPolicyURLOutput; +import com.volcengine.tos.model.object.PreSignedPostSignatureInput; +import com.volcengine.tos.model.object.PreSignedPostSignatureOutput; +import com.volcengine.tos.model.object.PreSignedURLInput; +import com.volcengine.tos.model.object.PreSignedURLOutput; +import com.volcengine.tos.model.object.PreSingedPolicyURLInput; +import com.volcengine.tos.model.object.PreSingedPolicyURLOutput; +import com.volcengine.tos.model.object.PutFetchTaskInput; +import com.volcengine.tos.model.object.PutFetchTaskOutput; +import com.volcengine.tos.model.object.PutObjectACLInput; +import com.volcengine.tos.model.object.PutObjectACLOutput; +import com.volcengine.tos.model.object.PutObjectFromFileInput; +import com.volcengine.tos.model.object.PutObjectFromFileOutput; +import com.volcengine.tos.model.object.PutObjectInput; +import com.volcengine.tos.model.object.PutObjectOutput; +import com.volcengine.tos.model.object.PutObjectTaggingInput; +import com.volcengine.tos.model.object.PutObjectTaggingOutput; +import com.volcengine.tos.model.object.PutSymlinkInput; +import com.volcengine.tos.model.object.PutSymlinkOutput; +import com.volcengine.tos.model.object.RenameObjectInput; +import com.volcengine.tos.model.object.RenameObjectOutput; +import com.volcengine.tos.model.object.RestoreObjectInput; +import com.volcengine.tos.model.object.RestoreObjectOutput; +import com.volcengine.tos.model.object.ResumableCopyObjectInput; +import com.volcengine.tos.model.object.ResumableCopyObjectOutput; +import com.volcengine.tos.model.object.SetObjectMetaInput; +import com.volcengine.tos.model.object.SetObjectMetaOutput; +import com.volcengine.tos.model.object.SetObjectTimeInput; +import com.volcengine.tos.model.object.SetObjectTimeOutput; +import com.volcengine.tos.model.object.UploadFileInput; +import com.volcengine.tos.model.object.UploadFileOutput; +import com.volcengine.tos.model.object.UploadFileV2Input; +import com.volcengine.tos.model.object.UploadFileV2Output; +import com.volcengine.tos.model.object.UploadPartCopyInput; +import com.volcengine.tos.model.object.UploadPartCopyOutput; +import com.volcengine.tos.model.object.UploadPartCopyV2Input; +import com.volcengine.tos.model.object.UploadPartCopyV2Output; +import com.volcengine.tos.model.object.UploadPartFromFileInput; +import com.volcengine.tos.model.object.UploadPartFromFileOutput; +import com.volcengine.tos.model.object.UploadPartInput; +import com.volcengine.tos.model.object.UploadPartOutput; +import com.volcengine.tos.model.object.UploadPartV2Input; +import com.volcengine.tos.model.object.UploadPartV2Output; +import com.volcengine.tos.transport.TransportConfig; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.tosfs.object.InputStreamProvider; +import org.apache.hadoop.fs.tosfs.object.Part; +import org.apache.hadoop.fs.tosfs.util.RetryableUtils; +import org.apache.hadoop.thirdparty.com.google.common.base.Throwables; +import org.apache.hadoop.thirdparty.com.google.common.io.CountingInputStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.net.SocketException; +import java.net.SocketTimeoutException; +import java.net.UnknownHostException; +import java.time.Duration; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.Callable; +import javax.net.ssl.SSLException; + +public class DelegationClient implements TOSV2 { + + private static final Logger LOG = LoggerFactory.getLogger(DelegationClient.class); + + private final Credentials provider; + private final TOSClientConfiguration config; + private int maxRetryTimes; + private TOSV2 client; + private volatile Credential usedCredential; + private final List nonRetryable409ErrorCodes; + + protected DelegationClient(TOSClientConfiguration configuration, int maxRetryTimes, + List nonRetryable409ErrorCodes) { + this.config = configuration; + this.maxRetryTimes = maxRetryTimes; + this.provider = configuration.getCredentials(); + this.usedCredential = provider.credential(); + this.client = new TOSV2ClientBuilder().build(configuration); + this.nonRetryable409ErrorCodes = nonRetryable409ErrorCodes; + } + + @VisibleForTesting + void setClient(TOSV2 client) { + this.client = client; + } + + public TOSV2 client() { + return client; + } + + @VisibleForTesting + void setMaxRetryTimes(int maxRetryTimes) { + this.maxRetryTimes = maxRetryTimes; + } + + public int maxRetryTimes() { + return maxRetryTimes; + } + + public TOSClientConfiguration config() { + return config; + } + + public Credential usedCredential() { + return usedCredential; + } + + @Override + public CreateBucketV2Output createBucket(String bucket) throws TosException { + return retry(() -> client.createBucket(bucket)); + } + + @Override + public CreateBucketV2Output createBucket(CreateBucketV2Input input) throws TosException { + return retry(() -> client.createBucket(input)); + } + + @Override + public HeadBucketV2Output headBucket(HeadBucketV2Input input) throws TosException { + return retry(() -> client.headBucket(input)); + } + + @Override + public DeleteBucketOutput deleteBucket(DeleteBucketInput input) throws TosException { + return retry(() -> client.deleteBucket(input)); + } + + @Override + public ListBucketsV2Output listBuckets(ListBucketsV2Input input) throws TosException { + return retry(() -> client.listBuckets(input)); + } + + @Override + public CreateBucketOutput createBucket(CreateBucketInput input) throws TosException { + return retry(() -> client.createBucket(input)); + } + + @Override + public HeadBucketOutput headBucket(String bucket) throws TosException { + return retry(() -> client.headBucket(bucket)); + } + + @Override + public DeleteBucketOutput deleteBucket(String bucket) throws TosException { + return retry(() -> client.deleteBucket(bucket)); + } + + @Override + public ListBucketsOutput listBuckets(ListBucketsInput input) throws TosException { + return retry(() -> client.listBuckets(input)); + } + + @Override + public PutBucketPolicyOutput putBucketPolicy(String bucket, String policy) throws TosException { + return retry(() -> client.putBucketPolicy(bucket, policy)); + } + + @Override + public PutBucketPolicyOutput putBucketPolicy(PutBucketPolicyInput input) throws TosException { + return retry(() -> client.putBucketPolicy(input)); + } + + @Override + public GetBucketPolicyOutput getBucketPolicy(String bucket) throws TosException { + return retry(() -> client.getBucketPolicy(bucket)); + } + + @Override + public GetBucketPolicyOutput getBucketPolicy(GetBucketPolicyInput input) throws TosException { + return retry(() -> client.getBucketPolicy(input)); + } + + @Override + public DeleteBucketPolicyOutput deleteBucketPolicy(String bucket) throws TosException { + return retry(() -> client.deleteBucketPolicy(bucket)); + } + + @Override + public GetObjectOutput getObject(String bucket, String objectKey, + RequestOptionsBuilder... builders) throws TosException { + return retry(() -> client.getObject(bucket, objectKey, builders)); + } + + @Override + public HeadObjectOutput headObject(String bucket, String objectKey, + RequestOptionsBuilder... builders) throws TosException { + return retry(() -> client.headObject(bucket, objectKey, builders)); + } + + @Override + public DeleteObjectOutput deleteObject(String bucket, String objectKey, + RequestOptionsBuilder... builders) throws TosException { + return retry(() -> client.deleteObject(bucket, objectKey, builders)); + } + + @Override + public DeleteMultiObjectsOutput deleteMultiObjects( + String bucket, + DeleteMultiObjectsInput input, + RequestOptionsBuilder... builders) + throws TosException { + return retry(() -> client.deleteMultiObjects(bucket, input, builders)); + } + + @Override + public PutObjectOutput putObject( + String bucket, String objectKey, InputStream inputStream, + RequestOptionsBuilder... builders) + throws TosException { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + public UploadFileOutput uploadFile( + String bucket, UploadFileInput input, + RequestOptionsBuilder... builders) throws TosException { + return retry(() -> client.uploadFile(bucket, input, builders)); + } + + @Override + public AppendObjectOutput appendObject( + String bucket, String objectKey, InputStream content, long offset, + RequestOptionsBuilder... builders) + throws TosException { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + public SetObjectMetaOutput setObjectMeta(String bucket, String objectKey, + RequestOptionsBuilder... builders) throws TosException { + return retry(() -> client.setObjectMeta(bucket, objectKey, builders)); + } + + @Override + public ListObjectsOutput listObjects(String bucket, ListObjectsInput input) throws TosException { + return retry(() -> client.listObjects(bucket, input)); + } + + @Override + public ListObjectVersionsOutput listObjectVersions(String bucket, ListObjectVersionsInput input) + throws TosException { + return retry(() -> client.listObjectVersions(bucket, input)); + } + + @Override + public CopyObjectOutput copyObject( + String bucket, String srcObjectKey, String dstObjectKey, + RequestOptionsBuilder... builders) + throws TosException { + return retry(() -> client.copyObject(bucket, srcObjectKey, dstObjectKey, builders)); + } + + @Override + public CopyObjectOutput copyObjectTo( + String bucket, String dstBucket, String dstObjectKey, + String srcObjectKey, + RequestOptionsBuilder... builders) + throws TosException { + return retry(() -> + client.copyObjectTo(bucket, dstBucket, dstObjectKey, srcObjectKey, builders)); + } + + @Override + public CopyObjectOutput copyObjectFrom( + String bucket, String srcBucket, String srcObjectKey, String dstObjectKey, + RequestOptionsBuilder... builders) + throws TosException { + return retry(() -> + client.copyObjectFrom(bucket, srcBucket, srcObjectKey, dstObjectKey, builders)); + } + + @Override + public UploadPartCopyOutput uploadPartCopy( + String bucket, UploadPartCopyInput input, + RequestOptionsBuilder... builders) throws TosException { + return retry(() -> client.uploadPartCopy(bucket, input, builders)); + } + + @Override + public PutObjectAclOutput putObjectAcl(String bucket, PutObjectAclInput input) + throws TosException { + return retry(() -> client.putObjectAcl(bucket, input)); + } + + @Override + public GetObjectAclOutput getObjectAcl( + String bucket, String objectKey, + RequestOptionsBuilder... builders) + throws TosException { + return retry(() -> client.getObjectAcl(bucket, objectKey, builders)); + } + + @Override + public CreateMultipartUploadOutput createMultipartUpload( + String bucket, String objectKey, + RequestOptionsBuilder... builders) + throws TosException { + return retry(() -> client.createMultipartUpload(bucket, objectKey, builders)); + } + + @Override + public UploadPartOutput uploadPart( + String bucket, UploadPartInput input, + RequestOptionsBuilder... builders) + throws TosException { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + public CompleteMultipartUploadOutput completeMultipartUpload( + String bucket, + CompleteMultipartUploadInput input) + throws TosException { + return retry(() -> client.completeMultipartUpload(bucket, input)); + } + + @Override + public AbortMultipartUploadOutput abortMultipartUpload( + String bucket, + AbortMultipartUploadInput input) + throws TosException { + return retry(() -> client.abortMultipartUpload(bucket, input)); + } + + @Override + public ListUploadedPartsOutput listUploadedParts( + String bucket, + ListUploadedPartsInput input, + RequestOptionsBuilder... builders) + throws TosException { + return retry(() -> client.listUploadedParts(bucket, input, builders)); + } + + @Override + public ListMultipartUploadsOutput listMultipartUploads( + String bucket, + ListMultipartUploadsInput input) + throws TosException { + return retry(() -> client.listMultipartUploads(bucket, input)); + } + + @Override + public String preSignedURL( + String httpMethod, String bucket, String objectKey, Duration ttl, + RequestOptionsBuilder... builders) + throws TosException { + return retry(() -> client.preSignedURL(httpMethod, bucket, objectKey, ttl, builders)); + } + + @Override + public DeleteBucketPolicyOutput deleteBucketPolicy(DeleteBucketPolicyInput input) + throws TosException { + return retry(() -> client.deleteBucketPolicy(input)); + } + + @Override + public PutBucketCORSOutput putBucketCORS(PutBucketCORSInput input) + throws TosException { + return retry(() -> client.putBucketCORS(input)); + } + + @Override + public GetBucketCORSOutput getBucketCORS(GetBucketCORSInput input) + throws TosException { + return retry(() -> client.getBucketCORS(input)); + } + + @Override + public DeleteBucketCORSOutput deleteBucketCORS(DeleteBucketCORSInput input) + throws TosException { + return retry(() -> client.deleteBucketCORS(input)); + } + + @Override + public PutBucketStorageClassOutput putBucketStorageClass(PutBucketStorageClassInput input) + throws TosException { + return retry(() -> client.putBucketStorageClass(input)); + } + + @Override + public GetBucketLocationOutput getBucketLocation(GetBucketLocationInput input) + throws TosException { + return retry(() -> client.getBucketLocation(input)); + } + + @Override + public PutBucketLifecycleOutput putBucketLifecycle(PutBucketLifecycleInput input) + throws TosException { + return retry(() -> client.putBucketLifecycle(input)); + } + + @Override + public GetBucketLifecycleOutput getBucketLifecycle(GetBucketLifecycleInput input) + throws TosException { + return retry(() -> client.getBucketLifecycle(input)); + } + + @Override + public DeleteBucketLifecycleOutput deleteBucketLifecycle(DeleteBucketLifecycleInput input) + throws TosException { + return retry(() -> client.deleteBucketLifecycle(input)); + } + + @Override + public PutBucketMirrorBackOutput putBucketMirrorBack(PutBucketMirrorBackInput input) + throws TosException { + return retry(() -> client.putBucketMirrorBack(input)); + } + + @Override + public GetBucketMirrorBackOutput getBucketMirrorBack(GetBucketMirrorBackInput input) + throws TosException { + return retry(() -> client.getBucketMirrorBack(input)); + } + + @Override + public DeleteBucketMirrorBackOutput deleteBucketMirrorBack(DeleteBucketMirrorBackInput input) + throws TosException { + return retry(() -> client.deleteBucketMirrorBack(input)); + } + + @Override + public PutBucketReplicationOutput putBucketReplication(PutBucketReplicationInput input) + throws TosException { + return retry(() -> client.putBucketReplication(input)); + } + + @Override + public GetBucketReplicationOutput getBucketReplication(GetBucketReplicationInput input) + throws TosException { + return retry(() -> client.getBucketReplication(input)); + } + + @Override + public DeleteBucketReplicationOutput deleteBucketReplication(DeleteBucketReplicationInput input) + throws TosException { + return retry(() -> client.deleteBucketReplication(input)); + } + + @Override + public PutBucketVersioningOutput putBucketVersioning(PutBucketVersioningInput input) + throws TosException { + return retry(() -> client.putBucketVersioning(input)); + } + + @Override + public GetBucketVersioningOutput getBucketVersioning(GetBucketVersioningInput input) + throws TosException { + return retry(() -> client.getBucketVersioning(input)); + } + + @Override + public PutBucketWebsiteOutput putBucketWebsite(PutBucketWebsiteInput input) + throws TosException { + return retry(() -> client.putBucketWebsite(input)); + } + + @Override + public GetBucketWebsiteOutput getBucketWebsite(GetBucketWebsiteInput input) + throws TosException { + return retry(() -> client.getBucketWebsite(input)); + } + + @Override + public DeleteBucketWebsiteOutput deleteBucketWebsite(DeleteBucketWebsiteInput input) + throws TosException { + return retry(() -> client.deleteBucketWebsite(input)); + } + + @Override + public PutBucketNotificationOutput putBucketNotification(PutBucketNotificationInput input) + throws TosException { + return retry(() -> client.putBucketNotification(input)); + } + + @Override + public GetBucketNotificationOutput getBucketNotification(GetBucketNotificationInput input) + throws TosException { + return retry(() -> client.getBucketNotification(input)); + } + + @Override + public PutBucketNotificationType2Output putBucketNotificationType2( + PutBucketNotificationType2Input input) throws TosException { + return retry(() -> client.putBucketNotificationType2(input)); + } + + @Override + public GetBucketNotificationType2Output getBucketNotificationType2( + GetBucketNotificationType2Input input) throws TosException { + return retry(() -> client.getBucketNotificationType2(input)); + } + + @Override + public PutBucketCustomDomainOutput putBucketCustomDomain(PutBucketCustomDomainInput input) + throws TosException { + return retry(() -> client.putBucketCustomDomain(input)); + } + + @Override + public ListBucketCustomDomainOutput listBucketCustomDomain(ListBucketCustomDomainInput input) + throws TosException { + return retry(() -> client.listBucketCustomDomain(input)); + } + + @Override + public DeleteBucketCustomDomainOutput deleteBucketCustomDomain( + DeleteBucketCustomDomainInput input) throws TosException { + return retry(() -> client.deleteBucketCustomDomain(input)); + } + + @Override + public PutBucketRealTimeLogOutput putBucketRealTimeLog(PutBucketRealTimeLogInput input) + throws TosException { + return retry(() -> client.putBucketRealTimeLog(input)); + } + + @Override + public GetBucketRealTimeLogOutput getBucketRealTimeLog(GetBucketRealTimeLogInput input) + throws TosException { + return retry(() -> client.getBucketRealTimeLog(input)); + } + + @Override + public DeleteBucketRealTimeLogOutput deleteBucketRealTimeLog(DeleteBucketRealTimeLogInput input) + throws TosException { + return retry(() -> deleteBucketRealTimeLog(input)); + } + + @Override + public PutBucketACLOutput putBucketACL(PutBucketACLInput input) throws TosException { + return retry(() -> client.putBucketACL(input)); + } + + @Override + public GetBucketACLOutput getBucketACL(GetBucketACLInput input) throws TosException { + return retry(() -> client.getBucketACL(input)); + } + + @Override + public PutBucketRenameOutput putBucketRename(PutBucketRenameInput input) throws TosException { + return retry(() -> client.putBucketRename(input)); + } + + @Override + public GetBucketRenameOutput getBucketRename(GetBucketRenameInput input) throws TosException { + return retry(() -> client.getBucketRename(input)); + } + + @Override + public DeleteBucketRenameOutput deleteBucketRename(DeleteBucketRenameInput input) + throws TosException { + return retry(() -> client.deleteBucketRename(input)); + } + + @Override + public PutBucketEncryptionOutput putBucketEncryption(PutBucketEncryptionInput input) + throws TosException { + return retry(() -> client.putBucketEncryption(input)); + } + + @Override + public GetBucketEncryptionOutput getBucketEncryption(GetBucketEncryptionInput input) + throws TosException { + return retry(() -> client.getBucketEncryption(input)); + } + + @Override + public DeleteBucketEncryptionOutput deleteBucketEncryption(DeleteBucketEncryptionInput input) + throws TosException { + return retry(() -> client.deleteBucketEncryption(input)); + } + + @Override + public PutBucketTaggingOutput putBucketTagging(PutBucketTaggingInput input) throws TosException { + return retry(() -> client.putBucketTagging(input)); + } + + @Override + public GetBucketTaggingOutput getBucketTagging(GetBucketTaggingInput input) throws TosException { + return retry(() -> client.getBucketTagging(input)); + } + + @Override + public DeleteBucketTaggingOutput deleteBucketTagging(DeleteBucketTaggingInput input) + throws TosException { + return retry(() -> client.deleteBucketTagging(input)); + } + + @Override + public PutBucketInventoryOutput putBucketInventory(PutBucketInventoryInput input) + throws TosException { + return retry(() -> client.putBucketInventory(input)); + } + + @Override + public GetBucketInventoryOutput getBucketInventory(GetBucketInventoryInput input) + throws TosException { + return retry(() -> client.getBucketInventory(input)); + } + + @Override + public ListBucketInventoryOutput listBucketInventory(ListBucketInventoryInput input) + throws TosException { + return retry(() -> client.listBucketInventory(input)); + } + + @Override + public DeleteBucketInventoryOutput deleteBucketInventory(DeleteBucketInventoryInput input) + throws TosException { + return retry(() -> client.deleteBucketInventory(input)); + } + + @Override + public GetObjectV2Output getObject(GetObjectV2Input input) throws TosException { + return retry(() -> client.getObject(input)); + } + + @Override + public GetObjectToFileOutput getObjectToFile(GetObjectToFileInput input) throws TosException { + return retry(() -> client.getObjectToFile(input)); + } + + @Override + public GetFileStatusOutput getFileStatus(GetFileStatusInput input) throws TosException { + return retry(() -> client.getFileStatus(input)); + } + + @Override + public UploadFileV2Output uploadFile(UploadFileV2Input input) throws TosException { + return retry(() -> client.uploadFile(input)); + } + + @Override + public DownloadFileOutput downloadFile(DownloadFileInput input) throws TosException { + return retry(() -> client.downloadFile(input)); + } + + @Override + public ResumableCopyObjectOutput resumableCopyObject(ResumableCopyObjectInput input) + throws TosException { + return retry(() -> client.resumableCopyObject(input)); + } + + @Override + public HeadObjectV2Output headObject(HeadObjectV2Input input) throws TosException { + return retry(() -> client.headObject(input)); + } + + @Override + public DeleteObjectOutput deleteObject(DeleteObjectInput input) throws TosException { + return retry(() -> client.deleteObject(input)); + } + + @Override + public DeleteMultiObjectsV2Output deleteMultiObjects(DeleteMultiObjectsV2Input input) + throws TosException { + return retry(() -> client.deleteMultiObjects(input)); + } + + public PutObjectOutput put( + String bucket, String key, InputStreamProvider streamProvider, + long contentLength, ACLType aclType) { + return retry(() -> client.putObject( + newPutObjectRequest(bucket, key, streamProvider, contentLength, aclType))); + } + + private PutObjectInput newPutObjectRequest( + String bucket, + String key, + InputStreamProvider streamProvider, + long contentLength, + ACLType aclType) { + + return PutObjectInput.builder() + .bucket(bucket) + .key(key) + .content(streamProvider.newStream()) + .contentLength(contentLength) + .options(new ObjectMetaRequestOptions() + .setAclType(aclType)) + .build(); + } + + public AppendObjectOutput appendObject(String bucket, String key, + InputStreamProvider streamProvider, long offset, long contentLength, String originalCrc64, + ACLType aclType) { + // originalCrc64 is needed when appending data to object. It should be the object's crc64 + // checksum if the object exists, and null if the object doesn't exist. + return retry(() -> client.appendObject( + newAppendObjectRequest(bucket, key, streamProvider, offset, contentLength, originalCrc64, + aclType))); + } + + private AppendObjectInput newAppendObjectRequest( + String bucket, + String key, + InputStreamProvider streamProvider, + long offset, + long contentLength, + String preCrc64ecma, + ACLType aclType) { + return AppendObjectInput.builder() + .bucket(bucket) + .key(key) + .content(streamProvider.newStream()) + .offset(offset) + .contentLength(contentLength) + .preHashCrc64ecma(preCrc64ecma) + .options(new ObjectMetaRequestOptions() + .setAclType(aclType)) + .build(); + } + + @Override + public PutObjectOutput putObject(PutObjectInput input) throws TosException { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + public PutObjectFromFileOutput putObjectFromFile(PutObjectFromFileInput input) + throws TosException { + return retry(() -> client.putObjectFromFile(input)); + } + + @Override + public AppendObjectOutput appendObject(AppendObjectInput input) + throws TosException { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + public SetObjectMetaOutput setObjectMeta(SetObjectMetaInput input) + throws TosException { + return retry(() -> client.setObjectMeta(input)); + } + + @Override + public SetObjectTimeOutput setObjectTime(SetObjectTimeInput input) throws TosException { + return retry(() -> client.setObjectTime(input)); + } + + @Override + public ListObjectsV2Output listObjects(ListObjectsV2Input input) + throws TosException { + return retry(() -> client.listObjects(input)); + } + + @Override + public ListObjectsType2Output listObjectsType2(ListObjectsType2Input input) + throws TosException { + return retry(() -> client.listObjectsType2(input)); + } + + @Override + public ListObjectVersionsV2Output listObjectVersions(ListObjectVersionsV2Input input) + throws TosException { + return retry(() -> client.listObjectVersions(input)); + } + + @Override + public CopyObjectV2Output copyObject(CopyObjectV2Input input) + throws TosException { + return retry(() -> client.copyObject(input)); + } + + @Override + public UploadPartCopyV2Output uploadPartCopy(UploadPartCopyV2Input input) + throws TosException { + return retry(() -> client.uploadPartCopy(input)); + } + + @Override + public PutObjectACLOutput putObjectAcl(PutObjectACLInput input) + throws TosException { + return retry(() -> client.putObjectAcl(input)); + } + + @Override + public GetObjectACLV2Output getObjectAcl(GetObjectACLV2Input input) + throws TosException { + return retry(() -> client.getObjectAcl(input)); + } + + @Override + public PutObjectTaggingOutput putObjectTagging(PutObjectTaggingInput input) + throws TosException { + return retry(() -> client.putObjectTagging(input)); + } + + @Override + public GetObjectTaggingOutput getObjectTagging(GetObjectTaggingInput input) + throws TosException { + return retry(() -> client.getObjectTagging(input)); + } + + @Override + public DeleteObjectTaggingOutput deleteObjectTagging(DeleteObjectTaggingInput input) + throws TosException { + return retry(() -> client.deleteObjectTagging(input)); + } + + @Override + public FetchObjectOutput fetchObject(FetchObjectInput input) throws TosException { + return retry(() -> client.fetchObject(input)); + } + + @Override + public PutFetchTaskOutput putFetchTask(PutFetchTaskInput input) throws TosException { + return retry(() -> client.putFetchTask(input)); + } + + @Override + public GetFetchTaskOutput getFetchTask(GetFetchTaskInput input) throws TosException { + return retry(() -> client.getFetchTask(input)); + } + + @Override + public CreateMultipartUploadOutput createMultipartUpload(CreateMultipartUploadInput input) + throws TosException { + return retry(() -> client.createMultipartUpload(input)); + } + + public Part uploadPart( + String bucket, + String key, + String uploadId, + int partNum, + InputStreamProvider streamProvider, + long contentLength, + ACLType aclType) { + return retry(() -> { + InputStream in = streamProvider.newStream(); + CountingInputStream countedIn = new CountingInputStream(in); + UploadPartV2Input request = UploadPartV2Input.builder() + .bucket(bucket) + .key(key) + .partNumber(partNum) + .uploadID(uploadId) + .content(countedIn) + .contentLength(contentLength) + .options(new ObjectMetaRequestOptions() + .setAclType(aclType)) + .build(); + UploadPartV2Output output = client.uploadPart(request); + return new Part(output.getPartNumber(), countedIn.getCount(), output.getEtag()); + }); + } + + @Override + public UploadPartV2Output uploadPart(UploadPartV2Input input) throws TosException { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + public UploadPartFromFileOutput uploadPartFromFile(UploadPartFromFileInput input) + throws TosException { + return retry(() -> client.uploadPartFromFile(input)); + } + + @Override + public CompleteMultipartUploadV2Output completeMultipartUpload( + CompleteMultipartUploadV2Input input) throws TosException { + return retry(() -> client.completeMultipartUpload(input)); + } + + @Override + public AbortMultipartUploadOutput abortMultipartUpload(AbortMultipartUploadInput input) + throws TosException { + return retry(() -> client.abortMultipartUpload(input)); + } + + @Override + public ListPartsOutput listParts(ListPartsInput input) throws TosException { + return retry(() -> client.listParts(input)); + } + + @Override + public ListMultipartUploadsV2Output listMultipartUploads(ListMultipartUploadsV2Input input) + throws TosException { + return retry(() -> client.listMultipartUploads(input)); + } + + @Override + public RenameObjectOutput renameObject(RenameObjectInput input) throws TosException { + return retry(() -> client.renameObject(input)); + } + + @Override + public RestoreObjectOutput restoreObject(RestoreObjectInput input) throws TosException { + return retry(() -> client.restoreObject(input)); + } + + @Override + public PutSymlinkOutput putSymlink(PutSymlinkInput input) throws TosException { + return retry(() -> client.putSymlink(input)); + } + + @Override + public GetSymlinkOutput getSymlink(GetSymlinkInput input) throws TosException { + return retry(() -> client.getSymlink(input)); + } + + @Override + public PreSignedURLOutput preSignedURL(PreSignedURLInput input) throws TosException { + return retry(() -> client.preSignedURL(input)); + } + + @Override + public PreSignedPostSignatureOutput preSignedPostSignature(PreSignedPostSignatureInput input) + throws TosException { + return retry(() -> client.preSignedPostSignature(input)); + } + + @Override + public PreSingedPolicyURLOutput preSingedPolicyURL(PreSingedPolicyURLInput input) + throws TosException { + return retry(() -> client.preSingedPolicyURL(input)); + } + + @Override + public PreSignedPolicyURLOutput preSignedPolicyURL(PreSignedPolicyURLInput input) + throws TosException { + return retry(() -> client.preSignedPolicyURL(input)); + } + + @Override + public void changeCredentials(Credentials credentials) { + retry(() -> { + client.changeCredentials(credentials); + return null; + }); + } + + @Override + public void changeRegionAndEndpoint(String region, String endpoint) { + retry(() -> { + client.changeRegionAndEndpoint(region, endpoint); + return null; + }); + } + + @Override + public void changeTransportConfig(TransportConfig conf) { + retry(() -> { + client.changeTransportConfig(conf); + return null; + }); + } + + @Override + public boolean refreshEndpointRegion(String s, String s1) { + return retry(() -> refreshEndpointRegion(s, s1)); + } + + @Override + public boolean refreshCredentials(String s, String s1, String s2) { + return retry(() -> refreshCredentials(s, s1, s2)); + } + + @Override + public void close() throws IOException { + client.close(); + } + + private void refresh() throws TosException { + Credential credential = provider.credential(); + if (credentialIsChanged(credential)) { + synchronized (this) { + if (credentialIsChanged(credential)) { + client.changeCredentials(provider); + usedCredential = credential; + } + } + } + } + + private boolean credentialIsChanged(Credential credential) { + return !Objects.equals(credential.getAccessKeyId(), usedCredential.getAccessKeyId()) + || !Objects.equals(credential.getAccessKeySecret(), usedCredential.getAccessKeySecret()) + || !Objects.equals(credential.getSecurityToken(), usedCredential.getSecurityToken()); + } + + private T retry(Callable callable) { + int attempt = 0; + while (true) { + attempt++; + try { + refresh(); + return callable.call(); + } catch (TosException e) { + if (attempt >= maxRetryTimes) { + LOG.error("Retry exhausted after {} times.", maxRetryTimes); + throw e; + } + if (isRetryableException(e, nonRetryable409ErrorCodes)) { + LOG.warn("Retry TOS request in the {} times, error: {}", attempt, + Throwables.getRootCause(e).getMessage()); + try { + // last time does not need to sleep + Thread.sleep(RetryableUtils.backoff(attempt)); + } catch (InterruptedException ex) { + throw new TosClientException("tos: request interrupted.", ex); + } + } else { + throw e; + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + @VisibleForTesting + static boolean isRetryableException(TosException e, List nonRetryable409ErrorCodes) { + return e.getStatusCode() >= HttpStatus.INTERNAL_SERVER_ERROR + || e.getStatusCode() == HttpStatus.TOO_MANY_REQUESTS + || e.getCause() instanceof SocketException + || e.getCause() instanceof UnknownHostException + || e.getCause() instanceof SSLException + || e.getCause() instanceof SocketTimeoutException + || e.getCause() instanceof InterruptedException + || isRetryableTosClientException(e) + || isRetryableTosServerException(e, nonRetryable409ErrorCodes); + } + + private static boolean isRetryableTosClientException(TosException e) { + return e instanceof TosClientException + && e.getCause() instanceof IOException + && !(e.getCause() instanceof EOFException); + } + + private static boolean isRetryableTosServerException(TosException e, + List nonRetryable409ErrorCodes) { + return e instanceof TosServerException + && e.getStatusCode() == HttpStatus.CONFLICT + && isRetryableTosConflictException((TosServerException) e, nonRetryable409ErrorCodes); + } + + private static boolean isRetryableTosConflictException(TosServerException e, + List nonRetryableCodes) { + String errorCode = e.getEc(); + return StringUtils.isEmpty(errorCode) || !nonRetryableCodes.contains(errorCode); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClientBuilder.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClientBuilder.java new file mode 100644 index 0000000000000..cc1640c2e8af6 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClientBuilder.java @@ -0,0 +1,185 @@ +/* + * 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.tosfs.object.tos; + +import com.volcengine.tos.TOSClientConfiguration; +import com.volcengine.tos.TosException; +import com.volcengine.tos.transport.TransportConfig; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.conf.TosKeys; +import org.apache.hadoop.fs.tosfs.object.Constants; +import org.apache.hadoop.fs.tosfs.object.tos.auth.CredentialsProvider; +import org.apache.hadoop.fs.tosfs.util.ParseUtils; +import org.apache.hadoop.fs.tosfs.util.TOSClientContextUtils; +import org.apache.hadoop.util.Preconditions; +import org.apache.hadoop.util.VersionInfo; + +import java.lang.reflect.InvocationTargetException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import static org.apache.hadoop.fs.tosfs.object.tos.TOS.TOS_SCHEME; + +public class DelegationClientBuilder { + + public static final int DISABLE_TOS_RETRY_VALUE = -1; + private static final String TOS_ENDPOINT_KEY = + ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key(TOS_SCHEME); + private static final String TOS_REGION_KEY = ConfKeys.FS_OBJECT_STORAGE_REGION.key(TOS_SCHEME); + + @VisibleForTesting + static final Map CACHE = new ConcurrentHashMap<>(); + + private String bucket; + private Configuration conf; + + public DelegationClientBuilder bucket(String bucketInput) { + this.bucket = bucketInput; + return this; + } + + public DelegationClientBuilder conf(Configuration confInput) { + this.conf = confInput; + return this; + } + + public DelegationClient build() throws TosException { + Preconditions.checkNotNull(bucket, "Bucket cannot be null"); + Preconditions.checkNotNull(conf, "Conf cannot be null"); + String endpoint = getAndCheckEndpoint(conf); + String region = getAndCheckRegion(conf, endpoint); + + if (conf.getBoolean(TosKeys.FS_TOS_DISABLE_CLIENT_CACHE, + TosKeys.FS_TOS_DISABLE_CLIENT_CACHE_DEFAULT)) { + return createNewClient(conf, endpoint, region, bucket, false); + } + return CACHE.computeIfAbsent(bucket, + client -> createNewClient(conf, endpoint, region, bucket, true)); + } + + private DelegationClient createNewClient(Configuration config, String endpoint, String region, + String bucketName, boolean cached) { + CredentialsProvider provider = createProvider(config, bucketName); + TOSClientConfiguration clientConfiguration = TOSClientConfiguration.builder() + .region(region) + .endpoint(endpoint) + .credentials(provider) + .enableCrc(config.getBoolean( + TosKeys.FS_TOS_CRC_CHECK_ENABLED, TosKeys.FS_TOS_CRC_CHECK_ENABLED_DEFAULT)) + .transportConfig(createTransportConfig(config)) + .userAgentProductName(config.get( + TosKeys.FS_TOS_USER_AGENT_PREFIX, TosKeys.FS_TOS_USER_AGENT_PREFIX_DEFAULT)) + .userAgentSoftName(Constants.TOS_FS) + .userAgentSoftVersion(VersionInfo.getVersion()) + .build(); + + int maxRetryTimes = config.getInt(TosKeys.FS_TOS_REQUEST_MAX_RETRY_TIMES, + TosKeys.FS_TOS_REQUEST_MAX_RETRY_TIMES_DEFAULT); + List nonRetryable409ErrorCodes = Arrays.asList( + config.getTrimmedStrings(TosKeys.FS_TOS_FAST_FAILURE_409_ERROR_CODES, + TosKeys.FS_TOS_FAST_FAILURE_409_ERROR_CODES_DEFAULT)); + + if (cached) { + return new CachedClient(clientConfiguration, maxRetryTimes, nonRetryable409ErrorCodes); + } else { + return new DelegationClient(clientConfiguration, maxRetryTimes, nonRetryable409ErrorCodes); + } + } + + private CredentialsProvider createProvider(Configuration config, String bucketName) { + try { + CredentialsProvider provider = (CredentialsProvider) Class.forName( + config.get(TosKeys.FS_TOS_CREDENTIALS_PROVIDER, + TosKeys.FS_TOS_CREDENTIALS_PROVIDER_DEFAULT)) + .getDeclaredConstructor() + .newInstance(); + provider.initialize(config, bucketName); + return provider; + } catch (ClassNotFoundException | + InstantiationException | + IllegalAccessException | + InvocationTargetException | + NoSuchMethodException e) { + throw new TosException(e); + } + } + + private String getAndCheckEndpoint(Configuration config) { + String endpoint = config.get(TOS_ENDPOINT_KEY); + if (StringUtils.isBlank(endpoint)) { + endpoint = ParseUtils.envAsString(TOS.ENV_TOS_ENDPOINT); + } + Preconditions.checkNotNull(endpoint, "%s cannot be null", TOS_ENDPOINT_KEY); + return endpoint.trim(); + } + + private String getAndCheckRegion(Configuration config, String endpoint) { + String region = config.get(TOS_REGION_KEY); + if (StringUtils.isNotBlank(region)) { + return region.trim(); + } + region = TOSClientContextUtils.parseRegion(endpoint); + Preconditions.checkNotNull(region, "%s cannot be null", TOS_REGION_KEY); + return region.trim(); + } + + private TransportConfig createTransportConfig(Configuration config) { + TransportConfig.TransportConfigBuilder builder = TransportConfig.builder(); + // Disable tos sdk retry with negative number since we have set retry strategy above TOS SDK, + // which cannot support retry all input streams via mark & reset API. + // It's hard to use it as there are some restrictions. + // the TOS SDK will reset the max retry count with 3 if the configured count equal to 0. + builder.maxRetryCount(DISABLE_TOS_RETRY_VALUE); + + builder.maxConnections(config.getInt(TosKeys.FS_TOS_HTTP_MAX_CONNECTIONS, + TosKeys.FS_TOS_HTTP_MAX_CONNECTIONS_DEFAULT)); + builder.idleConnectionTimeMills(config.getInt(TosKeys.FS_TOS_HTTP_IDLE_CONNECTION_TIME_MILLS, + TosKeys.FS_TOS_HTTP_IDLE_CONNECTION_TIME_MILLS_DEFAULT)); + builder.connectTimeoutMills(config.getInt(TosKeys.FS_TOS_HTTP_CONNECT_TIMEOUT_MILLS, + TosKeys.FS_TOS_HTTP_CONNECT_TIMEOUT_MILLS_DEFAULT)); + builder.readTimeoutMills(config.getInt(TosKeys.FS_TOS_HTTP_READ_TIMEOUT_MILLS, + TosKeys.FS_TOS_HTTP_READ_TIMEOUT_MILLS_DEFAULT)); + builder.writeTimeoutMills(config.getInt(TosKeys.FS_TOS_HTTP_WRITE_TIMEOUT_MILLS, + TosKeys.FS_TOS_HTTP_WRITE_TIMEOUT_MILLS_DEFAULT)); + builder.enableVerifySSL(config.getBoolean(TosKeys.FS_TOS_HTTP_ENABLE_VERIFY_SSL, + TosKeys.FS_TOS_HTTP_ENABLE_VERIFY_SSL_DEFAULT)); + builder.dnsCacheTimeMinutes(config.getInt(TosKeys.FS_TOS_HTTP_DNS_CACHE_TIME_MINUTES, + TosKeys.FS_TOS_HTTP_DNS_CACHE_TIME_MINUTES_DEFAULT)); + + return builder.build(); + } + + static class CachedClient extends DelegationClient { + + protected CachedClient(TOSClientConfiguration configuration, int maxRetryTimes, + List nonRetryable409ErrorCodes) { + super(configuration, maxRetryTimes, nonRetryable409ErrorCodes); + } + + @Override + public void close() { + // do nothing as this client may be shared by multiple upper-layer instances + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/GetObjectOutput.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/GetObjectOutput.java new file mode 100644 index 0000000000000..718cc7e39ef24 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/GetObjectOutput.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.tosfs.object.tos; + +import com.volcengine.tos.model.object.GetObjectV2Output; +import org.apache.hadoop.fs.tosfs.object.exceptions.ChecksumMismatchException; +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.util.Preconditions; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Arrays; + +public class GetObjectOutput { + private final GetObjectV2Output output; + private final byte[] checksum; + + public GetObjectOutput(GetObjectV2Output output, byte[] checksum) { + Preconditions.checkNotNull(checksum, "Checksum should not be null."); + this.output = output; + this.checksum = checksum; + } + + public GetObjectV2Output output() { + return output; + } + + public byte[] checksum() { + return checksum; + } + + public InputStream verifiedContent(byte[] expectedChecksum) throws IOException { + if (!Arrays.equals(expectedChecksum, checksum)) { + CommonUtils.runQuietly(this::forceClose); + throw new ChecksumMismatchException(expectedChecksum, checksum); + } + + return output.getContent(); + } + + public void forceClose() throws IOException { + output.forceClose(); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOS.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOS.java new file mode 100644 index 0000000000000..aac7b933478bb --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOS.java @@ -0,0 +1,1038 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.object.tos; + +import com.volcengine.tos.TOSV2; +import com.volcengine.tos.TosException; +import com.volcengine.tos.TosServerException; +import com.volcengine.tos.comm.common.ACLType; +import com.volcengine.tos.comm.common.BucketType; +import com.volcengine.tos.internal.util.TypeConverter; +import com.volcengine.tos.model.bucket.HeadBucketV2Input; +import com.volcengine.tos.model.bucket.HeadBucketV2Output; +import com.volcengine.tos.model.bucket.Tag; +import com.volcengine.tos.model.object.AbortMultipartUploadInput; +import com.volcengine.tos.model.object.AppendObjectOutput; +import com.volcengine.tos.model.object.CompleteMultipartUploadV2Input; +import com.volcengine.tos.model.object.CopyObjectV2Input; +import com.volcengine.tos.model.object.CreateMultipartUploadInput; +import com.volcengine.tos.model.object.CreateMultipartUploadOutput; +import com.volcengine.tos.model.object.DeleteError; +import com.volcengine.tos.model.object.DeleteMultiObjectsV2Input; +import com.volcengine.tos.model.object.DeleteMultiObjectsV2Output; +import com.volcengine.tos.model.object.DeleteObjectInput; +import com.volcengine.tos.model.object.DeleteObjectTaggingInput; +import com.volcengine.tos.model.object.GetFileStatusInput; +import com.volcengine.tos.model.object.GetFileStatusOutput; +import com.volcengine.tos.model.object.GetObjectBasicOutput; +import com.volcengine.tos.model.object.GetObjectTaggingInput; +import com.volcengine.tos.model.object.GetObjectTaggingOutput; +import com.volcengine.tos.model.object.GetObjectV2Input; +import com.volcengine.tos.model.object.GetObjectV2Output; +import com.volcengine.tos.model.object.HeadObjectV2Input; +import com.volcengine.tos.model.object.HeadObjectV2Output; +import com.volcengine.tos.model.object.ListMultipartUploadsV2Input; +import com.volcengine.tos.model.object.ListMultipartUploadsV2Output; +import com.volcengine.tos.model.object.ListObjectsType2Input; +import com.volcengine.tos.model.object.ListObjectsType2Output; +import com.volcengine.tos.model.object.ListedCommonPrefix; +import com.volcengine.tos.model.object.ListedObjectV2; +import com.volcengine.tos.model.object.ListedUpload; +import com.volcengine.tos.model.object.ObjectMetaRequestOptions; +import com.volcengine.tos.model.object.ObjectTobeDeleted; +import com.volcengine.tos.model.object.PutObjectOutput; +import com.volcengine.tos.model.object.PutObjectTaggingInput; +import com.volcengine.tos.model.object.RenameObjectInput; +import com.volcengine.tos.model.object.TagSet; +import com.volcengine.tos.model.object.UploadPartCopyV2Input; +import com.volcengine.tos.model.object.UploadPartCopyV2Output; +import com.volcengine.tos.model.object.UploadedPartV2; +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.conf.TosKeys; +import org.apache.hadoop.fs.tosfs.object.BucketInfo; +import org.apache.hadoop.fs.tosfs.object.ChecksumInfo; +import org.apache.hadoop.fs.tosfs.object.ChecksumType; +import org.apache.hadoop.fs.tosfs.object.Constants; +import org.apache.hadoop.fs.tosfs.object.DirectoryStorage; +import org.apache.hadoop.fs.tosfs.object.InputStreamProvider; +import org.apache.hadoop.fs.tosfs.object.MultipartUpload; +import org.apache.hadoop.fs.tosfs.object.ObjectConstants; +import org.apache.hadoop.fs.tosfs.object.ObjectContent; +import org.apache.hadoop.fs.tosfs.object.ObjectInfo; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectUtils; +import org.apache.hadoop.fs.tosfs.object.Part; +import org.apache.hadoop.fs.tosfs.object.exceptions.InvalidObjectKeyException; +import org.apache.hadoop.fs.tosfs.object.exceptions.NotAppendableException; +import org.apache.hadoop.fs.tosfs.object.request.ListObjectsRequest; +import org.apache.hadoop.fs.tosfs.object.response.ListObjectsResponse; +import org.apache.hadoop.fs.tosfs.util.LazyReload; +import org.apache.hadoop.thirdparty.com.google.common.base.Joiner; +import org.apache.hadoop.thirdparty.com.google.common.base.Strings; +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.util.Preconditions; +import org.apache.log4j.Level; +import org.apache.log4j.LogManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Date; +import java.util.Deque; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; + +import static org.apache.hadoop.fs.tosfs.object.tos.TOSErrorCodes.APPEND_NOT_APPENDABLE; +import static org.apache.hadoop.fs.tosfs.object.tos.TOSUtils.CHECKSUM_HEADER; +import static org.apache.hadoop.fs.tosfs.object.tos.TOSUtils.appendable; +import static org.apache.hadoop.fs.tosfs.object.tos.TOSUtils.crc64ecma; +import static org.apache.hadoop.fs.tosfs.object.tos.TOSUtils.parseChecksum; + +/** + * {@link TOS} will be initialized by the {@link ObjectStorage#initialize(Configuration, String)}. + */ +public class TOS implements DirectoryStorage { + + private static final Logger LOG = LoggerFactory.getLogger(TOS.class); + public static final String TOS_SCHEME = "tos"; + + public static final String ENV_TOS_ACCESS_KEY_ID = "TOS_ACCESS_KEY_ID"; + public static final String ENV_TOS_SECRET_ACCESS_KEY = "TOS_SECRET_ACCESS_KEY"; + public static final String ENV_TOS_SESSION_TOKEN = "TOS_SESSION_TOKEN"; + public static final String ENV_TOS_ENDPOINT = "TOS_ENDPOINT"; + + private static final int NOT_FOUND_CODE = 404; + private static final int PATH_CONFLICT_CODE = 409; + private static final int INVALID_RANGE_CODE = 416; + + private static final int MIN_PART_SIZE = 5 * 1024 * 1024; + private static final int MAX_PART_COUNT = 10000; + + private static final InputStream EMPTY_STREAM = new ByteArrayInputStream(new byte[0]); + + private Configuration conf; + private String bucket; + private DelegationClient client; + private long maxDrainBytes; + private int batchDeleteMaxRetries; + private List batchDeleteRetryCodes; + private long batchDeleteRetryInterval; + private int maxDeleteObjectsCount; + private int listObjectsCount; + // the max retry times during reading object content + private int maxInputStreamRetries; + private ACLType defaultAcl; + private ChecksumInfo checksumInfo; + private BucketInfo bucketInfo; + + static { + org.apache.log4j.Logger logger = LogManager.getLogger("com.volcengine.tos"); + String logLevel = System.getProperty("tos.log.level", "WARN"); + + LOG.debug("Reset the log level of com.volcengine.tos with {} ", logLevel); + logger.setLevel(Level.toLevel(logLevel.toUpperCase(), Level.WARN)); + } + + @Override + public void initialize(Configuration config, String bucketName) { + this.conf = config; + this.bucket = bucketName; + client = new DelegationClientBuilder().conf(config).bucket(bucketName).build(); + maxDrainBytes = + config.getLong(TosKeys.FS_TOS_MAX_DRAIN_BYTES, TosKeys.FS_TOS_MAX_DRAIN_BYTES_DEFAULT); + batchDeleteMaxRetries = config.getInt(TosKeys.FS_TOS_BATCH_DELETE_MAX_RETRIES, + TosKeys.FS_TOS_BATCH_DELETE_MAX_RETRIES_DEFAULT); + batchDeleteRetryCodes = Arrays.asList( + config.getTrimmedStrings(TosKeys.FS_TOS_BATCH_DELETE_RETRY_CODES, + TosKeys.FS_TOS_BATCH_DELETE_RETRY_CODES_DEFAULT)); + batchDeleteRetryInterval = config.getLong(TosKeys.FS_TOS_BATCH_DELETE_RETRY_INTERVAL, + TosKeys.FS_TOS_BATCH_DELETE_RETRY_INTERVAL_DEFAULT); + maxDeleteObjectsCount = config.getInt(TosKeys.FS_TOS_DELETE_OBJECTS_COUNT, + TosKeys.FS_TOS_DELETE_OBJECTS_COUNT_DEFAULT); + listObjectsCount = + config.getInt(TosKeys.FS_TOS_LIST_OBJECTS_COUNT, TosKeys.FS_TOS_LIST_OBJECTS_COUNT_DEFAULT); + maxInputStreamRetries = config.getInt(TosKeys.FS_TOS_MAX_READ_OBJECT_RETRIES, + TosKeys.FS_TOS_MAX_READ_OBJECT_RETRIES_DEFAULT); + defaultAcl = TypeConverter.convertACLType(config.get(TosKeys.FS_TOS_ACL_DEFAULT)); + + String algorithm = + config.get(TosKeys.FS_TOS_CHECKSUM_ALGORITHM, TosKeys.FS_TOS_CHECKSUM_ALGORITHM_DEFAULT); + ChecksumType checksumType = ChecksumType.valueOf( + config.get(TosKeys.FS_TOS_CHECKSUM_TYPE, TosKeys.FS_TOS_CHECKSUM_TYPE_DEFAULT) + .toUpperCase()); + Preconditions.checkArgument(CHECKSUM_HEADER.containsKey(checksumType), + "Checksum type %s is not supported by TOS.", checksumType.name()); + checksumInfo = new ChecksumInfo(algorithm, checksumType); + + bucketInfo = getBucketInfo(bucketName); + } + + @Override + public String scheme() { + return TOS_SCHEME; + } + + @Override + public Configuration conf() { + return conf; + } + + @Override + public BucketInfo bucket() { + return bucketInfo; + } + + private BucketInfo getBucketInfo(String bucketName) { + try { + HeadBucketV2Output res = + client.headBucket(HeadBucketV2Input.builder().bucket(bucketName).build()); + + // BUCKET_TYPE_FNS is the general purpose bucket, BUCKET_TYPE_HNS is directory bucket. + boolean directoryBucket = BucketType.BUCKET_TYPE_HNS.equals(res.getBucketType()); + + return new BucketInfo(bucketName, directoryBucket); + } catch (TosException e) { + if (e.getStatusCode() == NOT_FOUND_CODE) { + return null; + } + throw new RuntimeException(e); + } + } + + @VisibleForTesting + void setClient(DelegationClient client) { + this.client = client; + } + + private void checkAvailableClient() { + Preconditions.checkState(client != null, + "Encountered uninitialized ObjectStorage, call initialize(..) please."); + } + + @Override + public ObjectContent get(String key, long offset, long limit) { + checkAvailableClient(); + Preconditions.checkArgument(offset >= 0, "offset is a negative number: %s", offset); + + if (limit == 0) { + // Can not return empty stream when limit = 0, because the requested object might not exist. + if (head(key) != null) { + return new ObjectContent(Constants.MAGIC_CHECKSUM, EMPTY_STREAM); + } else { + throw new RuntimeException(String.format("Object %s doesn't exit", key)); + } + } + + long end = limit < 0 ? -1 : offset + limit - 1; + GetObjectFactory factory = (k, startOff, endOff) -> getObject(key, startOff, endOff); + ChainTOSInputStream chainStream = + new ChainTOSInputStream(factory, key, offset, end, maxDrainBytes, maxInputStreamRetries); + return new ObjectContent(chainStream.checksum(), chainStream); + } + + @Override + public Iterable listDir(String key, boolean recursive) { + if (recursive) { + if (bucket().isDirectory()) { + // The directory bucket only support list object with delimiter = '/', so if we want to + // list directory recursively, we have to list each dir step by step. + return bfsListDir(key); + } else { + return listAll(key, key); + } + } else { + return innerListDir(key, key, -1); + } + } + + private Iterable bfsListDir(String key) { + return new LazyReload<>(() -> { + final Deque dirQueue = new LinkedList<>(); + AtomicReference continueToken = new AtomicReference<>(""); + AtomicReference curDir = new AtomicReference<>(key); + + return buf -> { + // No more objects when isTruncated is false. + if (curDir.get() == null) { + return true; + } + + ListObjectsType2Input request = + createListObjectsType2Input(curDir.get(), curDir.get(), listObjectsCount, "/", + continueToken.get()); + ListObjectsType2Output response = client.listObjectsType2(request); + + if (response.getContents() != null) { + for (ListedObjectV2 obj : response.getContents()) { + buf.add(new ObjectInfo(obj.getKey(), obj.getSize(), obj.getLastModified(), + parseChecksum(obj, checksumInfo))); + } + } + + if (response.getCommonPrefixes() != null) { + for (ListedCommonPrefix prefix : response.getCommonPrefixes()) { + buf.add(new ObjectInfo(prefix.getPrefix(), 0, new Date(), Constants.MAGIC_CHECKSUM)); + dirQueue.add(prefix.getPrefix()); + } + } + + if (response.isTruncated()) { + continueToken.set(response.getNextContinuationToken()); + } else { + curDir.set(dirQueue.poll()); + continueToken.set(""); + } + + return curDir.get() == null; + }; + }); + } + + private Iterable innerListDir(String key, String startAfter, int limit) { + return new LazyReload<>(() -> { + AtomicReference continueToken = new AtomicReference<>(""); + AtomicBoolean isTruncated = new AtomicBoolean(true); + AtomicInteger remaining = new AtomicInteger(limit < 0 ? Integer.MAX_VALUE : limit); + + return buf -> { + // No more objects when isTruncated is false. + if (!isTruncated.get()) { + return true; + } + + int remainingKeys = remaining.get(); + int maxKeys = Math.min(listObjectsCount, remainingKeys); + ListObjectsType2Input request = + createListObjectsType2Input(key, startAfter, maxKeys, "/", continueToken.get()); + ListObjectsType2Output response = client.listObjectsType2(request); + + if (response.getContents() != null) { + for (ListedObjectV2 obj : response.getContents()) { + buf.add(new ObjectInfo(obj.getKey(), obj.getSize(), obj.getLastModified(), + parseChecksum(obj, checksumInfo))); + } + } + + if (response.getCommonPrefixes() != null) { + for (ListedCommonPrefix prefix : response.getCommonPrefixes()) { + buf.add(new ObjectInfo(prefix.getPrefix(), 0, new Date(), Constants.MAGIC_CHECKSUM)); + } + } + + isTruncated.set(response.isTruncated()); + remaining.compareAndSet(remainingKeys, remainingKeys - response.getKeyCount()); + continueToken.set(response.getNextContinuationToken()); + + return !isTruncated.get(); + }; + }); + } + + @Override + public void deleteDir(String key, boolean recursive) { + checkAvailableClient(); + if (recursive) { + if (conf.getBoolean(TosKeys.FS_TOS_RMR_SERVER_ENABLED, + TosKeys.FS_FS_TOS_RMR_SERVER_ENABLED_DEFAULT)) { + DeleteObjectInput request = + DeleteObjectInput.builder().bucket(bucket).recursive(true).key(key).build(); + try { + // It's a test feature, TOS SDK don't expose atomic delete dir capability currently. + Field f = DeleteObjectInput.class.getDeclaredField("recursiveByServer"); + f.setAccessible(true); + f.setBoolean(request, true); + } catch (Exception e) { + throw new RuntimeException(e); + } + client.deleteObject(request); + } else { + if (conf.getBoolean(TosKeys.FS_TOS_RMR_CLIENT_ENABLE, + TosKeys.FS_TOS_RMR_CLIENT_ENABLE_DEFAULT)) { + client.deleteObject( + DeleteObjectInput.builder().bucket(bucket).recursive(true).key(key).build()); + } else { + recursiveDeleteDir(key); + } + } + } else { + delete(key); + } + } + + @Override + public boolean isEmptyDir(String key) { + checkAvailableClient(); + return !innerListDir(key, key, 1).iterator().hasNext(); + } + + public void recursiveDeleteDir(String key) { + for (ObjectInfo obj : innerListDir(key, key, -1)) { + if (obj.isDir()) { + recursiveDeleteDir(obj.key()); + } else { + delete(obj.key()); + } + } + delete(key); + } + + interface GetObjectFactory { + /** + * Get object content for the given object key and range. + * + * @param key The object key + * @param offset The start offset of object content + * @param end The end offset of object content + * @return {@link GetObjectOutput} + */ + GetObjectOutput create(String key, long offset, long end); + } + + public GetObjectOutput getObject(String key, long offset, long end) { + checkAvailableClient(); + Preconditions.checkArgument(offset >= 0, "offset is a negative number: %s", offset); + + try { + GetObjectV2Input request = GetObjectV2Input.builder().bucket(bucket).key(key) + .options(ObjectMetaRequestOptions.builder().range(offset, end).build()).build(); + GetObjectV2Output output = client.getObject(request); + + byte[] checksum = parseChecksum(output.getRequestInfo().getHeader(), checksumInfo); + return new GetObjectOutput(output, checksum); + } catch (TosException e) { + if (e instanceof TosServerException) { + TosServerException tosException = (TosServerException) e; + if (tosException.getStatusCode() == INVALID_RANGE_CODE) { + ObjectInfo info = head(key); + // if the object is empty or the requested offset is equal to object size, + // return empty stream directly, otherwise, throw exception. + if (info.size() == 0 || offset == info.size()) { + return new GetObjectOutput( + new GetObjectV2Output(new GetObjectBasicOutput(), EMPTY_STREAM), info.checksum()); + } else { + throw new RuntimeException(e); + } + } + } + throw new RuntimeException(e); + } + } + + @Override + public byte[] put(String key, InputStreamProvider streamProvider, long contentLength) { + checkAvailableClient(); + PutObjectOutput res = client.put(bucket, key, streamProvider, contentLength, defaultAcl); + return ObjectInfo.isDir(key) ? + Constants.MAGIC_CHECKSUM : + parseChecksum(res.getRequestInfo().getHeader(), checksumInfo); + } + + @Override + public byte[] append(String key, InputStreamProvider streamProvider, long contentLength) { + if (bucketInfo.isDirectory()) { + return hnsAppend(key, streamProvider, contentLength); + } else { + return fnsAppend(key, streamProvider, contentLength); + } + } + + private byte[] hnsAppend(String key, InputStreamProvider streamProvider, long contentLength) { + checkAvailableClient(); + + long offset = 0; + String preCrc64; + + TosObjectInfo obj = innerHead(key); + if (obj == null) { + if (contentLength == 0) { + throw new NotAppendableException(String.format( + "%s is not appendable because append non-existed object with " + + "zero byte is not supported.", key)); + } + + // In HNS, append non-existed object is not allowed. Pre-create an empty object before + // performing appendObject. + PutObjectOutput res = client.put(bucket, key, () -> EMPTY_STREAM, 0, defaultAcl); + preCrc64 = res.getHashCrc64ecma(); + } else { + if (contentLength == 0) { + return obj.checksum(); + } + offset = obj.size(); + preCrc64 = obj.crc64ecma(); + } + + AppendObjectOutput res = + client.appendObject(bucket, key, streamProvider, offset, contentLength, preCrc64, + defaultAcl); + return ObjectInfo.isDir(key) ? Constants.MAGIC_CHECKSUM : + parseChecksum(res.getRequestInfo().getHeader(), checksumInfo); + } + + private byte[] fnsAppend(String key, InputStreamProvider streamProvider, long contentLength) { + checkAvailableClient(); + + TosObjectInfo obj = innerHead(key); + if (obj != null) { + if (!obj.appendable()) { + throw new NotAppendableException(String.format("%s is not appendable.", key)); + } + if (contentLength == 0) { + return obj.checksum(); + } + } else if (contentLength == 0) { + throw new NotAppendableException(String.format("%s is not appendable because append" + + " non-existed object with zero byte is not supported.", key)); + } + + long offset = obj == null ? 0 : obj.size(); + String preCrc64 = obj == null ? null : obj.crc64ecma(); + AppendObjectOutput res; + try { + res = client.appendObject(bucket, key, streamProvider, offset, contentLength, preCrc64, + defaultAcl); + } catch (TosServerException e) { + if (e.getStatusCode() == 409 && APPEND_NOT_APPENDABLE.equals(e.getEc())) { + throw new NotAppendableException(String.format("%s is not appendable.", key)); + } + throw e; + } + + return ObjectInfo.isDir(key) ? + Constants.MAGIC_CHECKSUM : + parseChecksum(res.getRequestInfo().getHeader(), checksumInfo); + } + + @Override + public void delete(String key) { + checkAvailableClient(); + client.deleteObject(DeleteObjectInput.builder().bucket(bucket).key(key).build()); + } + + @Override + public List batchDelete(List keys) { + checkAvailableClient(); + int totalKeyCnt = keys.size(); + + Preconditions.checkArgument(totalKeyCnt <= maxDeleteObjectsCount, + "The batch delete object count should <= %s", maxDeleteObjectsCount); + + + List failedKeys = innerBatchDelete(keys); + for (int retry = 1; retry < batchDeleteMaxRetries && !failedKeys.isEmpty(); retry++) { + if (isBatchDeleteRetryable(failedKeys)) { + try { + Thread.sleep(batchDeleteRetryInterval); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + + failedKeys = innerBatchDelete(deleteErrorKeys(failedKeys)); + } else { + LOG.warn("{} of {} objects deleted failed, and cannot be retried, detail: {}", + failedKeys.size(), + totalKeyCnt, + Joiner.on(",\n").join(failedKeys)); + break; + } + } + + if (!failedKeys.isEmpty()) { + LOG.warn("{} of {} objects deleted failed after retry {} times.", + failedKeys.size(), totalKeyCnt, batchDeleteMaxRetries); + } + + return deleteErrorKeys(failedKeys); + } + + @Override + public void deleteAll(String prefix) { + if (bucket().isDirectory()) { + deleteDir(prefix, true); + } else { + Iterable objects = listAll(prefix, ""); + ObjectUtils.deleteAllObjects(this, objects, + conf.getInt(ConfKeys.FS_BATCH_DELETE_SIZE.key(scheme()), + ConfKeys.FS_BATCH_DELETE_SIZE_DEFAULT)); + } + } + + private List innerBatchDelete(List keys) { + List toBeDeleted = Lists.newArrayList(); + for (String key : keys) { + toBeDeleted.add(ObjectTobeDeleted.builder().key(key).build()); + } + + DeleteMultiObjectsV2Output deletedRes = client.deleteMultiObjects(DeleteMultiObjectsV2Input + .builder() + .bucket(bucket) + .objects(toBeDeleted) + .build()); + + return deletedRes.getErrors() == null ? Lists.newArrayList() : deletedRes.getErrors(); + } + + private boolean isBatchDeleteRetryable(List failedKeys) { + for (DeleteError errorKey : failedKeys) { + if (batchDeleteRetryCodes.contains(errorKey.getCode())) { + LOG.warn("Failed to delete object, which might be deleted succeed after retry, detail: {}", + errorKey); + } else { + return false; + } + } + return true; + } + + private static List deleteErrorKeys(List errorKeys) { + List keys = Lists.newArrayList(); + for (DeleteError error : errorKeys) { + keys.add(error.getKey()); + } + return keys; + } + + @Override + public ObjectInfo head(String key) { + return innerHead(key); + } + + private TosObjectInfo innerHead(String key) { + checkAvailableClient(); + try { + HeadObjectV2Input request = HeadObjectV2Input.builder().bucket(bucket).key(key).build(); + HeadObjectV2Output response = client.headObject(request); + + // use crc64ecma/crc32c as checksum to compare object contents, don't use eTag as checksum + // value since PUT & MPU operations have different object etags for same content. + Map headers = response.getRequestInfo().getHeader(); + byte[] checksum = parseChecksum(headers, checksumInfo); + boolean isDir = bucket().isDirectory() ? response.isDirectory() : ObjectInfo.isDir(key); + + return new TosObjectInfo(key, response.getContentLength(), response.getLastModifiedInDate(), + checksum, isDir, + appendable(headers), crc64ecma(headers)); + } catch (TosException e) { + if (e.getStatusCode() == NOT_FOUND_CODE) { + return null; + } + + if (e.getStatusCode() == PATH_CONFLICT_CODE) { + // if a directory 'a/b/' exists in directory bucket, both headObject('a/b') and + // headObject('a/b/') will get directory info, but the response key should be 'a/b/'. + // But if a file 'a/b' exists in directory bucket, only headObject('a/b') will get file + // info, headObject('a/b/') will get 409 error. + throw new InvalidObjectKeyException(e); + } + + throw new RuntimeException(e); + } + } + + @Override + public Iterable list(ListObjectsRequest req) { + return new LazyReload<>(() -> { + AtomicReference continueToken = new AtomicReference<>(""); + AtomicBoolean isTruncated = new AtomicBoolean(true); + AtomicInteger remaining = + new AtomicInteger(req.maxKeys() < 0 ? Integer.MAX_VALUE : req.maxKeys()); + + return buf -> { + // No more objects when isTruncated is false. + if (!isTruncated.get()) { + return true; + } + + int remainingKeys = remaining.get(); + int maxKeys = Math.min(listObjectsCount, remainingKeys); + ListObjectsType2Input request = + createListObjectsType2Input(req.prefix(), req.startAfter(), maxKeys, req.delimiter(), + continueToken.get()); + ListObjectsType2Output response = client.listObjectsType2(request); + List objects = listObjectsOutputToObjectInfos(response); + List commonPrefixes = listObjectsOutputToCommonPrefixes(response); + buf.add(new ListObjectsResponse(objects, commonPrefixes)); + + if (maxKeys < listObjectsCount) { + isTruncated.set(false); + } else { + continueToken.set(response.getNextContinuationToken()); + remaining.compareAndSet(remainingKeys, remainingKeys - response.getKeyCount()); + if (remaining.get() == 0) { + isTruncated.set(false); + } else { + isTruncated.set(response.isTruncated()); + } + } + return !isTruncated.get(); + }; + }); + } + + private List listObjectsOutputToCommonPrefixes(ListObjectsType2Output listObjectsOutput) { + if (listObjectsOutput.getCommonPrefixes() == null) { + return Lists.newArrayList(); + } + + return listObjectsOutput.getCommonPrefixes() + .stream() + .map(ListedCommonPrefix::getPrefix) + .collect(Collectors.toList()); + } + + private List listObjectsOutputToObjectInfos( + ListObjectsType2Output listObjectsOutput) { + if (listObjectsOutput.getContents() == null) { + return Lists.newArrayList(); + } + return listObjectsOutput.getContents().stream() + .map(obj -> new ObjectInfo( + obj.getKey(), + obj.getSize(), + obj.getLastModified(), + parseChecksum(obj, checksumInfo))) + .collect(Collectors.toList()); + } + + private ListObjectsType2Input createListObjectsType2Input( + String prefix, String startAfter, int maxKeys, String delimiter, String continueToken) { + ListObjectsType2Input.ListObjectsType2InputBuilder builder = ListObjectsType2Input.builder() + .bucket(bucket) + .prefix(prefix) + .startAfter(startAfter) + .delimiter(delimiter) + .maxKeys(maxKeys); + + if (!Strings.isNullOrEmpty(continueToken)) { + builder.continuationToken(continueToken); + } + return builder.build(); + } + + @Override + public MultipartUpload createMultipartUpload(String key) { + checkAvailableClient(); + CreateMultipartUploadInput input = CreateMultipartUploadInput.builder() + .bucket(bucket) + .key(key) + .options(createMetaOptions()) + .build(); + CreateMultipartUploadOutput output = client.createMultipartUpload(input); + return new MultipartUpload(output.getKey(), output.getUploadID(), MIN_PART_SIZE, + MAX_PART_COUNT); + } + + @Override + public Part uploadPart( + String key, String uploadId, int partNum, + InputStreamProvider streamProvider, long contentLength) { + checkAvailableClient(); + return client.uploadPart(bucket, key, uploadId, partNum, streamProvider, contentLength, + defaultAcl); + } + + @Override + public byte[] completeUpload(String key, String uploadId, List uploadParts) { + checkAvailableClient(); + List uploadedPartsV2 = uploadParts.stream().map( + part -> UploadedPartV2.builder() + .etag(part.eTag()) + .partNumber(part.num()) + .size(part.size()) + .build() + ).collect(Collectors.toList()); + CompleteMultipartUploadV2Input input = CompleteMultipartUploadV2Input.builder() + .bucket(bucket) + .key(key) + .uploadID(uploadId) + .uploadedParts(uploadedPartsV2) + .build(); + return parseChecksum(client.completeMultipartUpload(input).getRequestInfo().getHeader(), + checksumInfo); + } + + @Override + public void abortMultipartUpload(String key, String uploadId) { + checkAvailableClient(); + AbortMultipartUploadInput input = AbortMultipartUploadInput.builder() + .bucket(bucket) + .key(key) + .uploadID(uploadId) + .build(); + client.abortMultipartUpload(input); + } + + @Override + public Iterable listUploads(String prefix) { + checkAvailableClient(); + return new LazyReload<>(() -> { + AtomicReference nextKeyMarker = new AtomicReference<>(""); + AtomicReference nextUploadIdMarker = new AtomicReference<>(""); + AtomicBoolean isTruncated = new AtomicBoolean(true); + return buf -> { + // No more uploads when isTruncated is false. + if (!isTruncated.get()) { + return true; + } + ListMultipartUploadsV2Input input = ListMultipartUploadsV2Input.builder() + .bucket(bucket) + .prefix(prefix) + .keyMarker(nextKeyMarker.get()) + .uploadIDMarker(nextUploadIdMarker.get()) + .build(); + ListMultipartUploadsV2Output output = client.listMultipartUploads(input); + isTruncated.set(output.isTruncated()); + if (output.getUploads() != null) { + // Fill the reloaded uploads into buffer. + for (ListedUpload upload : output.getUploads()) { + buf.add(new MultipartUpload(upload.getKey(), upload.getUploadID(), + ObjectConstants.MIN_PART_SIZE, ObjectConstants.MAX_PART_COUNT)); + } + LOG.info("Retrieve {} uploads with prefix: {}, marker: {}", + output.getUploads().size(), nextKeyMarker.get(), nextUploadIdMarker.get()); + } + // Refresh the nextKeyMarker and nextUploadMarker for the next reload. + nextKeyMarker.set(output.getNextKeyMarker()); + nextUploadIdMarker.set(output.getNextUploadIdMarker()); + + return !isTruncated.get(); + }; + }); + } + + @Override + public Part uploadPartCopy( + String srcKey, String dstKey, String uploadId, int partNum, long copySourceRangeStart, + long copySourceRangeEnd) { + Preconditions.checkArgument(!Strings.isNullOrEmpty(srcKey), "Source key should not be empty."); + Preconditions.checkArgument(!Strings.isNullOrEmpty(dstKey), "Dest key should not be empty."); + Preconditions.checkArgument(!Strings.isNullOrEmpty(uploadId), "Upload ID should not be empty."); + Preconditions.checkArgument(copySourceRangeStart >= 0, "CopySourceRangeStart must be >= 0."); + Preconditions.checkArgument(copySourceRangeEnd >= 0, "CopySourceRangeEnd must be >= 0."); + Preconditions.checkNotNull(copySourceRangeEnd >= copySourceRangeStart, + "CopySourceRangeEnd must be >= copySourceRangeStart."); + checkAvailableClient(); + UploadPartCopyV2Input input = UploadPartCopyV2Input.builder() + .bucket(bucket) + .key(dstKey) + .uploadID(uploadId) + .sourceBucket(bucket) + .sourceKey(srcKey) + .partNumber(partNum) + .copySourceRange(copySourceRangeStart, copySourceRangeEnd) + .options(createMetaOptions()) + .build(); + UploadPartCopyV2Output output = client.uploadPartCopy(input); + return new Part(output.getPartNumber(), copySourceRangeEnd - copySourceRangeStart + 1, + output.getEtag()); + } + + @Override + public void copy(String srcKey, String dstKey) { + checkAvailableClient(); + CopyObjectV2Input input = CopyObjectV2Input.builder() + .bucket(bucket) + .key(dstKey) + .srcBucket(bucket) + .srcKey(srcKey) + .options(createMetaOptions()) + .build(); + client.copyObject(input); + } + + private ObjectMetaRequestOptions createMetaOptions() { + return new ObjectMetaRequestOptions().setAclType(defaultAcl); + } + + @Override + public void rename(String srcKey, String dstKey) { + checkAvailableClient(); + Preconditions.checkArgument(!Objects.equals(srcKey, dstKey), + "Cannot rename to the same object"); + + RenameObjectInput request = RenameObjectInput.builder() + .bucket(bucket) + .key(srcKey) + .newKey(dstKey) + .build(); + client.renameObject(request); + } + + // TOS allows up to 10 tags. AWS S3 allows up to 10 tags too. + @Override + public void putTags(String key, Map newTags) { + checkAvailableClient(); + List tags = newTags.entrySet().stream() + .map(e -> new Tag().setKey(e.getKey()).setValue(e.getValue())) + .collect(Collectors.toList()); + + if (tags.size() > 0) { + client.putObjectTagging(createPutTagInput(bucket, key, tags)); + } else { + client.deleteObjectTagging(createDeleteTagInput(bucket, key)); + } + } + + @Override + public Map getTags(String key) { + Map result = new HashMap<>(); + for (Tag tag : getObjectTaggingList(key)) { + result.put(tag.getKey(), tag.getValue()); + } + return result; + } + + private List getObjectTaggingList(String key) { + checkAvailableClient(); + + GetObjectTaggingInput input = GetObjectTaggingInput.builder() + .bucket(bucket) + .key(key) + .build(); + GetObjectTaggingOutput output = client.getObjectTagging(input); + + TagSet tagSet = output.getTagSet(); + if (tagSet == null || tagSet.getTags() == null) { + return new ArrayList<>(); + } + return tagSet.getTags(); + } + + private static PutObjectTaggingInput createPutTagInput(String bucket, String key, + List tags) { + return PutObjectTaggingInput.builder() + .bucket(bucket) + .key(key) + .tagSet(TagSet.builder().tags(tags).build()) + .build(); + } + + private static DeleteObjectTaggingInput createDeleteTagInput(String bucket, String key) { + return DeleteObjectTaggingInput.builder() + .bucket(bucket) + .key(key) + .build(); + } + + /** + * Implement Hadoop FileSystem.getFileStatus semantics through + * {@link TOSV2#getFileStatus(GetFileStatusInput)}.
+ * + * The detail behavior are as follows: + *
    + *
  • Assume object 'a/b' exists in TOS, getFileStatus("a/b") will get object('a/b') succeed, + * getFileStatus("a/b/") will get 404.
  • + *
  • Assume object 'a/b/' exists in TOS, both getFileStatus("a/b") & getFileStatus("a/b/") + * will get object('a/b/') succeed
  • + *
  • Assume object 'a/b/c' exists in TOS, both getFileStatus("a/b") & getFileStatus("a/b/") + * will get object('a/b/') succeed.
  • + *
+ *

+ * And the following is the logic of {@link TOSV2#getFileStatus(GetFileStatusInput)}:
+ * Step 1: Head the specified key, if the head operation is successful, the response is filled + * with the actual object.
+ * Step 2: Append the key with the suffix '/' to perform list operation, if the list operation is + * successful, the response is filled with the first object from the listing results + * ; if there are no objects, return 404.
+ * + * @param key for the object. + * @return object + */ + private ObjectInfo getFileStatus(String key) { + checkAvailableClient(); + Preconditions.checkArgument(!Strings.isNullOrEmpty(key), "key should not be empty."); + + GetFileStatusInput input = GetFileStatusInput.builder() + .bucket(bucket) + .key(key) + .build(); + try { + GetFileStatusOutput output = client.getFileStatus(input); + if (key.equals(output.getKey()) && !ObjectInfo.isDir(output.getKey())) { + return new ObjectInfo(key, output.getSize(), output.getLastModifiedInDate(), + parseChecksum(output, checksumInfo)); + } else { + String dirKey = ObjectInfo.isDir(key) ? key : key + '/'; + + // If only the prefix exists but dir object key doesn't exist, will use the current date as + // the modified date. + Date lastModifiedInDate = + dirKey.equals(output.getKey()) ? output.getLastModifiedInDate() : new Date(); + return new ObjectInfo(dirKey, 0, lastModifiedInDate, Constants.MAGIC_CHECKSUM, true); + } + } catch (TosException e) { + // the specified object does not exist. + if (e.getStatusCode() == NOT_FOUND_CODE) { + return null; + } + + if (e.getStatusCode() == PATH_CONFLICT_CODE) { + throw new InvalidObjectKeyException(e); + } + throw new RuntimeException(e); + } + } + + @Override + public ObjectInfo objectStatus(String key) { + if (bucket().isDirectory()) { + return head(key); + } else if (conf.getBoolean(TosKeys.FS_TOS_GET_FILE_STATUS_ENABLED, + TosKeys.FS_TOS_GET_FILE_STATUS_ENABLED_DEFAULT)) { + return getFileStatus(key); + } else { + ObjectInfo obj = head(key); + if (obj == null && !ObjectInfo.isDir(key)) { + key = key + '/'; + obj = head(key); + } + + if (obj == null) { + Iterable objs = list(key, null, 1); + if (objs.iterator().hasNext()) { + obj = new ObjectInfo(key, 0, new Date(0), Constants.MAGIC_CHECKSUM, true); + } + } + + return obj; + } + } + + @Override + public ChecksumInfo checksumInfo() { + return checksumInfo; + } + + @Override + public void close() throws IOException { + client.close(); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOSErrorCodes.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOSErrorCodes.java new file mode 100644 index 0000000000000..87396f539dfab --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOSErrorCodes.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.tosfs.object.tos; + +import java.util.StringJoiner; + +public final class TOSErrorCodes { + private TOSErrorCodes() { + } + + // The 409 error codes of HNS + public static final String DELETE_NON_EMPTY_DIR = "0026-00000013"; + public static final String LOCATED_UNDER_A_FILE = "0026-00000020"; + public static final String COPY_BETWEEN_DIR_AND_FILE = "0026-00000021"; + public static final String PATH_LOCK_CONFLICT = "0026-00000022"; + public static final String RENAME_TO_AN_EXISTED_DIR = "0026-00000025"; + public static final String RENAME_TO_SUB_DIR = "0026-00000026"; + public static final String RENAME_BETWEEN_DIR_AND_FILE = "0026-00000027"; + + // The 409 error codes shared by HNS and FNS. + public static final String APPEND_OFFSET_NOT_MATCHED = "0017-00000208"; + public static final String APPEND_NOT_APPENDABLE = "0017-00000209"; + + + // The bellow error cannot be solved by retry the request except the code PATH_LOCK_CONFLICT, + // so need to fail fast. + public static final String FAST_FAILURE_CONFLICT_ERROR_CODES = new StringJoiner(",") + .add(DELETE_NON_EMPTY_DIR) + .add(LOCATED_UNDER_A_FILE) + .add(COPY_BETWEEN_DIR_AND_FILE) + .add(RENAME_TO_AN_EXISTED_DIR) + .add(RENAME_TO_SUB_DIR) + .add(RENAME_BETWEEN_DIR_AND_FILE) + .add(APPEND_OFFSET_NOT_MATCHED) + .add(APPEND_NOT_APPENDABLE) + .toString(); +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOSInputStream.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOSInputStream.java new file mode 100644 index 0000000000000..774a31f48bd59 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOSInputStream.java @@ -0,0 +1,121 @@ +/* + * 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.tosfs.object.tos; + +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.thirdparty.com.google.common.io.ByteStreams; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.util.concurrent.atomic.AtomicBoolean; + +class TOSInputStream extends InputStream { + private static final Logger LOG = LoggerFactory.getLogger(TOSInputStream.class); + + private final GetObjectOutput output; + private final InputStream stream; + private final AtomicBoolean closed = new AtomicBoolean(false); + + private long curOff; + private final long endOff; // range end offset (inclusive) + private final long maxDrainByteSize; + + TOSInputStream(GetObjectOutput output, long startOff, long endOff, long maxDrainByteSize, + byte[] expectedChecksum) throws IOException { + this.output = output; + this.stream = output.verifiedContent(expectedChecksum); + this.curOff = startOff; + this.endOff = endOff; + this.maxDrainByteSize = maxDrainByteSize; + } + + @Override + public int read() throws IOException { + int b = stream.read(); + curOff += 1; + return b; + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + int readed = 0; + int n; + do { + n = stream.read(b, off + readed, len - readed); + if (n > 0) { + readed += n; + } + } while (n > 0); + + if (readed == 0) { + return n; + } else { + curOff += readed; + return readed; + } + } + + // Only visible for testing. + GetObjectOutput getObjectOutput() { + return output; + } + + @Override + public void close() throws IOException { + if (closed.compareAndSet(false, true)) { + if (endOff >= 0) { + // The unread bytes is known. we just skip the bytes if gap <= expected drain size (to reuse + // the socket conn), otherwise we force close the socket conn without reading any bytes in + // the future. + long gap = endOff - curOff + 1; + if (gap <= maxDrainByteSize) { + // The close will try to drain bytes internally. + stream.close(); + } else { + CommonUtils.runQuietly(output::forceClose, false); + } + + } else { + // The unread bytes is unknown, we try to read the expected drain bytes to see if it's EOF + // now. If EOF then just close the stream to reuse the socket conn, otherwise close the + // connection directly for saving draining time. + try { + ByteStreams.skipFully(stream, maxDrainByteSize); + } catch (Exception e) { + if (e instanceof EOFException) { + LOG.debug("Stream is EOF now, just close the stream to reuse the socket connection."); + stream.close(); + } else { + LOG.debug("Stream skipFully encountered exception, force close the socket connection.", + e); + // Force close the socket connection. + CommonUtils.runQuietly(output::forceClose, false); + } + return; + } + + // Force close the socket connection. + CommonUtils.runQuietly(output::forceClose, false); + } + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOSUtils.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOSUtils.java new file mode 100644 index 0000000000000..e1900f05d3199 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOSUtils.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.tosfs.object.tos; + +import com.volcengine.tos.model.object.GetFileStatusOutput; +import com.volcengine.tos.model.object.ListedObjectV2; +import org.apache.hadoop.fs.tosfs.common.Bytes; +import org.apache.hadoop.fs.tosfs.object.ChecksumInfo; +import org.apache.hadoop.fs.tosfs.object.ChecksumType; +import org.apache.hadoop.fs.tosfs.object.Constants; +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; + +import java.util.Map; + +public final class TOSUtils { + private TOSUtils() {} + + // Checksum header. + public static final Map CHECKSUM_HEADER = ImmutableMap.of( + ChecksumType.CRC32C, "x-tos-hash-crc32c", + ChecksumType.CRC64ECMA, "x-tos-hash-crc64ecma" + ); + + // Object type header. Object is either 'Appendable' or 'Normal'. + public static final String OBJECT_TYPE_KEY = "x-tos-object-type"; + public static final String APPENDABLE_TYPE_VALUE = "Appendable"; + + // Checksum is magic checksum if the object doesn't support checksum type. + public static byte[] parseChecksum(Map headers, ChecksumInfo checksumInfo) { + ChecksumType type = checksumInfo.checksumType(); + String header = CHECKSUM_HEADER.get(type); + if (header == null) { + return Constants.MAGIC_CHECKSUM; + } + + String checksumStr = headers.get(header); + if (checksumStr == null) { + return Constants.MAGIC_CHECKSUM; + } + + return parseChecksumStringToBytes(checksumStr, type); + } + + // Checksum is magic checksum if the object doesn't support checksum type. + public static byte[] parseChecksum(ListedObjectV2 obj, ChecksumInfo checksumInfo) { + ChecksumType type = checksumInfo.checksumType(); + + String checksumStr; + if (type == ChecksumType.CRC32C) { + checksumStr = obj.getHashCrc32c(); + } else if (type == ChecksumType.CRC64ECMA) { + checksumStr = obj.getHashCrc64ecma(); + } else { + throw new IllegalArgumentException( + String.format("Checksum type %s is not supported by TOS.", type.name())); + } + + if (checksumStr == null) { + return Constants.MAGIC_CHECKSUM; + } + + return parseChecksumStringToBytes(checksumStr, type); + } + + // Checksum is magic checksum if the object doesn't support checksum type. + public static byte[] parseChecksum(GetFileStatusOutput obj, ChecksumInfo checksumInfo) { + ChecksumType type = checksumInfo.checksumType(); + + if (type == ChecksumType.CRC32C) { + return parseChecksumStringToBytes(obj.getCrc32(), type); + } else if (type == ChecksumType.CRC64ECMA) { + return parseChecksumStringToBytes(obj.getCrc64(), type); + } else { + throw new IllegalArgumentException( + String.format("Checksum type %s is not supported by TOS.", type.name())); + } + } + + public static byte[] parseChecksumStringToBytes(String checksum, ChecksumType type) { + if (checksum == null) { + return Constants.MAGIC_CHECKSUM; + } + + switch (type) { + case CRC32C: + case CRC64ECMA: + return Bytes.toBytes(Long.parseUnsignedLong(checksum)); + default: + throw new IllegalArgumentException( + String.format("Checksum type %s is not supported by TOS.", type.name())); + } + } + + public static String crc64ecma(Map headers) { + String header = CHECKSUM_HEADER.get(ChecksumType.CRC64ECMA); + return headers.get(header); + } + + public static boolean appendable(Map headers) { + String value = headers.get(OBJECT_TYPE_KEY); + return APPENDABLE_TYPE_VALUE.equals(value); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TosObjectInfo.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TosObjectInfo.java new file mode 100644 index 0000000000000..302b509ca1f7d --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TosObjectInfo.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.tosfs.object.tos; + +import org.apache.commons.codec.binary.Hex; +import org.apache.hadoop.fs.tosfs.object.ObjectInfo; +import org.apache.hadoop.thirdparty.com.google.common.base.MoreObjects; + +import java.util.Date; +import java.util.Objects; + +public class TosObjectInfo extends ObjectInfo { + private final String crc64ecma; + private final boolean appendable; + + public TosObjectInfo(String key, long size, Date mtime, byte[] checksum, boolean isDir, + boolean appendable, String crc64ecma) { + super(key, size, mtime, checksum, isDir); + this.crc64ecma = crc64ecma; + this.appendable = appendable; + } + + public String crc64ecma() { + return crc64ecma; + } + + public boolean appendable() { + return appendable; + } + + @Override + public boolean equals(Object o) { + if (!super.equals(o)) { + return false; + } + + if (!(o instanceof TosObjectInfo)) { + return false; + } + + TosObjectInfo that = (TosObjectInfo) o; + return Objects.equals(appendable, that.appendable) && Objects.equals(crc64ecma, that.crc64ecma); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), appendable, crc64ecma); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("key", key()) + .add("size", size()) + .add("mtime", mtime()) + .add("checksum", Hex.encodeHexString(checksum())) + .add("isDir", isDir()) + .add("appendable", appendable) + .add("crc64ecma", crc64ecma) + .toString(); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/AbstractCredentialsProvider.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/AbstractCredentialsProvider.java new file mode 100644 index 0000000000000..be7cf189b5476 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/AbstractCredentialsProvider.java @@ -0,0 +1,79 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.object.tos.auth; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.hadoop.fs.tosfs.conf.TosKeys.FS_TOS_ACCESS_KEY_ID; +import static org.apache.hadoop.fs.tosfs.conf.TosKeys.FS_TOS_SECRET_ACCESS_KEY; + +public abstract class AbstractCredentialsProvider implements CredentialsProvider { + + private static final Logger LOG = LoggerFactory.getLogger(AbstractCredentialsProvider.class); + + private volatile ExpireableCredential credential; + private Configuration conf; + private String bucket; + + @Override + public void initialize(Configuration config, String bucketName) { + this.conf = config; + this.bucket = bucketName; + } + + /** + * throw exception if no valid credential found, the response credential is not null. + * + * @return credential + */ + @Override + public ExpireableCredential credential() { + if (credential == null || credential.isExpired()) { + synchronized (this) { + if (credential == null || credential.isExpired()) { + LOG.debug("Credential expired, create a new credential"); + ExpireableCredential cred = createCredential(); + Preconditions.checkNotNull(cred.getAccessKeyId(), "%s cannot be null", + FS_TOS_ACCESS_KEY_ID); + Preconditions.checkNotNull(cred.getAccessKeySecret(), "%s cannot be null", + FS_TOS_SECRET_ACCESS_KEY); + credential = cred; + } + } + } + return credential; + } + + public Configuration conf() { + return conf; + } + + public String bucket() { + return bucket; + } + + /** + * Create expireable credential. + * + * throw exception if not credential found. + * @return expireable credential. + */ + protected abstract ExpireableCredential createCredential(); +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/CredentialsProvider.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/CredentialsProvider.java new file mode 100644 index 0000000000000..f16d0c47b6586 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/CredentialsProvider.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.tosfs.object.tos.auth; + +import com.volcengine.tos.auth.Credentials; +import org.apache.hadoop.conf.Configuration; + +import javax.annotation.Nullable; + +public interface CredentialsProvider extends Credentials { + + /** + * Initialize the credential provider. + * + * @param conf the {@link Configuration} used for building credential provider + * @param bucket the binding bucket, it can be null. + */ + void initialize(Configuration conf, @Nullable String bucket); +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/DefaultCredentialsProviderChain.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/DefaultCredentialsProviderChain.java new file mode 100644 index 0000000000000..ba3446c5bec71 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/DefaultCredentialsProviderChain.java @@ -0,0 +1,101 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.object.tos.auth; + +import com.volcengine.tos.TosException; +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.LinkedList; +import java.util.List; + +import static org.apache.hadoop.fs.tosfs.conf.TosKeys.FS_TOS_CUSTOM_CREDENTIAL_PROVIDER_CLASSES; +import static org.apache.hadoop.fs.tosfs.conf.TosKeys.FS_TOS_CUSTOM_CREDENTIAL_PROVIDER_CLASSES_DEFAULT; + +public class DefaultCredentialsProviderChain extends AbstractCredentialsProvider { + + private static final Logger LOG = LoggerFactory.getLogger(DefaultCredentialsProviderChain.class); + + private final List providers = new LinkedList<>(); + private volatile AbstractCredentialsProvider lastUsedProvider; + + @Override + public void initialize(Configuration config, String bucketName) { + super.initialize(config, bucketName); + loadAllCredentialProviders(); + } + + private void loadAllCredentialProviders() { + for (String providerClazz : getCustomProviderClasses()) { + try { + Class clazz = Class.forName(providerClazz); + AbstractCredentialsProvider credentialsProvider = + (AbstractCredentialsProvider) clazz.getDeclaredConstructor().newInstance(); + credentialsProvider.initialize(conf(), bucket()); + providers.add(credentialsProvider); + } catch (Exception e) { + LOG.error("Failed to initialize credential provider for {}", providerClazz, e); + // throw exception directly since the configurations are invalid. + throw new TosException(e); + } + } + } + + private String[] getCustomProviderClasses() { + String[] classes = conf().getStringCollection(FS_TOS_CUSTOM_CREDENTIAL_PROVIDER_CLASSES) + .toArray(new String[0]); + if (classes.length == 0) { + classes = FS_TOS_CUSTOM_CREDENTIAL_PROVIDER_CLASSES_DEFAULT; + } + return classes; + } + + @Override + protected ExpireableCredential createCredential() { + if (lastUsedProvider != null) { + return lastUsedProvider.credential(); + } else { + List exceptions = new LinkedList<>(); + for (AbstractCredentialsProvider provider : providers) { + try { + ExpireableCredential credential = provider.credential(); + LOG.debug("Access credential from {} successfully, choose it as the candidate provider", + provider.getClass().getName()); + lastUsedProvider = provider; + return credential; + } catch (Exception e) { + LOG.debug("Failed to access credential from provider {}", provider.getClass().getName(), + e); + exceptions.add(e); + } + } + String errorMsg = "Unable to load TOS credentials from any provider in the chain."; + RuntimeException runtimeException = new RuntimeException(errorMsg); + exceptions.forEach(runtimeException::addSuppressed); + throw runtimeException; + } + } + + @VisibleForTesting + AbstractCredentialsProvider lastUsedProvider() { + Preconditions.checkNotNull(lastUsedProvider, "provider cannot be null"); + return lastUsedProvider; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/EnvironmentCredentialsProvider.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/EnvironmentCredentialsProvider.java new file mode 100644 index 0000000000000..8355a72037a3d --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/EnvironmentCredentialsProvider.java @@ -0,0 +1,34 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.object.tos.auth; + +import org.apache.hadoop.fs.tosfs.object.tos.TOS; + +public class EnvironmentCredentialsProvider extends AbstractCredentialsProvider { + + public static final String NAME = EnvironmentCredentialsProvider.class.getName(); + + @Override + protected ExpireableCredential createCredential() { + return new ExpireableCredential( + System.getenv(TOS.ENV_TOS_ACCESS_KEY_ID), + System.getenv(TOS.ENV_TOS_SECRET_ACCESS_KEY), + System.getenv(TOS.ENV_TOS_SESSION_TOKEN), + Long.MAX_VALUE + ); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/ExpireableCredential.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/ExpireableCredential.java new file mode 100644 index 0000000000000..d97e2a98a8ab6 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/ExpireableCredential.java @@ -0,0 +1,67 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.object.tos.auth; + +import com.volcengine.tos.auth.Credential; +import org.apache.hadoop.fs.tosfs.conf.TosKeys; +import org.apache.hadoop.util.Preconditions; + +public class ExpireableCredential extends Credential { + public static final int EXPIRED_INTERVAL_MILLIS = 1000 * 60; // 1 minute + + private final long expireTimeMills; + + /** + * The credential is never expired, default sts value is null and expired is Long.MAX_VALUE. + * + * @param accessKeyId IAM AK. + * @param accessKeySecret IAM SK. + */ + public ExpireableCredential( + String accessKeyId, + String accessKeySecret) { + this(accessKeyId, accessKeySecret, "", Long.MAX_VALUE); + } + + /** + * Credential that can expire. + * + * @param accessKeyId IAM AK. + * @param accessKeySecret IAM SK. + * @param sessionToken Session token. + * @param expireTimeMills Session token expire time, + * the default value is the request time +6H if get it from the meta + * service. + */ + public ExpireableCredential( + String accessKeyId, + String accessKeySecret, + String sessionToken, + long expireTimeMills) { + super(accessKeyId, accessKeySecret, sessionToken); + Preconditions.checkNotNull(accessKeyId, + "%s cannot be null", TosKeys.FS_TOS_ACCESS_KEY_ID); + Preconditions.checkNotNull(accessKeySecret, + "%s cannot be null", TosKeys.FS_TOS_SECRET_ACCESS_KEY); + Preconditions.checkArgument(expireTimeMills > 0, "expiredTime must be > 0"); + this.expireTimeMills = expireTimeMills; + } + + public boolean isExpired() { + return expireTimeMills - System.currentTimeMillis() <= EXPIRED_INTERVAL_MILLIS; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/SimpleCredentialsProvider.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/SimpleCredentialsProvider.java new file mode 100644 index 0000000000000..391a35e9b38b5 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/SimpleCredentialsProvider.java @@ -0,0 +1,58 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.object.tos.auth; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.conf.TosKeys; + +import static org.apache.hadoop.fs.tosfs.conf.TosKeys.FS_TOS_ACCESS_KEY_ID; +import static org.apache.hadoop.fs.tosfs.conf.TosKeys.FS_TOS_SECRET_ACCESS_KEY; +import static org.apache.hadoop.fs.tosfs.conf.TosKeys.FS_TOS_SESSION_TOKEN; + +public class SimpleCredentialsProvider extends AbstractCredentialsProvider { + + public static final String NAME = SimpleCredentialsProvider.class.getName(); + + @Override + protected ExpireableCredential createCredential() { + String accessKey = + lookup(conf(), TosKeys.FS_TOS_BUCKET_ACCESS_KEY_ID.key(bucket()), FS_TOS_ACCESS_KEY_ID); + String secretKey = lookup(conf(), TosKeys.FS_TOS_BUCKET_SECRET_ACCESS_KEY.key(bucket()), + FS_TOS_SECRET_ACCESS_KEY); + String sessionToken = + lookup(conf(), TosKeys.FS_TOS_BUCKET_SESSION_TOKEN.key(bucket()), FS_TOS_SESSION_TOKEN); + if (StringUtils.isEmpty(sessionToken)) { + // This is a static ak sk configuration. + return new ExpireableCredential(accessKey, secretKey); + } else { + // This is an assume role configuration. Due to the ak, sk and token won't be refreshed in + // conf, set the expireTime to Long.MAX_VALUE. + return new ExpireableCredential(accessKey, secretKey, sessionToken, Long.MAX_VALUE); + } + } + + static String lookup(Configuration conf, String key, String fallbackKey) { + if (StringUtils.isNotEmpty(key)) { + String dynValue = conf.get(key); + if (StringUtils.isNotEmpty(dynValue)) { + return dynValue; + } + } + return conf.get(fallbackKey); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/package-info.java new file mode 100644 index 0000000000000..558ae48524191 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/package-info.java @@ -0,0 +1,26 @@ +/* + * 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. + */ + +/** + * Classes for hadoop-tos tos auth. + */ +@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"}) +@InterfaceStability.Evolving +package org.apache.hadoop.fs.tosfs.object.tos.auth; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/package-info.java new file mode 100644 index 0000000000000..eb06310552eaf --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/package-info.java @@ -0,0 +1,26 @@ +/* + * 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. + */ + +/** + * Classes for hadoop-tos tos. + */ +@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"}) +@InterfaceStability.Evolving +package org.apache.hadoop.fs.tosfs.object.tos; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/DefaultFsOps.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/DefaultFsOps.java new file mode 100644 index 0000000000000..82ff1c9c794b9 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/DefaultFsOps.java @@ -0,0 +1,189 @@ +/* + * 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.tosfs.ops; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException; +import org.apache.hadoop.fs.tosfs.RawFileStatus; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.object.Constants; +import org.apache.hadoop.fs.tosfs.object.ObjectInfo; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectUtils; +import org.apache.hadoop.fs.tosfs.object.request.ListObjectsRequest; +import org.apache.hadoop.fs.tosfs.object.response.ListObjectsResponse; +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.fs.tosfs.util.Iterables; +import org.apache.hadoop.util.Lists; + +import java.io.IOException; +import java.util.Date; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.function.Function; +import java.util.function.Predicate; + +import static org.apache.hadoop.fs.tosfs.object.ObjectUtils.SLASH; + +/** + * Provides rename, delete, list capabilities for general purpose bucket. + */ +public class DefaultFsOps implements FsOps { + private final ObjectStorage storage; + private final ExecutorService taskThreadPool; + private final Function objMapper; + private final RenameOp renameOp; + private final boolean asyncCreateParentDir; + + public DefaultFsOps( + ObjectStorage storage, + Configuration conf, + ExecutorService taskThreadPool, + Function objMapper) { + this.storage = storage; + this.taskThreadPool = taskThreadPool; + this.objMapper = objMapper; + this.renameOp = new RenameOp(conf, storage, taskThreadPool); + this.asyncCreateParentDir = + conf.getBoolean(ConfKeys.FS_ASYNC_CREATE_MISSED_PARENT.key(storage.scheme()), + ConfKeys.FS_ASYNC_CREATE_MISSED_PARENT_DEFAULT); + } + + @Override + public void renameFile(Path src, Path dst, long length) { + renameOp.renameFile(src, dst, length); + mkdirIfNecessary(src.getParent(), asyncCreateParentDir); + } + + @Override + public void renameDir(Path src, Path dst) { + renameOp.renameDir(src, dst); + mkdirIfNecessary(src.getParent(), asyncCreateParentDir); + } + + @Override + public void deleteFile(Path file) { + storage.delete(ObjectUtils.pathToKey(file)); + mkdirIfNecessary(file.getParent(), asyncCreateParentDir); + } + + @Override + public void deleteDir(Path dir, boolean recursive) throws IOException { + String dirKey = ObjectUtils.pathToKey(dir, true); + if (recursive) { + storage.deleteAll(dirKey); + } else { + if (isEmptyDirectory(dir)) { + storage.delete(dirKey); + } else { + throw new PathIsNotEmptyDirectoryException(dir.toString()); + } + } + } + + @Override + public Iterable listDir(Path dir, boolean recursive, + Predicate postFilter) { + String key = ObjectUtils.pathToKey(dir, true); + String delimiter = recursive ? null : SLASH; + + ListObjectsRequest req = ListObjectsRequest.builder() + .prefix(key) + .startAfter(key) + .delimiter(delimiter) + .build(); + return Iterables.transform(asObjectInfo(storage.list(req), postFilter), objMapper); + } + + @Override + public boolean isEmptyDirectory(Path dir) { + String key = ObjectUtils.pathToKey(dir, true); + ListObjectsRequest req = ListObjectsRequest.builder() + .prefix(key) + .startAfter(key) + .delimiter(SLASH) + .maxKeys(1) + .build(); + return !asObjectInfo(storage.list(req), s -> true).iterator().hasNext(); + } + + @Override + public void mkdirs(Path dir) { + if (dir.isRoot()) { + return; + } + String key = ObjectUtils.pathToKey(dir, true); + storage.put(key, new byte[0]); + + // Create parent dir if missed. + Path parentPath = dir.getParent(); + String parentKey = ObjectUtils.pathToKey(parentPath, true); + while (!parentPath.isRoot() && storage.head(parentKey) == null) { + storage.put(parentKey, new byte[0]); + parentPath = parentPath.getParent(); + parentKey = ObjectUtils.pathToKey(parentPath, true); + } + } + + private void mkdirIfNecessary(Path path, boolean async) { + if (path != null) { + CommonUtils.runQuietly(() -> { + Future future = taskThreadPool.submit(() -> { + String key = ObjectUtils.pathToKey(path, true); + if (!key.isEmpty() && storage.head(key) == null) { + mkdirs(ObjectUtils.keyToPath(key)); + } + }); + + if (!async) { + future.get(); + } + }); + } + } + + /** + * Convert ListObjectResponse iterable to FileStatus iterable, + * using file status acceptor to filter the expected objects and common prefixes. + * + * @param listResponses the iterable of ListObjectsResponse + * @param filter the file status acceptor + * @return the iterable of TosFileStatus + */ + private Iterable asObjectInfo(Iterable listResponses, + Predicate filter) { + Iterable> results = Iterables.transform(listResponses, listResp -> { + List objs = Lists.newArrayList(); + + // Add object files. + objs.addAll(listResp.objects()); + + // Add object directories. + for (String prefix : listResp.commonPrefixes()) { + objs.add(new ObjectInfo(prefix, 0, new Date(), Constants.MAGIC_CHECKSUM, true)); + } + + return objs; + }); + + return Iterables.filter(Iterables.concat(results), o -> filter.test(o.key())); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/DirectoryFsOps.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/DirectoryFsOps.java new file mode 100644 index 0000000000000..3a1b81cbc98c7 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/DirectoryFsOps.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.tosfs.ops; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException; +import org.apache.hadoop.fs.tosfs.RawFileStatus; +import org.apache.hadoop.fs.tosfs.object.DirectoryStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectInfo; +import org.apache.hadoop.fs.tosfs.object.ObjectUtils; +import org.apache.hadoop.fs.tosfs.util.Iterables; + +import java.io.IOException; +import java.util.function.Function; +import java.util.function.Predicate; + +/** + * Provides rename, delete, list capabilities for directory bucket. + */ +public class DirectoryFsOps implements FsOps { + private final DirectoryStorage storage; + private final Function objMapper; + + public DirectoryFsOps(DirectoryStorage storage, Function objMapper) { + this.storage = storage; + this.objMapper = objMapper; + } + + @Override + public void renameFile(Path src, Path dst, long length) { + innerRename(src, dst, false); + } + + @Override + public void renameDir(Path src, Path dst) { + innerRename(src, dst, true); + } + + private void innerRename(Path src, Path dst, boolean isDir) { + // Need to ensure the dest parent exist before rename file in directory bucket. + String dstParentKey = ObjectUtils.pathToKey(dst.getParent(), true); + if (!dstParentKey.isEmpty() && storage.head(dstParentKey) == null) { + mkdirs(dst.getParent()); + } + + String srcKey = ObjectUtils.pathToKey(src, isDir); + String dstKey = ObjectUtils.pathToKey(dst, isDir); + storage.rename(srcKey, dstKey); + } + + @Override + public void deleteFile(Path file) { + storage.delete(ObjectUtils.pathToKey(file)); + } + + @Override + public void deleteDir(Path dir, boolean recursive) throws IOException { + String dirKey = ObjectUtils.pathToKey(dir, true); + if (recursive) { + storage.deleteAll(dirKey); + } else { + if (isEmptyDirectory(dir)) { + storage.delete(dirKey); + } else { + throw new PathIsNotEmptyDirectoryException(dir.toString()); + } + } + } + + @Override + public Iterable listDir(Path dir, boolean recursive, + Predicate postFilter) { + String key = ObjectUtils.pathToKey(dir, true); + Iterable objs = + Iterables.filter(storage.listDir(key, recursive), obj -> postFilter.test(obj.key())); + return Iterables.transform(objs, objMapper); + } + + @Override + public boolean isEmptyDirectory(Path dir) { + return storage.isEmptyDir(ObjectUtils.pathToKey(dir, true)); + } + + @Override + public void mkdirs(Path dir) { + if (dir.isRoot()) { + return; + } + String key = ObjectUtils.pathToKey(dir, true); + // Directory bucket will create missed parent dirs automatically. + storage.put(key, new byte[0]); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/FsOps.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/FsOps.java new file mode 100644 index 0000000000000..659f4ac5c6b01 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/FsOps.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.tosfs.ops; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.RawFileStatus; + +import java.io.IOException; +import java.util.function.Predicate; + +public interface FsOps { + + /** + * Rename file from source path to dest path. + * + * @param src the source path. + * @param dst the dest path. + * @param length the length of source file. + * @throws IOException if any io error happen. + */ + void renameFile(Path src, Path dst, long length) throws IOException; + + /** + * Rename dir from source path to dest path. + * + * @param src the source path. + * @param dst the dest path. + * @throws IOException if any io error happen. + */ + void renameDir(Path src, Path dst) throws IOException; + + /** + * Delete the given file. + * + * @param file the given file path. + * @throws IOException if any io error happen. + */ + void deleteFile(Path file) throws IOException; + + /** + * Delete the given dir. + * + * @param dir the given dir path. + * @param recursive indicate whether delete dir recursively. + * @throws IOException if any io error happen. + */ + void deleteDir(Path dir, boolean recursive) throws IOException; + + /** + * List the sub dirs and files with given dir. + * Return empty collection if the path doesn't exist, or is a file, or is an empty dir. + * + * @param dir the listed path. + * @param recursive indicated whether list all sub dirs/files or not. + * @param postFilter filter the result after getting listing response. + * @return the status of sub dirs and files. + */ + Iterable listDir(Path dir, boolean recursive, Predicate postFilter); + + /** + * Check is the input dir empty. + * + * @param dir the dir to check. + * @return true if path don't have any children. + */ + boolean isEmptyDirectory(Path dir); + + /** + * Create dir and parent dirs if don't exist. + * + * @param dir the dir to be created. + * @throws IOException if any io error happen. + */ + void mkdirs(Path dir) throws IOException; +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/RenameOp.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/RenameOp.java new file mode 100644 index 0000000000000..c139c6d9d9092 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/RenameOp.java @@ -0,0 +1,230 @@ +/* + * 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.tosfs.ops; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.common.Tasks; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.object.MultipartUpload; +import org.apache.hadoop.fs.tosfs.object.ObjectInfo; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectUtils; +import org.apache.hadoop.fs.tosfs.object.Part; +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Comparator; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +public class RenameOp { + private static final Logger LOG = LoggerFactory.getLogger(RenameOp.class); + private static final int RENAME_RETRY_TIMES = 3; + + private final Configuration conf; + private final ObjectStorage storage; + private final ExecutorService renamePool; + // Whether enable object storage atomic rename object capability. + private final boolean renameObjectEnabled; + + public RenameOp(Configuration conf, ObjectStorage storage, ExecutorService taskThreadPool) { + this.conf = conf; + this.storage = storage; + this.renamePool = taskThreadPool; + this.renameObjectEnabled = + conf.getBoolean(ConfKeys.FS_OBJECT_RENAME_ENABLED.key(storage.scheme()), + ConfKeys.FS_OBJECT_RENAME_ENABLED_DEFAULT); + } + + public void renameDir(Path src, Path dst) { + String srcKey = ObjectUtils.pathToKey(src, true); + String dstKey = ObjectUtils.pathToKey(dst, true); + renameDir(srcKey, dstKey); + } + + public void renameFile(Path src, Path dst, long length) { + String srcKey = ObjectUtils.pathToKey(src, false); + String dstKey = ObjectUtils.pathToKey(dst, false); + renameFile(srcKey, dstKey, length); + } + + /** + * Renames each object after listing all objects with given src key via renaming semantic if + * object storage supports atomic rename semantic, otherwise renaming all objects via + * copy & delete. + * + * @param srcKey the source dir key, ending with slash. + * @param dstKey the destination parent dir key, ending with slash. + */ + private void renameDir(String srcKey, String dstKey) { + Iterable objs = storage.listAll(srcKey, ""); + if (renameObjectEnabled) { + Tasks.foreach(objs) + .executeWith(renamePool) + .throwFailureWhenFinished() + .retry(RENAME_RETRY_TIMES) + .revertWith(sourceInfo -> { + String newDstKey = dstKey + sourceInfo.key().substring(srcKey.length()); + String newSrcKey = sourceInfo.key(); + LOG.debug("Try to rollback dest key {} to source key {}", newDstKey, newSrcKey); + + storage.rename(newDstKey, newSrcKey); + }) + .run(sourceInfo -> { + String newDstKey = dstKey + sourceInfo.key().substring(srcKey.length()); + String newSrcKey = sourceInfo.key(); + LOG.debug("Try to rename src key {} to dest key {}", newSrcKey, newDstKey); + + storage.rename(newSrcKey, newDstKey); + }); + } else { + Tasks.foreach(objs) + .executeWith(renamePool) + .throwFailureWhenFinished() + .retry(RENAME_RETRY_TIMES) + .revertWith(sourceInfo -> { + String newDstKey = dstKey + sourceInfo.key().substring(srcKey.length()); + storage.delete(newDstKey); + }) + .run(sourceInfo -> { + String newDstKey = dstKey + sourceInfo.key().substring(srcKey.length()); + LOG.debug("Try to rename src key {} to dest key {}", sourceInfo.key(), newDstKey); + + try { + if (ObjectInfo.isDir(newDstKey)) { + mkdir(newDstKey); + } else { + copyFile(sourceInfo.key(), newDstKey, sourceInfo.size()); + } + } catch (IOException e) { + throw new UncheckedIOException( + String.format("Failed to copy source file %s to dest file %s", sourceInfo.key(), + newDstKey), e); + } + }); + + // Delete all the source keys, since we've already copied them into destination keys. + storage.deleteAll(srcKey); + } + } + + private void renameFile(String srcKey, String dstKey, long fileSize) { + if (renameObjectEnabled) { + storage.rename(srcKey, dstKey); + } else { + Tasks.foreach(0) + .throwFailureWhenFinished() + .retry(RENAME_RETRY_TIMES) + .revertWith(obj -> storage.delete(dstKey)) + .run(obj -> { + try { + copyFile(srcKey, dstKey, fileSize); + } catch (IOException e) { + throw new UncheckedIOException( + String.format("Failed to copy source file %s to dest file %s", srcKey, dstKey), + e); + } + }); + + Tasks.foreach(0) + .throwFailureWhenFinished() + .retry(RENAME_RETRY_TIMES) + .run(obj -> storage.delete(srcKey)); + } + } + + private void copyFile(String srcKey, String dstKey, long srcSize) throws IOException { + long byteSizePerPart = conf.getLong(ConfKeys.FS_MULTIPART_SIZE.key(storage.scheme()), + ConfKeys.FS_MULTIPART_SIZE_DEFAULT); + long multiPartCopyThreshold = + conf.getLong(ConfKeys.FS_MULTIPART_COPY_THRESHOLD.key(storage.scheme()), + ConfKeys.FS_MULTIPART_COPY_THRESHOLD_DEFAULT); + if (srcSize > multiPartCopyThreshold) { + uploadPartCopy(srcKey, srcSize, dstKey, byteSizePerPart); + } else { + storage.copy(srcKey, dstKey); + } + } + + private void uploadPartCopy(String srcKey, long srcSize, String dstKey, long byteSizePerPart) { + final MultipartUpload multipartUpload = storage.createMultipartUpload(dstKey); + try { + Preconditions.checkState(byteSizePerPart >= multipartUpload.minPartSize(), + "Configured upload part size %s must be greater than or equals to the minimal part" + + " size %s, please check configure key %s.", byteSizePerPart, + multipartUpload.minPartSize(), ConfKeys.FS_MULTIPART_SIZE.key(storage.scheme())); + + AtomicInteger partNumGetter = new AtomicInteger(0); + List> results = Lists.newArrayList(); + for (long start = 0, end; start < srcSize; start += byteSizePerPart) { + end = Math.min(start + byteSizePerPart, srcSize) - 1; + Preconditions.checkArgument(end >= 0, "Invalid copy range start: %s, end: %s", start, end); + // Submit upload part copy task to the thread pool. + CompletableFuture result = asyncUploadPartCopy(srcKey, multipartUpload, + partNumGetter.incrementAndGet(), start, end); + results.add(result); + } + + // Waiting for all the upload parts to be finished. + List parts = results.stream() + .map(CompletableFuture::join) + .sorted(Comparator.comparing(Part::num)) + .collect(Collectors.toList()); + + finishUpload(multipartUpload.key(), multipartUpload.uploadId(), parts); + } catch (Exception e) { + LOG.error("Encountering error when upload part copy", e); + CommonUtils.runQuietly( + () -> storage.abortMultipartUpload(multipartUpload.key(), multipartUpload.uploadId())); + throw e; + } + } + + protected void finishUpload(String key, String uploadId, List uploadParts) { + storage.completeUpload(key, uploadId, uploadParts); + } + + private CompletableFuture asyncUploadPartCopy( + String srcKey, MultipartUpload multipartUpload, int partNum, + long copyRangeStart, long copyRangeEnd) { + return CompletableFuture.supplyAsync( + () -> storage.uploadPartCopy(srcKey, multipartUpload.key(), multipartUpload.uploadId(), + partNum, copyRangeStart, copyRangeEnd), renamePool) + .whenComplete((part, err) -> { + if (err != null) { + LOG.error("Failed to upload part copy, src key: {}, multipartUpload: {}, partNum: {}," + + " copy range start: {}, copy range end: {}", srcKey, multipartUpload, partNum, + copyRangeStart, copyRangeEnd, err); + } + }); + } + + private void mkdir(String key) { + storage.put(key, new byte[0]); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/package-info.java new file mode 100644 index 0000000000000..ecada495c292e --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/package-info.java @@ -0,0 +1,26 @@ +/* + * 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. + */ + +/** + * Classes for hadoop-tos ops. + */ +@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"}) +@InterfaceStability.Evolving +package org.apache.hadoop.fs.tosfs.ops; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/package-info.java new file mode 100644 index 0000000000000..16822fc4ccb37 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/package-info.java @@ -0,0 +1,26 @@ +/* + * 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. + */ + +/** + * Classes for hadoop-tos tosfs. + */ +@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"}) +@InterfaceStability.Evolving +package org.apache.hadoop.fs.tosfs; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/CommonUtils.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/CommonUtils.java new file mode 100644 index 0000000000000..e9b42b4198002 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/CommonUtils.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.tosfs.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public final class CommonUtils { + private static final Logger LOG = LoggerFactory.getLogger(CommonUtils.class); + + private CommonUtils() {} + + public static void runQuietly(RunWithException run) { + runQuietly(run, true); + } + + public static void runQuietly(RunWithException run, boolean logError) { + try { + run.run(); + } catch (Exception e) { + if (logError) { + LOG.info("Encounter error but can be ignored: ", e); + } + } + } + + public interface RunWithException { + void run() throws Exception; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/FSUtils.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/FSUtils.java new file mode 100644 index 0000000000000..42ec266dc7bbc --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/FSUtils.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.tosfs.util; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.util.Preconditions; + +import java.net.URI; + +public final class FSUtils { + private static final String OVERFLOW_ERROR_HINT = + FSExceptionMessages.TOO_MANY_BYTES_FOR_DEST_BUFFER + + ": request length = %s, with offset = %s, buffer capacity = %s"; + + private FSUtils() { + } + + public static void checkReadParameters(byte[] buffer, int offset, int length) { + Preconditions.checkArgument(buffer != null, "Null buffer"); + Preconditions.checkArgument(offset >= 0 && offset <= buffer.length, + "offset: %s is out of range [%s, %s]", offset, 0, buffer.length); + Preconditions.checkArgument(length >= 0, "length: %s is negative", length); + Preconditions.checkArgument(buffer.length >= offset + length, + OVERFLOW_ERROR_HINT, length, offset, (buffer.length - offset)); + } + + public static URI normalizeURI(URI fsUri, Configuration hadoopConfig) { + final String scheme = fsUri.getScheme(); + final String authority = fsUri.getAuthority(); + + if (scheme == null && authority == null) { + fsUri = FileSystem.getDefaultUri(hadoopConfig); + } else if (scheme != null && authority == null) { + URI defaultUri = FileSystem.getDefaultUri(hadoopConfig); + if (scheme.equals(defaultUri.getScheme()) && defaultUri.getAuthority() != null) { + fsUri = defaultUri; + } + } + return fsUri; + } + + public static String scheme(Configuration conf, URI uri) { + if (uri.getScheme() == null || uri.getScheme().isEmpty()) { + return FileSystem.getDefaultUri(conf).getScheme(); + } else { + return uri.getScheme(); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/FuseUtils.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/FuseUtils.java new file mode 100644 index 0000000000000..27219f9081a4a --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/FuseUtils.java @@ -0,0 +1,28 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.util; + +public final class FuseUtils { + public static final String ENV_TOS_ENABLE_FUSE = "TOS_ENABLE_FUSE"; + + private FuseUtils() { + } + + public static boolean fuseEnabled() { + return ParseUtils.envAsBoolean(ENV_TOS_ENABLE_FUSE, false); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Iterables.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Iterables.java new file mode 100644 index 0000000000000..e4e68cba4ac43 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Iterables.java @@ -0,0 +1,128 @@ +/* + * 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.tosfs.util; + +import org.apache.hadoop.util.Preconditions; +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.function.Function; +import java.util.function.Predicate; + +public final class Iterables { + + private Iterables() {} + + public static Iterable transform(final Iterable fromIterable, + final Function function) { + Preconditions.checkNotNull(fromIterable); + Preconditions.checkNotNull(function); + return () -> new Iterator() { + private Iterator iterator = fromIterable.iterator(); + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public T next() { + return function.apply(iterator.next()); + } + }; + } + + public static Iterable filter(final Iterable unfiltered, + final Predicate predicate) { + Preconditions.checkNotNull(unfiltered); + Preconditions.checkNotNull(predicate); + return () -> new Iterator() { + private Iterator iterator = unfiltered.iterator(); + private boolean advance = true; + private T value; + + @Override + public boolean hasNext() { + if (!advance) { + return true; + } + + while (iterator.hasNext()) { + value = iterator.next(); + if (predicate.test(value)) { + advance = false; + return true; + } + } + + return false; + } + + @Override + public T next() { + if (hasNext()) { + advance = true; + return value; + } + throw new NoSuchElementException("No more items in iterator."); + } + }; + } + + public static Iterable concat( + Iterable> inputs) { + return () -> new ConcatenatedIterator<>(inputs.iterator()); + } + + private static class ConcatenatedIterator implements Iterator { + // Iterators is the iterator of iterables. + private final Iterator> iterators; + private Iterator curIter; + + ConcatenatedIterator(Iterator> iterators) { + Preconditions.checkNotNull(iterators, "Iterators should not be null."); + this.iterators = iterators; + } + + @Override + public boolean hasNext() { + while (curIter == null || !curIter.hasNext()) { + if (curIter != null) { + curIter = null; + } + + if (!iterators.hasNext()) { + return false; + } + + curIter = iterators.next().iterator(); + } + return true; + } + + @Override + public T next() { + if (hasNext()) { + return curIter.next(); + } + throw new NoSuchElementException("No more elements"); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/JsonCodec.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/JsonCodec.java new file mode 100644 index 0000000000000..5b2d7371c73d2 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/JsonCodec.java @@ -0,0 +1,46 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.util; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.PropertyAccessor; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +public class JsonCodec { + private static final ObjectMapper MAPPER = new ObjectMapper() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, true) + .configure(SerializationFeature.INDENT_OUTPUT, true) + .setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY); + private final Class clazz; + + public JsonCodec(Class clazz) { + this.clazz = clazz; + } + + public byte[] toBytes(T instance) throws IOException { + return MAPPER.writeValueAsBytes(instance); + } + + public T fromBytes(byte[] data) throws IOException { + return MAPPER.readValue(new String(data, 0, data.length, StandardCharsets.UTF_8), clazz); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/LazyReload.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/LazyReload.java new file mode 100644 index 0000000000000..60428abe78197 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/LazyReload.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.tosfs.util; + +import java.util.Iterator; +import java.util.function.Supplier; + +public class LazyReload implements Iterable { + private final Supplier> reload; + + public LazyReload(Supplier> reload) { + this.reload = reload; + } + + @Override + public Iterator iterator() { + return new LazyReloadIter<>(reload.get()); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/LazyReloadIter.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/LazyReloadIter.java new file mode 100644 index 0000000000000..43c6e30fcd3f8 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/LazyReloadIter.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.tosfs.util; + +import org.apache.hadoop.util.Lists; + +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +public class LazyReloadIter implements Iterator { + private final List buf = Lists.newArrayList(); + private final Reload reload; + private int cur = 0; + private boolean exhausted = false; + + public LazyReloadIter(Reload reload) { + this.reload = reload; + } + + @Override + public boolean hasNext() { + if (exhausted && buf.isEmpty()) { + return false; + } + if (cur >= buf.size()) { + // Reset the buffer and load more elements. + buf.clear(); + cur = 0; + // Reload the next batch. + boolean exhaust = reload.fill(buf); + while (buf.isEmpty() && !exhaust) { + exhaust = reload.fill(buf); + } + + if (exhaust) { + this.exhausted = true; + } + + return !buf.isEmpty(); + } + return true; + } + + @Override + public T next() { + if (hasNext()) { + return buf.get(cur++); + } else { + throw new NoSuchElementException(); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/ParseUtils.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/ParseUtils.java new file mode 100644 index 0000000000000..0851d86715964 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/ParseUtils.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.tosfs.util; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.util.Preconditions; + +public final class ParseUtils { + private static final String ERROR_MSG = "Failed to parse value %s as %s, property key %s"; + + private ParseUtils() { + } + + public static String envAsString(String key) { + return envAsString(key, true); + } + + public static String envAsString(String key, boolean allowNull) { + String value = System.getenv(key); + if (!allowNull) { + Preconditions.checkNotNull(value, "os env key: %s cannot be null", key); + } + return value; + } + + public static String envAsString(String key, String defaultValue) { + String value = System.getenv(key); + return StringUtils.isEmpty(value) ? defaultValue : value; + } + + public static boolean envAsBoolean(String key, boolean defaultValue) { + String value = System.getenv(key); + if (StringUtils.isEmpty(value)) { + return defaultValue; + } + checkBoolean(key, value); + return Boolean.parseBoolean(value); + } + + public static boolean isBoolean(String value) { + return "true".equalsIgnoreCase(value) || "false".equalsIgnoreCase(value); + } + + public static void checkBoolean(String key, String value) { + if (!isBoolean(value)) { + throw new IllegalArgumentException(String.format(ERROR_MSG, value, "boolean", key)); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Range.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Range.java new file mode 100644 index 0000000000000..e3f971d5fb4fe --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Range.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.tosfs.util; + +import org.apache.hadoop.thirdparty.com.google.common.base.MoreObjects; +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.util.Preconditions; + +import java.util.List; +import java.util.Objects; + +public final class Range { + private final long off; + private final long len; + + private Range(long off, long len) { + this.off = off; + this.len = len; + } + + public static Range of(long off, long len) { + return new Range(off, len); + } + + public long off() { + return off; + } + + public long len() { + return len; + } + + public long end() { + return off + len; + } + + public boolean include(long pos) { + return pos >= off && pos < off + len; + } + + public boolean overlap(Range r) { + return r.off() < end() && off() < r.end(); + } + + @Override + public int hashCode() { + return Objects.hash(off, len); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof Range)) { + return false; + } + + Range that = (Range) o; + return Objects.equals(off, that.off) + && Objects.equals(len, that.len); + } + + public String toString() { + return MoreObjects.toStringHelper(this) + .add("offset", off) + .add("length", len) + .toString(); + } + + public static List split(long totalSize, long width) { + Preconditions.checkArgument(totalSize >= 0, "Size %s must be >= 0", totalSize); + Preconditions.checkArgument(width > 0, "Width %s must be positive", width); + + long remain = totalSize % width; + long rangeNum = totalSize / width; + + List ranges = Lists.newArrayListWithCapacity((int) rangeNum + (remain == 0 ? 0 : 1)); + for (int i = 0; i < rangeNum; i++) { + ranges.add(Range.of(i * width, width)); + } + + if (remain > 0) { + ranges.add(Range.of(totalSize - remain, remain)); + } + + return ranges; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Reload.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Reload.java new file mode 100644 index 0000000000000..fb608d6e79bdb --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Reload.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.tosfs.util; + +import java.util.List; + +public interface Reload { + /** + * @param buf contains the filled data + * @return true if no need to fill again since the source is exhausted, otherwise false. + */ + boolean fill(List buf); +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/RemoteIterators.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/RemoteIterators.java new file mode 100644 index 0000000000000..9e3909bc1d7a9 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/RemoteIterators.java @@ -0,0 +1,123 @@ +/* + * 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.tosfs.util; + +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.thirdparty.com.google.common.base.MoreObjects; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Iterator; +import java.util.NoSuchElementException; + +public final class RemoteIterators { + + private RemoteIterators() { + } + + /** + * Create an iterator from a singleton. + * + * @param singleton instance + * @param type + * @return a remote iterator + */ + public static RemoteIterator fromSingleton(@Nullable T singleton) { + return new SingletonIterator<>(singleton); + } + + /** + * Create an iterator from an iterable and a transformation function. + * + * @param source type + * @param result type + * @param iterator source + * @param mapper transformation + * @return a remote iterator + */ + public static RemoteIterator fromIterable(Iterable iterator, + FunctionRaisingIOE mapper) { + return new IterableRemoteIterator<>(iterator, mapper); + } + + public interface FunctionRaisingIOE { + + /** + * Apply the function. + * + * @param s argument 1 + * @return result + * @throws IOException Any IO failure + */ + T apply(S s) throws IOException; + } + + private static final class IterableRemoteIterator implements RemoteIterator { + private final Iterator sourceIterator; + private final FunctionRaisingIOE mapper; + + private IterableRemoteIterator(Iterable source, FunctionRaisingIOE mapper) { + this.sourceIterator = source.iterator(); + this.mapper = mapper; + } + + @Override + public boolean hasNext() { + return sourceIterator.hasNext(); + } + + @Override + public T next() throws IOException { + return mapper.apply(sourceIterator.next()); + } + } + + private static final class SingletonIterator implements RemoteIterator { + private final T singleton; + + private boolean processed; + + private SingletonIterator(@Nullable T singleton) { + this.singleton = singleton; + this.processed = singleton == null; + } + + @Override + public boolean hasNext() { + return !processed; + } + + @Override + public T next() { + if (hasNext()) { + processed = true; + return singleton; + } else { + throw new NoSuchElementException(); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("singleton", singleton) + .toString(); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/RetryableUtils.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/RetryableUtils.java new file mode 100644 index 0000000000000..72354ad399b1b --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/RetryableUtils.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.tosfs.util; + +import java.util.concurrent.ThreadLocalRandom; + +public final class RetryableUtils { + private static final ThreadLocalRandom RANDOM = ThreadLocalRandom.current(); + + private RetryableUtils() { + } + + /** + * Copy from {@link com.volcengine.tos.internal.util.TosUtils#backoff(int)}. + * + * @param attempt the attempt count + * @return backoff milliseconds + */ + public static long backoff(int attempt) { + if (attempt == 0) { + return 1000L; + } else { + double backoff = 1.0; + + double max; + for (max = 10.0; backoff < max && attempt > 0; --attempt) { + backoff *= 1.6; + } + + if (backoff > max) { + backoff = max; + } + + backoff *= 1.0 + 0.2 * (RANDOM.nextDouble(1.0) * 2.0 - 1.0); + return backoff < 0.0 ? 0L : (long) (backoff * 1000.0); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Serializer.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Serializer.java new file mode 100644 index 0000000000000..2dfc69100fd1f --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Serializer.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.tosfs.util; + +import java.io.IOException; + +public interface Serializer { + byte[] serialize() throws IOException; +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/TOSClientContextUtils.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/TOSClientContextUtils.java new file mode 100644 index 0000000000000..c83d7e1aa73d8 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/TOSClientContextUtils.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.tosfs.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public final class TOSClientContextUtils { + public static final Logger LOG = LoggerFactory.getLogger(TOSClientContextUtils.class); + + private TOSClientContextUtils() { + } + + public static String normalizeEndpoint(String endpoint) { + for (String scheme : new String[]{"https://", "http://", "tos://"}) { + if (endpoint.startsWith(scheme)) { + return endpoint.substring(scheme.length()); + } + } + return endpoint; + } + + public static String parseRegion(String endpoint) { + String region = null; + String newEndpoint = normalizeEndpoint(endpoint); + String[] parts = newEndpoint.split("\\."); + if (parts.length == 3) { + // Endpoint is formatted like 'tos-.volces.com' + region = parts[0].replace("tos-", ""); + } else if (parts.length == 4) { + // Endpoint is formatted like '.tos-.volces.com' + region = parts[1].replace("tos-", ""); + } else if (parts.length == 6) { + // Endpoint is formatted like '.tos..privatelink.volces.com' + region = parts[2]; + } else if (parts.length == 7) { + // Endpoint is formatted like '..tos..privatelink.volces.com' + region = parts[3]; + } + LOG.debug("parse region [{}] from endpoint [{}]", region, endpoint); + return region; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/UUIDUtils.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/UUIDUtils.java new file mode 100644 index 0000000000000..d7b5ae8a6f51a --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/UUIDUtils.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.tosfs.util; + +import java.util.UUID; + +public final class UUIDUtils { + private UUIDUtils() { + } + + public static String random() { + UUID uuid = UUID.randomUUID(); + return uuid.toString().replace("-", ""); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/package-info.java new file mode 100644 index 0000000000000..c48a527e91515 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/package-info.java @@ -0,0 +1,26 @@ +/* + * 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. + */ + +/** + * Classes for hadoop-tos util. + */ +@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"}) +@InterfaceStability.Evolving +package org.apache.hadoop.fs.tosfs.util; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/site/markdown/cloud-storage/index.md b/hadoop-cloud-storage-project/hadoop-tos/src/site/markdown/cloud-storage/index.md new file mode 100644 index 0000000000000..d6379679f37f8 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/site/markdown/cloud-storage/index.md @@ -0,0 +1,430 @@ + + +# Integration of VolcanoEngine TOS in Hadoop + +## Overview + +TOS is the object storage service of Volcano Engine, which is a cloud vendor launched by ByteDance. +Hadoop-tos is a connector between computing systems and underlying storage. For systems like +Hadoop MR, Hive(both mr and tez) and Spark, hadoop-tos helps them use TOS as the underlying storage +system instead of HDFS. + +## Quick Start + +In quick start, we will use hadoop shell command to access a tos bucket. + +### Requirements + +1. A Volcano Engine account. Use the account to create a TOS bucket. +2. A dev environment that can access TOS. E.g. a local server or a volcano engine cloud server. +3. Install hadoop to the dev environment. Hadoop is installed at `$HADOOP_HOME`. + +### Usage +* Compile hadoop-tos bundle tar. The hadoop-tos bundle is not packaged in hadoop final tar file. +So we have to compile it manually. Download the hadoop project, and build it with command below. +```bash +mvn package -DskipTests -pl org.apache.hadoop:hadoop-tos +``` + +* The bundle jar file is placed at +`$HADOOP_HOME/hadoop-cloud-storage-project/hadoop-tos/target/hadoop-tos-{VERSION}.jar`. + +* Copy the bundle jar to hdfs lib path. The hdfs lib path is +`$HADOOP_HOME/share/hadoop/hdfs/lib`. Remember copying to all hadoop nodes. +```bash +cp hadoop-tos-{VERSION}.jar $HADOOP_HOME/share/hadoop/hdfs/lib/ +``` + +* Configure properties below. + +```xml + + + fs.defaultFS + tos://{your_bucket_name}/ + + The name of the default file system. Make it your tos bucket. + + + + + fs.tos.endpoint + + + Object storage endpoint to connect to, which should include both region and object domain name. + e.g. 'fs.tos.endpoint'='tos-cn-beijing.volces.com'. + + + + + fs.tos.impl + org.apache.hadoop.fs.tosfs.TosFileSystem + + The implementation class of the tos FileSystem. + + + + + fs.AbstractFileSystem.tos.impl + org.apache.hadoop.fs.tosfs.TosFS + + The implementation class of the tos AbstractFileSystem. + + + + + fs.tos.access-key-id + + + The access key of volcano engine's user or role. + + + + + fs.tos.secret-access-key + + + The secret key of the access key specified by 'fs.tos.access-key-id'. + + + +``` + +* Use hadoop shell command to access TOS. + +```bash +# 1. List root dir. +hadoop fs -ls / + +# 2. Make directory. +hadoop fs -mkdir /hadoop-tos + +# 3. Write and read. +echo "hello tos." > hello.txt +hadoop fs -put hello.txt /hadoop-tos/ +hadoop fs -cat /hadoop-tos/hello.txt + +# 4. Delete file and directory. +hadoop fs -rm -r /hadoop-tos/ +``` + +## Introduction + +This is a brief introduction of hadoop-tos design and basic functions. The following contents are +based on flat mode by default. The differences between hierarchy mode will be explained at the end +of each section. + +### TOS + +TOS is the object storage service of Volcano Engine. It is similar to ASW S3, Azure Blob Storage +and Aliyun OSS, and has some unique features, such as object fast copy, object fast rename, +CRC32C checksum etc. Learn more details about TOS from: https://www.volcengine.com/product/TOS. + +TOS has 2 modes: the flat mode and the hierarchy mode. In flat mode, there are no directories, +all objects are files indexed by the object names. User can use 'slash' in the object name to +logically divide objects into different "directories", though the "directories divided by the slash" +are not real. Cleanup a logic directory is to clean all the files with "directory path" as prefix. + +In hierarchy mode, there are directories and files. A directory object is the object whose name +ends with slash. All objects start with the directory object name are the directory object's +consecutive objects, and together they form a directory tree. A directory object can't contain +any data. Delete or rename a directory object will clean or rename all objects under the +directory tree atomically. + +TOS has some distinctive features that are very useful in bigdata scenarios. + +1. The fast copy feature enables users to duplicate objects without copying data, even huge objects + could be copied within tens of milliseconds. +2. The fast rename feature enables users to rename one object with a new name without copying data. +3. TOS supports CRC32C checksum. It enables user to compare files' checksums between HDFS and TOS. + +### Directory and file + +This section illustrates how hadoop-tos transforms TOS to a hadoop FileSystem. TOS requires object's +name must not start with slash, must not contain consecutive slash and must not be empty. Here is +the transformation rules. + +* Object name is divided by slash to form hierarchy. +* An object whose name ends with slash is a directory. +* An object whose name doesn't end with slash is a file. +* A file's parents are directories, no matter whether the parent exists or not. + +For example, supposing we have 2 objects "user/table/" and "user/table/part-0". The first object +is mapped to "/user/table" in hadoop and is a directory. The second object is mapped to +"/user/table/part-0" as a file. The non-existent object "user/" is mapped to "/user" as a directory +because it's the parent of file "/user/table/part-0". + +| Object name | Object existence | FileSystem path | FileSystem Type | +|-------------------|------------------|--------------------|-----------------| +| user/table/ | yes | /user/table | Directory | +| user/table/part-0 | yes | /user/table/part-0 | File | +| user/ | no | /user | Directory | + +The FileSystem requirements above are not enforced rules in flat mode, users can construct +cases violating the requirements above. For example, creating a file with its parent is a file. In +hierarchy mode, the requirements are enforced rules controlled by TOS service, so there won't be +semantic violations. + +### List, Rename and Delete + +List, rename and delete are costly operations in flat mode. Since the namespace is flat, to list +a directory, the client needs to scan all objects with directory as the prefix and filter with +delimiter. For rename and delete directory, the client needs to first list the directory to get all +objects and then rename or delete objects one by one. So they are not atomic operations and costs a +lot comparing to hdfs. + +The idiosyncrasies of hierarchy mode is supporting directory. So it can list very fast and +support atomic rename and delete directory. Rename or delete failure in flat mode may leave +the bucket in an inconsistent state, the hierarchy mode won't have this problem. + +### Read and write file + +The read behaviour in hadoop-tos is very like reading an HDFS file. The challenge is how to keep the +input stream consistent with object. If the object is changed after we open the file, the input +stream should fail. This is implemented by saving the file checksum when open file. If the +file is changed while reading, the input stream will compare the checksum and trigger an exception. + +The write behaviour in hadoop-tos is slightly different from hdfs. Firstly, the append interface +is not supported. Secondly, the file is not visible until it is successfully closed. Finally, +when 2 clients try to write one file, the last client to close the file will override the previous +one. + +Both read and write has many performance optimizations. E.g. range read, connection reuse, local +write buffer, put for small files, multipart-upload for big files etc. + +### Permissions + +TOS permission model is different from hadoop filesystem permission model. TOS supports permissions +based on IAM, Bucket Policy, Bucket and Object ACL, while hadoop filesystem permission model uses +mode and acl. There is no way to mapped tos permission to hadoop filesystem permission, so we have +to use fake permissions in TosFileSystem and TosFS. Users can read and change the filesystem +permissions, they can only be seen but not effective. Permission control eventually depends on TOS +permission model. + +### Times + +Hadoop-tos supports last modified time and doesn't support access time. For files, the last modified +time is the object's modified time. For directories, if the directory object doesn't exist, the last +modified time is the current system time. If the directory object exists, the last modified time is +the object's modify time when `getFileStatus` and current system time when `listStatus`. + +### File checksum + +TOS supports CRC64ECMA checksum by default, it is mapped to Hadoop FileChecksum. We can +retrieve it by calling `FileSystem#getFileChecksum`. +To be compatible with HDFS, TOS provides optional CRC32C checksum. When we distcp +between HDFS and TOS, we can rely on distcp checksum mechanisms to keep data consistent. +To use CRC32C, configure keys below. +```xml + + + fs.tos.checksum.enabled + true + + + fs.tos.checksum-algorithm + COMPOSITE-CRC32C + + + fs.tos.checksum-type + CRC32C + + +``` + +### Credential + +TOS client uses access key id and secret access key to authenticate with tos service. There are 2 +ways to configure them. First is adding to hadoop configuration, such as adding to core-site.xml or +configuring through `-D` parameter. The second is setting environment variable, hadoop-tos will +search for environment variables automatically. + +To configure ak, sk in hadoop configuration, using the key below. + +```xml + + + + + fs.tos.access-key-id + + + The accessKey key to access the tos object storage. + + + + fs.tos.secret-access-key + + + The secret access key to access the object storage. + + + + fs.tos.session-token + + + The session token to access the object storage. + + + + + + fs.tos.bucket.{bucket_name}.access-key-id + + + The access key to access the object storage for the configured bucket. + + + + fs.tos.bucket.{bucket_name}.secret-access-key + + + The secret access key to access the object storage for the configured bucket. + + + + fs.tos.bucket.{bucket_name}.session-token + + + The session token to access the object storage for the configured bucket. + + + +``` + +The ak, sk in environment variables have the top priority and automatically fall back to hadoop +configuration if not found. The priority could be changed +by `fs.tos.credential.provider.custom.classes`. + +### Committer + +Hadoop-tos provides MapReduce job committer for better performance. By default, hadoop uses FileOutputCommitter +which will rename files many times. First when tasks commit, files will be renamed to a output path. +Then when job commits, the files will be renamed from output path to the final path. When using hdfs, +rename is not a problem because it only changes meta. But in TOS, by default the rename is implemented +as copy and delete and costs a lot. + +TOS committer is an optimized implementation for object storage. When task commits, it won't complete +multipart-upload for the files, instead it write pending set files including all the information to +complete multipart-upload. Then when job commits, it reads all the pending files and completes all +multipart-uploads. + +An alternative way is turning on TOS renameObject switch and still use FileOutputFormat. Objects are +renamed with only meta change. The performance is slightly slower than hadoop-tos committer, because +when using TOS rename objects, each file is committed first and then renamed to the final path. There +is 1 commit request and 2 rename request to TOS. But with hadoop-tos committer, the object is +postponing committed and there is no rename request overhead. + +To enable hadoop-tos committer, configure the key value below. +```xml + + + + mapred.output.committer.class + org.apache.hadoop.fs.tosfs.commit.mapred.Committer + + + + mapreduce.outputcommitter.factory.scheme.tos + org.apache.hadoop.fs.tosfs.commit.CommitterFactory + + +``` + +To enable tos objectRename, first turn on the object rename switch on tos, then configure the key +value below. +```xml + + + fs.tos.rename.enabled + true + + +``` + +## Properties Summary + +| properties | description | default value | required | +|-------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------|----------| +| fs.tos.access-key-id | The accessKey key to access the tos object storage | NONE | YES | +| fs.tos.secret-access-key | The secret access key to access the object storage | NONE | YES | +| fs.tos.session-token | The session token to access the object storage | NONE | NO | +| fs.%s.endpoint | Object storage endpoint to connect to, which should include both region and object domain name. | NONE | NO | +| fs.%s.region | The region of the object storage, e.g. fs.tos.region. Parsing template "fs.%s.endpoint" to know the region. | NONE | NO | +| fs.tos.bucket.%s.access-key-id | The access key to access the object storage for the configured bucket, where %s is the bucket name. | NONE | NO | +| fs.tos.bucket.%s.secret-access-key | The secret access key to access the object storage for the configured bucket, where %s is the bucket name | NONE | NO | +| fs.tos.bucket.%s.session-token | The session token to access the object storage for the configured bucket, where %s is the bucketname | NONE | NO | +| fs.tos.credentials.provider | Default credentials provider chain that looks for credentials in this order: SimpleCredentialsProvider,EnvironmentCredentialsProvider | org.apache.hadoop.fs.tosfs.object.tos.auth.DefaultCredentialsProviderChain | NO | +| fs.tos.credential.provider.custom.classes | User customized credential provider classes, separate provider class name with comma if there are multiple providers. | org.apache.hadoop.fs.tosfs.object.tos.auth.EnvironmentCredentialsProvider,org.apache.hadoop.fs.tosfs.object.tos.auth.SimpleCredentialsProvider | NO | +| fs.tos.credentials.provider | Default credentials provider chain that looks for credentials in this order: SimpleCredentialsProvider,EnvironmentCredentialsProvider. | org.apache.hadoop.fs.tosfs.object.tos.auth.DefaultCredentialsProviderChain | NONE | +| fs.tos.credential.provider.custom.classes | User customized credential provider classes, separate provider class name with comma if there are multiple providers. | org.apache.hadoop.fs.tosfs.object.tos.auth.EnvironmentCredentialsProvider,org.apache.hadoop.fs.tosfs.object.tos.auth.SimpleCredentialsProvider | NONE | +| fs.tos.http.maxConnections | The maximum number of connections to the TOS service that a client can create. | 1024 | NONE | +| fs.tos.http.idleConnectionTimeMills | The time that a connection thread can be in idle state, larger than which the thread will be terminated. | 60000 | NONE | +| fs.tos.http.connectTimeoutMills | The connect timeout that the tos client tries to connect to the TOS service. | 10000 | NONE | +| fs.tos.http.readTimeoutMills | The reading timeout when reading data from tos. Note that it is configured for the tos client sdk, not hadoop-tos. | 30000 | NONE | +| fs.tos.http.writeTimeoutMills | The writing timeout when uploading data to tos. Note that it is configured for the tos client sdk, not hadoop-tos. | 30000 | NONE | +| fs.tos.http.enableVerifySSL | Enables SSL connections to TOS or not. | true | NONE | +| fs.tos.http.dnsCacheTimeMinutes | The timeout (in minutes) of the dns cache used in tos client. | 0 | NONE | +| fs.tos.rmr.server.enabled | Used for directory bucket, whether enable recursive delete capability in TOS server, which will atomic delete all objects under given dir(inclusive), otherwise the client will list all sub objects, and then send batch delete request to TOS to delete dir. | false | NONE | +| fs.tos.rmr.client.enabled | If fs.tos.rmr.client.enabled is true, client will list all objects under the given dir and delete them by batch. Set value with true will use the recursive delete capability of TOS SDK, otherwise will delete object one by one via preorder tree walk. | true | NONE | +| fs.tos.user.agent.prefix | The prefix will be used as the product name in TOS SDK. The final user agent pattern is '{prefix}/TOS_FS/{hadoop tos version}'. | HADOOP-TOS | NONE | +| fs.tos.max-drain-bytes | The threshold indicates whether reuse the socket connection to optimize read performance during closing tos object inputstream of get object. If the remaining bytes is less than max drain bytes during closing the inputstream, will just skip the bytes instead of closing the socket connection. | 1024 * 1024L | NONE | +| fs.tos.client.disable.cache | Whether disable the tos http client cache in the current JVM. | false | NONE | +| fs.tos.batch.delete.objects-count | The batch size when deleting the objects in batches. | 1000 | NONE | +| fs.tos.batch.delete.max-retries | The maximum retry times when deleting objects in batches failed. | 20 | NONE | +| fs.tos.batch.delete.retry-codes | The codes from TOS deleteMultiObjects response, client will resend the batch delete request to delete the failed keys again if the response only contains these codes, otherwise won't send request anymore. | ExceedAccountQPSLimit,ExceedAccountRateLimit,ExceedBucketQPSLimit,ExceedBucketRateLimit,InternalError,ServiceUnavailable,SlowDown,TooManyRequests | NONE | +| fs.tos.batch.delete.retry.interval | The retry interval (in milliseconds) when deleting objects in batches failed. | 1000 | NONE | +| fs.tos.list.objects-count | The batch size of listing object per request for the given object storage, such as listing a directory, searching for all objects whose path starts with the directory path, and returning them as a list. | 1000 | NONE | +| fs.tos.request.max.retry.times | The maximum retry times of sending request via TOS client, client will resend the request if got retryable exceptions, e.g. SocketException, UnknownHostException, SSLException, InterruptedException, SocketTimeoutException, or got TOO_MANY_REQUESTS, INTERNAL_SERVER_ERROR http codes. | 20 | NONE | +| fs.tos.fast-fail-409-error-codes | The fast-fail error codes means the error cannot be solved by retrying the request. TOS client won't retry the request if receiving a 409 http status code and if the error code is in the configured non-retryable error code list. | 0026-00000013,0026-00000020,0026-00000021,0026-00000025,0026-00000026,0026-00000027 ,0017-00000208,0017-00000209 | NONE | +| fs.tos.inputstream.max.retry.times | The maximum retry times of reading object content via TOS client, client will resend the request to create a new input stream if getting unexpected end of stream error during reading the input stream. | 5 | NONE | +| fs.tos.crc.check.enable | Enable the crc check when uploading files to tos or not. | true | NONE | +| fs.tos.get-file-status.enabled | Whether enable tos getFileStatus API or not, which returns the object info directly in one RPC request, otherwise, might need to send three RPC requests to get object info. For example, there is a key 'a/b/c' exists in TOS, and we want to get object status of 'a/b', the GetFileStatus('a/b') will return the prefix 'a/b/' as a directory object directly. If this property is disabled, we need to head('a/b') at first, and then head('a/b/'), and last call list('a/b/', limit=1) to get object info. Using GetFileStatus API can reduce the RPC call times. | true | NONE | +| fs.tos.checksum-algorithm | The key indicates the name of the tos checksum algorithm. Specify the algorithm name to compare checksums between different storage systems. For example to compare checksums between hdfs and tos, we need to configure the algorithm name to COMPOSITE-CRC32C. | TOS-CHECKSUM | NONE | +| fs.tos.checksum-type | The key indicates how to retrieve file checksum from tos, error will be thrown if the configured checksum type is not supported by tos. The supported checksum types are: CRC32C, CRC64ECMA. | CRC64ECMA | NONE | +| fs.objectstorage.%s.impl | The object storage implementation for the defined scheme. For example, we can delegate the scheme 'abc' to TOS (or other object storage),and access the TOS object storage as 'abc://bucket/path/to/key' | NONE | NO | +| fs.%s.delete.batch-size | The batch size of deleting multiple objects per request for the given object storage. e.g. fs.tos.delete.batch-size | 250 | NONE | +| fs.%s.multipart.size | The multipart upload part size of the given object storage, e.g. fs.tos.multipart.size. | 8388608 | NONE | +| fs.%s.multipart.copy-threshold | The threshold (larger than this value) to enable multipart upload during copying objects in the given object storage. If the copied data size is less than threshold, will copy data via executing copyObject instead of uploadPartCopy. E.g. fs.tos.multipart.copy-threshold | 5242880 | NONE | +| fs.%s.multipart.threshold | The threshold which control whether enable multipart upload during writing data to the given object storage, if the write data size is less than threshold, will write data via simple put instead of multipart upload. E.g. fs.tos.multipart.threshold. | 10485760 | NONE | +| fs.%s.multipart.staging-buffer-size | The max byte size which will buffer the staging data in-memory before flushing to the staging file. It will decrease the random write in local staging disk dramatically if writing plenty of small files. | 4096 | NONE | +| fs.%s.multipart.staging-dir | The multipart upload part staging dir(s) of the given object storage. e.g. fs.tos.multipart.staging-dir. Separate the staging dirs with comma if there are many staging dir paths. | ${java.io.tmpdir}/multipart-staging-dir | NONE | +| fs.%s.missed.parent.dir.async-create | True to create the missed parent dir asynchronously during deleting or renaming a file or dir. | true | NONE | +| fs.%s.rename.enabled | Whether using rename semantic of object storage during rename files, otherwise using copy + delete. Please ensure that the object storage support and enable rename semantic and before enable it, and also ensure grant rename permission to the requester. If you are using TOS, you have to send putBucketRename request before sending rename request, otherwise MethodNotAllowed exception will be thrown. | false | NONE | +| fs.%s.task.thread-pool-size | The range size when open object storage input stream. Value must be positive. The size of thread pool used for running tasks in parallel for the given object fs, e.g. delete objects, copy files. the key example: fs.tos.task.thread-pool-size. | Long.MAX_VALUE | NONE | +| fs.%s.multipart.thread-pool-size | The size of thread pool used for uploading multipart in parallel for the given object storage, e.g. fs.tos.multipart.thread-pool-size | Max value of 2 and available processors. | NONE | +| fs.%s.checksum.enabled | The toggle indicates whether enable checksum during getting file status for the given object. E.g. fs.tos.checksum.enabled | Max value of 2 and available processors. | NONE | +| fs.filestore.checksum-algorithm | The key indicates the name of the filestore checksum algorithm. Specify the algorithm name to satisfy different storage systems. For example, the hdfs style name is COMPOSITE-CRC32 and COMPOSITE-CRC32C. | TOS-CHECKSUM | NO | +| fs.filestore.checksum-type | The key indicates how to retrieve file checksum from filestore, error will be thrown if the configured checksum type is not supported. The supported checksum type is: MD5. | MD5 | NO | + +## Running unit tests in hadoop-tos module + +Unit tests need to connect to tos service. Setting the 6 environment variables below to run unit +tests. + +```bash +export TOS_ACCESS_KEY_ID={YOUR_ACCESS_KEY} +export TOS_SECRET_ACCESS_KEY={YOUR_SECRET_ACCESS_KEY} +export TOS_ENDPOINT={TOS_SERVICE_ENDPOINT} +export FILE_STORAGE_ROOT=/tmp/local_dev/ +export TOS_BUCKET={YOUR_BUCKET_NAME} +export TOS_UNIT_TEST_ENABLED=true +``` + +Then cd to hadoop project root directory, and run the test command below. + +```bash +mvn -Dtest=org.apache.hadoop.fs.tosfs.** test -pl org.apache.hadoop:hadoop-tos +``` \ No newline at end of file diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestEnv.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestEnv.java new file mode 100644 index 0000000000000..c95967bec50be --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestEnv.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.tosfs; + +import org.apache.hadoop.fs.tosfs.util.ParseUtils; + +public final class TestEnv { + public static final String ENV_TOS_UNIT_TEST_ENABLED = "TOS_UNIT_TEST_ENABLED"; + private static final boolean TOS_TEST_ENABLED; + + static { + TOS_TEST_ENABLED = ParseUtils.envAsBoolean(ENV_TOS_UNIT_TEST_ENABLED, true); + } + + private TestEnv() {} + + public static boolean checkTestEnabled() { + return TOS_TEST_ENABLED; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestRawFSUtils.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestRawFSUtils.java new file mode 100644 index 0000000000000..511a13b84214a --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestRawFSUtils.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.tosfs; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestRawFSUtils { + + @Test + public void testIsAncestor() { + assertTrue(RawFSUtils.inSubtree("/", "/")); + assertTrue(RawFSUtils.inSubtree("/", "/a")); + assertTrue(RawFSUtils.inSubtree("/a", "/a")); + assertFalse(RawFSUtils.inSubtree("/a", "/")); + assertTrue(RawFSUtils.inSubtree("/", "/a/b/c")); + assertFalse(RawFSUtils.inSubtree("/a/b/c", "/")); + assertTrue(RawFSUtils.inSubtree("/", "/a/b/c.txt")); + assertFalse(RawFSUtils.inSubtree("/a/b/c.txt", "/")); + assertTrue(RawFSUtils.inSubtree("/a/b/", "/a/b")); + assertTrue(RawFSUtils.inSubtree("/a/b/", "/a/b/c")); + assertFalse(RawFSUtils.inSubtree("/a/b/c", "/a/b")); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestRawFileSystem.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestRawFileSystem.java new file mode 100644 index 0000000000000..05c7bcb7e46dc --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestRawFileSystem.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.tosfs; + +import org.apache.hadoop.conf.Configuration; +import org.junit.jupiter.api.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class TestRawFileSystem { + @Test + public void testInitializeFileSystem() throws URISyntaxException, IOException { + Configuration conf = new Configuration(); + try (RawFileSystem fs = new RawFileSystem()) { + fs.initialize(new URI("filestore://bucket_a/a/b/c"), conf); + assertEquals("bucket_a", fs.bucket()); + + fs.initialize(new URI("filestore://bucket-/a/b/c"), conf); + assertEquals("bucket-", fs.bucket()); + + fs.initialize(new URI("filestore://-bucket/a/b/c"), conf); + assertEquals("-bucket", fs.bucket()); + } + } + + @Test + public void testBucketNotExist() { + Configuration conf = new Configuration(); + RawFileSystem fs = new RawFileSystem(); + assertThrows(FileNotFoundException.class, + () -> fs.initialize(new URI("tos://not-exist-bucket/"), conf), "Bucket doesn't exist."); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestTosChecksum.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestTosChecksum.java new file mode 100644 index 0000000000000..87b80ae77378b --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestTosChecksum.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.tosfs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileChecksum; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.conf.FileStoreKeys; +import org.apache.hadoop.fs.tosfs.conf.TosKeys; +import org.apache.hadoop.fs.tosfs.object.ChecksumType; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectStorageFactory; +import org.apache.hadoop.fs.tosfs.util.TempFiles; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.apache.hadoop.fs.tosfs.util.UUIDUtils; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Stream; + +import static org.apache.hadoop.fs.tosfs.object.tos.TOS.TOS_SCHEME; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestTosChecksum { + private static final String FILE_STORE_ROOT = TempFiles.newTempDir("TestTosChecksum"); + private static final String ALGORITHM_NAME = "mock-algorithm"; + private static final String PREFIX = UUIDUtils.random(); + + private ObjectStorage objectStorage; + + public void setObjectStorage(ObjectStorage objectStorage) { + this.objectStorage = objectStorage; + } + + + static Stream provideArguments() throws URISyntaxException { + List values = new ArrayList<>(); + + // Case 1: file store. + Configuration fileStoreConf = new Configuration(); + fileStoreConf.set(FileStoreKeys.FS_FILESTORE_CHECKSUM_ALGORITHM, ALGORITHM_NAME); + fileStoreConf.set(FileStoreKeys.FS_FILESTORE_CHECKSUM_TYPE, ChecksumType.MD5.name()); + fileStoreConf.set(ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key("filestore"), FILE_STORE_ROOT); + URI uri0 = new URI("filestore://" + TestUtility.bucket() + "/"); + + values.add(Arguments.of( + ChecksumType.MD5, + fileStoreConf, + uri0, + ObjectStorageFactory.create(uri0.getScheme(), uri0.getAuthority(), fileStoreConf) + )); + + // Case 2: tos. + Configuration tosConf = new Configuration(); + tosConf.set(TosKeys.FS_TOS_CHECKSUM_ALGORITHM, ALGORITHM_NAME); + tosConf.set(TosKeys.FS_TOS_CHECKSUM_TYPE, ChecksumType.CRC32C.name()); + URI uri1 = new URI(TOS_SCHEME + "://" + TestUtility.bucket() + "/"); + + values.add(Arguments.of( + ChecksumType.CRC32C, + tosConf, + uri1, + ObjectStorageFactory.create(uri1.getScheme(), uri1.getAuthority(), tosConf) + )); + + return values.stream(); + } + + @AfterEach + public void tearDown() { + objectStorage.deleteAll(PREFIX); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testChecksumInfo(ChecksumType type, Configuration conf, URI uri, + ObjectStorage objectStore) { + setObjectStorage(objectStore); + + assertEquals(ALGORITHM_NAME, objectStore.checksumInfo().algorithm()); + assertEquals(type, objectStore.checksumInfo().checksumType()); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testFileChecksum(ChecksumType type, Configuration conf, URI uri, + ObjectStorage objectStore) throws Exception { + setObjectStorage(objectStore); + + try (RawFileSystem fs = new RawFileSystem()) { + fs.initialize(uri, conf); + Path file = new Path("/" + PREFIX, "testFileChecksum"); + fs.create(file).close(); + FileChecksum checksum = fs.getFileChecksum(file, Long.MAX_VALUE); + assertEquals(ALGORITHM_NAME, checksum.getAlgorithmName()); + + String key = file.toString().substring(1); + byte[] checksumData = objectStore.head(key).checksum(); + assertArrayEquals(checksumData, checksum.getBytes()); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestTosFileSystem.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestTosFileSystem.java new file mode 100644 index 0000000000000..8b70f261416b6 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestTosFileSystem.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.tosfs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; + +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + +public class TestTosFileSystem { + + @BeforeAll + public static void before() { + assumeTrue(TestEnv.checkTestEnabled()); + } + + @Test + public void testUriVerification() throws URISyntaxException, IOException { + Configuration conf = new Configuration(false); + conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://cluster-0/"); + + TosFileSystem tfs = new TosFileSystem(); + assertThrows(IllegalArgumentException.class, + () -> tfs.initialize(new URI("hdfs://cluster/"), conf), "Expect invalid uri error."); + assertThrows(IllegalArgumentException.class, () -> tfs.initialize(new URI("/path"), conf), + "Expect invalid uri error."); + tfs.initialize(new URI(String.format("tos://%s/", TestUtility.bucket())), conf); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/BaseJobSuite.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/BaseJobSuite.java new file mode 100644 index 0000000000000..2ce55281c85d9 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/BaseJobSuite.java @@ -0,0 +1,268 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.object.MultipartUpload; +import org.apache.hadoop.fs.tosfs.object.ObjectInfo; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectUtils; +import org.apache.hadoop.fs.tosfs.util.ParseUtils; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.thirdparty.com.google.common.collect.Iterables; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public abstract class BaseJobSuite { + private static final Logger LOG = LoggerFactory.getLogger(BaseJobSuite.class); + public static final int DEFAULT_APP_ATTEMPT_ID = 1; + protected static final Text KEY_1 = new Text("key1"); + protected static final Text KEY_2 = new Text("key2"); + protected static final Text VAL_1 = new Text("val1"); + protected static final Text VAL_2 = new Text("val2"); + + private Job job; + private String jobId; + private FileSystem fs; + private Path outputPath; + private ObjectStorage storage; + + private final boolean dumpObjectStorage = ParseUtils.envAsBoolean("DUMP_OBJECT_STORAGE", false); + + protected abstract Path magicPartPath(); + + protected abstract Path magicPendingSetPath(); + + protected abstract void assertSuccessMarker() throws IOException; + + protected abstract void assertSummaryReport(Path reportDir) throws IOException; + + protected abstract void assertNoTaskAttemptPath() throws IOException; + + protected void assertMagicPathExist(Path output) throws IOException { + Path magicPath = CommitUtils.magicPath(output); + assertTrue(fs.exists(magicPath), String.format("Magic path: %s should exist", magicPath)); + } + + protected void assertMagicPathNotExist(Path output) throws IOException { + Path magicPath = CommitUtils.magicPath(output); + assertFalse(fs.exists(magicPath), String.format("Magic path: %s should not exist", magicPath)); + } + + protected abstract boolean skipTests(); + + public Path magicPendingPath() { + Path magicPart = magicPartPath(); + return new Path(magicPart.getParent(), magicPart.getName() + ".pending"); + } + + public Path magicJobPath() { + return CommitUtils.magicPath(outputPath); + } + + public String magicPartKey() { + return ObjectUtils.pathToKey(magicPartPath()); + } + + public String destPartKey() { + return MagicOutputStream.toDestKey(magicPartPath()); + } + + public Job job() { + return job; + } + + public String jobId() { + return jobId; + } + + public FileSystem fs() { + return fs; + } + + public Path outputPath() { + return outputPath; + } + + public ObjectStorage storage() { + return storage; + } + + public void setJob(Job value) { + this.job = value; + } + + public void setJobId(String value) { + this.jobId = value; + } + + public void setFs(FileSystem value) { + this.fs = value; + } + + public void setOutputPath(Path value) { + this.outputPath = value; + } + + public void setObjectStorage(ObjectStorage value) { + this.storage = value; + } + + public void assertHasMagicKeys() { + Iterable objects = storage.listAll(ObjectUtils.pathToKey(magicJobPath(), true), ""); + assertTrue( + Iterables.any(objects, o -> o.key().contains(CommitUtils.MAGIC) && o.key().contains(jobId)), + "Should have some __magic object keys"); + } + + public void assertHasBaseKeys() { + Iterable objects = storage.listAll(ObjectUtils.pathToKey(magicJobPath(), true), ""); + assertTrue( + Iterables.any(objects, o -> o.key().contains(CommitUtils.BASE) && o.key().contains(jobId)), + "Should have some __base object keys"); + } + + public void assertNoMagicPendingFile() { + String magicPendingKey = String.format("%s.pending", magicPartKey()); + assertNull(storage.head(magicPendingKey), "Magic pending key should exist"); + } + + public void assertHasMagicPendingFile() { + String magicPendingKey = String.format("%s.pending", magicPartKey()); + assertNotNull(storage.head(magicPendingKey), "Magic pending key should exist"); + } + + public void assertNoMagicMultipartUpload() { + Iterable uploads = + storage.listUploads(ObjectUtils.pathToKey(magicJobPath(), true)); + boolean anyMagicUploads = Iterables.any(uploads, u -> u.key().contains(CommitUtils.MAGIC)); + assertFalse(anyMagicUploads, "Should have no magic multipart uploads"); + } + + public void assertNoMagicObjectKeys() { + Iterable objects = storage.listAll(ObjectUtils.pathToKey(magicJobPath(), true), ""); + boolean anyMagicUploads = + Iterables.any(objects, o -> o.key().contains(CommitUtils.MAGIC) && o.key().contains(jobId)); + assertFalse(anyMagicUploads, "Should not have any magic keys"); + } + + public void assertHasPendingSet() { + Iterable objects = storage.listAll(ObjectUtils.pathToKey(magicJobPath(), true), ""); + boolean anyPendingSet = Iterables.any(objects, + o -> o.key().contains(CommitUtils.PENDINGSET_SUFFIX) && o.key().contains(jobId)); + assertTrue(anyPendingSet, "Should have the expected .pendingset file"); + } + + public void assertPendingSetAtRightLocation() { + Iterable objects = storage.listAll(ObjectUtils.pathToKey(magicJobPath(), true), ""); + Path magicJobAttemptPath = + CommitUtils.magicJobAttemptPath(job().getJobID().toString(), DEFAULT_APP_ATTEMPT_ID, + outputPath); + String inQualifiedPath = magicJobAttemptPath.toUri().getPath().substring(1); + Iterable filtered = Iterables.filter(objects, + o -> o.key().contains(CommitUtils.PENDINGSET_SUFFIX) && o.key().contains(jobId)); + boolean pendingSetAtRightLocation = Iterables.any(filtered, + o -> o.key().startsWith(inQualifiedPath) && o.key().contains(jobId)); + assertTrue(pendingSetAtRightLocation, + "The .pendingset file should locate at the job's magic output path."); + } + + public void assertMultipartUpload(int expectedUploads) { + // Note: should be care in concurrent case: they need to check the same output path. + Iterable uploads = + storage.listUploads(ObjectUtils.pathToKey(outputPath, true)); + long actualUploads = StreamSupport.stream(uploads.spliterator(), false).count(); + assertEquals(expectedUploads, actualUploads); + } + + public void assertPartFiles(int num) throws IOException { + FileStatus[] files = fs.listStatus(outputPath, + f -> !MagicOutputStream.isMagic(new Path(f.toUri())) && f.toUri().toString() + .contains("part-")); + assertEquals(num, files.length); + Iterable objects = storage.listAll(ObjectUtils.pathToKey(outputPath, true), ""); + List infos = Arrays.stream(Iterables.toArray(objects, ObjectInfo.class)) + .filter(o -> o.key().contains("part-")).collect(Collectors.toList()); + assertEquals(num, infos.size(), + String.format("Part files number should be %d, but got %d", num, infos.size())); + } + + public void assertNoPartFiles() throws IOException { + FileStatus[] files = fs.listStatus(outputPath, + f -> !MagicOutputStream.isMagic(new Path(f.toUri())) && f.toUri().toString() + .contains("part-")); + assertEquals(0, files.length); + Iterable objects = storage.listAll(ObjectUtils.pathToKey(outputPath, true), ""); + boolean anyPartFile = Iterables.any(objects, o -> o.key().contains("part-")); + assertFalse(anyPartFile, "Should have no part files"); + } + + public void dumpObjectStorage() { + if (dumpObjectStorage) { + LOG.info("===> Dump object storage - Start <==="); + dumpObjectKeys(); + dumpMultipartUploads(); + LOG.info("===> Dump object storage - End <==="); + } + } + + public void dumpObjectKeys() { + String prefix = ObjectUtils.pathToKey(magicJobPath()); + LOG.info("Dump object keys with prefix {}", prefix); + storage.listAll("", "").forEach(o -> LOG.info("Dump object keys - {}", o)); + } + + public void dumpMultipartUploads() { + String prefix = ObjectUtils.pathToKey(magicJobPath()); + LOG.info("Dump multi part uploads with prefix {}", prefix); + storage.listUploads("") + .forEach(u -> LOG.info("Dump multipart uploads - {}", u)); + } + + public void verifyPartContent() throws IOException { + String partKey = destPartKey(); + LOG.info("Part key to verify is: {}", partKey); + try (InputStream in = storage.get(partKey).stream()) { + byte[] data = IOUtils.toByteArray(in); + String expected = String.format("%s\t%s\n%s\t%s\n", KEY_1, VAL_1, KEY_2, VAL_2); + assertEquals(expected, new String(data, StandardCharsets.UTF_8)); + } + } + + public void assertSuccessMarkerNotExist() throws IOException { + Path succPath = CommitUtils.successMarker(outputPath); + assertFalse(fs.exists(succPath), String.format("%s should not exists", succPath)); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/CommitterTestBase.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/CommitterTestBase.java new file mode 100644 index 0000000000000..93f6ca434fb4d --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/CommitterTestBase.java @@ -0,0 +1,433 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.fs.tosfs.util.UUIDUtils; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.List; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assumptions.assumeFalse; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + +public abstract class CommitterTestBase { + private Configuration conf; + private FileSystem fs; + private Path outputPath; + private TaskAttemptID job1Task0Attempt0; + private TaskAttemptID job2Task1Attempt0; + private Path reportDir; + + @BeforeEach + public void setup() throws IOException { + conf = newConf(); + fs = FileSystem.get(conf); + String uuid = UUIDUtils.random(); + outputPath = fs.makeQualified(new Path("/test/" + uuid)); + job1Task0Attempt0 = JobSuite.createTaskAttemptId(randomTrimmedJobId(), 0, 0); + job2Task1Attempt0 = JobSuite.createTaskAttemptId(randomTrimmedJobId(), 1, 0); + + reportDir = fs.makeQualified(new Path("/report/" + uuid)); + fs.mkdirs(reportDir); + conf.set(Committer.COMMITTER_SUMMARY_REPORT_DIR, reportDir.toUri().toString()); + } + + protected abstract Configuration newConf(); + + @AfterEach + public void teardown() { + CommonUtils.runQuietly(() -> fs.delete(outputPath, true)); + IOUtils.closeStream(fs); + } + + @BeforeAll + public static void before() { + assumeTrue(TestEnv.checkTestEnabled()); + } + + @AfterAll + public static void afterClass() { + List committerThreads = Thread.getAllStackTraces().keySet() + .stream() + .map(Thread::getName) + .filter(n -> n.startsWith(Committer.THREADS_PREFIX)) + .collect(Collectors.toList()); + assertTrue(committerThreads.isEmpty(), "Outstanding committer threads"); + } + + private static String randomTrimmedJobId() { + SimpleDateFormat formatter = new SimpleDateFormat("yyyyMMdd"); + return String.format("%s%04d_%04d", formatter.format(new Date()), + (long) (Math.random() * 1000), + (long) (Math.random() * 1000)); + } + + private static String randomFormedJobId() { + return String.format("job_%s", randomTrimmedJobId()); + } + + @Test + public void testSetupJob() throws IOException { + JobSuite suite = JobSuite.create(conf, job1Task0Attempt0, outputPath); + assumeFalse(suite.skipTests()); + // Setup job. + suite.setupJob(); + suite.dumpObjectStorage(); + suite.assertHasMagicKeys(); + } + + @Test + public void testSetupJobWithOrphanPaths() throws IOException, InterruptedException { + JobSuite suite = JobSuite.create(conf, job1Task0Attempt0, outputPath); + assumeFalse(suite.skipTests()); + // Orphan success marker. + Path successPath = CommitUtils.successMarker(outputPath); + CommitUtils.save(fs, successPath, new byte[]{}); + assertTrue(fs.exists(successPath)); + + // Orphan job path. + Path jobPath = CommitUtils.magicJobPath(suite.committer().jobId(), outputPath); + fs.mkdirs(jobPath); + assertTrue(fs.exists(jobPath), "The job path should be existing"); + Path subPath = new Path(jobPath, "tmp.pending"); + CommitUtils.save(fs, subPath, new byte[]{}); + assertTrue(fs.exists(subPath), "The sub path under job path should be existing."); + FileStatus jobPathStatus = fs.getFileStatus(jobPath); + + Thread.sleep(1000L); + suite.setupJob(); + suite.dumpObjectStorage(); + suite.assertHasMagicKeys(); + + assertFalse(fs.exists(successPath), "Should have deleted the success path"); + assertTrue(fs.exists(jobPath), "Should have re-created the job path"); + assertFalse(fs.exists(subPath), "Should have deleted the sub path under the job path"); + } + + @Test + public void testSetupTask() throws IOException { + JobSuite suite = JobSuite.create(conf, job1Task0Attempt0, outputPath); + assumeFalse(suite.skipTests()); + // Remaining attempt task path. + Path taskAttemptBasePath = + CommitUtils.magicTaskAttemptBasePath(suite.taskAttemptContext(), outputPath); + Path subTaskAttemptPath = new Path(taskAttemptBasePath, "tmp.pending"); + CommitUtils.save(fs, subTaskAttemptPath, new byte[]{}); + assertTrue(fs.exists(taskAttemptBasePath)); + assertTrue(fs.exists(subTaskAttemptPath)); + + // Setup job. + suite.setupJob(); + suite.assertHasMagicKeys(); + // It will clear all the job path once we've set up the job. + assertFalse(fs.exists(taskAttemptBasePath)); + assertFalse(fs.exists(subTaskAttemptPath)); + + // Left some the task paths. + CommitUtils.save(fs, subTaskAttemptPath, new byte[]{}); + assertTrue(fs.exists(taskAttemptBasePath)); + assertTrue(fs.exists(subTaskAttemptPath)); + + // Setup task. + suite.setupTask(); + assertFalse(fs.exists(subTaskAttemptPath)); + } + + @Test + public void testCommitTask() throws Exception { + JobSuite suite = JobSuite.create(conf, job1Task0Attempt0, outputPath); + assumeFalse(suite.skipTests()); + + // Setup job + suite.setupJob(); + suite.dumpObjectStorage(); + suite.assertHasMagicKeys(); + + // Setup task + suite.setupTask(); + + // Write records. + suite.assertNoMagicPendingFile(); + suite.assertMultipartUpload(0); + suite.writeOutput(); + suite.dumpObjectStorage(); + suite.assertHasMagicPendingFile(); + suite.assertNoMagicMultipartUpload(); + suite.assertMultipartUpload(1); + // Assert the pending file content. + Path pendingPath = suite.magicPendingPath(); + byte[] pendingData = CommitUtils.load(suite.fs(), pendingPath); + Pending pending = Pending.deserialize(pendingData); + assertEquals(suite.destPartKey(), pending.destKey()); + assertEquals(20, pending.length()); + assertEquals(1, pending.parts().size()); + + // Commit the task. + suite.commitTask(); + + // Verify the pending set file. + suite.assertHasPendingSet(); + // Assert the pending set file content. + Path pendingSetPath = suite.magicPendingSetPath(); + byte[] pendingSetData = CommitUtils.load(suite.fs(), pendingSetPath); + PendingSet pendingSet = PendingSet.deserialize(pendingSetData); + assertEquals(suite.job().getJobID().toString(), pendingSet.jobId()); + assertEquals(1, pendingSet.commits().size()); + assertEquals(pending, pendingSet.commits().get(0)); + assertEquals(pendingSet.extraData(), ImmutableMap.of(CommitUtils.TASK_ATTEMPT_ID, + suite.taskAttemptContext().getTaskAttemptID().toString())); + + // Complete the multipart upload and verify the results. + ObjectStorage storage = suite.storage(); + storage.completeUpload(pending.destKey(), pending.uploadId(), pending.parts()); + suite.verifyPartContent(); + } + + @Test + public void testAbortTask() throws Exception { + JobSuite suite = JobSuite.create(conf, job1Task0Attempt0, outputPath); + assumeFalse(suite.skipTests()); + suite.setupJob(); + suite.setupTask(); + + // Pre-check before the output write. + suite.assertNoMagicPendingFile(); + suite.assertMultipartUpload(0); + + // Execute the output write. + suite.writeOutput(); + + // Post-check after the output write. + suite.assertHasMagicPendingFile(); + suite.assertNoMagicMultipartUpload(); + suite.assertMultipartUpload(1); + // Assert the pending file content. + Path pendingPath = suite.magicPendingPath(); + byte[] pendingData = CommitUtils.load(suite.fs(), pendingPath); + Pending pending = Pending.deserialize(pendingData); + assertEquals(suite.destPartKey(), pending.destKey()); + assertEquals(20, pending.length()); + assertEquals(1, pending.parts().size()); + + // Abort the task. + suite.abortTask(); + + // Verify the state after aborting task. + suite.assertNoMagicPendingFile(); + suite.assertNoMagicMultipartUpload(); + suite.assertMultipartUpload(0); + suite.assertNoTaskAttemptPath(); + } + + @Test + public void testCommitJob() throws Exception { + JobSuite suite = JobSuite.create(conf, job1Task0Attempt0, outputPath); + assumeFalse(suite.skipTests()); + suite.setupJob(); + suite.setupTask(); + suite.writeOutput(); + suite.commitTask(); + + // Commit the job. + suite.assertNoPartFiles(); + suite.commitJob(); + // Verify the output. + suite.assertNoMagicMultipartUpload(); + suite.assertNoMagicObjectKeys(); + suite.assertSuccessMarker(); + suite.assertSummaryReport(reportDir); + suite.verifyPartContent(); + } + + + @Test + public void testCommitJobFailed() throws Exception { + JobSuite suite = JobSuite.create(conf, job1Task0Attempt0, outputPath); + assumeFalse(suite.skipTests()); + suite.setupJob(); + suite.setupTask(); + suite.writeOutput(); + suite.commitTask(); + + // Commit the job. + suite.assertNoPartFiles(); + suite.commitJob(); + } + + @Test + public void testCommitJobSuccessMarkerFailed() throws Exception { + JobSuite suite = JobSuite.create(conf, job1Task0Attempt0, outputPath); + assumeFalse(suite.skipTests()); + suite.setupJob(); + suite.setupTask(); + suite.writeOutput(); + suite.commitTask(); + + CommitUtils.injectError("marker"); + // Commit the job. + suite.assertNoPartFiles(); + assertThrows(IOException.class, suite::commitJob, "Expect commit job error."); + CommitUtils.removeError("marker"); + + // Verify the output. + suite.assertNoMagicMultipartUpload(); + suite.assertNoMagicObjectKeys(); + suite.assertSuccessMarkerNotExist(); + assertEquals(0, suite.fs().listStatus(suite.outputPath()).length); + } + + @Test + public void testTaskCommitAfterJobCommit() throws Exception { + JobSuite suite = JobSuite.create(conf, job1Task0Attempt0, outputPath); + assumeFalse(suite.skipTests()); + + suite.setupJob(); + suite.setupTask(); + suite.writeOutput(); + suite.commitTask(); + + // Commit the job + suite.assertNoPartFiles(); + suite.commitJob(); + // Verify the output. + suite.assertNoMagicMultipartUpload(); + suite.assertNoMagicObjectKeys(); + suite.assertSuccessMarker(); + suite.verifyPartContent(); + + // Commit the task again. + assertThrows(FileNotFoundException.class, suite::commitTask); + } + + @Test + public void testTaskCommitWithConsistentJobId() throws Exception { + Configuration config = newConf(); + String consistentJobId = randomFormedJobId(); + config.set(CommitUtils.SPARK_WRITE_UUID, consistentJobId); + JobSuite suite = JobSuite.create(config, job1Task0Attempt0, outputPath); + assumeFalse(suite.skipTests()); + + // By now, we have two "jobId"s, one is spark uuid, and the other is the jobId in taskAttempt. + // The job committer will adopt the former. + suite.setupJob(); + + // Next, we clear spark uuid, and set the jobId of taskAttempt to another value. In this case, + // the committer will take the jobId of taskAttempt as the final jobId, which is not consistent + // with the one that committer holds. + config.unset(CommitUtils.SPARK_WRITE_UUID); + String anotherJobId = randomTrimmedJobId(); + TaskAttemptID taskAttemptId1 = + JobSuite.createTaskAttemptId(anotherJobId, JobSuite.DEFAULT_APP_ATTEMPT_ID); + final TaskAttemptContext attemptContext1 = + JobSuite.createTaskAttemptContext(config, taskAttemptId1, JobSuite.DEFAULT_APP_ATTEMPT_ID); + + assertThrows(IllegalArgumentException.class, () -> suite.setupTask(attemptContext1), + "JobId set in the context"); + + // Even though we use another taskAttempt, as long as we ensure the spark uuid is consistent, + // the jobId in committer is consistent. + config.set(CommitUtils.SPARK_WRITE_UUID, consistentJobId); + config.set(FileOutputFormat.OUTDIR, outputPath.toString()); + anotherJobId = randomTrimmedJobId(); + TaskAttemptID taskAttemptId2 = + JobSuite.createTaskAttemptId(anotherJobId, JobSuite.DEFAULT_APP_ATTEMPT_ID); + TaskAttemptContext attemptContext2 = + JobSuite.createTaskAttemptContext(config, taskAttemptId2, JobSuite.DEFAULT_APP_ATTEMPT_ID); + + suite.setupTask(attemptContext2); + // Write output must use the same task context with setup task. + suite.writeOutput(attemptContext2); + // Commit task must use the same task context with setup task. + suite.commitTask(attemptContext2); + suite.assertPendingSetAtRightLocation(); + + // Commit the job + suite.assertNoPartFiles(); + suite.commitJob(); + + // Verify the output. + suite.assertNoMagicMultipartUpload(); + suite.assertNoMagicObjectKeys(); + suite.assertSuccessMarker(); + suite.verifyPartContent(); + } + + @Test + public void testConcurrentJobs() throws Exception { + JobSuite suite1 = JobSuite.create(conf, job1Task0Attempt0, outputPath); + JobSuite suite2 = JobSuite.create(conf, job2Task1Attempt0, outputPath); + assumeFalse(suite1.skipTests()); + assumeFalse(suite2.skipTests()); + suite1.setupJob(); + suite2.setupJob(); + suite1.setupTask(); + suite2.setupTask(); + suite1.writeOutput(); + suite2.writeOutput(); + suite1.commitTask(); + suite2.commitTask(); + + // Job2 commit the job. + suite2.assertNoPartFiles(); + suite2.commitJob(); + suite2.assertPartFiles(1); + + suite2.assertNoMagicMultipartUpload(); + suite2.assertNoMagicObjectKeys(); + suite2.assertSuccessMarker(); + suite2.assertSummaryReport(reportDir); + suite2.verifyPartContent(); + suite2.assertMagicPathExist(outputPath); + + // Job1 commit the job. + suite1.commitJob(); + suite2.assertPartFiles(2); + + // Verify the output. + suite1.assertNoMagicMultipartUpload(); + suite1.assertNoMagicObjectKeys(); + suite1.assertSuccessMarker(); + suite1.assertSummaryReport(reportDir); + suite1.verifyPartContent(); + suite1.assertMagicPathNotExist(outputPath); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/JobSuite.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/JobSuite.java new file mode 100644 index 0000000000000..9582179646105 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/JobSuite.java @@ -0,0 +1,228 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.object.ObjectStorageFactory; +import org.apache.hadoop.fs.tosfs.object.ObjectUtils; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.JobID; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.mapreduce.TaskType; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory; +import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; +import org.apache.hadoop.mapreduce.task.JobContextImpl; +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; +import org.apache.hadoop.net.NetUtils; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public final class JobSuite extends BaseJobSuite { + private static final CommitterFactory FACTORY = new CommitterFactory(); + private final JobContext jobContext; + private final TaskAttemptContext taskAttemptContext; + private final Committer committer; + + private JobSuite(FileSystem fs, Configuration conf, TaskAttemptID taskAttemptId, int appAttemptId, + Path outputPath) throws IOException { + setFs(fs); + // Initialize the job instance. + setJob(Job.getInstance(conf)); + job().setJobID(JobID.forName(CommitUtils.buildJobId(conf, taskAttemptId.getJobID()))); + this.jobContext = createJobContext(job().getConfiguration(), taskAttemptId); + setJobId(CommitUtils.buildJobId(jobContext)); + this.taskAttemptContext = + createTaskAttemptContext(job().getConfiguration(), taskAttemptId, appAttemptId); + + // Set job output directory. + FileOutputFormat.setOutputPath(job(), outputPath); + setOutputPath(outputPath); + setObjectStorage(ObjectStorageFactory.create(outputPath.toUri().getScheme(), + outputPath.toUri().getAuthority(), conf)); + + // Initialize committer. + this.committer = (Committer) FACTORY.createOutputCommitter(outputPath, taskAttemptContext); + } + + public static JobSuite create(Configuration conf, TaskAttemptID taskAttemptId, Path outDir) + throws IOException { + FileSystem fs = outDir.getFileSystem(conf); + return new JobSuite(fs, conf, taskAttemptId, DEFAULT_APP_ATTEMPT_ID, outDir); + } + + public static TaskAttemptID createTaskAttemptId(String trimmedJobId, int attemptId) { + String attempt = String.format("attempt_%s_m_000000_%d", trimmedJobId, attemptId); + return TaskAttemptID.forName(attempt); + } + + public static TaskAttemptID createTaskAttemptId(String trimmedJobId, int taskId, int attemptId) { + String[] parts = trimmedJobId.split("_"); + return new TaskAttemptID(parts[0], Integer.parseInt(parts[1]), TaskType.MAP, taskId, attemptId); + } + + public static JobContext createJobContext(Configuration jobConf, TaskAttemptID taskAttemptId) { + return new JobContextImpl(jobConf, taskAttemptId.getJobID()); + } + + public static TaskAttemptContext createTaskAttemptContext( + Configuration jobConf, TaskAttemptID taskAttemptId, int appAttemptId) throws IOException { + // Set the key values for job configuration. + jobConf.set(MRJobConfig.TASK_ATTEMPT_ID, taskAttemptId.toString()); + jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, appAttemptId); + jobConf.set(PathOutputCommitterFactory.COMMITTER_FACTORY_CLASS, + CommitterFactory.class.getName()); + return new TaskAttemptContextImpl(jobConf, taskAttemptId); + } + + public void setupJob() throws IOException { + committer.setupJob(jobContext); + } + + public void setupTask() throws IOException { + committer.setupTask(taskAttemptContext); + } + + // This method simulates the scenario that the job may set up task with a different + // taskAttemptContext, e.g., for a spark job. + public void setupTask(TaskAttemptContext taskAttemptCxt) throws IOException { + committer.setupTask(taskAttemptCxt); + } + + public void writeOutput() throws Exception { + writeOutput(taskAttemptContext); + } + + // This method simulates the scenario that the job may set up task with a different + // taskAttemptContext, e.g., for a spark job. + public void writeOutput(TaskAttemptContext taskAttemptCxt) throws Exception { + RecordWriter writer = new TextOutputFormat<>().getRecordWriter(taskAttemptCxt); + NullWritable nullKey = NullWritable.get(); + NullWritable nullVal = NullWritable.get(); + Object[] keys = new Object[]{KEY_1, nullKey, null, nullKey, null, KEY_2}; + Object[] vals = new Object[]{VAL_1, nullVal, null, null, nullVal, VAL_2}; + try { + assertEquals(keys.length, vals.length); + for (int i = 0; i < keys.length; i++) { + writer.write(keys[i], vals[i]); + } + } finally { + writer.close(taskAttemptCxt); + } + } + + public boolean needsTaskCommit() { + return committer.needsTaskCommit(taskAttemptContext); + } + + public void commitTask() throws IOException { + committer.commitTask(taskAttemptContext); + } + + // This method simulates the scenario that the job may set up task with a different + // taskAttemptContext, e.g., for a spark job. + public void commitTask(TaskAttemptContext taskAttemptCxt) throws IOException { + committer.commitTask(taskAttemptCxt); + } + + public void abortTask() throws IOException { + committer.abortTask(taskAttemptContext); + } + + public void commitJob() throws IOException { + committer.commitJob(jobContext); + } + + @Override + public Path magicPartPath() { + return new Path(committer.getWorkPath(), + FileOutputFormat.getUniqueFile(taskAttemptContext, "part", "")); + } + + @Override + public Path magicPendingSetPath() { + return CommitUtils.magicTaskPendingSetPath(taskAttemptContext, outputPath()); + } + + public TaskAttemptContext taskAttemptContext() { + return taskAttemptContext; + } + + public Committer committer() { + return committer; + } + + @Override + public void assertNoTaskAttemptPath() throws IOException { + Path path = CommitUtils.magicTaskAttemptBasePath(taskAttemptContext, outputPath()); + assertFalse(fs().exists(path), "Task attempt path should be not existing"); + String pathToKey = ObjectUtils.pathToKey(path); + assertNull(storage().head(pathToKey), "Should have no task attempt path key"); + } + + @Override + protected boolean skipTests() { + return storage().bucket().isDirectory(); + } + + @Override + public void assertSuccessMarker() throws IOException { + Path succPath = CommitUtils.successMarker(outputPath()); + assertTrue(fs().exists(succPath), String.format("%s should be exists", succPath)); + SuccessData successData = SuccessData.deserialize(CommitUtils.load(fs(), succPath)); + assertEquals(SuccessData.class.getName(), successData.name()); + assertTrue(successData.success()); + assertEquals(NetUtils.getHostname(), successData.hostname()); + assertEquals(CommitUtils.COMMITTER_NAME, successData.committer()); + assertEquals( + String.format("Task committer %s", taskAttemptContext.getTaskAttemptID()), + successData.description()); + assertEquals(job().getJobID().toString(), successData.jobId()); + assertEquals(1, successData.filenames().size()); + assertEquals(destPartKey(), successData.filenames().get(0)); + } + + @Override + public void assertSummaryReport(Path reportDir) throws IOException { + Path reportPath = CommitUtils.summaryReport(reportDir, job().getJobID().toString()); + assertTrue(fs().exists(reportPath), String.format("%s should be exists", reportPath)); + SuccessData reportData = SuccessData.deserialize(CommitUtils.load(fs(), reportPath)); + assertEquals(SuccessData.class.getName(), reportData.name()); + assertTrue(reportData.success()); + assertEquals(NetUtils.getHostname(), reportData.hostname()); + assertEquals(CommitUtils.COMMITTER_NAME, reportData.committer()); + assertEquals( + String.format("Task committer %s", taskAttemptContext.getTaskAttemptID()), + reportData.description()); + assertEquals(job().getJobID().toString(), reportData.jobId()); + assertEquals(1, reportData.filenames().size()); + assertEquals(destPartKey(), reportData.filenames().get(0)); + assertEquals("clean", reportData.diagnostics().get("stage")); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/MRJobTestBase.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/MRJobTestBase.java new file mode 100644 index 0000000000000..ae887051ada08 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/MRJobTestBase.java @@ -0,0 +1,241 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.examples.terasort.TeraGen; +import org.apache.hadoop.examples.terasort.TeraSort; +import org.apache.hadoop.examples.terasort.TeraSortConfigKeys; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.fs.tosfs.object.ObjectInfo; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectStorageFactory; +import org.apache.hadoop.fs.tosfs.object.ObjectUtils; +import org.apache.hadoop.fs.tosfs.util.UUIDUtils; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.WordCount; +import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster; +import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig; +import org.apache.hadoop.util.ToolRunner; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + +public abstract class MRJobTestBase { + private static final Logger LOG = LoggerFactory.getLogger(MRJobTestBase.class); + + private static Configuration conf = new Configuration(); + private static MiniMRYarnCluster yarnCluster; + + private static FileSystem fs; + + private static Path testDataPath; + + public static void setConf(Configuration newConf) { + conf = newConf; + } + + @BeforeAll + public static void beforeClass() throws IOException { + assumeTrue(TestEnv.checkTestEnabled()); + + conf.setBoolean(JHAdminConfig.MR_HISTORY_CLEANER_ENABLE, false); + conf.setBoolean(YarnConfiguration.NM_DISK_HEALTH_CHECK_ENABLE, false); + conf.setInt(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 100); + + conf.set("mapreduce.outputcommitter.factory.scheme.tos", + CommitterFactory.class.getName()); // 3x newApiCommitter=true. + conf.set("mapred.output.committer.class", + Committer.class.getName()); // 2x and 3x newApiCommitter=false. + conf.set("mapreduce.outputcommitter.class", + org.apache.hadoop.fs.tosfs.commit.Committer.class.getName()); // 2x newApiCommitter=true. + + // Start the yarn cluster. + yarnCluster = new MiniMRYarnCluster("yarn-" + System.currentTimeMillis(), 2); + LOG.info("Default filesystem: {}", conf.get("fs.defaultFS")); + LOG.info("Default filesystem implementation: {}", conf.get("fs.AbstractFileSystem.tos.impl")); + + yarnCluster.init(conf); + yarnCluster.start(); + + fs = FileSystem.get(conf); + testDataPath = new Path("/mr-test-" + UUIDUtils.random()) + .makeQualified(fs.getUri(), fs.getWorkingDirectory()); + } + + @AfterAll + public static void afterClass() throws IOException { + if (!TestEnv.checkTestEnabled()) { + return; + } + + fs.delete(testDataPath, true); + if (yarnCluster != null) { + yarnCluster.stop(); + } + } + + @AfterEach + public void after() throws IOException { + } + + @Test + public void testTeraGen() throws Exception { + Path teraGenPath = + new Path(testDataPath, "teraGen").makeQualified(fs.getUri(), fs.getWorkingDirectory()); + Path output = new Path(teraGenPath, "output"); + JobConf jobConf = new JobConf(yarnCluster.getConfig()); + jobConf.addResource(conf); + jobConf.setInt(TeraSortConfigKeys.SAMPLE_SIZE.key(), 1000); + jobConf.setInt(TeraSortConfigKeys.NUM_PARTITIONS.key(), 10); + jobConf.setBoolean(TeraSortConfigKeys.USE_SIMPLE_PARTITIONER.key(), false); + + String[] args = new String[]{Integer.toString(1000), output.toString()}; + int result = ToolRunner.run(jobConf, new TeraGen(), args); + assertEquals(0, result, String.format("teragen %s", StringUtils.join(" ", args))); + + // Verify the success data. + ObjectStorage storage = ObjectStorageFactory.create( + output.toUri().getScheme(), output.toUri().getAuthority(), conf); + int byteSizes = 0; + + Path success = new Path(output, CommitUtils.SUCCESS); + byte[] serializedData = CommitUtils.load(fs, success); + SuccessData successData = SuccessData.deserialize(serializedData); + assertTrue(successData.success(), "Should execute successfully"); + // Assert the destination paths. + assertEquals(2, successData.filenames().size()); + successData.filenames().sort(String::compareTo); + assertEquals(ObjectUtils.pathToKey(new Path(output, "part-m-00000")), + successData.filenames().get(0)); + assertEquals(ObjectUtils.pathToKey(new Path(output, "part-m-00001")), + successData.filenames().get(1)); + + for (String partFileKey : successData.filenames()) { + ObjectInfo objectInfo = storage.head(partFileKey); + assertNotNull(objectInfo, "Output file should be existing"); + byteSizes += objectInfo.size(); + } + + assertEquals(byteSizes, 100 /* Each row 100 bytes */ * 1000 /* total 1000 rows */); + } + + @Test + public void testTeraSort() throws Exception { + Path teraGenPath = + new Path(testDataPath, "teraGen").makeQualified(fs.getUri(), fs.getWorkingDirectory()); + Path inputPath = new Path(teraGenPath, "output"); + Path outputPath = new Path(teraGenPath, "sortOutput"); + JobConf jobConf = new JobConf(yarnCluster.getConfig()); + jobConf.addResource(conf); + jobConf.setInt(TeraSortConfigKeys.SAMPLE_SIZE.key(), 1000); + jobConf.setInt(TeraSortConfigKeys.NUM_PARTITIONS.key(), 10); + jobConf.setBoolean(TeraSortConfigKeys.USE_SIMPLE_PARTITIONER.key(), false); + String[] args = new String[]{inputPath.toString(), outputPath.toString()}; + int result = ToolRunner.run(jobConf, new TeraSort(), args); + assertEquals(0, result, String.format("terasort %s", StringUtils.join(" ", args))); + + // Verify the success data. + ObjectStorage storage = ObjectStorageFactory + .create(outputPath.toUri().getScheme(), outputPath.toUri().getAuthority(), conf); + int byteSizes = 0; + + Path success = new Path(outputPath, CommitUtils.SUCCESS); + byte[] serializedData = CommitUtils.load(fs, success); + SuccessData successData = SuccessData.deserialize(serializedData); + assertTrue(successData.success(), "Should execute successfully"); + // Assert the destination paths. + assertEquals(1, successData.filenames().size()); + successData.filenames().sort(String::compareTo); + assertEquals(ObjectUtils.pathToKey(new Path(outputPath, "part-r-00000")), + successData.filenames().get(0)); + + for (String partFileKey : successData.filenames()) { + ObjectInfo objectInfo = storage.head(partFileKey); + assertNotNull(objectInfo, "Output file should be existing"); + byteSizes += objectInfo.size(); + } + + assertEquals(byteSizes, 100 /* Each row 100 bytes */ * 1000 /* total 1000 rows */); + } + + @Disabled + @Test + public void testWordCount() throws Exception { + Path wordCountPath = + new Path(testDataPath, "wc").makeQualified(fs.getUri(), fs.getWorkingDirectory()); + Path output = new Path(wordCountPath, "output"); + Path input = new Path(wordCountPath, "input"); + JobConf jobConf = new JobConf(yarnCluster.getConfig()); + jobConf.addResource(conf); + + if (!fs.mkdirs(input)) { + throw new IOException("Mkdirs failed to create " + input.toString()); + } + + DataOutputStream file = fs.create(new Path(input, "part-0")); + file.writeBytes("a a b c"); + file.close(); + + String[] args = new String[]{input.toString(), output.toString()}; + int result = ToolRunner.run(jobConf, new WordCount(), args); + assertEquals(0, result, String.format("WordCount %s", StringUtils.join(" ", args))); + + // Verify the success path. + assertTrue(fs.exists(new Path(output, CommitUtils.SUCCESS))); + assertTrue(fs.exists(new Path(output, "part-00000"))); + + Path success = new Path(output, CommitUtils.SUCCESS); + assertTrue(CommitUtils.load(fs, success).length != 0, "Success file must be not empty"); + + byte[] serializedData = CommitUtils.load(fs, new Path(output, "part-00000")); + String outputAsStr = new String(serializedData); + Map resAsMap = getResultAsMap(outputAsStr); + assertEquals(2, (int) resAsMap.get("a")); + assertEquals(1, (int) resAsMap.get("b")); + assertEquals(1, (int) resAsMap.get("c")); + } + + private Map getResultAsMap(String outputAsStr) { + Map result = new HashMap<>(); + for (String line : outputAsStr.split("\n")) { + String[] tokens = line.split("\t"); + assertTrue(tokens.length > 1, + String.format("Not enough tokens in in string %s from output %s", line, outputAsStr)); + result.put(tokens[0], Integer.parseInt(tokens[1])); + } + return result; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/TestCommitter.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/TestCommitter.java new file mode 100644 index 0000000000000..017b98e9564e5 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/TestCommitter.java @@ -0,0 +1,29 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.util.TestUtility; + +public class TestCommitter extends CommitterTestBase { + @Override + protected Configuration newConf() { + Configuration conf = new Configuration(); + conf.set("fs.defaultFS", String.format("tos://%s", TestUtility.bucket())); + return conf; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/TestMRJob.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/TestMRJob.java new file mode 100644 index 0000000000000..a27cb71206ebd --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/TestMRJob.java @@ -0,0 +1,49 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.conf.TosKeys; +import org.apache.hadoop.fs.tosfs.object.tos.TOS; +import org.apache.hadoop.fs.tosfs.util.ParseUtils; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.junit.jupiter.api.BeforeAll; + +import java.io.IOException; + +public class TestMRJob extends MRJobTestBase { + + @BeforeAll + public static void beforeClass() throws IOException { + // Create the new configuration and set it to the IT Case. + Configuration newConf = new Configuration(); + newConf.set("fs.defaultFS", String.format("tos://%s", TestUtility.bucket())); + // Application in yarn cluster cannot read the environment variables from user bash, so here we + // set it into the config manually. + newConf.set(ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key("tos"), + ParseUtils.envAsString(TOS.ENV_TOS_ENDPOINT, false)); + newConf.set(TosKeys.FS_TOS_ACCESS_KEY_ID, + ParseUtils.envAsString(TOS.ENV_TOS_ACCESS_KEY_ID, false)); + newConf.set(TosKeys.FS_TOS_SECRET_ACCESS_KEY, + ParseUtils.envAsString(TOS.ENV_TOS_SECRET_ACCESS_KEY, false)); + + MRJobTestBase.setConf(newConf); + // Continue to prepare the IT Case environments. + MRJobTestBase.beforeClass(); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/TestMagicOutputStream.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/TestMagicOutputStream.java new file mode 100644 index 0000000000000..0844b130bebbb --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/TestMagicOutputStream.java @@ -0,0 +1,203 @@ +/* + * 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.tosfs.commit; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.common.ThreadPools; +import org.apache.hadoop.fs.tosfs.object.MultipartUpload; +import org.apache.hadoop.fs.tosfs.object.ObjectStorageTestBase; +import org.apache.hadoop.fs.tosfs.object.ObjectUtils; +import org.apache.hadoop.fs.tosfs.object.Part; +import org.apache.hadoop.fs.tosfs.object.staging.StagingPart; +import org.apache.hadoop.fs.tosfs.object.staging.State; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import java.util.concurrent.ExecutorService; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + +public class TestMagicOutputStream extends ObjectStorageTestBase { + + private static ExecutorService threadPool; + + @BeforeAll + public static void beforeClass() { + threadPool = ThreadPools.newWorkerPool("TestMagicOutputStream-pool"); + } + + @AfterAll + public static void afterClass() { + if (!threadPool.isShutdown()) { + threadPool.shutdown(); + } + } + + private static Path path(String p) { + return new Path(p); + } + + private static Path path(Path parent, String child) { + return new Path(parent, child); + } + + @Test + public void testCreateDestKey() { + Object[][] testCases = new Object[][]{ + new Object[]{path("tos://bucket/__magic/a.txt"), "a.txt"}, + new Object[] {path("tos://bucket/output/__magic/job-1/tasks/tasks-attempt-0/a.txt"), + "output/a.txt"}, + new Object[]{path("tos://bucket/__magic/job0/task0/__base/a.txt"), "a.txt"}, + new Object[] {path("tos://bucket/output/__magic/job0/task0/__base/part/part-m-1000"), + "output/part/part-m-1000"}, + new Object[]{path("tos://bucket/a/b/c/__magic/__base/d/e/f"), "a/b/c/d/e/f"}, + new Object[]{path("tos://bucket/a/b/c/__magic/d/e/f"), "a/b/c/f"}, + }; + + for (Object[] input : testCases) { + String actualDestKey = MagicOutputStream.toDestKey((Path) input[0]); + assertEquals(actualDestKey, input[1], "Unexpected destination key."); + } + } + + @Test + public void testNonMagicPath() { + try (MagicOutputStream ignored = new TestingMagicOutputStream(path(testDir(), "non-magic"))) { + fail("Cannot create magic output stream for non-magic path"); + } catch (Exception ignored) { + } + } + + @Test + public void testWriteZeroByte() throws IOException { + Path magic = path(path(testDir(), CommitUtils.MAGIC), "zero-byte.txt"); + MagicOutputStream out = new TestingMagicOutputStream(magic); + // write zero-byte and close. + out.close(); + assertStagingFiles(0, out.stagingParts()); + + // Read and validate the .pending contents + try (InputStream in = getStorage().get(out.pendingKey()).stream()) { + byte[] data = IOUtils.toByteArray(in); + Pending commit = Pending.deserialize(data); + assertEquals(getStorage().bucket().name(), commit.bucket()); + assertEquals(out.destKey(), commit.destKey()); + assertTrue(StringUtils.isNoneEmpty(commit.uploadId())); + assertTrue(commit.createdTimestamp() > 0); + assertEquals(1, commit.parts().size()); + assertEquals(0, commit.length()); + assertEquals(out.upload().uploadId(), commit.uploadId()); + } + } + + public void testWrite(int len) throws IOException { + Path magic = path(path(testDir(), CommitUtils.MAGIC), len + ".txt"); + int uploadPartSize = 8 << 20; + int partNum = (len - 1) / (8 << 20) + 1; + + MagicOutputStream out = new TestingMagicOutputStream(magic); + byte[] data = TestUtility.rand(len); + out.write(data); + out.close(); + + assertStagingFiles(partNum, out.stagingParts()); + assertEquals(ObjectUtils.pathToKey(magic) + CommitUtils.PENDING_SUFFIX, out.pendingKey()); + + Pending commit; + try (InputStream in = getStorage().get(out.pendingKey()).stream()) { + byte[] serializedData = IOUtils.toByteArray(in); + commit = Pending.deserialize(serializedData); + assertEquals(getStorage().bucket().name(), commit.bucket()); + assertEquals(out.destKey(), commit.destKey()); + assertTrue(commit.createdTimestamp() > 0); + assertEquals(len, commit.length()); + assertEquals(out.upload().uploadId(), commit.uploadId()); + // Verify the upload part list. + assertEquals(partNum, commit.parts().size()); + if (!commit.parts().isEmpty()) { + for (int i = 0; i < partNum - 1; i += 1) { + assertEquals(uploadPartSize, commit.parts().get(i).size()); + } + Part lastPart = commit.parts().get(partNum - 1); + assertTrue(lastPart.size() > 0 && lastPart.size() <= uploadPartSize); + } + } + + // List multipart uploads + int uploadsNum = 0; + for (MultipartUpload upload : getStorage().listUploads(out.destKey())) { + uploadsNum += 1; + assertEquals(out.upload(), upload); + } + assertEquals(1L, uploadsNum); + + // The target object is still not visible for object storage. + assertNull(getStorage().head(out.destKey())); + + // Complete the upload and validate the content. + getStorage().completeUpload(out.destKey(), out.upload().uploadId(), commit.parts()); + try (InputStream in = getStorage().get(out.destKey()).stream()) { + assertArrayEquals(data, IOUtils.toByteArray(in)); + } + } + + @Test + public void testWrite1MB() throws IOException { + testWrite(1 << 20); + } + + @Test + public void testWrite24MB() throws IOException { + testWrite(24 << 20); + } + + @Test + public void testWrite100MB() throws IOException { + testWrite(100 << 20); + } + + private static void assertStagingFiles(int expectedNum, List stagings) { + assertEquals(expectedNum, stagings.size()); + for (StagingPart staging : stagings) { + assertEquals(State.CLEANED, staging.state()); + } + } + + private class TestingMagicOutputStream extends MagicOutputStream { + + TestingMagicOutputStream(Path magic) { + super(fs(), getStorage(), threadPool, tosConf(), magic); + } + + protected void persist(Path p, byte[] data) { + getStorage().put(ObjectUtils.pathToKey(p), data); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/mapred/CommitterTestBase.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/mapred/CommitterTestBase.java new file mode 100644 index 0000000000000..e34088c23d2b2 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/mapred/CommitterTestBase.java @@ -0,0 +1,381 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit.mapred; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.fs.tosfs.commit.CommitUtils; +import org.apache.hadoop.fs.tosfs.commit.Pending; +import org.apache.hadoop.fs.tosfs.commit.PendingSet; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.fs.tosfs.util.UUIDUtils; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.TaskAttemptContext; +import org.apache.hadoop.mapred.TaskAttemptID; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assumptions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.List; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public abstract class CommitterTestBase { + private Configuration conf; + private FileSystem fs; + private Path outputPath; + private TaskAttemptID taskAttempt0; + private Path reportDir; + + @BeforeEach + public void setup() throws IOException { + conf = newConf(); + fs = FileSystem.get(conf); + String uuid = UUIDUtils.random(); + outputPath = fs.makeQualified(new Path("/test/" + uuid)); + taskAttempt0 = JobSuite.createTaskAttemptId(randomTrimmedJobId(), 0); + + reportDir = fs.makeQualified(new Path("/report/" + uuid)); + fs.mkdirs(reportDir); + conf.set(org.apache.hadoop.fs.tosfs.commit.Committer.COMMITTER_SUMMARY_REPORT_DIR, + reportDir.toUri().toString()); + } + + protected abstract Configuration newConf(); + + @AfterEach + public void teardown() { + CommonUtils.runQuietly(() -> fs.delete(outputPath, true)); + IOUtils.closeStream(fs); + } + + @BeforeClass + public static void beforeClass() { + Assumptions.assumeTrue(TestEnv.checkTestEnabled()); + } + + @AfterAll + public static void afterClass() { + if (!TestEnv.checkTestEnabled()) { + return; + } + + List committerThreads = Thread.getAllStackTraces().keySet() + .stream() + .map(Thread::getName) + .filter(n -> n.startsWith(org.apache.hadoop.fs.tosfs.commit.Committer.THREADS_PREFIX)) + .collect(Collectors.toList()); + assertTrue(committerThreads.isEmpty(), "Outstanding committer threads"); + } + + private static String randomTrimmedJobId() { + SimpleDateFormat formatter = new SimpleDateFormat("yyyyMMdd"); + return String.format("%s%04d_%04d", formatter.format(new Date()), + (long) (Math.random() * 1000), + (long) (Math.random() * 1000)); + } + + private static String randomFormedJobId() { + return String.format("job_%s", randomTrimmedJobId()); + } + + @Test + public void testSetupJob() throws IOException { + JobSuite suite = JobSuite.create(conf, taskAttempt0, outputPath); + Assume.assumeFalse(suite.skipTests()); + + // Setup job. + suite.setupJob(); + suite.dumpObjectStorage(); + suite.assertHasMagicKeys(); + } + + @Test + public void testSetupJobWithOrphanPaths() throws IOException, InterruptedException { + JobSuite suite = JobSuite.create(conf, taskAttempt0, outputPath); + Assume.assumeFalse(suite.skipTests()); + + // Orphan success marker. + Path successPath = CommitUtils.successMarker(outputPath); + CommitUtils.save(fs, successPath, new byte[]{}); + assertTrue(fs.exists(successPath), "The success file should exist."); + + // Orphan job path. + Path jobPath = CommitUtils.magicJobPath(suite.committer().jobId(), outputPath); + fs.mkdirs(jobPath); + assertTrue(fs.exists(jobPath), "The job path should exist."); + Path subPath = new Path(jobPath, "tmp.pending"); + CommitUtils.save(fs, subPath, new byte[]{}); + assertTrue(fs.exists(subPath), "The sub path under job path should be existing."); + FileStatus jobPathStatus = fs.getFileStatus(jobPath); + + Thread.sleep(1000L); + suite.setupJob(); + suite.dumpObjectStorage(); + suite.assertHasMagicKeys(); + + assertFalse(fs.exists(successPath), "Should have deleted the success path"); + assertTrue(fs.exists(jobPath), "Should have re-created the job path"); + assertFalse(fs.exists(subPath), "Should have deleted the sub path under the job path"); + } + + @Test + public void testSetupTask() throws IOException { + JobSuite suite = JobSuite.create(conf, taskAttempt0, outputPath); + Assume.assumeFalse(suite.skipTests()); + + // Remaining attempt task path. + Path taskAttemptBasePath = + CommitUtils.magicTaskAttemptBasePath(suite.taskAttemptContext(), outputPath); + Path subTaskAttemptPath = new Path(taskAttemptBasePath, "tmp.pending"); + CommitUtils.save(fs, subTaskAttemptPath, new byte[]{}); + assertTrue(fs.exists(taskAttemptBasePath)); + assertTrue(fs.exists(subTaskAttemptPath)); + + // Setup job. + suite.setupJob(); + suite.assertHasMagicKeys(); + // It will clear all the job path once we've set up the job. + assertFalse(fs.exists(taskAttemptBasePath)); + assertFalse(fs.exists(subTaskAttemptPath)); + + // Left some the task paths. + CommitUtils.save(fs, subTaskAttemptPath, new byte[]{}); + assertTrue(fs.exists(taskAttemptBasePath)); + assertTrue(fs.exists(subTaskAttemptPath)); + + // Setup task. + suite.setupTask(); + assertFalse(fs.exists(subTaskAttemptPath)); + } + + @Test + public void testCommitTask() throws Exception { + JobSuite suite = JobSuite.create(conf, taskAttempt0, outputPath); + Assume.assumeFalse(suite.skipTests()); + // Setup job + suite.setupJob(); + suite.dumpObjectStorage(); + suite.assertHasMagicKeys(); + + // Setup task + suite.setupTask(); + + // Write records. + suite.assertNoMagicPendingFile(); + suite.assertMultipartUpload(0); + suite.writeOutput(); + suite.dumpObjectStorage(); + suite.assertHasMagicPendingFile(); + suite.assertNoMagicMultipartUpload(); + suite.assertMultipartUpload(1); + // Assert the pending file content. + Path pendingPath = suite.magicPendingPath(); + byte[] pendingData = CommitUtils.load(suite.fs(), pendingPath); + Pending pending = Pending.deserialize(pendingData); + assertEquals(suite.destPartKey(), pending.destKey()); + assertEquals(20, pending.length()); + assertEquals(1, pending.parts().size()); + + // Commit the task. + suite.commitTask(); + + // Verify the pending set file. + suite.assertHasPendingSet(); + // Assert the pending set file content. + Path pendingSetPath = suite.magicPendingSetPath(); + byte[] pendingSetData = CommitUtils.load(suite.fs(), pendingSetPath); + PendingSet pendingSet = PendingSet.deserialize(pendingSetData); + assertEquals(suite.job().getJobID().toString(), pendingSet.jobId()); + assertEquals(1, pendingSet.commits().size()); + assertEquals(pending, pendingSet.commits().get(0)); + assertEquals(pendingSet.extraData(), ImmutableMap.of(CommitUtils.TASK_ATTEMPT_ID, + suite.taskAttemptContext().getTaskAttemptID().toString())); + + // Complete the multipart upload and verify the results. + ObjectStorage storage = suite.storage(); + storage.completeUpload(pending.destKey(), pending.uploadId(), pending.parts()); + suite.verifyPartContent(); + } + + @Test + public void testAbortTask() throws Exception { + JobSuite suite = JobSuite.create(conf, taskAttempt0, outputPath); + Assume.assumeFalse(suite.skipTests()); + suite.setupJob(); + suite.setupTask(); + + // Pre-check before the output write. + suite.assertNoMagicPendingFile(); + suite.assertMultipartUpload(0); + + // Execute the output write. + suite.writeOutput(); + + // Post-check after the output write. + suite.assertHasMagicPendingFile(); + suite.assertNoMagicMultipartUpload(); + suite.assertMultipartUpload(1); + // Assert the pending file content. + Path pendingPath = suite.magicPendingPath(); + byte[] pendingData = CommitUtils.load(suite.fs(), pendingPath); + Pending pending = Pending.deserialize(pendingData); + assertEquals(suite.destPartKey(), pending.destKey()); + assertEquals(20, pending.length()); + assertEquals(1, pending.parts().size()); + + // Abort the task. + suite.abortTask(); + + // Verify the state after aborting task. + suite.assertNoMagicPendingFile(); + suite.assertNoMagicMultipartUpload(); + suite.assertMultipartUpload(0); + suite.assertNoTaskAttemptPath(); + } + + @Test + public void testCommitJob() throws Exception { + JobSuite suite = JobSuite.create(conf, taskAttempt0, outputPath); + Assume.assumeFalse(suite.skipTests()); + suite.setupJob(); + suite.setupTask(); + suite.writeOutput(); + suite.commitTask(); + + // Commit the job. + suite.assertNoPartFiles(); + suite.commitJob(); + // Verify the output. + suite.assertNoMagicMultipartUpload(); + suite.assertNoMagicObjectKeys(); + suite.assertSuccessMarker(); + suite.assertSummaryReport(reportDir); + suite.verifyPartContent(); + } + + + @Test + public void testCommitJobFailed() throws Exception { + JobSuite suite = JobSuite.create(conf, taskAttempt0, outputPath); + Assume.assumeFalse(suite.skipTests()); + suite.setupJob(); + suite.setupTask(); + suite.writeOutput(); + suite.commitTask(); + + // Commit the job. + suite.assertNoPartFiles(); + suite.commitJob(); + } + + @Test + public void testTaskCommitAfterJobCommit() throws Exception { + JobSuite suite = JobSuite.create(conf, taskAttempt0, outputPath); + Assume.assumeFalse(suite.skipTests()); + suite.setupJob(); + suite.setupTask(); + suite.writeOutput(); + suite.commitTask(); + + // Commit the job + suite.assertNoPartFiles(); + suite.commitJob(); + // Verify the output. + suite.assertNoMagicMultipartUpload(); + suite.assertNoMagicObjectKeys(); + suite.assertSuccessMarker(); + suite.verifyPartContent(); + + // Commit the task again. + assertThrows(FileNotFoundException.class, suite::commitTask); + } + + @Test + public void testTaskCommitWithConsistentJobId() throws Exception { + Configuration config = newConf(); + String consistentJobId = randomFormedJobId(); + config.set(CommitUtils.SPARK_WRITE_UUID, consistentJobId); + JobSuite suite = JobSuite.create(config, taskAttempt0, outputPath); + Assume.assumeFalse(suite.skipTests()); + + // By now, we have two "jobId"s, one is spark uuid, and the other is the jobId in taskAttempt. + // The job committer will adopt the former. + suite.setupJob(); + + // Next, we clear spark uuid, and set the jobId of taskAttempt to another value. In this case, + // the committer will take the jobId of taskAttempt as the final jobId, which is not consistent + // with the one that committer holds. + config.unset(CommitUtils.SPARK_WRITE_UUID); + JobConf jobConf = new JobConf(config); + String anotherJobId = randomTrimmedJobId(); + TaskAttemptID taskAttemptId1 = + JobSuite.createTaskAttemptId(anotherJobId, JobSuite.DEFAULT_APP_ATTEMPT_ID); + final TaskAttemptContext attemptContext1 = + JobSuite.createTaskAttemptContext(jobConf, taskAttemptId1, JobSuite.DEFAULT_APP_ATTEMPT_ID); + + assertThrows(IllegalArgumentException.class, () -> suite.setupTask(attemptContext1), + "JobId set in the context"); + + // Even though we use another taskAttempt, as long as we ensure the spark uuid is consistent, + // the jobId in committer is consistent. + config.set(CommitUtils.SPARK_WRITE_UUID, consistentJobId); + config.set(FileOutputFormat.OUTDIR, outputPath.toString()); + jobConf = new JobConf(config); + anotherJobId = randomTrimmedJobId(); + TaskAttemptID taskAttemptId2 = + JobSuite.createTaskAttemptId(anotherJobId, JobSuite.DEFAULT_APP_ATTEMPT_ID); + TaskAttemptContext attemptContext2 = + JobSuite.createTaskAttemptContext(jobConf, taskAttemptId2, JobSuite.DEFAULT_APP_ATTEMPT_ID); + + suite.setupTask(attemptContext2); + // Write output must use the same task context with setup task. + suite.writeOutput(attemptContext2); + // Commit task must use the same task context with setup task. + suite.commitTask(attemptContext2); + suite.assertPendingSetAtRightLocation(); + + // Commit the job + suite.assertNoPartFiles(); + suite.commitJob(); + + // Verify the output. + suite.assertNoMagicMultipartUpload(); + suite.assertNoMagicObjectKeys(); + suite.assertSuccessMarker(); + suite.verifyPartContent(); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/mapred/JobSuite.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/mapred/JobSuite.java new file mode 100644 index 0000000000000..5376faec2cfe9 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/mapred/JobSuite.java @@ -0,0 +1,227 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit.mapred; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.commit.BaseJobSuite; +import org.apache.hadoop.fs.tosfs.commit.CommitUtils; +import org.apache.hadoop.fs.tosfs.commit.SuccessData; +import org.apache.hadoop.fs.tosfs.object.ObjectStorageFactory; +import org.apache.hadoop.fs.tosfs.object.ObjectUtils; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.FileOutputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.JobContext; +import org.apache.hadoop.mapred.JobContextImpl; +import org.apache.hadoop.mapred.JobID; +import org.apache.hadoop.mapred.RecordWriter; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapred.TaskAttemptContext; +import org.apache.hadoop.mapred.TaskAttemptContextImpl; +import org.apache.hadoop.mapred.TaskAttemptID; +import org.apache.hadoop.mapred.TextOutputFormat; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.net.NetUtils; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public final class JobSuite extends BaseJobSuite { + private final JobContext jobContext; + private final TaskAttemptContext taskAttemptContext; + private final Committer committer; + + private JobSuite(FileSystem fs, JobConf conf, + TaskAttemptID taskAttemptId, int appAttemptId, Path outputPath) + throws IOException { + setFs(fs); + // Initialize the job instance. + setJob(Job.getInstance(conf)); + job().setJobID(JobID.forName(CommitUtils.buildJobId(conf, taskAttemptId.getJobID()))); + this.jobContext = createJobContext(conf, taskAttemptId); + this.taskAttemptContext = createTaskAttemptContext(conf, taskAttemptId, appAttemptId); + setJobId(CommitUtils.buildJobId(jobContext)); + + // Set job output directory. + FileOutputFormat.setOutputPath(conf, outputPath); + setOutputPath(outputPath); + setObjectStorage(ObjectStorageFactory.create(outputPath.toUri().getScheme(), + outputPath.toUri().getAuthority(), conf)); + + // Initialize committer. + this.committer = new Committer(); + this.committer.setupTask(taskAttemptContext); + } + + public static JobSuite create(Configuration conf, TaskAttemptID taskAttemptId, Path outDir) + throws IOException { + FileSystem fs = outDir.getFileSystem(conf); + return new JobSuite(fs, new JobConf(conf), taskAttemptId, DEFAULT_APP_ATTEMPT_ID, outDir); + } + + public static TaskAttemptID createTaskAttemptId(String trimmedJobId, int attemptId) { + String attempt = String.format("attempt_%s_m_000000_%d", trimmedJobId, attemptId); + return TaskAttemptID.forName(attempt); + } + + public static JobContext createJobContext(JobConf jobConf, TaskAttemptID taskAttemptId) { + return new JobContextImpl(jobConf, taskAttemptId.getJobID()); + } + + public static TaskAttemptContext createTaskAttemptContext( + JobConf jobConf, TaskAttemptID taskAttemptId, int appAttemptId) throws IOException { + // Set the key values for job configuration. + jobConf.set(MRJobConfig.TASK_ATTEMPT_ID, taskAttemptId.toString()); + jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, appAttemptId); + jobConf.set("mapred.output.committer.class", + Committer.class.getName()); // 2x and 3x newApiCommitter=false. + return new TaskAttemptContextImpl(jobConf, taskAttemptId); + } + + public void setupJob() throws IOException { + committer.setupJob(jobContext); + } + + public void setupTask() throws IOException { + committer.setupTask(taskAttemptContext); + } + + // This method simulates the scenario that the job may set up task with a different + // taskAttemptContext, e.g., for a spark job. + public void setupTask(TaskAttemptContext taskAttemptCxt) throws IOException { + committer.setupTask(taskAttemptCxt); + } + + public void writeOutput() throws Exception { + writeOutput(taskAttemptContext); + } + + // This method simulates the scenario that the job may set up task with a different + // taskAttemptContext, e.g., for a spark job. + public void writeOutput(TaskAttemptContext taskAttemptCxt) throws Exception { + RecordWriter writer = new TextOutputFormat<>().getRecordWriter(fs(), + taskAttemptCxt.getJobConf(), + CommitUtils.buildJobId(taskAttemptCxt), + taskAttemptCxt.getProgressible()); + NullWritable nullKey = NullWritable.get(); + NullWritable nullVal = NullWritable.get(); + Object[] keys = new Object[]{KEY_1, nullKey, null, nullKey, null, KEY_2}; + Object[] vals = new Object[]{VAL_1, nullVal, null, null, nullVal, VAL_2}; + try { + assertEquals(keys.length, vals.length); + for (int i = 0; i < keys.length; i++) { + writer.write(keys[i], vals[i]); + } + } finally { + writer.close(Reporter.NULL); + } + } + + public boolean needsTaskCommit() throws IOException { + return committer.needsTaskCommit(taskAttemptContext); + } + + public void commitTask() throws IOException { + committer.commitTask(taskAttemptContext); + } + + // This method simulates the scenario that the job may set up task with a different + // taskAttemptContext, e.g., for a spark job. + public void commitTask(TaskAttemptContext taskAttemptCxt) throws IOException { + committer.commitTask(taskAttemptCxt); + } + + public void abortTask() throws IOException { + committer.abortTask(taskAttemptContext); + } + + public void commitJob() throws IOException { + committer.commitJob(jobContext); + } + + @Override + public Path magicPartPath() { + return new Path(committer.getWorkPath(), committer.jobId()); + } + + @Override + public Path magicPendingSetPath() { + return CommitUtils.magicTaskPendingSetPath(taskAttemptContext, outputPath()); + } + + public TaskAttemptContext taskAttemptContext() { + return taskAttemptContext; + } + + public Committer committer() { + return committer; + } + + @Override + public void assertNoTaskAttemptPath() throws IOException { + Path path = CommitUtils.magicTaskAttemptBasePath(taskAttemptContext, outputPath()); + assertFalse(fs().exists(path), "Task attempt path should be not existing"); + String pathToKey = ObjectUtils.pathToKey(path); + assertNull(storage().head(pathToKey), "Should have no task attempt path key"); + } + + @Override + protected boolean skipTests() { + return storage().bucket().isDirectory(); + } + + @Override + public void assertSuccessMarker() throws IOException { + Path succPath = CommitUtils.successMarker(outputPath()); + assertTrue(fs().exists(succPath), String.format("%s should be exists", succPath)); + SuccessData successData = SuccessData.deserialize(CommitUtils.load(fs(), succPath)); + assertEquals(SuccessData.class.getName(), successData.name()); + assertTrue(successData.success()); + assertEquals(NetUtils.getHostname(), successData.hostname()); + assertEquals(CommitUtils.COMMITTER_NAME, successData.committer()); + assertEquals( + String.format("Task committer %s", taskAttemptContext.getTaskAttemptID()), + successData.description()); + assertEquals(job().getJobID().toString(), successData.jobId()); + assertEquals(1, successData.filenames().size()); + assertEquals(destPartKey(), successData.filenames().get(0)); + } + + @Override + public void assertSummaryReport(Path reportDir) throws IOException { + Path reportPath = CommitUtils.summaryReport(reportDir, job().getJobID().toString()); + assertTrue(fs().exists(reportPath), String.format("%s should be exists", reportPath)); + SuccessData reportData = SuccessData.deserialize(CommitUtils.load(fs(), reportPath)); + assertEquals(SuccessData.class.getName(), reportData.name()); + assertTrue(reportData.success()); + assertEquals(NetUtils.getHostname(), reportData.hostname()); + assertEquals(CommitUtils.COMMITTER_NAME, reportData.committer()); + assertEquals(String.format("Task committer %s", taskAttemptContext.getTaskAttemptID()), + reportData.description()); + assertEquals(job().getJobID().toString(), reportData.jobId()); + assertEquals(1, reportData.filenames().size()); + assertEquals(destPartKey(), reportData.filenames().get(0)); + assertEquals("clean", reportData.diagnostics().get("stage")); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/mapred/TestCommitter.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/mapred/TestCommitter.java new file mode 100644 index 0000000000000..48f90ec3d30de --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/commit/mapred/TestCommitter.java @@ -0,0 +1,29 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.commit.mapred; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.util.TestUtility; + +public class TestCommitter extends CommitterTestBase { + @Override + protected Configuration newConf() { + Configuration conf = new Configuration(); + conf.set("fs.defaultFS", String.format("tos://%s", TestUtility.bucket())); + return conf; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestChecksum.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestChecksum.java new file mode 100644 index 0000000000000..bdc14e05e84e1 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestChecksum.java @@ -0,0 +1,137 @@ +/* + * 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.tosfs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileChecksum; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.contract.AbstractFSContractTestBase; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.tosfs.RawFileSystem; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.object.ObjectUtils; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; +import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset; + +public class TestChecksum extends AbstractFSContractTestBase { + @BeforeClass + public static void before() { + Assume.assumeTrue(TestEnv.checkTestEnabled()); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new TosContract(conf); + } + + private Path testCreateNewFile(String fileName, byte[] data, boolean useBuilder) + throws IOException { + describe("Foundational 'create a file' test, using builder API=" + useBuilder); + Path path = path(fileName, useBuilder); + + writeDataset(getFileSystem(), path, data, data.length, 1024 * 1024, false, useBuilder); + ContractTestUtils.verifyFileContents(getFileSystem(), path, data); + + return path; + } + + private Path path(String filepath, boolean useBuilder) throws IOException { + return super.path(filepath + (useBuilder ? "" : "-builder")); + } + + @Test + public void testCheckSumWithSimplePut() throws IOException { + byte[] data = dataset(256, 'a', 'z'); + Path path1 = testCreateNewFile("file1", data, true); + Path path2 = testCreateNewFile("file2", data, true); + Path path3 = testCreateNewFile("file3", dataset(512, 'a', 'z'), true); + + FileChecksum expected = getFileSystem().getFileChecksum(path1); + assertEquals("Checksum value should be same among objects with same content", + expected, getFileSystem().getFileChecksum(path2)); + assertEquals("Checksum value should be same among multiple call for same object", + expected, getFileSystem().getFileChecksum(path1)); + assertNotEquals( + "Checksum value should be different for different objects with different content", expected, + getFileSystem().getFileChecksum(path3)); + + Path renamed = path("renamed"); + getFileSystem().rename(path1, renamed); + assertEquals("Checksum value should not change after rename", + expected, getFileSystem().getFileChecksum(renamed)); + } + + @Test + public void testCheckSumShouldSameViaPutAndMPU() throws IOException { + byte[] data = TestUtility.rand(11 << 20); + + // simple put + Path singleFile = path("singleFile"); + RawFileSystem fs = (RawFileSystem) getFileSystem(); + fs.storage().put(ObjectUtils.pathToKey(singleFile), data); + + // MPU upload data, the default threshold is 10MB + Path mpuFile = testCreateNewFile("mpuFile", data, true); + + assertEquals(fs.getFileChecksum(singleFile), fs.getFileChecksum(mpuFile)); + } + + @Test + public void testDisableCheckSum() throws IOException { + Path path1 = testCreateNewFile("file1", dataset(256, 'a', 'z'), true); + Path path2 = testCreateNewFile("file2", dataset(512, 'a', 'z'), true); + assertNotEquals(getFileSystem().getFileChecksum(path1), getFileSystem().getFileChecksum(path2)); + + // disable checksum + Configuration newConf = new Configuration(getFileSystem().getConf()); + newConf.setBoolean(ConfKeys.FS_CHECKSUM_ENABLED.key("tos"), false); + FileSystem newFS = FileSystem.get(newConf); + + assertEquals(newFS.getFileChecksum(path1), newFS.getFileChecksum(path2)); + } + + @Test + public void testGetDirChecksum() throws IOException { + FileSystem fs = getFileSystem(); + + Path dir1 = path("dir1", true); + Path dir2 = path("dir2", true); + assertPathDoesNotExist("directory already exists", dir1); + assertPathDoesNotExist("directory already exists", dir2); + fs.mkdirs(dir1); + + assertThrows("Path is not a file", FileNotFoundException.class, + () -> getFileSystem().getFileChecksum(dir1)); + assertThrows("No such file or directory", FileNotFoundException.class, + () -> getFileSystem().getFileChecksum(dir2)); + + assertDeleted(dir1, false); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestCreate.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestCreate.java new file mode 100644 index 0000000000000..a9b4f0833980f --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestCreate.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.tosfs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractCreateTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.junit.jupiter.api.BeforeAll; + +import static org.junit.jupiter.api.Assumptions.assumeTrue; + +public class TestCreate extends AbstractContractCreateTest { + + @BeforeAll + public static void before() { + assumeTrue(TestEnv.checkTestEnabled()); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new TosContract(conf); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestDelete.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestDelete.java new file mode 100644 index 0000000000000..31363432669a4 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestDelete.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.tosfs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.AbstractContractDeleteTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; + +import static org.apache.hadoop.fs.tosfs.object.ObjectTestUtils.assertDirExist; +import static org.apache.hadoop.fs.tosfs.object.ObjectTestUtils.assertObjectNotExist; + +public class TestDelete extends AbstractContractDeleteTest { + + @BeforeClass + public static void before() { + Assume.assumeTrue(TestEnv.checkTestEnabled()); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new TosContract(conf); + } + + @Test + public void testParentDirCreatedAfterDeleteSubChildren() throws IOException { + Path path = path("testParentDirCreatedAfterDeleteSubChildren/"); + Path file1 = new Path(path, "f1"); + Path file2 = new Path(path, "f2"); + ContractTestUtils.writeTextFile(getFileSystem(), file1, + "the first file", true); + ContractTestUtils.writeTextFile(getFileSystem(), file2, + "the second file", true); + assertPathExists("file1 not created", file1); + assertPathExists("file1 not created", file2); + + assertObjectNotExist(path, false); + assertObjectNotExist(path, true); + + assertDeleted(file1, false); + assertPathExists("parent path should exist", path); + + assertObjectNotExist(path, false); + assertDirExist(path); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestDistCp.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestDistCp.java new file mode 100644 index 0000000000000..a87ac6e690b93 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestDistCp.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.tosfs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.tools.contract.AbstractContractDistCpTest; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assumptions.assumeTrue; + +public class TestDistCp extends AbstractContractDistCpTest { + + @BeforeAll + public static void before() { + assumeTrue(TestEnv.checkTestEnabled()); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new TosContract(conf); + } + + // ignore this test case as there is intermittent IllegalStateException issue + @Test + public void testDistCpWithIterator() { + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestGetFileStatus.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestGetFileStatus.java new file mode 100644 index 0000000000000..2c631121bef42 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestGetFileStatus.java @@ -0,0 +1,169 @@ +/* + * 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.tosfs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.tosfs.RawFileStatus; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.conf.TosKeys; +import org.apache.hadoop.fs.tosfs.object.Constants; +import org.apache.hadoop.fs.tosfs.util.UUIDUtils; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +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; + +@RunWith(Parameterized.class) +public class TestGetFileStatus extends AbstractContractGetFileStatusTest { + + private final boolean getFileStatusEnabled; + + @Parameterized.Parameters(name = "getFileStatusEnabled={0}") + public static List createParameters() { + return Arrays.asList(false, true); + } + + public TestGetFileStatus(boolean getFileStatusEnabled) { + this.getFileStatusEnabled = getFileStatusEnabled; + } + + @BeforeClass + public static void before() { + Assume.assumeTrue(TestEnv.checkTestEnabled()); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + conf.setBoolean(TosKeys.FS_TOS_GET_FILE_STATUS_ENABLED, getFileStatusEnabled); + conf.setBoolean(ConfKeys.FS_ASYNC_CREATE_MISSED_PARENT.key("tos"), false); + return new TosContract(conf); + } + + @Test + public void testDirModificationTimeShouldNotBeZero() throws IOException { + FileSystem fs = getFileSystem(); + Path path = getContract().getTestPath(); + fs.delete(path, true); + + Path subfolder = path.suffix('/' + this.methodName.getMethodName() + "-" + UUIDUtils.random()); + mkdirs(subfolder); + + FileStatus fileStatus = fs.getFileStatus(path); + assertTrue(fileStatus.getModificationTime() > 0); + } + + @Test + public void testThrowExceptionWhenListStatusForNonExistPath() { + FileSystem fs = getFileSystem(); + Path path = getContract().getTestPath(); + + assertThrows("Path doesn't exist", FileNotFoundException.class, + () -> fs.listStatusIterator(new Path(path, "testListStatusForNonExistPath"))); + } + + @Test + public void testPathStatNonexistentFile() { + FileSystem fs = getFileSystem(); + // working dir does not exist. + Path file = new Path(getContract().getTestPath(), this.methodName.getMethodName()); + assertThrows("Path doesn't exist", FileNotFoundException.class, () -> fs.getFileStatus(file)); + } + + @Test + public void testPathStatExistentFile() throws IOException { + FileSystem fs = getFileSystem(); + Path file = new Path(getContract().getTestPath(), this.methodName.getMethodName()); + + int size = 1 << 20; + byte[] data = dataset(size, 'a', 'z'); + createFile(fs, file, true, data); + FileStatus status = fs.getFileStatus(file); + Assert.assertTrue(status.isFile()); + Assert.assertTrue(status.getModificationTime() > 0); + Assert.assertEquals(size, status.getLen()); + } + + @Test + public void testPathStatEmptyDirectory() throws IOException { + FileSystem fs = getFileSystem(); + Path workingPath = new Path(getContract().getTestPath(), this.methodName.getMethodName()); + mkdirs(workingPath); + + FileStatus dirStatus = fs.getFileStatus(workingPath); + Assert.assertTrue(dirStatus.isDirectory()); + Assert.assertTrue(dirStatus.getModificationTime() > 0); + if (dirStatus instanceof RawFileStatus) { + Assert.assertArrayEquals(Constants.MAGIC_CHECKSUM, ((RawFileStatus) dirStatus).checksum()); + } + } + + @Test + public void testPathStatWhenCreateSubDir() throws IOException { + FileSystem fs = getFileSystem(); + Path workintPath = new Path(getContract().getTestPath(), this.methodName.getMethodName()); + // create sub directory directly. + Path subDir = new Path(workintPath, UUIDUtils.random()); + mkdirs(subDir); + Assert.assertTrue(fs.getFileStatus(subDir).isDirectory()); + + // can get FileStatus of working dir. + Assert.assertTrue(fs.getFileStatus(workintPath).isDirectory()); + // delete sub directory. + fs.delete(subDir, true); + // still cat get FileStatus of working dir. + Assert.assertTrue(fs.getFileStatus(workintPath).isDirectory()); + } + + @Test + public void testPathStatDirNotExistButSubFileExist() throws IOException { + FileSystem fs = getFileSystem(); + // working dir does not exist. + Path workintPath = new Path(getContract().getTestPath(), this.methodName.getMethodName()); + assertThrows("Path doesn't exist", FileNotFoundException.class, + () -> fs.getFileStatus(workintPath)); + + // create sub file in working dir directly. + Path file = workintPath.suffix('/' + UUIDUtils.random()); + touch(fs, file); + + // can get FileStatus of working dir. + Assert.assertTrue(fs.getFileStatus(workintPath).isDirectory()); + + // delete sub file, will create parent directory. + fs.delete(file, false); + Assert.assertTrue(fs.getFileStatus(workintPath).isDirectory()); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestMkdir.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestMkdir.java new file mode 100644 index 0000000000000..cfe12524bdfc1 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestMkdir.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.tosfs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractMkdirTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.junit.Assume; +import org.junit.BeforeClass; + +public class TestMkdir extends AbstractContractMkdirTest { + + @BeforeClass + public static void before() { + Assume.assumeTrue(TestEnv.checkTestEnabled()); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new TosContract(conf); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestOpen.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestOpen.java new file mode 100644 index 0000000000000..29abb10241622 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestOpen.java @@ -0,0 +1,75 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.AbstractContractOpenTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.tosfs.RawFileStatus; +import org.apache.hadoop.fs.tosfs.RawFileSystem; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.fs.tosfs.object.exceptions.ChecksumMismatchException; +import org.apache.hadoop.fs.tosfs.util.Range; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; +import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset; + +public class TestOpen extends AbstractContractOpenTest { + + @BeforeClass + public static void before() { + Assume.assumeTrue(TestEnv.checkTestEnabled()); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new TosContract(conf); + } + + @Test + public void testOpenAExpiredFile() throws IOException { + Path file = path("testOpenAOutageFile"); + FileSystem fs = getFileSystem(); + byte[] data = dataset(256, 'a', 'z'); + writeDataset(fs, file, data, data.length, 1024 * 1024, true); + + FileStatus fileStatus = fs.getFileStatus(file); + if (fs instanceof RawFileSystem) { + byte[] expectChecksum = ((RawFileStatus) fileStatus).checksum(); + FSDataInputStream fsDataInputStream = + ((RawFileSystem) fs).open(file, expectChecksum, Range.of(0, Long.MAX_VALUE)); + fsDataInputStream.close(); + + // update the file + data = dataset(512, 'a', 'z'); + writeDataset(fs, file, data, data.length, 1024 * 1024, true); + + FSDataInputStream newStream = + ((RawFileSystem) fs).open(file, expectChecksum, Range.of(0, Long.MAX_VALUE)); + assertThrows("the file is expired", ChecksumMismatchException.class, () -> newStream.read()); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestRename.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestRename.java new file mode 100644 index 0000000000000..d63e340516f5b --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestRename.java @@ -0,0 +1,281 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.contract; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.AbstractContractRenameTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Test; + +import java.io.IOException; +import java.io.InputStream; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; +import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset; + +public class TestRename extends AbstractContractRenameTest { + + @BeforeClass + public static void before() { + Assume.assumeTrue(TestEnv.checkTestEnabled()); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + // Add follow two keys into hadoop configuration. + String defaultScheme = FileSystem.getDefaultUri(conf).getScheme(); + Configuration newConf = new Configuration(conf); + newConf.setLong(ConfKeys.FS_MULTIPART_SIZE.key(defaultScheme), + ConfKeys.FS_MULTIPART_SIZE_DEFAULT); + newConf.setLong(ConfKeys.FS_MULTIPART_THRESHOLD.key(defaultScheme), + ConfKeys.FS_MULTIPART_THRESHOLD_DEFAULT); + + return new TosContract(newConf); + } + + @Test + public void testSucceedRenameFile() throws IOException { + describe("check if source file and dest file exists when succeed to rename"); + Path renameSrc = path("renameSrc"); + Path renameDest = path("renameDst"); + FileSystem fs = getFileSystem(); + byte[] data = dataset(256, 'a', 'z'); + writeDataset(fs, renameSrc, data, data.length, 1024 * 1024, true); + boolean renamed = rename(renameSrc, renameDest); + assertTrue(renamed); + assertPathExists("dest file should exist when succeed to rename", renameDest); + assertPathDoesNotExist("source file should not exist when succeed to rename", renameSrc); + } + + @Test + public void testSucceedRenameDir() throws IOException { + describe("check if source dir and dest dir exists when succeed to rename"); + Path renameSrc = path("renameSrc"); + Path renameDest = path("renameDst"); + int fileNums = 10; + int byteSize = 10 << 20; // trigger multipart upload + FileSystem fs = getFileSystem(); + for (int i = 0; i < fileNums; i++) { + byte[] data = dataset(byteSize >> i, 'a', 'z'); + writeDataset(fs, new Path(renameSrc, String.format("src%02d", i)), data, data.length, + 1024 * 1024, true); + } + boolean renamed = rename(renameSrc, renameDest); + assertTrue(renamed); + for (int i = 0; i < fileNums; i++) { + Path srcFilePath = new Path(renameSrc, String.format("src%02d", i)); + Path dstFilePath = new Path(renameDest, String.format("src%02d", i)); + byte[] data = dataset(byteSize >> i, 'a', 'z'); + assertPathExists("dest file should exist when succeed to rename", dstFilePath); + assertPathDoesNotExist("source file should not exist when succeed to rename", srcFilePath); + try (InputStream is = fs.open(dstFilePath)) { + assertArrayEquals(data, IOUtils.toByteArray(is)); + } + } + } + + @Test + public void testFailedRename() throws IOException { + describe("check if source file and dest file exists when failed to rename"); + Path renameSrc = path("src/renameSrc"); + Path renameDest = path("src/renameSrc/renameDst"); + FileSystem fs = getFileSystem(); + byte[] data = dataset(256, 'a', 'z'); + writeDataset(fs, renameSrc, data, data.length, 1024 * 1024, true); + boolean renamed; + try { + renamed = rename(renameSrc, renameDest); + } catch (IOException e) { + renamed = false; + } + assertFalse(renamed); + assertPathExists("source file should exist when failed to rename", renameSrc); + assertPathDoesNotExist("dest file should not exist when failed to rename", renameDest); + } + + @Test + public void testRenameSmallFile() throws IOException { + testRenameFileByPut(1 << 20); + testRenameFileByPut(3 << 20); + } + + @Test + public void testRenameLargeFile() throws IOException { + testRenameFileByUploadParts(16 << 20); + testRenameFileByUploadParts(10 << 20); + } + + @Test + public void testRenameDirWithSubFileAndSubDir() throws IOException { + FileSystem fs = getFileSystem(); + + Path renameSrc = path("dir/renameSrc"); + Path renameDest = path("dir/renameDst"); + int size = 1024; + byte[] data = dataset(size, 'a', 'z'); + String fileName = "file.txt"; + writeDataset(fs, new Path(renameSrc, fileName), data, data.length, 1024, true); + + String dirName = "dir"; + Path dirPath = new Path(renameSrc, dirName); + mkdirs(dirPath); + assertPathExists("source dir should exist", dirPath); + + boolean renamed = fs.rename(renameSrc, renameDest); + + assertTrue(renamed); + Path srcFilePath = new Path(renameSrc, fileName); + Path dstFilePath = new Path(renameDest, fileName); + assertPathExists("dest file should exist when succeed to rename", dstFilePath); + assertPathDoesNotExist("source file should not exist when succeed to rename", srcFilePath); + + assertPathExists("dest dir should exist when succeed to rename", new Path(renameDest, dirName)); + assertPathDoesNotExist("source dir should not exist when succeed to rename", + new Path(renameSrc, dirName)); + + ContractTestUtils.cleanup("TEARDOWN", fs, getContract().getTestPath()); + } + + public void testRenameFileByPut(int size) throws IOException { + describe("check if use put method when rename file"); + Path renameSrc = path("renameSrc"); + Path renameDest = path("renameDst"); + FileSystem fs = getFileSystem(); + + byte[] data = dataset(size, 'a', 'z'); + String fileName = String.format("%sMB.txt", size >> 20); + writeDataset(fs, new Path(renameSrc, fileName), data, data.length, 1024 * 1024, true); + boolean renamed = fs.rename(renameSrc, renameDest); + + assertTrue(renamed); + Path srcFilePath = new Path(renameSrc, fileName); + Path dstFilePath = new Path(renameDest, fileName); + assertPathExists("dest file should exist when succeed to rename", dstFilePath); + assertPathDoesNotExist("source file should not exist when succeed to rename", srcFilePath); + + assertPathExists("dest src should exist when succeed to rename", renameDest); + assertPathDoesNotExist("source src should not exist when succeed to rename", renameSrc); + + try (InputStream is = fs.open(dstFilePath)) { + assertArrayEquals(data, IOUtils.toByteArray(is)); + } + ContractTestUtils.cleanup("TEARDOWN", fs, getContract().getTestPath()); + } + + @Test + public void testCreateParentDirAfterRenameSubFile() throws IOException { + FileSystem fs = getFileSystem(); + + Path srcDir = path("srcDir"); + Path destDir = path("destDir"); + + assertPathDoesNotExist("Src dir should not exist", srcDir); + assertPathDoesNotExist("Dest dir should not exist", destDir); + int size = 1 << 20; + byte[] data = dataset(size, 'a', 'z'); + String fileName = String.format("%sMB.txt", size >> 20); + Path srcFile = new Path(srcDir, fileName); + Path destFile = new Path(destDir, fileName); + writeDataset(fs, srcFile, data, data.length, 1024 * 1024, true); + + assertPathExists("Src file should exist", srcFile); + assertPathExists("Src dir should exist", srcDir); + + mkdirs(destDir); + assertPathExists("Dest dir should exist", destDir); + + boolean renamed = fs.rename(srcFile, destFile); + assertTrue(renamed); + + assertPathExists("Dest file should exist", destFile); + assertPathExists("Dest dir should exist", destDir); + assertPathDoesNotExist("Src file should not exist", srcFile); + assertPathExists("Src dir should exist", srcDir); + } + + @Test + public void testCreateParentDirAfterRenameSubDir() throws IOException { + FileSystem fs = getFileSystem(); + + Path srcDir = path("srcDir"); + Path destDir = path("destDir"); + + assertPathDoesNotExist("Src dir should not exist", srcDir); + assertPathDoesNotExist("Dest dir should not exist", destDir); + + String subDirName = String.format("subDir"); + Path srcSubDir = new Path(srcDir, subDirName); + Path destDestDir = new Path(destDir, subDirName); + mkdirs(srcSubDir); + + assertPathExists("Src sub dir should exist", srcSubDir); + assertPathExists("Src dir should exist", srcDir); + + mkdirs(destDir); + assertPathExists("Dest dir should exist", destDir); + + boolean renamed = fs.rename(srcSubDir, destDestDir); + assertTrue(renamed); + + assertPathExists("Dest sub dir should exist", destDestDir); + assertPathExists("Dest dir should exist", destDir); + assertPathDoesNotExist("Src sub dir should not exist", srcSubDir); + assertPathExists("Src sub dir should exist", srcDir); + } + + public void testRenameFileByUploadParts(int size) throws IOException { + describe("check if use upload parts method when rename file"); + Path renameSrc = path("renameSrc"); + Path renameDest = path("renameDst"); + FileSystem fs = getFileSystem(); + + byte[] data = dataset(size, 'a', 'z'); + String fileName = String.format("%sMB.txt", size >> 20); + writeDataset(fs, new Path(renameSrc, fileName), data, data.length, 1024 * 1024, true); + boolean renamed = fs.rename(renameSrc, renameDest); + + assertTrue(renamed); + Path srcFilePath = new Path(renameSrc, fileName); + Path dstFilePath = new Path(renameDest, fileName); + assertPathExists("dest file should exist when succeed to rename", dstFilePath); + assertPathDoesNotExist("source file should not exist when succeed to rename", srcFilePath); + + try (InputStream is = fs.open(dstFilePath)) { + assertArrayEquals(data, IOUtils.toByteArray(is)); + } + ContractTestUtils.cleanup("TEARDOWN", fs, getContract().getTestPath()); + } + + @Ignore + @Test + public void testRenameFileUnderFileSubdir() { + } + + @Ignore + @Test + public void testRenameFileUnderFile() { + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestRootDir.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestRootDir.java new file mode 100644 index 0000000000000..6086d0a810b63 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestRootDir.java @@ -0,0 +1,37 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.junit.Assume; +import org.junit.BeforeClass; + +public class TestRootDir extends AbstractContractRootDirectoryTest { + + @BeforeClass + public static void before() { + Assume.assumeTrue(TestEnv.checkTestEnabled()); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new TosContract(conf); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestSeek.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestSeek.java new file mode 100644 index 0000000000000..ad5bef20508ec --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestSeek.java @@ -0,0 +1,37 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractSeekTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.junit.Assume; +import org.junit.BeforeClass; + +public class TestSeek extends AbstractContractSeekTest { + + @BeforeClass + public static void before() { + Assume.assumeTrue(TestEnv.checkTestEnabled()); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new TosContract(conf); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestUnbuffer.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestUnbuffer.java new file mode 100644 index 0000000000000..06e5677c17286 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestUnbuffer.java @@ -0,0 +1,37 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractUnbufferTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.junit.Assume; +import org.junit.BeforeClass; + +public class TestUnbuffer extends AbstractContractUnbufferTest { + + @BeforeClass + public static void before() { + Assume.assumeTrue(TestEnv.checkTestEnabled()); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new TosContract(conf); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestXAttr.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestXAttr.java new file mode 100644 index 0000000000000..7acf58d24f0d0 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestXAttr.java @@ -0,0 +1,177 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.XAttrSetFlag; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.contract.AbstractFSContractTestBase; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.fs.tosfs.common.Bytes; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Arrays; +import java.util.EnumSet; +import java.util.List; +import java.util.Map; + +public class TestXAttr extends AbstractFSContractTestBase { + private static final String XATTR_NAME = "xAttrName"; + private static final byte[] XATTR_VALUE = "xAttrValue".getBytes(); + + @BeforeClass + public static void before() { + Assume.assumeTrue(TestEnv.checkTestEnabled()); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new TosContract(conf); + } + + @Test + public void testGetNonExistedXAttr() throws Exception { + FileSystem fs = getFileSystem(); + Path path = path("testSetAndGet/file"); + fs.create(path).close(); + + fs.setXAttr(path, XATTR_NAME, XATTR_VALUE); + assertThrows("Not found.", IOException.class, + () -> fs.getXAttr(path, "non-exist")); + assertThrows("Not found.", IOException.class, + () -> fs.getXAttrs(path, Arrays.asList("non-exist"))); + assertThrows("Not found.", IOException.class, + () -> fs.getXAttrs(path, Arrays.asList("non-exist", XATTR_NAME))); + } + + @Test + public void testSetAndGetWhenPathNotExist() throws Exception { + FileSystem fs = getFileSystem(); + Path path = path("testXAttrWhenPathNotExist/file"); + fs.delete(path); + + assertThrows("No such file", FileNotFoundException.class, + () -> fs.setXAttr(path, XATTR_NAME, XATTR_VALUE)); + assertThrows("No such file", FileNotFoundException.class, + () -> fs.getXAttrs(path)); + assertThrows("No such file", FileNotFoundException.class, + () -> fs.removeXAttr(path, "name")); + } + + @Test + public void testSetAndGet() throws Exception { + FileSystem fs = getFileSystem(); + Path path = path("testSetAndGet/file"); + fs.create(path).close(); + + fs.setXAttr(path, XATTR_NAME, XATTR_VALUE); + assertArrayEquals(XATTR_VALUE, fs.getXAttr(path, XATTR_NAME)); + } + + @Test + public void testSetAndGetNonExistedObject() throws Exception { + FileSystem fs = getFileSystem(); + Path path = path("testSetAndGetOnNonExistedObject/dir-0/dir-1/file"); + fs.create(path).close(); + + Path nonExistedPath = path.getParent().getParent(); + fs.setXAttr(nonExistedPath, XATTR_NAME, XATTR_VALUE); + assertThrows("Not found.", IOException.class, + () -> fs.getXAttr(nonExistedPath, XATTR_NAME)); + } + + @Test + public void testSetAndGetOnExistedObjectDir() throws Exception { + FileSystem fs = getFileSystem(); + Path path = path("testSetAndGetOnDir/dir-0/dir-1"); + fs.mkdirs(path); + + fs.setXAttr(path, XATTR_NAME, XATTR_VALUE); + assertThrows("Not found.", IOException.class, + () -> fs.getXAttr(path, XATTR_NAME)); + } + + @Test + public void testGetAndListAll() throws Exception { + FileSystem fs = getFileSystem(); + Path path = path("testGetAndListAll/file"); + fs.create(path).close(); + + int size = 10; + for (int i = 0; i < size; i++) { + fs.setXAttr(path, XATTR_NAME + i, Bytes.toBytes("VALUE" + i)); + } + + Map result = fs.getXAttrs(path); + assertEquals(size, result.size()); + for (int i = 0; i < size; i++) { + assertEquals("VALUE" + i, Bytes.toString(result.get(XATTR_NAME + i))); + } + + List names = fs.listXAttrs(path); + assertEquals(size, names.size()); + for (int i = 0; i < size; i++) { + assertTrue(names.contains(XATTR_NAME + i)); + } + } + + @Test + public void testRemove() throws Exception { + FileSystem fs = getFileSystem(); + Path path = path("testRemove/file"); + fs.create(path).close(); + + int size = 10; + for (int i = 0; i < size; i++) { + fs.setXAttr(path, XATTR_NAME + i, Bytes.toBytes("VALUE" + i)); + } + + for (int i = 0; i < size; i++) { + fs.removeXAttr(path, XATTR_NAME + i); + String name = XATTR_NAME + i; + assertThrows("Not found.", IOException.class, + () -> fs.getXAttr(path, name)); + assertEquals(size - 1 - i, fs.listXAttrs(path).size()); + } + } + + @Test + public void testXAttrFlag() throws Exception { + FileSystem fs = getFileSystem(); + Path path = path("testXAttrFlag/file"); + fs.create(path).close(); + + String key = XATTR_NAME; + byte[] value = XATTR_VALUE; + assertThrows("The CREATE flag must be specified", IOException.class, + () -> fs.setXAttr(path, key, value, EnumSet.of(XAttrSetFlag.REPLACE))); + fs.setXAttr(path, key, value, EnumSet.of(XAttrSetFlag.CREATE)); + assertArrayEquals(value, fs.getXAttr(path, key)); + + byte[] newValue = Bytes.toBytes("new value"); + assertThrows("The REPLACE flag must be specified", IOException.class, + () -> fs.setXAttr(path, key, newValue, EnumSet.of(XAttrSetFlag.CREATE))); + fs.setXAttr(path, key, newValue, EnumSet.of(XAttrSetFlag.REPLACE)); + assertArrayEquals(newValue, fs.getXAttr(path, key)); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TosContract.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TosContract.java new file mode 100644 index 0000000000000..0f1ce3a4c7508 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TosContract.java @@ -0,0 +1,61 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.contract; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.AbstractBondedFSContract; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.apache.hadoop.fs.tosfs.util.UUIDUtils; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TosContract extends AbstractBondedFSContract { + private static final Logger LOG = LoggerFactory.getLogger(TosContract.class); + private final String testDir; + + public TosContract(Configuration conf) { + super(conf); + addConfResource("contract/tos.xml"); + // Set the correct contract test path if there is a provided bucket name from environment. + if (StringUtils.isNoneEmpty(TestUtility.bucket())) { + conf.set("fs.contract.test.fs.tos", String.format("tos://%s/", TestUtility.bucket())); + } + + testDir = "/test-" + UUIDUtils.random(); + } + + @BeforeClass + public static void before() { + Assume.assumeTrue(TestEnv.checkTestEnabled()); + } + + @Override + public String getScheme() { + return "tos"; + } + + @Override + public Path getTestPath() { + LOG.info("the test dir is: {}", testDir); + return new Path(testDir); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/ObjectStorageTestBase.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/ObjectStorageTestBase.java new file mode 100644 index 0000000000000..484a414389ba9 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/ObjectStorageTestBase.java @@ -0,0 +1,108 @@ +/* + * 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.tosfs.object; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.apache.hadoop.fs.tosfs.util.UUIDUtils; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.io.TempDir; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +public class ObjectStorageTestBase { + private static final Logger LOG = LoggerFactory.getLogger(ObjectStorageTestBase.class); + private Configuration conf; + private Configuration tosConf; + private Path testDir; + private FileSystem fs; + private String scheme; + private ObjectStorage storage; + + @TempDir + private java.nio.file.Path tempDir; + + @BeforeEach + public void setUp() throws IOException { + LOG.info("The test temporary folder is {}", tempDir); + + String tempDirPath = tempDir.toAbsolutePath().toString(); + conf = new Configuration(); + conf.set(ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key("filestore"), tempDirPath); + conf.set("fs.filestore.impl", LocalFileSystem.class.getName()); + tosConf = new Configuration(conf); + // Set the environment variable for ObjectTestUtils#assertObject + TestUtility.setSystemEnv(FileStore.ENV_FILE_STORAGE_ROOT, tempDirPath); + + testDir = new Path("filestore://" + FileStore.DEFAULT_BUCKET + "/", UUIDUtils.random()); + fs = testDir.getFileSystem(conf); + scheme = testDir.toUri().getScheme(); + storage = ObjectStorageFactory.create(scheme, testDir.toUri().getAuthority(), tosConf); + } + + @AfterEach + public void tearDown() throws IOException { + if (storage != null) { + // List all keys with test dir prefix and delete them. + String prefix = ObjectUtils.pathToKey(testDir); + CommonUtils.runQuietly(() -> storage.deleteAll(prefix)); + // List all multipart uploads and abort them. + CommonUtils.runQuietly(() -> { + for (MultipartUpload upload : storage.listUploads(prefix)) { + LOG.info("Abort the multipart upload {}", upload); + storage.abortMultipartUpload(upload.key(), upload.uploadId()); + } + }); + + storage.close(); + } + } + + public Configuration conf() { + return conf; + } + + public Configuration tosConf() { + return tosConf; + } + + public Path testDir() { + return testDir; + } + + public FileSystem fs() { + return fs; + } + + public String scheme() { + return scheme; + } + + public ObjectStorage getStorage() { + return storage; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/ObjectTestUtils.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/ObjectTestUtils.java new file mode 100644 index 0000000000000..d8c23552c5e37 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/ObjectTestUtils.java @@ -0,0 +1,127 @@ +/* + * 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.tosfs.object; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Iterator; +import java.util.Objects; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public final class ObjectTestUtils { + + public static final byte[] EMPTY_BYTES = new byte[] {}; + + private ObjectTestUtils() { + } + + /** + * Assert that all the parent directories should be existing. + * + * @param path to validate, can be directory or file. + */ + public static void assertParentDirExist(Path path) throws IOException { + for (Path p = path.getParent(); p != null && p.getParent() != null; p = p.getParent()) { + assertObject(p, EMPTY_BYTES, true); + } + } + + /** + * Assert that all the parent directories and current directory should be existing. + * + * @param path to validate, must be a directory. + */ + public static void assertDirExist(Path path) throws IOException { + // All parent directories exist. + assertParentDirExist(path); + // The current directory exist. + assertObject(path, EMPTY_BYTES, true); + } + + public static void assertObjectNotExist(Path path) throws IOException { + assertObjectNotExist(path, false); + } + + public static void assertObjectNotExist(Path path, boolean isDir) throws IOException { + ObjectStorage store = + ObjectStorageFactory.create(path.toUri().getScheme(), path.toUri().getHost(), + new Configuration()); + String objectKey = ObjectUtils.pathToKey(path, isDir); + ObjectInfo info = store.head(objectKey); + assertNull(info, String.format("Object key %s shouldn't exist in backend storage.", objectKey)); + + store.close(); + } + + public static void assertObject(Path path, byte[] data) throws IOException { + assertObject(path, data, false); + } + + public static void assertObject(Path path, byte[] data, boolean isDir) throws IOException { + ObjectStorage store = + ObjectStorageFactory.create(path.toUri().getScheme(), path.toUri().getHost(), + new Configuration()); + String objectKey = ObjectUtils.pathToKey(path, isDir); + // Verify the existence of object. + ObjectInfo info = store.head(objectKey); + assertNotNull(info, String.format("there should be an key %s in object storage", objectKey)); + assertEquals(info.key(), objectKey); + assertEquals(data.length, info.size()); + // Verify the data content. + try (InputStream in = store.get(objectKey, 0, -1).stream()) { + byte[] actual = IOUtils.toByteArray(in); + assertArrayEquals(data, actual, "Unexpected binary"); + } + + store.close(); + } + + public static void assertMultipartUploadExist(Path path, String uploadId) throws IOException { + ObjectStorage store = + ObjectStorageFactory.create(path.toUri().getScheme(), path.toUri().getHost(), + new Configuration()); + String objectKey = ObjectUtils.pathToKey(path, false); + + Iterator uploadIterator = store.listUploads(objectKey).iterator(); + assertTrue(uploadIterator.hasNext()); + assertMultipartUploadIdExist(uploadIterator, uploadId); + + store.close(); + } + + private static void assertMultipartUploadIdExist(Iterator uploadIterator, + String uploadId) { + boolean exist = false; + while (uploadIterator.hasNext()) { + if (Objects.equals(uploadIterator.next().uploadId(), uploadId)) { + exist = true; + } + } + assertTrue(exist); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestDirectoryStorage.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestDirectoryStorage.java new file mode 100644 index 0000000000000..5d3965770afdf --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestDirectoryStorage.java @@ -0,0 +1,224 @@ +/* + * 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.tosfs.object; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.fs.tosfs.conf.TosKeys; +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.apache.hadoop.fs.tosfs.util.UUIDUtils; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +import static org.apache.hadoop.fs.tosfs.util.TestUtility.scheme; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + +public class TestDirectoryStorage { + private final ObjectStorage storage; + + public TestDirectoryStorage() { + Configuration conf = new Configuration(); + storage = + ObjectStorageFactory.createWithPrefix(String.format("%s-%s/", scheme(), UUIDUtils.random()), + scheme(), TestUtility.bucket(), conf); + } + + @BeforeAll + public static void before() { + assumeTrue(TestEnv.checkTestEnabled()); + } + + @AfterEach + public void tearDown() { + CommonUtils.runQuietly(() -> storage.deleteAll("")); + for (MultipartUpload upload : storage.listUploads("")) { + storage.abortMultipartUpload(upload.key(), upload.uploadId()); + } + } + + @Test + public void testListEmptyDir() { + String key = "testListEmptyDir/"; + mkdir(key); + assertNotNull(directoryStorage().head(key)); + + assertFalse(directoryStorage().listDir(key, false).iterator().hasNext()); + assertFalse(directoryStorage().listDir(key, false).iterator().hasNext()); + assertTrue(directoryStorage().isEmptyDir(key)); + } + + @Test + public void testListNonExistDir() { + String key = "testListNonExistDir/"; + assertNull(directoryStorage().head(key)); + + assertFalse(directoryStorage().listDir(key, false).iterator().hasNext()); + assertFalse(directoryStorage().listDir(key, false).iterator().hasNext()); + assertTrue(directoryStorage().isEmptyDir(key)); + } + + @Test + public void testRecursiveList() { + String root = "root/"; + String file1 = "root/file1"; + String file2 = "root/afile2"; + String dir1 = "root/dir1/"; + String file3 = "root/dir1/file3"; + + mkdir(root); + mkdir(dir1); + touchFile(file1, TestUtility.rand(8)); + touchFile(file2, TestUtility.rand(8)); + touchFile(file3, TestUtility.rand(8)); + + Assertions.assertThat(directoryStorage().listDir(root, false)) + .hasSize(3) + .extracting(ObjectInfo::key) + .contains(dir1, file1, file2); + + Assertions.assertThat(directoryStorage().listDir(root, true)) + .hasSize(4) + .extracting(ObjectInfo::key) + .contains(dir1, file1, file2, file3); + } + + @Test + public void testRecursiveListWithSmallBatch() { + Configuration conf = new Configuration(directoryStorage().conf()); + conf.setInt(TosKeys.FS_TOS_LIST_OBJECTS_COUNT, 5); + directoryStorage().initialize(conf, directoryStorage().bucket().name()); + + String root = "root/"; + mkdir(root); + + // Create 2 files start with 'a', 2 sub dirs start with 'b', 2 files start with 'c' + for (int i = 1; i <= 2; i++) { + touchFile("root/a-file-" + i, TestUtility.rand(8)); + mkdir("root/b-dir-" + i + "/"); + touchFile("root/c-file-" + i, TestUtility.rand(8)); + } + + // Create two files under each sub dirs. + for (int j = 1; j <= 2; j++) { + touchFile(String.format("root/b-dir-%d/file1", j), TestUtility.rand(8)); + touchFile(String.format("root/b-dir-%d/file2", j), TestUtility.rand(8)); + } + + Assertions.assertThat(directoryStorage().listDir(root, false)) + .hasSize(6) + .extracting(ObjectInfo::key) + .contains( + "root/a-file-1", "root/a-file-2", + "root/b-dir-1/", "root/b-dir-2/", + "root/c-file-1", "root/c-file-2"); + + Assertions.assertThat(directoryStorage().listDir(root, true)) + .hasSize(10) + .extracting(ObjectInfo::key) + .contains( + "root/a-file-1", "root/a-file-2", + "root/b-dir-1/", "root/b-dir-1/file1", "root/b-dir-1/file2", + "root/b-dir-2/", "root/b-dir-2/file1", "root/b-dir-2/file2", + "root/c-file-1", "root/c-file-2"); + } + + @Test + public void testRecursiveListRoot() { + String root = "root/"; + String dir1 = "root/dir1/"; + mkdir(root); + mkdir(dir1); + + Assertions.assertThat(directoryStorage().listDir("", true)) + .hasSize(2) + .extracting(ObjectInfo::key) + .contains("root/", "root/dir1/"); + } + + @Test + public void testDeleteEmptyDir() { + String dir = "a/b/"; + mkdir(dir); + + directoryStorage().deleteDir(dir, false); + assertNull(directoryStorage().head(dir)); + } + + @Test + public void testDeleteNonEmptyDir() { + String dir = "a/b/"; + String subDir = "a/b/c/"; + String file = "a/b/file.txt"; + mkdir(dir); + mkdir(subDir); + touchFile(file, new byte[10]); + + assertThrows(RuntimeException.class, () -> directoryStorage().deleteDir(dir, false)); + assertNotNull(directoryStorage().head(dir)); + assertNotNull(directoryStorage().head(subDir)); + assertNotNull(directoryStorage().head(file)); + + directoryStorage().deleteDir(dir, true); + assertNull(directoryStorage().head(dir)); + assertNull(directoryStorage().head(subDir)); + assertNull(directoryStorage().head(file)); + } + + @Test + public void testRecursiveDeleteDirViaTosSDK() { + Configuration conf = new Configuration(directoryStorage().conf()); + conf.setBoolean(TosKeys.FS_TOS_RMR_CLIENT_ENABLE, true); + directoryStorage().initialize(conf, directoryStorage().bucket().name()); + + testDeleteNonEmptyDir(); + } + + // TOS doesn't enable recursive delete in server side currently. + @Disabled + @Test + public void testAtomicDeleteDir() { + Configuration conf = new Configuration(directoryStorage().conf()); + conf.setBoolean(TosKeys.FS_TOS_RMR_SERVER_ENABLED, true); + directoryStorage().initialize(conf, directoryStorage().bucket().name()); + + testDeleteNonEmptyDir(); + } + + private void touchFile(String key, byte[] data) { + directoryStorage().put(key, data); + } + + private void mkdir(String key) { + directoryStorage().put(key, new byte[0]); + } + + private DirectoryStorage directoryStorage() { + assumeTrue(storage.bucket().isDirectory()); + return (DirectoryStorage) storage; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectMultiRangeInputStream.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectMultiRangeInputStream.java new file mode 100644 index 0000000000000..ee73e9db6d1e8 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectMultiRangeInputStream.java @@ -0,0 +1,451 @@ +/* + * 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.tosfs.object; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.common.Bytes; +import org.apache.hadoop.fs.tosfs.common.ThreadPools; +import org.apache.hadoop.fs.tosfs.object.exceptions.ChecksumMismatchException; +import org.apache.hadoop.fs.tosfs.util.Range; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestObjectMultiRangeInputStream extends ObjectStorageTestBase { + private static ExecutorService threadPool; + + @BeforeAll + public static void beforeClass() { + threadPool = ThreadPools.newWorkerPool("TestObjectInputStream-pool"); + } + + @AfterAll + public static void afterClass() { + if (!threadPool.isShutdown()) { + threadPool.shutdown(); + } + } + + @Test + public void testSequentialAndRandomRead() throws IOException { + Path outPath = new Path(testDir(), "testSequentialAndRandomRead.txt"); + String key = ObjectUtils.pathToKey(outPath); + byte[] rawData = TestUtility.rand(5 << 20); + getStorage().put(key, rawData); + + ObjectContent content = getStorage().get(key); + assertArrayEquals(rawData, IOUtils.toByteArray(content.stream())); + + try (ObjectMultiRangeInputStream in = new ObjectMultiRangeInputStream(threadPool, getStorage(), + ObjectUtils.pathToKey(outPath), rawData.length, Long.MAX_VALUE, content.checksum())) { + // sequential read + assertEquals(0, in.getPos()); + assertEquals(0, in.nextExpectPos()); + + byte[] b = new byte[1024]; + int readCnt = in.read(b); + assertEquals(readCnt, b.length); + assertArrayEquals(Arrays.copyOfRange(rawData, 0, 1024), b); + assertEquals(1024, in.getPos()); + assertEquals(1024, in.nextExpectPos()); + + readCnt = in.read(b); + assertEquals(readCnt, b.length); + assertArrayEquals(Arrays.copyOfRange(rawData, 1024, 2048), b); + assertEquals(2048, in.getPos()); + assertEquals(2048, in.nextExpectPos()); + + // random read forward + in.seek(4 << 20); + assertEquals(4 << 20, in.getPos()); + assertEquals(2048, in.nextExpectPos()); + + readCnt = in.read(b); + assertEquals(readCnt, b.length); + assertArrayEquals(Arrays.copyOfRange(rawData, 4 << 20, 1024 + (4 << 20)), b); + assertEquals((4 << 20) + 1024, in.getPos()); + assertEquals((4 << 20) + 1024, in.nextExpectPos()); + + // random read back + in.seek(2 << 20); + assertEquals(2 << 20, in.getPos()); + assertEquals((4 << 20) + 1024, in.nextExpectPos()); + + readCnt = in.read(b); + assertEquals(readCnt, b.length); + assertArrayEquals(Arrays.copyOfRange(rawData, 2 << 20, 1024 + (2 << 20)), b); + assertEquals((2 << 20) + 1024, in.getPos()); + assertEquals((2 << 20) + 1024, in.nextExpectPos()); + } + } + + private InputStream getStream(String key) { + return getStorage().get(key).stream(); + } + + @Test + public void testReadSingleByte() throws IOException { + int len = 10; + Path outPath = new Path(testDir(), "testReadSingleByte.txt"); + byte[] data = TestUtility.rand(len); + String key = ObjectUtils.pathToKey(outPath); + byte[] checksum = getStorage().put(key, data); + + try (InputStream in = new ObjectMultiRangeInputStream(threadPool, getStorage(), key, + data.length, Long.MAX_VALUE, checksum)) { + for (int i = 0; i < data.length; i++) { + assertTrue(in.read() >= 0); + } + assertEquals(-1, in.read()); + } + } + + @Test + public void testReadStreamButTheFileChangedDuringReading() throws IOException { + int len = 2048; + Path outPath = new Path(testDir(), "testReadStreamButTheFileChangedDuringReading.txt"); + byte[] data = TestUtility.rand(len); + String key = ObjectUtils.pathToKey(outPath); + byte[] checksum = getStorage().put(key, data); + + try (InputStream in = new ObjectMultiRangeInputStream(threadPool, getStorage(), key, + data.length, 1024, checksum)) { + byte[] read = new byte[1024]; + int n = in.read(read); + assertEquals(1024, n); + + getStorage().put(key, TestUtility.rand(1024)); + assertThrows(ChecksumMismatchException.class, () -> in.read(read), "The file is staled"); + } + } + + @Test + public void testRead100M() throws IOException { + testSequentialReadData(100 << 20, 6 << 20); + testSequentialReadData(100 << 20, 5 << 20); + } + + @Test + public void testRead10M() throws IOException { + testSequentialReadData(10 << 20, 4 << 20); + testSequentialReadData(10 << 20, 5 << 20); + } + + @Test + public void testParallelRead10M() throws IOException, ExecutionException, InterruptedException { + testParallelRandomRead(10 << 20, 4 << 20); + testParallelRandomRead(10 << 20, 5 << 20); + } + + @Test + public void testRead100b() throws IOException { + testSequentialReadData(100, 40); + testSequentialReadData(100, 50); + testSequentialReadData(100, 100); + testSequentialReadData(100, 101); + } + + private void testSequentialReadData(int dataSize, int partSize) throws IOException { + Path outPath = new Path(testDir(), String.format("%d-%d.txt", dataSize, partSize)); + String key = ObjectUtils.pathToKey(outPath); + byte[] rawData = TestUtility.rand(dataSize); + getStorage().put(key, rawData); + + ObjectContent content = getStorage().get(key); + assertArrayEquals(rawData, IOUtils.toByteArray(content.stream())); + + int batchSize = (dataSize - 1) / partSize + 1; + try (InputStream in = new ObjectMultiRangeInputStream(threadPool, getStorage(), + ObjectUtils.pathToKey(outPath), rawData.length, Long.MAX_VALUE, content.checksum())) { + for (int i = 0; i < batchSize; i++) { + int start = i * partSize; + int end = Math.min(dataSize, start + partSize); + byte[] expectArr = Arrays.copyOfRange(rawData, start, end); + + byte[] b = new byte[end - start]; + int ret = in.read(b, 0, b.length); + + assertEquals(b.length, ret); + assertArrayEquals(expectArr, b, String.format("the read bytes mismatched at batch: %d", i)); + } + assertEquals(-1, in.read()); + } + } + + private void testParallelRandomRead(int dataSize, int partSize) + throws IOException, ExecutionException, InterruptedException { + + Path outPath = new Path(testDir(), String.format("%d-%d.txt", dataSize, partSize)); + String key = ObjectUtils.pathToKey(outPath); + byte[] rawData = TestUtility.rand(dataSize); + getStorage().put(key, rawData); + + ObjectContent content = getStorage().get(key); + assertArrayEquals(rawData, IOUtils.toByteArray(content.stream())); + + Random random = new Random(); + List> tasks = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + int position = random.nextInt(rawData.length); + tasks.add(threadPool.submit( + () -> testReadDataFromSpecificPosition(rawData, outPath, position, partSize, + content.checksum()))); + } + + for (Future task : tasks) { + assertTrue(task.get()); + } + } + + private boolean testReadDataFromSpecificPosition( + final byte[] rawData, + final Path objPath, + final int startPosition, + final int partSize, + byte[] checksum) { + int rawDataSize = rawData.length; + int batchSize = (rawDataSize - startPosition - 1) / partSize + 1; + try (ObjectMultiRangeInputStream in = new ObjectMultiRangeInputStream(threadPool, getStorage(), + ObjectUtils.pathToKey(objPath), rawDataSize, Long.MAX_VALUE, checksum)) { + in.seek(startPosition); + + for (int i = 0; i < batchSize; i++) { + int start = startPosition + i * partSize; + int end = Math.min(rawDataSize, start + partSize); + byte[] expectArr = Arrays.copyOfRange(rawData, start, end); + + byte[] b = new byte[end - start]; + int ret = in.read(b, 0, b.length); + + assertEquals(b.length, ret); + assertArrayEquals(expectArr, b, String.format("the read bytes mismatched at batch: %d", i)); + } + assertEquals(-1, in.read()); + return true; + } catch (IOException e) { + return false; + } + } + + @Test + public void testParallelReadFromOneInputStream() throws IOException, ExecutionException, + InterruptedException { + testParallelReadFromOneInputStreamImpl(10 << 20, 512, 10); + testParallelReadFromOneInputStreamImpl(10 << 20, 64, 100); + testParallelReadFromOneInputStreamImpl(1 << 20, 2 << 20, 5); + } + + public void testParallelReadFromOneInputStreamImpl(int dataSize, int batchSize, int parallel) + throws IOException, ExecutionException, InterruptedException { + + Path outPath = new Path(testDir(), + String.format("%d-%d-testParallelReadFromOneInputStreamImpl.txt", dataSize, batchSize)); + String key = ObjectUtils.pathToKey(outPath); + byte[] rawData = TestUtility.rand(dataSize); + getStorage().put(key, rawData); + ObjectContent content = getStorage().get(key); + assertArrayEquals(rawData, IOUtils.toByteArray(content.stream())); + + AtomicInteger sum = new AtomicInteger(0); + CopyOnWriteArrayList readBytes = new CopyOnWriteArrayList(); + List> futures = new ArrayList<>(); + try (ObjectMultiRangeInputStream inputStream = new ObjectMultiRangeInputStream(threadPool, + getStorage(), ObjectUtils.pathToKey(outPath), rawData.length, Long.MAX_VALUE, + content.checksum())) { + for (int i = 0; i < parallel; i++) { + futures.add(threadPool.submit(() -> { + byte[] data = new byte[batchSize]; + try { + int count; + while ((count = inputStream.read(data)) != -1) { + sum.getAndAdd(count); + readBytes.add(Arrays.copyOfRange(data, 0, count)); + data = new byte[batchSize]; + } + } catch (IOException e) { + throw new RuntimeException(e); + } + })); + } + + for (Future future : futures) { + future.get(); + + } + assertEquals(rawData.length, sum.get()); + } + + byte[] actualBytes = new byte[rawData.length]; + int offset = 0; + for (byte[] bytes : readBytes) { + System.arraycopy(bytes, 0, actualBytes, offset, bytes.length); + offset += bytes.length; + } + + Arrays.sort(actualBytes); + Arrays.sort(rawData); + assertArrayEquals(rawData, actualBytes); + } + + @Test + public void testPositionalRead() throws IOException { + Path outPath = new Path(testDir(), "testPositionalRead.txt"); + String key = ObjectUtils.pathToKey(outPath); + int fileSize = 5 << 20; + byte[] rawData = TestUtility.rand(fileSize); + getStorage().put(key, rawData); + ObjectContent content = getStorage().get(key); + assertArrayEquals(rawData, IOUtils.toByteArray(content.stream())); + + Random rand = ThreadLocalRandom.current(); + + try (ObjectMultiRangeInputStream in = new ObjectMultiRangeInputStream(threadPool, getStorage(), + ObjectUtils.pathToKey(outPath), fileSize, Long.MAX_VALUE, content.checksum())) { + for (int i = 0; i < 100; i++) { + int pos = rand.nextInt(fileSize); + int len = rand.nextInt(fileSize); + + int expectSize = Math.min(fileSize - pos, len); + byte[] actual = new byte[expectSize]; + int actualLen = in.read(pos, actual, 0, expectSize); + + assertEquals(expectSize, actualLen); + assertArrayEquals(Bytes.toBytes(rawData, pos, expectSize), actual); + } + } + } + + @Test + public void testReadAcrossRange() throws IOException { + Path outPath = new Path(testDir(), "testReadAcrossRange.txt"); + String key = ObjectUtils.pathToKey(outPath); + int fileSize = 1 << 10; + byte[] rawData = TestUtility.rand(fileSize); + getStorage().put(key, rawData); + ObjectContent content = getStorage().get(key); + assertArrayEquals(rawData, IOUtils.toByteArray(content.stream())); + + try (ObjectMultiRangeInputStream in = new ObjectMultiRangeInputStream( + ThreadPools.defaultWorkerPool(), getStorage(), key, fileSize, 10, content.checksum())) { + byte[] data = new byte[fileSize / 2]; + for (int i = 0; i < 2; i++) { + assertEquals(data.length, in.read(data)); + assertEquals((i + 1) * data.length, in.getPos()); + assertArrayEquals(Bytes.toBytes(rawData, i * data.length, data.length), data); + } + } + } + + @Test + public void testStorageRange() throws IOException { + Path outPath = new Path(testDir(), "testStorageRange.txt"); + String key = ObjectUtils.pathToKey(outPath); + int fileSize = 5 << 20; + byte[] rawData = TestUtility.rand(fileSize); + getStorage().put(key, rawData); + ObjectContent content = getStorage().get(key); + assertArrayEquals(rawData, IOUtils.toByteArray(content.stream())); + + int oneMB = 1 << 20; + long rangeOpenLen = oneMB; + try (ObjectMultiRangeInputStream in = new ObjectMultiRangeInputStream( + ThreadPools.defaultWorkerPool(), getStorage(), key, fileSize, rangeOpenLen, + content.checksum())) { + assertNull(in.stream()); + + // Init range. + in.read(); + assertEquals(Range.of(0, rangeOpenLen), in.stream().range()); + // Range doesn't change. + in.read(new byte[(int) (rangeOpenLen - 1)], 0, (int) (rangeOpenLen - 1)); + assertEquals(Range.of(0, rangeOpenLen), in.stream().range()); + + // Move to next range. + in.read(); + assertEquals(Range.of(rangeOpenLen, rangeOpenLen), in.stream().range()); + + // Seek and move. + in.seek(rangeOpenLen * 3 + 10); + in.read(); + assertEquals(Range.of(rangeOpenLen * 3, rangeOpenLen), in.stream().range()); + + // Seek small and range doesn't change. + in.seek(in.getPos() + 1); + in.read(); + assertEquals(Range.of(rangeOpenLen * 3, rangeOpenLen), in.stream().range()); + + // Seek big and range changes. + in.seek(rangeOpenLen * 2); + in.read(new byte[(int) (rangeOpenLen - 10)], 0, (int) (rangeOpenLen - 10)); + assertEquals(Range.of(rangeOpenLen * 2, rangeOpenLen), in.stream().range()); + // Old range has 10 bytes left. Seek 10 bytes then read 10 bytes. Old range can't read any + // bytes, so range changes. + assertEquals(rangeOpenLen * 3 - 10, in.getPos()); + in.seek(rangeOpenLen * 3); + in.read(new byte[10], 0, 10); + assertEquals(Range.of(rangeOpenLen * 3, rangeOpenLen), in.stream().range()); + + // Read big buffer. + in.seek(10); + in.read(new byte[oneMB * 3], 0, oneMB * 3); + assertEquals(oneMB * 3 + 10, in.getPos()); + assertEquals(Range.of(3 * rangeOpenLen, rangeOpenLen), in.stream().range()); + } + + try (ObjectMultiRangeInputStream in = new ObjectMultiRangeInputStream(threadPool, getStorage(), + ObjectUtils.pathToKey(outPath), fileSize, Long.MAX_VALUE, content.checksum())) { + assertNull(in.stream()); + + // Init range. + in.read(); + assertEquals(Range.of(0, fileSize), in.stream().range()); + + // Range doesn't change. + in.read(new byte[oneMB], 0, oneMB); + assertEquals(Range.of(0, fileSize), in.stream().range()); + + // Seek and move. + long pos = oneMB * 3 + 10; + in.seek(pos); + in.read(); + assertEquals(Range.of(0, fileSize), in.stream().range()); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectOutputStream.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectOutputStream.java new file mode 100644 index 0000000000000..7c459dc29dece --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectOutputStream.java @@ -0,0 +1,423 @@ +/* + * 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.tosfs.object; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.common.ThreadPools; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.object.staging.StagingPart; +import org.apache.hadoop.fs.tosfs.object.staging.State; +import org.apache.hadoop.fs.tosfs.util.FSUtils; +import org.apache.hadoop.fs.tosfs.util.TempFiles; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.apache.hadoop.thirdparty.com.google.common.base.Joiner; +import org.apache.hadoop.util.Lists; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestObjectOutputStream extends ObjectStorageTestBase { + + private static ExecutorService threadPool; + + @BeforeAll + public static void beforeClass() { + threadPool = ThreadPools.newWorkerPool("TestObjectOutputStream-pool"); + } + + @AfterAll + public static void afterClass() { + if (!threadPool.isShutdown()) { + threadPool.shutdown(); + } + } + + @Test + public void testMkStagingDir() throws ExecutionException, InterruptedException, IOException { + try (TempFiles tmp = TempFiles.of()) { + List tmpDirs = Lists.newArrayList(); + for (int i = 0; i < 3; i++) { + tmpDirs.add(tmp.newDir()); + } + Configuration newConf = new Configuration(tosConf()); + newConf.set(ConfKeys.FS_MULTIPART_STAGING_DIR.key("filestore"), Joiner.on(",").join(tmpDirs)); + + // Start multiple threads to open streams to create staging dir. + List> futures = Collections.synchronizedList(new ArrayList<>()); + for (int i = 0; i < 10; i++) { + futures.add(threadPool.submit(() -> + new ObjectOutputStream(getStorage(), threadPool, newConf, path("none.txt"), true))); + } + for (Future f : futures) { + f.get().close(); + } + } + } + + @Test + public void testWriteZeroByte() throws IOException { + Path zeroByteTxt = path("zero-byte.txt"); + ObjectOutputStream out = + new ObjectOutputStream(getStorage(), threadPool, tosConf(), zeroByteTxt, true); + // write zero-byte and close. + out.write(new byte[0], 0, 0); + out.close(); + assertStagingPart(0, out.stagingParts()); + + // Read and validate the dest object contents + ObjectTestUtils.assertObject(zeroByteTxt, ObjectTestUtils.EMPTY_BYTES); + } + + @Test + public void testWriteZeroByteWithoutAllowPut() throws IOException { + Path zeroByteTxt = path("zero-byte-without-allow-put.txt"); + ObjectOutputStream out = + new ObjectOutputStream(getStorage(), threadPool, tosConf(), zeroByteTxt, false); + // write zero-byte and close. + out.close(); + assertStagingPart(0, out.stagingParts()); + + // Read and validate the dest object content. + ObjectTestUtils.assertObject(zeroByteTxt, ObjectTestUtils.EMPTY_BYTES); + } + + @Test + public void testDeleteStagingFileWhenUploadPartsOK() throws IOException { + Path path = path("data.txt"); + ObjectOutputStream out = + new ObjectOutputStream(getStorage(), threadPool, tosConf(), path, true); + byte[] data = TestUtility.rand((int) (ConfKeys.FS_MULTIPART_SIZE_DEFAULT * 2)); + out.write(data); + out.waitForPartsUpload(); + for (StagingPart part : out.stagingParts()) { + assertEquals(State.CLEANED, part.state()); + } + out.close(); + for (StagingPart part : out.stagingParts()) { + assertEquals(State.CLEANED, part.state()); + } + } + + @Test + public void testDeleteStagingFileWithClose() throws IOException { + Path path = path("data.txt"); + ObjectOutputStream out = + new ObjectOutputStream(getStorage(), threadPool, tosConf(), path, true); + byte[] data = TestUtility.rand((int) (ConfKeys.FS_MULTIPART_SIZE_DEFAULT * 2)); + out.write(data); + out.close(); + for (StagingPart part : out.stagingParts()) { + assertEquals(State.CLEANED, part.state()); + } + } + + @Test + public void testDeleteSimplePutStagingFile() throws IOException { + Path smallTxt = path("small.txt"); + ObjectOutputStream out = + new ObjectOutputStream(getStorage(), threadPool, tosConf(), smallTxt, true); + byte[] data = TestUtility.rand(4 << 20); + out.write(data); + for (StagingPart part : out.stagingParts()) { + assertTrue(part.size() > 0); + } + out.close(); + for (StagingPart part : out.stagingParts()) { + assertEquals(State.CLEANED, part.state()); + } + } + + @Test + public void testSimplePut() throws IOException { + Path smallTxt = path("small.txt"); + ObjectOutputStream out = + new ObjectOutputStream(getStorage(), threadPool, tosConf(), smallTxt, true); + byte[] data = TestUtility.rand(4 << 20); + out.write(data); + out.close(); + assertStagingPart(1, out.stagingParts()); + assertNull(out.upload(), "Should use the simple PUT to upload object for small file."); + + // Read and validate the dest object content. + ObjectTestUtils.assertObject(smallTxt, data); + } + + public void testWrite(int uploadPartSize, int len) throws IOException { + Configuration newConf = new Configuration(tosConf()); + newConf.setLong(ConfKeys.FS_MULTIPART_SIZE.key(FSUtils.scheme(conf(), testDir().toUri())), + uploadPartSize); + + Path outPath = path(len + ".txt"); + int partNum = (len - 1) / uploadPartSize + 1; + + byte[] data = TestUtility.rand(len); + ObjectOutputStream out = + new ObjectOutputStream(getStorage(), threadPool, newConf, outPath, true); + try { + out.write(data); + } finally { + out.close(); + } + + assertStagingPart(partNum, out.stagingParts()); + ObjectTestUtils.assertObject(outPath, data); + + // List multipart uploads + int uploadsNum = 0; + for (MultipartUpload ignored : getStorage().listUploads(out.destKey())) { + uploadsNum += 1; + } + assertEquals(0L, uploadsNum); + } + + @Test + public void testParallelWriteOneOutPutStream() throws IOException, ExecutionException, + InterruptedException { + testParallelWriteOneOutPutStreamImpl(5 << 20, 10, 128); + testParallelWriteOneOutPutStreamImpl(5 << 20, 10, 1 << 20); + testParallelWriteOneOutPutStreamImpl(5 << 20, 10, 2 << 20); + testParallelWriteOneOutPutStreamImpl(5 << 20, 10, 6 << 20); + } + + public void testParallelWriteOneOutPutStreamImpl(int partSize, int epochs, int batchSize) + throws IOException, ExecutionException, InterruptedException { + Configuration newConf = new Configuration(tosConf()); + newConf.setLong(ConfKeys.FS_MULTIPART_SIZE.key(FSUtils.scheme(conf(), testDir().toUri())), + partSize); + + String file = + String.format("%d-%d-%d-testParallelWriteOneOutPutStream.txt", partSize, epochs, batchSize); + Path outPath = path(file); + try (ObjectOutputStream out = new ObjectOutputStream(getStorage(), threadPool, newConf, outPath, + true)) { + List> futures = new ArrayList<>(); + for (int i = 0; i < epochs; i++) { + final int index = i; + futures.add(threadPool.submit(() -> { + try { + out.write(dataset(batchSize, index)); + } catch (IOException e) { + throw new RuntimeException(e); + } + })); + } + + // wait for all tasks finished + for (Future future : futures) { + future.get(); + } + } + + try (InputStream inputStream = getStorage().get(ObjectUtils.pathToKey(outPath)).stream()) { + List ret = new ArrayList<>(); + byte[] data = new byte[batchSize]; + while (inputStream.read(data) != -1) { + ret.add(data); + data = new byte[batchSize]; + } + + assertEquals(epochs, ret.size()); + List sortedRet = ret.stream() + .sorted(Comparator.comparingInt(o -> o[0])) + .collect(Collectors.toList()); + + int j = 0; + for (byte[] e : sortedRet) { + assertArrayEquals(dataset(batchSize, j), e); + j++; + } + } + } + + public static byte[] dataset(int len, int base) { + byte[] dataset = new byte[len]; + for (int i = 0; i < len; i++) { + dataset[i] = (byte) (base); + } + return dataset; + } + + @Test + public void testWrite1MB() throws IOException { + testWrite(5 << 20, 1 << 20); + testWrite(8 << 20, 1 << 20); + testWrite(16 << 20, 1 << 20); + } + + @Test + public void testWrite24MB() throws IOException { + testWrite(5 << 20, 24 << 20); + testWrite(8 << 20, 24 << 20); + testWrite(16 << 20, 24 << 20); + } + + @Test + public void testWrite100MB() throws IOException { + testWrite(5 << 20, 100 << 20); + testWrite(8 << 20, 100 << 20); + testWrite(16 << 20, 100 << 20); + } + + private void testMultipartThreshold(int partSize, int multipartThreshold, int dataSize) + throws IOException { + Configuration newConf = new Configuration(tosConf()); + newConf.setLong(ConfKeys.FS_MULTIPART_SIZE.key(scheme()), partSize); + newConf.setLong(ConfKeys.FS_MULTIPART_THRESHOLD.key(scheme()), multipartThreshold); + Path outPath = + path(String.format("threshold-%d-%d-%d.txt", partSize, multipartThreshold, dataSize)); + + byte[] data = TestUtility.rand(dataSize); + ObjectOutputStream out = + new ObjectOutputStream(getStorage(), threadPool, newConf, outPath, true); + try { + // Verify for every 1MB data writing, unless reaching the threshold. + int upperLimit = Math.min(multipartThreshold, dataSize); + int curOff = 0; + for (; curOff < upperLimit; curOff += (1 << 20)) { + int end = Math.min(curOff + (1 << 20), upperLimit); + out.write(Arrays.copyOfRange(data, curOff, end)); + + List uploads = Lists.newArrayList(getStorage().listUploads(out.destKey())); + if (end < multipartThreshold) { + assertEquals(0, uploads.size(), + "Shouldn't has any uploads because it just use simple PUT"); + } else { + assertEquals(1, uploads.size(), "Switch to use MPU."); + } + assertEquals((end - 1) / partSize + 1, out.stagingParts().size()); + } + + // Verify for every 1MB data writing, unless reaching the data size. + for (; curOff < dataSize; curOff += (1 << 20)) { + int end = Math.min(curOff + (1 << 20), dataSize); + out.write(Arrays.copyOfRange(data, curOff, end)); + + List uploads = Lists.newArrayList(getStorage().listUploads(out.destKey())); + assertEquals(1, uploads.size()); + assertEquals(out.destKey(), uploads.get(0).key()); + assertEquals((end - 1) / partSize + 1, out.stagingParts().size()); + } + } finally { + out.close(); + } + + assertStagingPart((dataSize - 1) / partSize + 1, out.stagingParts()); + ObjectTestUtils.assertObject(outPath, data); + + List uploads = Lists.newArrayList(getStorage().listUploads(out.destKey())); + assertEquals(0, uploads.size()); + } + + @Test + public void testMultipartThreshold2MB() throws IOException { + testMultipartThreshold(5 << 20, 2 << 20, 1 << 20); + testMultipartThreshold(5 << 20, 2 << 20, (2 << 20) - 1); + testMultipartThreshold(5 << 20, 2 << 20, 2 << 20); + testMultipartThreshold(5 << 20, 2 << 20, 4 << 20); + testMultipartThreshold(5 << 20, 2 << 20, 5 << 20); + testMultipartThreshold(5 << 20, 2 << 20, (5 << 20) + 1); + testMultipartThreshold(5 << 20, 2 << 20, 6 << 20); + testMultipartThreshold(5 << 20, 2 << 20, 10 << 20); + testMultipartThreshold(5 << 20, 2 << 20, 20 << 20); + } + + @Test + public void testMultipartThreshold5MB() throws IOException { + testMultipartThreshold(5 << 20, 5 << 20, 1 << 20); + testMultipartThreshold(5 << 20, 5 << 20, 4 << 20); + testMultipartThreshold(5 << 20, 5 << 20, 5 << 20); + testMultipartThreshold(5 << 20, 5 << 20, 5 << 20); + testMultipartThreshold(5 << 20, 5 << 20, 6 << 20); + testMultipartThreshold(5 << 20, 5 << 20, 10 << 20); + testMultipartThreshold(5 << 20, 5 << 20, 20 << 20); + } + + @Test + public void testMultipartThreshold10MB() throws IOException { + testMultipartThreshold(5 << 20, 10 << 20, 1 << 20); + testMultipartThreshold(5 << 20, 10 << 20, 10 << 20); + testMultipartThreshold(5 << 20, 10 << 20, 11 << 20); + testMultipartThreshold(5 << 20, 10 << 20, 15 << 20); + testMultipartThreshold(5 << 20, 10 << 20, 20 << 20); + testMultipartThreshold(5 << 20, 10 << 20, 40 << 20); + testMultipartThreshold(5 << 20, 10 << 20, 30 << 20); + } + + @Test + public void testCloseStreamTwice() throws IOException { + int len = 100; + Path outPath = path(len + ".txt"); + int partNum = 1; + + byte[] data = TestUtility.rand(len); + ObjectOutputStream out = + new ObjectOutputStream(getStorage(), threadPool, tosConf(), outPath, true); + try { + out.write(data); + out.close(); + } finally { + out.close(); + } + + assertStagingPart(partNum, out.stagingParts()); + ObjectTestUtils.assertObject(outPath, data); + } + + @Test + public void testWriteClosedStream() throws IOException { + byte[] data = TestUtility.rand(10); + Path outPath = path("testWriteClosedStream.txt"); + try (ObjectOutputStream out = new ObjectOutputStream(getStorage(), threadPool, tosConf(), + outPath, true)) { + out.close(); + out.write(data); + } catch (IllegalStateException e) { + assertEquals("OutputStream is closed.", e.getMessage()); + } + } + + private static void assertStagingPart(int expectedNum, List parts) { + assertEquals(expectedNum, parts.size()); + for (StagingPart part : parts) { + assertTrue(part.size() > 0); + } + } + + private Path path(String name) { + return new Path(testDir(), name); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectRangeInputStream.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectRangeInputStream.java new file mode 100644 index 0000000000000..45db5c66fb6bc --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectRangeInputStream.java @@ -0,0 +1,144 @@ +/* + * 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.tosfs.object; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.util.Range; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Arrays; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestObjectRangeInputStream extends ObjectStorageTestBase { + + @Test + public void testRead() throws IOException { + Path outPath = new Path(testDir(), "testRead.txt"); + String key = ObjectUtils.pathToKey(outPath); + byte[] rawData = TestUtility.rand(1 << 10); + getStorage().put(key, rawData); + ObjectContent content = getStorage().get(key); + assertArrayEquals(rawData, IOUtils.toByteArray(content.stream())); + + int position = 100; + int len = 200; + try (ObjectRangeInputStream ri = new ObjectRangeInputStream(getStorage(), key, + Range.of(position, len), content.checksum())) { + // Test read byte. + assertEquals(rawData[position] & 0xff, ri.read()); + + // Test read buffer. + byte[] buffer = new byte[len]; + assertEquals(buffer.length - 1, ri.read(buffer, 0, buffer.length)); + assertArrayEquals( + Arrays.copyOfRange(rawData, position + 1, position + len), + Arrays.copyOfRange(buffer, 0, buffer.length - 1)); + assertEquals(0, ri.available()); + + assertEquals(-1, ri.read()); + assertEquals(-1, ri.read(buffer, 0, buffer.length)); + } + } + + @Test + public void testRangeExceedInnerStream() throws IOException { + Path outPath = new Path(testDir(), "testRangeExceedInnerStream.txt"); + String key = ObjectUtils.pathToKey(outPath); + byte[] rawData = TestUtility.rand(10); + getStorage().put(key, rawData); + ObjectContent content = getStorage().get(key); + assertArrayEquals(rawData, IOUtils.toByteArray(content.stream())); + + int position = 10; + int badLen = 10; + try (ObjectRangeInputStream ri = new ObjectRangeInputStream(getStorage(), key, + Range.of(position, badLen), content.checksum())) { + byte[] buffer = new byte[1]; + assertEquals(-1, ri.read()); + assertEquals(-1, ri.read(buffer, 0, buffer.length)); + } + } + + @Test + public void testRangeInclude() throws IOException { + Path outPath = new Path(testDir(), "testRangeInclude.txt"); + String key = ObjectUtils.pathToKey(outPath); + byte[] rawData = TestUtility.rand(10); + getStorage().put(key, rawData); + ObjectContent content = getStorage().get(key); + assertArrayEquals(rawData, IOUtils.toByteArray(content.stream())); + + long pos = 100; + long len = 300; + + try (ObjectRangeInputStream in = new ObjectRangeInputStream(getStorage(), key, + Range.of(pos, len), content.checksum())) { + assertEquals(Range.of(pos, len), in.range()); + + assertTrue(in.include(pos)); + assertTrue(in.include((pos + len) / 2)); + assertTrue(in.include(pos + len - 1)); + + assertFalse(in.include(pos - 1)); + assertFalse(in.include(pos + len)); + } + } + + @Test + public void testSeek() throws IOException { + Path outPath = new Path(testDir(), "testSeek.txt"); + String key = ObjectUtils.pathToKey(outPath); + byte[] rawData = TestUtility.rand(1 << 10); + getStorage().put(key, rawData); + ObjectContent content = getStorage().get(key); + assertArrayEquals(rawData, IOUtils.toByteArray(content.stream())); + + long pos = 100; + long len = 300; + + try (ObjectRangeInputStream in = new ObjectRangeInputStream(getStorage(), key, + Range.of(pos, len), content.checksum())) { + assertEquals(pos, in.getPos()); + + Exception error = assertThrows(IllegalArgumentException.class, () -> in.seek(-1), "Overflow"); + assertTrue(error.getMessage().contains("must be in range Range{offset=100, length=300}")); + error = assertThrows(IllegalArgumentException.class, () -> in.seek(99), "Overflow"); + assertTrue(error.getMessage().contains("must be in range Range{offset=100, length=300}")); + error = assertThrows(IllegalArgumentException.class, () -> in.seek(401), "Overflow"); + assertTrue(error.getMessage().contains("must be in range Range{offset=100, length=300}")); + error = assertThrows(IllegalArgumentException.class, () -> in.seek(1 << 20), "Overflow"); + assertTrue(error.getMessage().contains("must be in range Range{offset=100, length=300}")); + + in.seek(399); + assertTrue(0 <= in.read()); + assertEquals(-1, in.read()); + + in.seek(100); + assertTrue(in.read() >= 0); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectStorage.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectStorage.java new file mode 100644 index 0000000000000..1c18219404f26 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectStorage.java @@ -0,0 +1,1495 @@ +/* + * 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.tosfs.object; + +import com.volcengine.tos.TosServerException; +import org.apache.commons.codec.digest.DigestUtils; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.fs.tosfs.conf.TosKeys; +import org.apache.hadoop.fs.tosfs.object.exceptions.InvalidObjectKeyException; +import org.apache.hadoop.fs.tosfs.object.exceptions.NotAppendableException; +import org.apache.hadoop.fs.tosfs.object.request.ListObjectsRequest; +import org.apache.hadoop.fs.tosfs.object.response.ListObjectsResponse; +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.fs.tosfs.util.TempFiles; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.apache.hadoop.fs.tosfs.util.UUIDUtils; +import org.apache.hadoop.thirdparty.com.google.common.collect.Maps; +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.util.Preconditions; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assumptions.assumeFalse; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + +public class TestObjectStorage { + private static final String FILE_STORE_ROOT = TempFiles.newTempDir("TestObjectStorage"); + private ObjectStorage storage; + + public static Stream provideArguments() { + assumeTrue(TestEnv.checkTestEnabled()); + + List values = new ArrayList<>(); + for (ObjectStorage store : TestUtility.createTestObjectStorage(FILE_STORE_ROOT)) { + values.add(Arguments.of(store)); + } + return values.stream(); + } + + private void setEnv(ObjectStorage objectStore) { + this.storage = objectStore; + } + + @AfterEach + public void tearDown() { + CommonUtils.runQuietly(() -> storage.deleteAll("")); + for (MultipartUpload upload : storage.listUploads("")) { + storage.abortMultipartUpload(upload.key(), upload.uploadId()); + } + } + + @AfterAll + public static void afterClass() throws Exception { + CommonUtils.runQuietly(() -> TempFiles.deleteDir(FILE_STORE_ROOT)); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testHeadNonExistObject(ObjectStorage store) { + setEnv(store); + assertNull(storage.head("a/b/c.txt")); + + byte[] data = TestUtility.rand(256); + storage.put("a/b/c.txt", data); + assertNotNull(storage.head("a/b/c.txt")); + + assertNull(storage.head("a/b/c/non-exits")); + if (storage.bucket().isDirectory()) { + assertThrows(InvalidObjectKeyException.class, () -> storage.head("a/b/c.txt/non-exits")); + } else { + assertNull(storage.head("a/b/c.txt/non-exits")); + } + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testHeadExistObject(ObjectStorage store) { + setEnv(store); + byte[] data = TestUtility.rand(256); + String key = "testHeadExistObject.txt"; + storage.put(key, data); + + ObjectInfo obj = storage.head(key); + assertEquals(key, obj.key()); + assertFalse(obj.isDir()); + if (storage.bucket().isDirectory()) { + assertThrows(InvalidObjectKeyException.class, () -> storage.head(key + "/")); + } else { + assertNull(storage.head(key + "/")); + } + + String dirKey = "testHeadExistObject/"; + storage.put(dirKey, new byte[0]); + obj = storage.head(dirKey); + assertEquals(dirKey, obj.key()); + assertTrue(obj.isDir()); + + if (storage.bucket().isDirectory()) { + obj = storage.head("testHeadExistObject"); + assertEquals("testHeadExistObject", obj.key()); + assertTrue(obj.isDir()); + } else { + assertNull(storage.head("testHeadExistObject")); + } + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testGetAndDeleteNonExistFile(ObjectStorage store) { + setEnv(store); + // ensure file is not exist + assertNull(storage.head("a/b/c.txt")); + + assertThrows(RuntimeException.class, () -> storage.get("a/b/c.txt", 0, 0)); + assertThrows(RuntimeException.class, () -> storage.get("a/b/c.txt", 0, 1)); + + // Allow to delete a non-exist object. + storage.delete("a/b/c.txt"); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testPutAndDeleteFileWithEmptyKey(ObjectStorage store) { + setEnv(store); + assertThrows(RuntimeException.class, () -> storage.put("", new byte[0])); + assertThrows(RuntimeException.class, () -> storage.put(null, new byte[0])); + assertThrows(RuntimeException.class, () -> storage.delete(null)); + assertThrows(RuntimeException.class, () -> storage.head("")); + assertThrows(RuntimeException.class, () -> storage.head(null)); + assertThrows(RuntimeException.class, () -> getStream("")); + assertThrows(RuntimeException.class, () -> getStream(null)); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testPutObjectButContentLengthDisMatch(ObjectStorage store) throws IOException { + setEnv(store); + byte[] data = TestUtility.rand(256); + String key = "a/truncated.txt"; + + // The final object data will be truncated if content length is smaller. + byte[] checksum = storage.put(key, () -> new ByteArrayInputStream(data), 200); + assertArrayEquals(Arrays.copyOfRange(data, 0, 200), IOUtils.toByteArray(getStream(key))); + ObjectInfo info = storage.head(key); + assertEquals(key, info.key()); + assertEquals(200, info.size()); + assertArrayEquals(checksum, info.checksum()); + + // Will create object failed is the content length is bigger. + assertThrows(RuntimeException.class, + () -> storage.put(key, () -> new ByteArrayInputStream(data), 300)); + } + + private InputStream getStream(String key) { + return storage.get(key).stream(); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testPutAndGetFile(ObjectStorage store) throws IOException { + setEnv(store); + byte[] data = TestUtility.rand(256); + String key = "a/test.txt"; + byte[] checksum = storage.put(key, data); + assertArrayEquals(data, IOUtils.toByteArray(getStream(key))); + + if (storage.bucket().isDirectory()) { + // Directory bucket will create missed parent dir. + assertArrayEquals(new byte[0], IOUtils.toByteArray(getStream("a"))); + assertArrayEquals(new byte[0], IOUtils.toByteArray(getStream("a/"))); + } else { + assertNull(storage.head("a")); + assertNull(storage.head("a/")); + } + + ObjectInfo info = storage.head(key); + assertEquals(key, info.key()); + assertEquals(data.length, info.size()); + assertArrayEquals(checksum, info.checksum()); + + ObjectContent content = storage.get(key); + assertArrayEquals(info.checksum(), content.checksum()); + assertArrayEquals(data, IOUtils.toByteArray(content.stream())); + + assertArrayEquals(data, IOUtils.toByteArray(getStream(key, 0, -1))); + assertThrows(RuntimeException.class, () -> storage.get(key, -1, -1), "offset is negative"); + assertThrows(RuntimeException.class, () -> storage.get(key + "/", 0, -1), + "path not found or resource type is invalid"); + + assertArrayEquals(data, IOUtils.toByteArray(getStream(key, 0, 256))); + assertArrayEquals(data, IOUtils.toByteArray(getStream(key, 0, 512))); + + byte[] secondHalfData = Arrays.copyOfRange(data, 128, 256); + assertArrayEquals(secondHalfData, IOUtils.toByteArray(getStream(key, 128, -1))); + assertArrayEquals(secondHalfData, IOUtils.toByteArray(getStream(key, 128, 256))); + assertArrayEquals(secondHalfData, IOUtils.toByteArray(getStream(key, 128, 257))); + assertArrayEquals(new byte[0], IOUtils.toByteArray(getStream(key, 128, 0))); + + ObjectContent partContent = storage.get(key, 8, 32); + assertArrayEquals(info.checksum(), partContent.checksum()); + assertArrayEquals(Arrays.copyOfRange(data, 8, 40), + IOUtils.toByteArray(partContent.stream())); + assertArrayEquals(data, IOUtils.toByteArray(getStream(key))); + + assertThrows(RuntimeException.class, () -> storage.get(key, 257, 8), + "offset is bigger than object length"); + assertArrayEquals(new byte[0], IOUtils.toByteArray(getStream(key, 256, 8))); + + assertArrayEquals(new byte[0], IOUtils.toByteArray(getStream(key, 0, 0))); + assertArrayEquals(new byte[0], IOUtils.toByteArray(getStream(key, 1, 0))); + + + // assert the original data is not changed during random get request + assertArrayEquals(data, IOUtils.toByteArray(getStream(key))); + + storage.delete(key); + assertNull(storage.head(key)); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testAppendAndGetFile(ObjectStorage store) throws Exception { + setEnv(store); + String key = "a/testAppendAndGetFile.txt"; + + // Append zero bytes. + assertThrows(NotAppendableException.class, () -> storage.append(key, new byte[0]), + "Append non-existed object with zero byte is not supported."); + + // Append 256 bytes. + byte[] data = TestUtility.rand(256); + byte[] checksum = storage.append(key, data); + assertArrayEquals(data, IOUtils.toByteArray(getStream(key))); + + // Append zero bytes. + byte[] newChecksum = storage.append(key, new byte[0]); + assertArrayEquals(checksum, newChecksum); + checksum = newChecksum; + + // Append one byte. + newChecksum = storage.append(key, new byte[1]); + assertFalse(Arrays.equals(checksum, newChecksum)); + assertArrayEquals(newChecksum, storage.head(key).checksum()); + checksum = newChecksum; + + // Append 1024 byte. + data = TestUtility.rand(1024); + newChecksum = storage.append(key, data); + assertFalse(Arrays.equals(checksum, newChecksum)); + assertArrayEquals(newChecksum, storage.head(key).checksum()); + + storage.delete(key); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testAppendLengthNotMatch(ObjectStorage store) { + setEnv(store); + byte[] data = TestUtility.rand(256); + String key = "a/testAppendLengthNotMatch.txt"; + storage.append(key, () -> new ByteArrayInputStream(data), 128); + assertEquals(128, storage.head(key).size()); + + assertThrows(RuntimeException.class, + () -> storage.append(key, () -> new ByteArrayInputStream(data), 1024), + "Expect unexpected end of stream error."); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testHeadAndListAndObjectStatusShouldGetSameObjectInfo(ObjectStorage store) { + setEnv(store); + String key = "testHeadAndListObjectCheckSum.txt"; + byte[] data = TestUtility.rand(256); + byte[] checksum = storage.put(key, data); + + ObjectInfo obj = storage.head(key); + assertEquals(obj, storage.objectStatus(key)); + if (!storage.bucket().isDirectory()) { + List objects = toList(storage.list(key, null, 1)); + assertEquals(1, objects.size()); + assertEquals(obj, objects.get(0)); + assertArrayEquals(checksum, objects.get(0).checksum()); + } + + + key = "testHeadAndListObjectCheckSum/"; + checksum = storage.put(key, new byte[0]); + obj = storage.head(key); + assertEquals(obj, storage.objectStatus(key)); + if (!storage.bucket().isDirectory()) { + List objects = toList(storage.list(key, null, 1)); + assertEquals(1, objects.size()); + assertEquals(obj, objects.get(0)); + assertArrayEquals(checksum, objects.get(0).checksum()); + } + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testObjectStatus(ObjectStorage store) { + setEnv(store); + // test get file status + String key = "a/b/testObjectStatus.txt"; + byte[] data = TestUtility.rand(256); + byte[] checksum = storage.put(key, data); + + ObjectInfo obj = storage.head(key); + assertArrayEquals(checksum, obj.checksum()); + assertEquals(obj, storage.objectStatus(key)); + + if (storage.bucket().isDirectory()) { + assertThrows(InvalidObjectKeyException.class, () -> storage.head(key + "/")); + assertThrows(InvalidObjectKeyException.class, () -> storage.objectStatus(key + "/")); + } else { + assertNull(storage.head(key + "/")); + assertNull(storage.objectStatus(key + "/")); + } + + // test get dir status + String dirKey = "a/b/dir/"; + checksum = storage.put(dirKey, new byte[0]); + obj = storage.head(dirKey); + assertEquals(Constants.MAGIC_CHECKSUM, checksum); + assertArrayEquals(Constants.MAGIC_CHECKSUM, checksum); + assertArrayEquals(checksum, obj.checksum()); + assertTrue(obj.isDir()); + assertEquals(dirKey, obj.key()); + assertEquals(obj, storage.objectStatus(dirKey)); + + if (storage.bucket().isDirectory()) { + assertNotNull(storage.head("a/b/dir")); + assertEquals("a/b/dir", storage.objectStatus("a/b/dir").key()); + } else { + assertNull(storage.head("a/b/dir")); + assertEquals(dirKey, storage.objectStatus("a/b/dir").key()); + } + + // test get dir status of prefix + String prefix = "a/b/"; + obj = storage.objectStatus(prefix); + assertEquals(prefix, obj.key()); + assertEquals(Constants.MAGIC_CHECKSUM, obj.checksum()); + assertTrue(obj.isDir()); + + if (storage.bucket().isDirectory()) { + assertEquals(obj, storage.head(prefix)); + assertEquals("a/b", storage.objectStatus("a/b").key()); + } else { + assertNull(storage.head(prefix)); + assertEquals(prefix, storage.objectStatus("a/b").key()); + } + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testPutAndGetDirectory(ObjectStorage store) throws IOException { + setEnv(store); + String key = "a/b/"; + byte[] data = new byte[0]; + storage.put(key, data); + + ObjectInfo info = storage.head(key); + assertEquals(key, info.key()); + assertEquals(data.length, info.size()); + assertArrayEquals(data, IOUtils.toByteArray(getStream(key))); + assertArrayEquals(data, IOUtils.toByteArray(getStream(key, 0, 256))); + + // test create the same dir again + storage.put(key, data); + + storage.delete(key); + assertNull(storage.head(key)); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testOverwriteFile(ObjectStorage store) throws IOException { + setEnv(store); + String key = "a/test.txt"; + byte[] data1 = TestUtility.rand(256); + byte[] data2 = TestUtility.rand(128); + + storage.put(key, data1); + assertArrayEquals(data1, IOUtils.toByteArray(getStream(key, 0, -1))); + + storage.put(key, data2); + assertArrayEquals(data2, IOUtils.toByteArray(getStream(key, 0, -1))); + + storage.delete(key); + assertNull(storage.head(key)); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testListObjectsWithEmptyDelimiters(ObjectStorage store) { + setEnv(store); + // Directory bucket only supports list with delimiter = '/' currently. + assumeFalse(storage.bucket().isDirectory()); + String key1 = "a/b/c/d"; + String key2 = "a/b"; + + byte[] data = TestUtility.rand(256); + for (int i = 0; i < 10; i++) { + storage.put(String.format("%s/file-%d.txt", key1, i), data); + storage.put(String.format("%s/file-%d.txt", key2, i), data); + } + + // list 100 objects under 'a/', there are total 20 objects. + ListObjectsResponse response = list("a/", "", 100, ""); + assertEquals(20, response.objects().size()); + assertEquals(0, response.commonPrefixes().size()); + assertEquals("a/b/c/d/file-0.txt", response.objects().get(0).key()); + assertEquals("a/b/file-9.txt", response.objects().get(19).key()); + + // list 20 objects and there only have 20 objects under 'a/' + response = list("a/", "", 20, ""); + assertEquals(20, response.objects().size()); + assertEquals(0, response.commonPrefixes().size()); + assertEquals("a/b/c/d/file-0.txt", response.objects().get(0).key()); + assertEquals("a/b/file-9.txt", response.objects().get(19).key()); + + // list the top 10 objects among 20 objects + response = list("a/", "", 10, ""); + assertEquals(10, response.objects().size()); + assertEquals(0, response.commonPrefixes().size()); + assertEquals("a/b/c/d/file-0.txt", response.objects().get(0).key()); + assertEquals("a/b/c/d/file-9.txt", response.objects().get(9).key()); + + // list the next 5 objects behind a/b/c/d/file-9.txt among 20 objects + response = list("a/", "a/b/c/d/file-9.txt", 5, ""); + assertEquals(5, response.objects().size()); + assertEquals(0, response.commonPrefixes().size()); + assertEquals("a/b/file-0.txt", response.objects().get(0).key()); + assertEquals("a/b/file-4.txt", response.objects().get(4).key()); + + // list the next 10 objects behind a/b/c/d/file-9.txt among 20 objects + response = list("a/", "a/b/c/d/file-9.txt", 10, ""); + assertEquals(10, response.objects().size()); + assertEquals(0, response.commonPrefixes().size()); + assertEquals("a/b/file-0.txt", response.objects().get(0).key()); + assertEquals("a/b/file-9.txt", response.objects().get(9).key()); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testListEmptyDirWithSlashDelimiter(ObjectStorage store) { + setEnv(store); + String key = "a/b/"; + storage.put(key, new byte[0]); + + ListObjectsResponse response = list(key, null, 10, "/"); + assertEquals(1, response.objects().size()); + assertEquals(0, response.commonPrefixes().size()); + assertEquals("a/b/", response.objects().get(0).key()); + + response = list(key, key, 10, "/"); + assertEquals(0, response.objects().size()); + assertEquals(0, response.commonPrefixes().size()); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testDeleteMultipleKeys(ObjectStorage store) { + setEnv(store); + String prefix = "a/b"; + byte[] data = TestUtility.rand(256); + + List keys = Lists.newArrayList(); + for (int i = 0; i < 50; i++) { + String existingKey = String.format("%s/existing-file-%d.txt", prefix, i); + storage.put(existingKey, data); + keys.add(existingKey); + + String unExistingKey = String.format("%s/unExisting-file-%d.txt", prefix, i); + keys.add(unExistingKey); + } + + List failedKeys = storage.batchDelete(keys); + + for (String key : failedKeys) { + assertNotNull(storage.head(key)); + } + + for (String key : keys) { + if (!failedKeys.contains(key)) { + assertNull(storage.head(key)); + } + } + + assertThrows(IllegalArgumentException.class, () -> storage.batchDelete( + IntStream.range(0, 1001).mapToObj(String::valueOf).collect(Collectors.toList())), + "The deleted keys size should be <= 1000"); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testListObjectsWithEmptyMarkers(ObjectStorage store) { + setEnv(store); + String key1 = "a/b/c/d"; + String key2 = "a/b"; + String key3 = "a1/b1"; + + // create the folder to compatible with directory bucket. + storage.put("a/", new byte[0]); + storage.put("a/b/", new byte[0]); + storage.put("a/b/c/", new byte[0]); + storage.put("a/b/c/d/", new byte[0]); + storage.put("a1/", new byte[0]); + storage.put("a1/b1/", new byte[0]); + + byte[] data = TestUtility.rand(256); + for (int i = 0; i < 10; i++) { + storage.put(String.format("%s/file-%d.txt", key1, i), data); + storage.put(String.format("%s/file-%d.txt", key2, i), data); + storage.put(String.format("%s/file-%d.txt", key3, i), data); + } + + // group objects by '/' under 'a/' + ListObjectsResponse response = list("a/", null, 100, "/"); + assertEquals(1, response.objects().size()); + assertEquals("a/", response.objects().get(0).key()); + assertEquals(1, response.commonPrefixes().size()); + assertEquals("a/b/", response.commonPrefixes().get(0)); + + response = list("a", null, 100, "/"); + assertEquals(0, response.objects().size()); + assertEquals(2, response.commonPrefixes().size()); + assertEquals("a/", response.commonPrefixes().get(0)); + assertEquals("a1/", response.commonPrefixes().get(1)); + + // group objects by '/' under 'a/b/' and group objects by 'b/' under 'a', they are same + response = list("a/b/", null, 100, "/"); + assertEquals(11, response.objects().size()); + assertEquals(1, response.commonPrefixes().size()); + assertEquals("a/b/c/", response.commonPrefixes().get(0)); + assertEquals("a/b/", response.objects().get(0).key()); + assertEquals("a/b/file-0.txt", response.objects().get(1).key()); + assertEquals("a/b/file-9.txt", response.objects().get(10).key()); + + response = list("a/b", null, 100, "/"); + assertEquals(0, response.objects().size()); + assertEquals(1, response.commonPrefixes().size()); + assertEquals("a/b/", response.commonPrefixes().get(0)); + + if (!storage.bucket().isDirectory()) { + // Directory bucket only supports list with delimiter = '/' currently. + response = list("a", null, 100, "b/"); + assertEquals(13, response.objects().size()); + assertEquals(1, response.commonPrefixes().size()); + assertEquals("a/b/", response.commonPrefixes().get(0)); + assertEquals("a/", response.objects().get(0).key()); + assertEquals("a1/", response.objects().get(1).key()); + assertEquals("a1/b1/", response.objects().get(2).key()); + assertEquals("a1/b1/file-0.txt", response.objects().get(3).key()); + assertEquals("a1/b1/file-9.txt", response.objects().get(12).key()); + + response = list("a/", null, 100, "b/"); + assertEquals(1, response.objects().size()); + assertEquals(1, response.commonPrefixes().size()); + assertEquals("a/b/", response.commonPrefixes().get(0)); + assertEquals("a/", response.objects().get(0).key()); + } + + // group objects by different delimiter under 'a/b/c/d/' or 'a/b/c/d' + response = list("a/b/c/d/", null, 100, "/"); + assertEquals(11, response.objects().size()); + assertEquals(0, response.commonPrefixes().size()); + assertEquals("a/b/c/d/", response.objects().get(0).key()); + + response = list("a/b/c/d/", null, 5, "/"); + assertEquals(5, response.objects().size()); + assertEquals(0, response.commonPrefixes().size()); + assertEquals("a/b/c/d/", response.objects().get(0).key()); + + response = list("a/b/c/d", null, 100, "/"); + assertEquals(0, response.objects().size()); + assertEquals(1, response.commonPrefixes().size()); + assertEquals("a/b/c/d/", response.commonPrefixes().get(0)); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testListObjectWithLimitObjectAndCommonPrefixes(ObjectStorage store) { + setEnv(store); + String key1 = "a/b/c/d"; + String key2 = "a/b"; + String key3 = "a1/b1"; + + byte[] data = TestUtility.rand(256); + for (int i = 0; i < 10; i++) { + storage.put(String.format("%s/file-%d.txt", key1, i), data); + storage.put(String.format("%s/file-%d.txt", key2, i), data); + storage.put(String.format("%s/file-%d.txt", key3, i), data); + } + + List dirKeys = Lists.newArrayList("a/b/d/", "a/b/e/", "a/b/f/", "a/b/g/"); + for (String key : dirKeys) { + storage.put(key, new byte[0]); + } + + // group objects by '/' under 'a/b/', and limit top 5 objects among 10 objects and 1 common + // prefix. + ListObjectsResponse response = list("a/b/", "a/b/", 5, "/"); + assertEquals(1, response.objects().size()); + assertEquals(4, response.commonPrefixes().size()); + assertEquals("a/b/c/", response.commonPrefixes().get(0)); + assertEquals("a/b/d/", response.commonPrefixes().get(1)); + assertEquals("a/b/e/", response.commonPrefixes().get(2)); + assertEquals("a/b/f/", response.commonPrefixes().get(3)); + assertEquals("a/b/file-0.txt", response.objects().get(0).key()); + + response = list("a/b/", "a/b/", 14, "/"); + assertEquals(10, response.objects().size()); + assertEquals(4, response.commonPrefixes().size()); + assertEquals("a/b/c/", response.commonPrefixes().get(0)); + assertEquals("a/b/d/", response.commonPrefixes().get(1)); + assertEquals("a/b/e/", response.commonPrefixes().get(2)); + assertEquals("a/b/f/", response.commonPrefixes().get(3)); + assertEquals("a/b/file-0.txt", response.objects().get(0).key()); + assertEquals("a/b/file-9.txt", response.objects().get(9).key()); + + response = list("a/b/", "a/b/", 15, "/"); + assertEquals(10, response.objects().size()); + assertEquals(5, response.commonPrefixes().size()); + assertEquals("a/b/c/", response.commonPrefixes().get(0)); + assertEquals("a/b/d/", response.commonPrefixes().get(1)); + assertEquals("a/b/e/", response.commonPrefixes().get(2)); + assertEquals("a/b/f/", response.commonPrefixes().get(3)); + assertEquals("a/b/g/", response.commonPrefixes().get(4)); + assertEquals("a/b/file-0.txt", response.objects().get(0).key()); + assertEquals("a/b/file-9.txt", response.objects().get(9).key()); + + // a/b/h-file-0.txt is behind from a/b/g/ + storage.put("a/b/h-file-0.txt", data); + response = list("a/b/", "a/b/", 15, "/"); + assertEquals(10, response.objects().size()); + assertEquals(5, response.commonPrefixes().size()); + assertEquals("a/b/c/", response.commonPrefixes().get(0)); + assertEquals("a/b/d/", response.commonPrefixes().get(1)); + assertEquals("a/b/e/", response.commonPrefixes().get(2)); + assertEquals("a/b/f/", response.commonPrefixes().get(3)); + assertEquals("a/b/g/", response.commonPrefixes().get(4)); + assertEquals("a/b/file-0.txt", response.objects().get(0).key()); + assertEquals("a/b/file-9.txt", response.objects().get(9).key()); + + response = list("a/b/", "a/b/", 20, "/"); + assertEquals(11, response.objects().size()); + assertEquals(5, response.commonPrefixes().size()); + assertEquals("a/b/c/", response.commonPrefixes().get(0)); + assertEquals("a/b/d/", response.commonPrefixes().get(1)); + assertEquals("a/b/e/", response.commonPrefixes().get(2)); + assertEquals("a/b/f/", response.commonPrefixes().get(3)); + assertEquals("a/b/g/", response.commonPrefixes().get(4)); + assertEquals("a/b/file-0.txt", response.objects().get(0).key()); + assertEquals("a/b/h-file-0.txt", response.objects().get(10).key()); + + response = list("a/b/", "a/b/", 1, "/"); + assertEquals(0, response.objects().size()); + assertEquals(1, response.commonPrefixes().size()); + assertEquals("a/b/c/", response.commonPrefixes().get(0)); + + response = list("a/b/", "a/b/", 2, "/"); + assertEquals(0, response.objects().size()); + assertEquals(2, response.commonPrefixes().size()); + assertEquals("a/b/c/", response.commonPrefixes().get(0)); + assertEquals("a/b/d/", response.commonPrefixes().get(1)); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testListedIteratorIsIdempotent(ObjectStorage store) { + setEnv(store); + String key1 = "a/b/c/d"; + + byte[] data = TestUtility.rand(256); + for (int i = 0; i < 10; i++) { + storage.put(String.format("%s/file-%d.txt", key1, i), data); + } + + Iterable res; + if (storage.bucket().isDirectory()) { + res = ((DirectoryStorage) storage).listDir("a/b/c/d/", true); + } else { + res = storage.list("a/b/c/d/", "a/b/c/d/", 10); + } + Iterator batch1 = res.iterator(); + Iterator batch2 = res.iterator(); + + for (int i = 0; i < 10; i++) { + assertTrue(batch1.hasNext()); + ObjectInfo obj = batch1.next(); + assertEquals(String.format("a/b/c/d/file-%d.txt", i), obj.key()); + } + assertFalse(batch1.hasNext()); + + for (int i = 0; i < 10; i++) { + assertTrue(batch2.hasNext()); + ObjectInfo obj = batch2.next(); + assertEquals(String.format("a/b/c/d/file-%d.txt", i), obj.key()); + } + assertFalse(batch2.hasNext()); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testListObjectsWithSmallBatch(ObjectStorage store) { + setEnv(store); + assumeFalse(storage.bucket().isDirectory()); + String key1 = "a/b/c/d/"; + + byte[] data = TestUtility.rand(256); + for (int i = 0; i < 10; i++) { + storage.put(String.format("%sfile-%d.txt", key1, i), data); + } + + // change list object count + Configuration newConf = new Configuration(storage.conf()); + newConf.setInt(TosKeys.FS_TOS_LIST_OBJECTS_COUNT, 5); + storage.initialize(newConf, storage.bucket().name()); + + List maxKeys = Arrays.asList(5, 10, 9, 20, -1); + for (int maxKey : maxKeys) { + Iterator objs = storage.list(key1, key1, maxKey).iterator(); + int end = Math.min(maxKey == -1 ? 10 : maxKey, 10); + for (int i = 0; i < end; i++) { + assertTrue(objs.hasNext()); + ObjectInfo obj = objs.next(); + assertEquals(String.format("a/b/c/d/file-%d.txt", i), obj.key()); + } + assertFalse(objs.hasNext()); + } + + // reset list object count + newConf = new Configuration(storage.conf()); + newConf.setInt(TosKeys.FS_TOS_LIST_OBJECTS_COUNT, 1000); + storage.initialize(newConf, storage.bucket().name()); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testListObjectsWithSpecificDelimiters(ObjectStorage store) { + setEnv(store); + assumeFalse(storage.bucket().isDirectory()); + String key1 = "a/b/c/d"; + String key2 = "a/b"; + String key3 = "a1/b1"; + + byte[] data = TestUtility.rand(256); + for (int i = 0; i < 10; i++) { + storage.put(String.format("%s/file-%d.txt", key1, i), data); + storage.put(String.format("%s/file-%d.txt", key2, i), data); + storage.put(String.format("%s/file-%d.txt", key3, i), data); + } + + ListObjectsResponse response = list("a", "", 11, "b/"); + assertEquals(10, response.objects().size()); + assertEquals(1, response.commonPrefixes().size()); + assertEquals("a/b/", response.commonPrefixes().get(0)); + assertEquals("a1/b1/file-0.txt", response.objects().get(0).key()); + assertEquals("a1/b1/file-9.txt", response.objects().get(9).key()); + + response = list("a", "", 5, "b/"); + assertEquals(4, response.objects().size()); + assertEquals(1, response.commonPrefixes().size()); + assertEquals("a/b/", response.commonPrefixes().get(0)); + assertEquals("a1/b1/file-0.txt", response.objects().get(0).key()); + assertEquals("a1/b1/file-3.txt", response.objects().get(3).key()); + + response = list("a", "a1/b1/file-3.txt", 5, "b/"); + assertEquals(5, response.objects().size()); + assertEquals(0, response.commonPrefixes().size()); + assertEquals("a1/b1/file-4.txt", response.objects().get(0).key()); + assertEquals("a1/b1/file-8.txt", response.objects().get(4).key()); + + response = list("a", "a1/b1/file-3.txt", 6, "b/"); + assertEquals(6, response.objects().size()); + assertEquals(0, response.commonPrefixes().size()); + assertEquals("a1/b1/file-4.txt", response.objects().get(0).key()); + assertEquals("a1/b1/file-9.txt", response.objects().get(5).key()); + + response = list("a", "a/b/file-3.txt", 5, "b/"); + assertEquals(4, response.objects().size()); + assertEquals(1, response.commonPrefixes().size()); + assertEquals("a/b/", response.commonPrefixes().get(0)); + assertEquals("a1/b1/file-0.txt", response.objects().get(0).key()); + assertEquals("a1/b1/file-3.txt", response.objects().get(3).key()); + + response = list("a", "a/b/file-3.txt", 10, "b/"); + assertEquals(9, response.objects().size()); + assertEquals(1, response.commonPrefixes().size()); + assertEquals("a/b/", response.commonPrefixes().get(0)); + assertEquals("a1/b1/file-0.txt", response.objects().get(0).key()); + assertEquals("a1/b1/file-8.txt", response.objects().get(8).key()); + + response = list("a", "a/b/file-3.txt", 11, "b/"); + assertEquals(10, response.objects().size()); + assertEquals(1, response.commonPrefixes().size()); + assertEquals("a/b/", response.commonPrefixes().get(0)); + assertEquals("a1/b1/file-0.txt", response.objects().get(0).key()); + assertEquals("a1/b1/file-9.txt", response.objects().get(9).key()); + + response = list("a", "a/b/", 1, "b/"); + assertEquals(1, response.objects().size()); + assertEquals(0, response.commonPrefixes().size()); + + response = list("a/b/c/d", "", 100, "/file"); + assertEquals(0, response.objects().size()); + assertEquals(1, response.commonPrefixes().size()); + assertEquals("a/b/c/d/file", response.commonPrefixes().get(0)); + + response = list("a/b/c/d/", "", 100, "file"); + assertEquals(0, response.objects().size()); + assertEquals(1, response.commonPrefixes().size()); + assertEquals("a/b/c/d/file", response.commonPrefixes().get(0)); + + + // group objects by different delimiter under 'a1' or 'a1/' + response = list("a1", "", 100, ""); + assertEquals(10, response.objects().size()); + assertEquals(0, response.commonPrefixes().size()); + + response = list("a1", "", 100, "/"); + assertEquals(0, response.objects().size()); + assertEquals(1, response.commonPrefixes().size()); + assertEquals("a1/", response.commonPrefixes().get(0)); + + response = list("a1/", "", 100, "/"); + assertEquals(0, response.objects().size()); + assertEquals(1, response.commonPrefixes().size()); + assertEquals("a1/b1/", response.commonPrefixes().get(0)); + + response = list("a1/", "", 1, "/"); + assertEquals(0, response.objects().size()); + assertEquals(1, response.commonPrefixes().size()); + assertEquals("a1/b1/", response.commonPrefixes().get(0)); + + // group objects by non-exist delimiter under 'a1' or 'a1/' + response = list("a1", "", 100, "non-exist"); + assertEquals(10, response.objects().size()); + assertEquals(0, response.commonPrefixes().size()); + + response = list("a1/", "", 100, "non-exist"); + assertEquals(10, response.objects().size()); + assertEquals(0, response.commonPrefixes().size()); + + // test the sequent of common prefixes + response = list("a", "", 100, "b"); + assertEquals(0, response.objects().size()); + assertEquals(2, response.commonPrefixes().size()); + assertEquals("a/b", response.commonPrefixes().get(0)); + assertEquals("a1/b", response.commonPrefixes().get(1)); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testOverwriteDirectoryWithAFile(ObjectStorage store) throws IOException { + setEnv(store); + String dirKey = "a/b/"; + String key = "a/b"; + storage.delete("a/"); + + byte[] data1 = new byte[0]; + byte[] data2 = TestUtility.rand(128); + + storage.put(dirKey, data1); + assertArrayEquals(data1, IOUtils.toByteArray(getStream(dirKey, 0, 256))); + + if (!storage.bucket().isDirectory()) { + // Directory bucket doesn't allow overwrote if the resource type is changed. + storage.put(key, data2); + assertArrayEquals(data2, IOUtils.toByteArray(getStream(key, 0, 256))); + } + + storage.delete(key); + storage.delete(dirKey); + assertNull(storage.head(key)); + assertNull(storage.head(dirKey)); + } + + private InputStream getStream(String key, long off, long limit) { + return storage.get(key, off, limit).stream(); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testDeleteNonEmptyDir(ObjectStorage store) throws IOException { + setEnv(store); + storage.put("a/", new byte[0]); + storage.put("a/b/", new byte[0]); + assertArrayEquals(new byte[0], IOUtils.toByteArray(getStream("a/b/", 0, 256))); + + ListObjectsResponse response = list("a/b/", "a/b/", 100, "/"); + assertEquals(0, response.objects().size()); + assertEquals(0, response.commonPrefixes().size()); + + if (!storage.bucket().isDirectory()) { + // Directory bucket only supports list with delimiter = '/'. + response = list("a/b/", "a/b/", 100, null); + assertEquals(0, response.objects().size()); + assertEquals(0, response.commonPrefixes().size()); + } + + storage.delete("a/b/"); + assertNull(storage.head("a/b/")); + assertNull(storage.head("a/b")); + assertNotNull(storage.head("a/")); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testRecursiveDelete(ObjectStorage store) { + setEnv(store); + storage.put("a/", new byte[0]); + storage.put("a/b/", new byte[0]); + storage.put("a/b/c1/", new byte[0]); + storage.put("a/b/c2/", new byte[0]); + storage.put("a/b/c3/", new byte[0]); + assertNotNull(storage.head("a/")); + assertNotNull(storage.head("a/b/")); + assertNotNull(storage.head("a/b/c1/")); + assertNotNull(storage.head("a/b/c2/")); + assertNotNull(storage.head("a/b/c3/")); + + storage.delete("a/b/c3/"); + assertNull(storage.head("a/b/c3/")); + + storage.deleteAll(""); + assertNull(storage.head("a/b/c1/")); + assertNull(storage.head("a/b/c2/")); + assertNull(storage.head("a/b/")); + assertNull(storage.head("a/")); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testListObjectKeys(ObjectStorage store) { + setEnv(store); + assumeFalse(storage.bucket().isDirectory()); + byte[] dirBytes = new byte[0]; + byte[] fileBytes = TestUtility.rand(128); + storage.put("a/b1/", dirBytes); + storage.put("a/b2/c0/", dirBytes); + storage.put("a/b2/c1/d1.txt", fileBytes); + storage.put("a/b2/c1/e1.txt", fileBytes); + storage.put("a/b2/c2.txt", fileBytes); + + // list single dir + List ret = toList(storage.list("a/b1", "", 10)); + assertEquals(1, ret.size()); + assertEquals("a/b1/", ret.get(0).key()); + assertEquals(0, ret.get(0).size()); + + ret = toList(storage.list("a/b1/", "", 10)); + assertEquals(1, ret.size()); + assertEquals("a/b1/", ret.get(0).key()); + assertEquals(0, ret.get(0).size()); + + // list single file + ret = toList(storage.list("a/b2/c1/d1.txt", "", 10)); + assertEquals(1, ret.size()); + assertEquals("a/b2/c1/d1.txt", ret.get(0).key()); + assertEquals(fileBytes.length, ret.get(0).size()); + + // list multiple files & dirs + ret = toList(storage.list("a/b2", "", 10)); + assertEquals(4, ret.size()); + assertEquals("a/b2/c0/", ret.get(0).key()); + assertEquals("a/b2/c1/d1.txt", ret.get(1).key()); + assertEquals("a/b2/c1/e1.txt", ret.get(2).key()); + assertEquals("a/b2/c2.txt", ret.get(3).key()); + assertEquals(dirBytes.length, ret.get(0).size()); + + // list single file with marker + ret = toList(storage.list("a/b2", "a/b2/c1/e1.txt", 10)); + assertEquals(1, ret.size()); + assertEquals("a/b2/c2.txt", ret.get(0).key()); + assertEquals(fileBytes.length, ret.get(0).size()); + + // list multiple files with marker + ret = toList(storage.list("a/b2", "a/b2/c1/", 10)); + assertEquals(3, ret.size()); + assertEquals("a/b2/c1/d1.txt", ret.get(0).key()); + assertEquals("a/b2/c1/e1.txt", ret.get(1).key()); + assertEquals("a/b2/c2.txt", ret.get(2).key()); + assertEquals(fileBytes.length, ret.get(0).size()); + + // list multiple files & dirs with part path as prefix + ret = toList(storage.list("a/b2/c", "", 10)); + assertEquals(4, ret.size()); + assertEquals("a/b2/c0/", ret.get(0).key()); + assertEquals("a/b2/c1/d1.txt", ret.get(1).key()); + assertEquals("a/b2/c1/e1.txt", ret.get(2).key()); + assertEquals("a/b2/c2.txt", ret.get(3).key()); + assertEquals(dirBytes.length, ret.get(0).size()); + + ret = toList(storage.list("a/b2/c", "", 2)); + assertEquals(2, ret.size()); + assertEquals("a/b2/c0/", ret.get(0).key()); + + ret = toList(storage.list("a/b2/c1/d1.", "", 10)); + assertEquals(1, ret.size()); + assertEquals("a/b2/c1/d1.txt", ret.get(0).key()); + assertEquals(fileBytes.length, ret.get(0).size()); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testListAllObjectKeys(ObjectStorage store) { + setEnv(store); + assumeFalse(storage.bucket().isDirectory()); + byte[] dirBytes = new byte[0]; + byte[] fileBytes = TestUtility.rand(128); + storage.put("a/b1/", dirBytes); + storage.put("a/b2/c0/", dirBytes); + storage.put("a/b2/c1/d1.txt", fileBytes); + storage.put("a/b2/c1/e1.txt", fileBytes); + storage.put("a/b2/c2.txt", dirBytes); + + // list single dir + List ret = Lists.newArrayList(storage.listAll("a/b1", "")); + assertEquals(1, ret.size()); + assertEquals("a/b1/", ret.get(0).key()); + assertEquals(0, ret.get(0).size()); + + // list single file + ret = Lists.newArrayList(storage.listAll("a/b2/c1/d1.txt", "")); + assertEquals(1, ret.size()); + assertEquals("a/b2/c1/d1.txt", ret.get(0).key()); + assertEquals(fileBytes.length, ret.get(0).size()); + + // list multiple files & dirs + ret = Lists.newArrayList(storage.listAll("a/b2", "")); + assertEquals(4, ret.size()); + assertEquals("a/b2/c0/", ret.get(0).key()); + assertEquals("a/b2/c1/d1.txt", ret.get(1).key()); + assertEquals("a/b2/c1/e1.txt", ret.get(2).key()); + assertEquals("a/b2/c2.txt", ret.get(3).key()); + assertEquals(dirBytes.length, ret.get(0).size()); + + // list multiple files & dirs with part path as prefix + ret = Lists.newArrayList(storage.listAll("a/b2/c", "")); + assertEquals(4, ret.size()); + assertEquals("a/b2/c0/", ret.get(0).key()); + assertEquals("a/b2/c1/d1.txt", ret.get(1).key()); + assertEquals("a/b2/c1/e1.txt", ret.get(2).key()); + assertEquals("a/b2/c2.txt", ret.get(3).key()); + assertEquals(dirBytes.length, ret.get(0).size()); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testListEmptyKeys(ObjectStorage store) { + setEnv(store); + if (storage.bucket().isDirectory()) { + assertEquals(0, + Lists.newArrayList(((DirectoryStorage) storage).listDir("not-exist", true)).size()); + } else { + assertEquals(0, Lists.newArrayList(storage.list("not-exist", "", 2)).size()); + } + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testMultiUploadEmptyFile(ObjectStorage store) { + setEnv(store); + String key = "a/b/empty.txt"; + MultipartUpload upload = storage.createMultipartUpload(key); + assertThrows(Exception.class, + () -> storage.completeUpload(key, upload.uploadId(), Lists.newArrayList())); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testMultiUploadZeroByte(ObjectStorage store) throws IOException { + setEnv(store); + String key = "a/b/zero.txt"; + MultipartUpload upload = storage.createMultipartUpload(key); + Part part = + storage.uploadPart(key, upload.uploadId(), 1, () -> new ByteArrayInputStream(new byte[0]), + 0); + storage.completeUpload(key, upload.uploadId(), Lists.newArrayList(part)); + assertArrayEquals(ObjectTestUtils.EMPTY_BYTES, IOUtils.toByteArray(getStream(key))); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testMultiUploadFile(ObjectStorage store) throws IOException { + setEnv(store); + String key1 = "a/b/c/e.txt"; + String uploadId1 = storage.createMultipartUpload(key1).uploadId(); + assertNotEquals(uploadId1, ""); + + byte[] dataset = multipleUpload(key1, uploadId1, 2, true); + assertArrayEquals(dataset, IOUtils.toByteArray(getStream(key1))); + + String key2 = "a/b/e/e.txt"; + String uploadId2 = storage.createMultipartUpload(key2).uploadId(); + assertNotEquals(uploadId2, ""); + + dataset = multipleUpload(key2, uploadId2, 3, true); + assertArrayEquals(dataset, IOUtils.toByteArray(getStream(key2))); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testPutAndCompleteMPUWithSameContent(ObjectStorage store) throws IOException { + setEnv(store); + String mpu = "a/b/mpu.txt"; + String put = "a/b/put.txt"; + byte[] dataset = TestUtility.rand(11 << 20); + byte[] checksum = multipleUpload(mpu, dataset); + + storage.put(put, dataset); + + ObjectInfo mputObj = storage.head(mpu); + ObjectInfo putObj = storage.head(put); + assertArrayEquals(checksum, mputObj.checksum()); + assertArrayEquals(checksum, putObj.checksum()); + + if (!storage.bucket().isDirectory()) { + List objectInfo = toList(storage.list(mpu, null, 10)); + assertEquals(mputObj, objectInfo.get(0)); + } + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testListUploads(ObjectStorage store) { + setEnv(store); + String key1 = "a/b/c/e.txt"; + String uploadId1 = storage.createMultipartUpload(key1).uploadId(); + assertNotEquals(uploadId1, ""); + multipleUpload(key1, uploadId1, 2, false); + + String key2 = "a/b/e/e.txt"; + String uploadId2 = storage.createMultipartUpload(key2).uploadId(); + assertNotEquals(uploadId2, ""); + multipleUpload(key2, uploadId2, 3, false); + + Iterable iterable = storage.listUploads(""); + List uploads = Lists.newArrayList(iterable.iterator()); + assertEquals(2, uploads.size()); + assertEquals(key1, uploads.get(0).key()); + assertEquals(uploadId1, uploads.get(0).uploadId()); + assertEquals(key2, uploads.get(1).key()); + assertEquals(uploadId2, uploads.get(1).uploadId()); + + // check iterator is idempotent + uploads = Lists.newArrayList(iterable.iterator()); + assertEquals(2, uploads.size()); + assertEquals(key1, uploads.get(0).key()); + assertEquals(uploadId1, uploads.get(0).uploadId()); + assertEquals(key2, uploads.get(1).key()); + assertEquals(uploadId2, uploads.get(1).uploadId()); + + uploads = Lists.newArrayList(storage.listUploads("a/b/")); + assertEquals(2, uploads.size()); + assertEquals(key1, uploads.get(0).key()); + assertEquals(uploadId1, uploads.get(0).uploadId()); + assertEquals(key2, uploads.get(1).key()); + assertEquals(uploadId2, uploads.get(1).uploadId()); + + uploads = Lists.newArrayList(storage.listUploads("a/b/c/")); + assertEquals(1, uploads.size()); + assertEquals(key1, uploads.get(0).key()); + assertEquals(uploadId1, uploads.get(0).uploadId()); + + storage.abortMultipartUpload(key1, uploadId1); + storage.abortMultipartUpload(key2, uploadId2); + assertEquals(0, Lists.newArrayList((storage.listUploads("a/b/"))).size()); + } + + private byte[] multipleUpload(String key, String uploadId, int partCnt, boolean completeUpload) { + int partSize = 5 * 1024 * 1024; + byte[] dataset = new byte[partCnt * partSize]; + byte[] partData = TestUtility.rand(partSize); + try { + int offset = 0; + List parts = new ArrayList<>(); + for (int i = 1; i <= partCnt; i++) { + Part part = storage.uploadPart(key, uploadId, i, () -> new ByteArrayInputStream(partData), + partData.length); + parts.add(part); + System.arraycopy(partData, 0, dataset, offset, partData.length); + offset += partData.length; + } + if (completeUpload) { + storage.completeUpload(key, uploadId, parts); + } + } catch (RuntimeException e) { + storage.abortMultipartUpload(key, uploadId); + } + return dataset; + } + + private byte[] multipleUpload(String key, byte[] dataset) throws IOException { + int partSize = 5 * 1024 * 1024; + int partCnt = (int) Math.ceil((double) dataset.length / partSize); + + String uploadId = storage.createMultipartUpload(key).uploadId(); + assertNotEquals(uploadId, ""); + + try { + List parts = new ArrayList<>(); + for (int i = 0; i < partCnt; i++) { + int start = i * partSize; + int end = Math.min(dataset.length, start + partSize); + byte[] partData = Arrays.copyOfRange(dataset, start, end); + + Part part = + storage.uploadPart(key, uploadId, i + 1, () -> new ByteArrayInputStream(partData), + partData.length); + + assertEquals(DigestUtils.md5Hex(partData), part.eTag().replace("\"", "")); + parts.add(part); + } + + byte[] checksum = storage.completeUpload(key, uploadId, parts); + assertArrayEquals(dataset, IOUtils.toByteArray(getStream(key))); + + return checksum; + } catch (IOException | RuntimeException e) { + storage.abortMultipartUpload(key, uploadId); + throw e; + } + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testUploadPartCopy10MB(ObjectStorage store) { + setEnv(store); + String srcKey = "src10MB.txt"; + String dstKey = "dst10MB.txt"; + testUploadPartCopy(srcKey, dstKey, 10 << 20); // 10MB + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testUploadPartCopy100MB(ObjectStorage store) { + setEnv(store); + String srcKey = "src100MB.txt"; + String dstKey = "dst100MB.txt"; + testUploadPartCopy(srcKey, dstKey, 100 << 20); // 100MB + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testUploadPartCopy65MB(ObjectStorage store) { + setEnv(store); + String srcKey = "src65MB.txt"; + String dstKey = "dst65MB.txt"; + testUploadPartCopy(srcKey, dstKey, 65 << 20); // 65MB + } + + private void testUploadPartCopy(String srcKey, String key, int fileSize) { + MultipartUpload srcMultipartUpload = storage.createMultipartUpload(srcKey); + long partSize = 5 << 20; + int partCnt = (int) (fileSize / partSize + (fileSize % partSize == 0 ? 0 : 1)); + byte[] data = + multipleUpload(srcMultipartUpload.key(), srcMultipartUpload.uploadId(), partCnt, true); + MultipartUpload dstMultipartUpload = storage.createMultipartUpload(key); + long copyPartRangeStart = 0L; + List results = Lists.newArrayList(); + try { + for (int i = 0; i < partCnt; i++) { + Part result = storage.uploadPartCopy(srcKey, key, dstMultipartUpload.uploadId(), i + 1, + copyPartRangeStart, Math.min(copyPartRangeStart + partSize, fileSize) - 1); + results.add(result); + copyPartRangeStart += partSize; + } + storage.completeUpload(key, dstMultipartUpload.uploadId(), results); + assertArrayEquals(data, IOUtils.toByteArray(getStream(key))); + } catch (Exception e) { + storage.abortMultipartUpload(key, dstMultipartUpload.uploadId()); + } + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testCopy0MB(ObjectStorage store) throws IOException { + setEnv(store); + String srcKey = "src0MB.txt"; + String dstKey = "dst0MB.txt"; + testCopy(srcKey, dstKey, 0); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testCopy5MB(ObjectStorage store) throws IOException { + setEnv(store); + String srcKey = "src5MB.txt"; + String dstKey = "dst5MB.txt"; + testCopy(srcKey, dstKey, 5 << 20); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testCopy10MB(ObjectStorage store) throws IOException { + setEnv(store); + String srcKey = "src10MB.txt"; + String dstKey = "dst10MB.txt"; + testCopy(srcKey, dstKey, 10 << 20); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testRename(ObjectStorage store) throws IOException { + setEnv(store); + String srcKey = "src.txt"; + String dstKey = "dst.txt"; + + // Rename source to a un-exist object + renameObject(srcKey, dstKey, 256); + renameObject(srcKey, dstKey, 0); + + // Overwrite an existing object + renameObjectWhenDestExist(srcKey, dstKey, 256, 0); + renameObjectWhenDestExist(srcKey, dstKey, 0, 256); + + assertNull(storage.head(srcKey)); + assertThrows(RuntimeException.class, () -> storage.rename(srcKey, dstKey), + "Source key not found"); + + assertThrows(RuntimeException.class, () -> renameObject(srcKey, srcKey, 256), + "Cannot rename to the same object"); + } + + private void renameObjectWhenDestExist(String srcKey, String dstKey, int srcSize, int destSize) + throws IOException { + byte[] dstData = new byte[destSize]; + storage.put(dstKey, dstData, 0, destSize); + assertArrayEquals(dstData, IOUtils.toByteArray(getStream(dstKey))); + + renameObject(srcKey, dstKey, srcSize); + } + + private void renameObject(String srcKey, String dstKey, int fileSize) throws IOException { + byte[] data = new byte[fileSize]; + storage.put(srcKey, data, 0, fileSize); + assertArrayEquals(data, IOUtils.toByteArray(getStream(srcKey))); + + storage.rename(srcKey, dstKey); + assertArrayEquals(data, IOUtils.toByteArray(getStream(dstKey))); + assertNull(storage.head(srcKey)); + + storage.delete(dstKey); + assertNull(storage.head(dstKey)); + } + + private void testCopy(String srcKey, String dstKey, int fileSize) throws IOException { + byte[] data = new byte[fileSize]; + storage.put(srcKey, data, 0, fileSize); + storage.copy(srcKey, dstKey); + assertArrayEquals(data, IOUtils.toByteArray(getStream(dstKey))); + } + + private ListObjectsResponse list(String prefix, String startAfter, int limit, String delimiter) { + Preconditions.checkArgument(limit <= 1000, "Cannot list more than 1000 objects."); + ListObjectsRequest request = ListObjectsRequest.builder() + .prefix(prefix) + .startAfter(startAfter) + .maxKeys(limit) + .delimiter(delimiter) + .build(); + Iterator iterator = storage.list(request).iterator(); + if (iterator.hasNext()) { + return iterator.next(); + } else { + return new ListObjectsResponse(new ArrayList<>(), new ArrayList<>()); + } + } + + private static List toList(final Iterable iterable) { + return StreamSupport.stream(iterable.spliterator(), false) + .collect(Collectors.toList()); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testObjectTagging(ObjectStorage store) { + setEnv(store); + assumeFalse(storage.bucket().isDirectory()); + if (storage instanceof FileStore) { + return; + } + + // create key. + String key = "ObjectTagging"; + String tagPrefix = "tag" + UUIDUtils.random() + "_"; + String valuePrefix = "value" + UUIDUtils.random() + "_"; + storage.put(key, new byte[0], 0, 0); + + Map tagsMap = new HashMap<>(); + for (int i = 0; i < 10; i++) { + tagsMap.put(tagPrefix + i, valuePrefix + i); + } + + // 1. put and get when key exists. + storage.putTags(key, tagsMap); + Map tags = storage.getTags(key); + assertEquals(10, tags.keySet().size()); + assertTrue(Maps.difference(tagsMap, tags).areEqual()); + + // 2. put and get when key doesn't exist. + assertThrows(TosServerException.class, () -> storage.putTags("non-exist-key", tagsMap), + "NoSuchKey"); + assertThrows(TosServerException.class, () -> storage.getTags("non-exist-key"), "doesn't exist"); + + // 3. tag threshold. + Map bigMap = new HashMap<>(tagsMap); + bigMap.put(tagPrefix + 11, valuePrefix + 11); + assertThrows(RuntimeException.class, () -> storage.putTags(key, bigMap), "exceed limit of 10"); + + // 4. put tag with null tagName. + Map nullKeyTag = new HashMap<>(); + nullKeyTag.put(null, "some value"); + assertThrows(TosServerException.class, () -> storage.putTags(key, nullKeyTag), + "TagKey you have provided is invalid"); + + // 5. put tag with null value. + Map nullValueTag = new HashMap<>(); + nullValueTag.put("some-key", null); + storage.putTags(key, nullValueTag); + assertNull(storage.getTags(key).get("some-key")); + + // 6. remove tags. + Map emptyTag = new HashMap<>(); + storage.putTags(key, emptyTag); + assertEquals(0, storage.getTags(key).size()); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testObjectChecksum(ObjectStorage store) throws IOException { + setEnv(store); + byte[] data = TestUtility.rand(256); + String key = "a/truncated.txt"; + + // Read object at the end offset. + byte[] checksum = storage.put(key, () -> new ByteArrayInputStream(data), 200); + ObjectContent objContent = storage.get(key, 200, -1); + objContent.stream().close(); + assertArrayEquals(checksum, objContent.checksum()); + + // Read empty object. + checksum = storage.put(key, () -> new ByteArrayInputStream(new byte[0]), 0); + objContent = storage.get(key, 0, -1); + objContent.stream().close(); + assertArrayEquals(checksum, objContent.checksum()); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestChainTOSInputStream.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestChainTOSInputStream.java new file mode 100644 index 0000000000000..26533d74c7d75 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestChainTOSInputStream.java @@ -0,0 +1,245 @@ +/* + * 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.tosfs.object.tos; + +import com.volcengine.tos.model.object.GetObjectBasicOutput; +import com.volcengine.tos.model.object.GetObjectV2Output; +import org.apache.hadoop.fs.tosfs.common.Bytes; +import org.apache.hadoop.fs.tosfs.object.Constants; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.junit.jupiter.api.Test; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Arrays; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestChainTOSInputStream { + + private static final int DATA_SIZE = 1 << 20; + private static final byte[] DATA = TestUtility.rand(DATA_SIZE); + + @Test + public void testRetryReadData() throws IOException { + int readLen = DATA_SIZE - 1; + int cutOff = readLen / 2; + try (ChainTOSInputStream stream = createTestChainTOSInputStream(DATA, 0, DATA_SIZE - 1, 1024, + cutOff)) { + // The read length is more than the cut-off position, and equal to data length, + // so the first stream will throw IOException, and fallback to the second stream. + byte[] data = new byte[readLen]; + int n = stream.read(data); + assertEquals(readLen, n); + assertArrayEquals(Bytes.toBytes(DATA, 0, readLen), data); + } + + try (ChainTOSInputStream stream = createTestChainTOSInputStream(DATA, 0, DATA_SIZE - 1, 1024, + cutOff)) { + // The read length is more than data length, so the first stream will throw IOException, + // and fallback to the second stream. + byte[] data = new byte[readLen + 2]; + int n = stream.read(data); + assertEquals(readLen, n); + assertArrayEquals(Bytes.toBytes(DATA, 0, readLen), Bytes.toBytes(data, 0, n)); + } + + readLen = DATA_SIZE / 3; + cutOff = DATA_SIZE / 2; + try (ChainTOSInputStream stream = createTestChainTOSInputStream(DATA, 0, DATA_SIZE, 1024, + cutOff)) { + for (int i = 0; i <= 3; i++) { + // The cut-off position is between (readLen, 2 * readLen), so the data of first read come + // from the first stream, and then the second read will meet IOException, and fallback to + // the second stream. + byte[] data = new byte[readLen]; + int n = stream.read(data); + + int off = i * readLen; + int len = Math.min(readLen, DATA_SIZE - off); + + assertEquals(len, n); + assertArrayEquals(Bytes.toBytes(DATA, off, len), Bytes.toBytes(data, 0, len)); + } + } + + int smallDataSize = 1 << 10; + cutOff = smallDataSize / 2; + byte[] smallData = TestUtility.rand(1 << 10); + try (ChainTOSInputStream stream = createTestChainTOSInputStream(smallData, 0, smallDataSize, + 1024, cutOff)) { + for (int i = 0; i < smallDataSize; i++) { + // The cut-off position is 512, the 512th read operation will meet IOException, + // and then fallback to the second stream. + int read = stream.read(); + assertEquals(smallData[i] & 0xFF, read); + } + } + } + + @Test + public void testSkipAndRead() throws IOException { + int cutOff = (DATA_SIZE - 1) / 2; + try (ChainTOSInputStream stream = createTestChainTOSInputStream(DATA, 0, DATA_SIZE - 1, 1024, + cutOff)) { + // The skip pos is equal to cut-off pos, once skip finished, the first read operation will + // meet IOException, and the fallback to the second stream. + int readPos = (DATA_SIZE - 1) / 2; + stream.skip(readPos); + + int readLen = 1024; + byte[] data = new byte[readLen]; + int n = stream.read(data); + assertEquals(readLen, n); + assertArrayEquals(Bytes.toBytes(DATA, readPos, readLen), data); + } + + try (ChainTOSInputStream stream = createTestChainTOSInputStream(DATA, 0, DATA_SIZE - 1, 1024, + cutOff)) { + // The skip pos is more than cut-off pos, the skip operation will throw IOException, + // and the fallback to the second stream and skip(readPos) again + int readPos = cutOff + 1024; + stream.skip(readPos); + + int readLen = 1024; + byte[] data = new byte[readLen]; + int n = stream.read(data); + assertEquals(readLen, n); + assertArrayEquals(Bytes.toBytes(DATA, readPos, readLen), data); + } + + try (ChainTOSInputStream stream = createTestChainTOSInputStream(DATA, 0, DATA_SIZE - 1, 1024, + cutOff)) { + // The skip pos = cut-off pos - 1025, the skip operation will succeed on the first stream, + // the 1024 bytes read operation also succeed on the first stream, + // but the next 1024 bytes read operation will fail on the first stream, and fallback to the + // second stream + int readPos = cutOff - 1024 - 1; + stream.skip(readPos); + + int readLen = 1024; + byte[] data = new byte[readLen]; + int n = stream.read(data); + assertEquals(readLen, n); + assertArrayEquals(Bytes.toBytes(DATA, readPos, readLen), data); + + n = stream.read(data); + assertEquals(readLen, n); + assertArrayEquals(Bytes.toBytes(DATA, readPos + 1024, readLen), data); + } + + try (ChainTOSInputStream stream = createTestChainTOSInputStream(DATA, 0, DATA_SIZE - 1, 1024, + cutOff)) { + // 1. Skip 1024 bytes and then read 1024 bytes from the first stream. + // 2. And then skip cut-off - 512 bytes, the target off = 1024 + 1024 + cut-off - 512, + // which is bigger than cut-off pos, so the second skip operation will fail, + // and then fallback to the second stream. + // 3. Read 1024 bytes + int readPos = 1024; + stream.skip(readPos); + + int readLen = 1024; + byte[] data = new byte[readLen]; + int n = stream.read(data); + assertEquals(readLen, n); + assertArrayEquals(Bytes.toBytes(DATA, readPos, readLen), data); + + int skipPos = cutOff - 512; + stream.skip(skipPos); + + n = stream.read(data); + assertEquals(readLen, n); + int targetOff = readPos + 1024 + skipPos; + assertArrayEquals(Bytes.toBytes(DATA, targetOff, readLen), data); + } + } + + /** + * The ChainTOSInputStream contains two stream created by TestObjectFactory. + * Once the read pos of first stream is more than cutPos, the stream will throw IOException with + * unexpect end of stream error msg, but the second stream will contain the remaining data. + */ + private ChainTOSInputStream createTestChainTOSInputStream(byte[] data, long startOff, long endOff, + long maxDrainSize, long cutPos) { + String key = "dummy-key"; + TOS.GetObjectFactory factory = new TestObjectFactory(data, Arrays.asList(cutPos, -1L)); + return new ChainTOSInputStream(factory, key, startOff, endOff, maxDrainSize, 1); + } + + private static class TestObjectFactory implements TOS.GetObjectFactory { + private final byte[] data; + private final List streamBreakPoses; + private int streamIndex = 0; + + TestObjectFactory(byte[] data, List streamBreakPoses) { + this.data = data; + this.streamBreakPoses = streamBreakPoses; + } + + @Override + public GetObjectOutput create(String key, long offset, long end) { + long len = Math.min(end, data.length) - offset; + ByteArrayInputStream dataIn = new ByteArrayInputStream(this.data, (int) offset, (int) len); + + if (streamIndex < streamBreakPoses.size()) { + return new GetObjectOutput(new GetObjectV2Output(new GetObjectBasicOutput(), + new UnExpectedEndOfStream(dataIn, streamBreakPoses.get(streamIndex++))), + Constants.MAGIC_CHECKSUM); + } else { + throw new RuntimeException("No more output"); + } + } + } + + private static class UnExpectedEndOfStream extends InputStream { + private final ByteArrayInputStream delegate; + private final long breakPos; + private int readPos; + + UnExpectedEndOfStream(ByteArrayInputStream stream, long breakPos) { + delegate = stream; + this.breakPos = breakPos; + } + + @Override + public int read() throws IOException { + if (breakPos != -1 && readPos >= breakPos) { + throw new IOException("unexpected end of stream on dummy source."); + } else { + int n = delegate.read(); + readPos += 1; + return n; + } + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + if (breakPos != -1 && readPos >= breakPos) { + throw new IOException("unexpected end of stream on dummy source."); + } else { + int n = delegate.read(b, off, len); + readPos += n; + return n; + } + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestDelegationClientBuilder.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestDelegationClientBuilder.java new file mode 100644 index 0000000000000..fabc26c23c852 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestDelegationClientBuilder.java @@ -0,0 +1,462 @@ +/* + * 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.tosfs.object.tos; + +import com.volcengine.tos.TOSV2; +import com.volcengine.tos.TOSV2ClientBuilder; +import com.volcengine.tos.TosClientException; +import com.volcengine.tos.TosException; +import com.volcengine.tos.TosServerException; +import com.volcengine.tos.auth.Credential; +import com.volcengine.tos.auth.StaticCredentials; +import com.volcengine.tos.comm.HttpStatus; +import com.volcengine.tos.model.object.DeleteObjectInput; +import com.volcengine.tos.model.object.HeadObjectV2Input; +import com.volcengine.tos.model.object.HeadObjectV2Output; +import com.volcengine.tos.model.object.ListObjectsV2Input; +import com.volcengine.tos.model.object.ListObjectsV2Output; +import com.volcengine.tos.model.object.PutObjectInput; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.fs.tosfs.common.Tasks; +import org.apache.hadoop.fs.tosfs.common.ThreadPools; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.conf.TosKeys; +import org.apache.hadoop.fs.tosfs.object.tos.auth.EnvironmentCredentialsProvider; +import org.apache.hadoop.fs.tosfs.object.tos.auth.SimpleCredentialsProvider; +import org.apache.hadoop.fs.tosfs.util.ParseUtils; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.apache.hadoop.fs.tosfs.util.UUIDUtils; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; +import org.junit.jupiter.api.TestReporter; + +import java.io.ByteArrayInputStream; +import java.io.EOFException; +import java.io.IOException; +import java.net.SocketException; +import java.net.SocketTimeoutException; +import java.net.UnknownHostException; +import java.util.Arrays; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import java.util.stream.IntStream; +import javax.net.ssl.SSLException; + +import static org.apache.hadoop.fs.tosfs.object.tos.DelegationClient.isRetryableException; +import static org.apache.hadoop.fs.tosfs.object.tos.TOS.TOS_SCHEME; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; +import static org.junit.jupiter.api.Assumptions.assumeTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class TestDelegationClientBuilder { + + private static final String TEST_KEY = UUIDUtils.random(); + private static final String TEST_DATA = "1234567890"; + private static final String ENV_ACCESS_KEY = + ParseUtils.envAsString(TOS.ENV_TOS_ACCESS_KEY_ID, false); + private static final String ENV_SECRET_KEY = + ParseUtils.envAsString(TOS.ENV_TOS_SECRET_ACCESS_KEY, false); + private static final String ENV_ENDPOINT = ParseUtils.envAsString(TOS.ENV_TOS_ENDPOINT, false); + + // Maximum retry times of the tos http client. + public static final String MAX_RETRY_COUNT_KEY = "fs.tos.http.maxRetryCount"; + + @BeforeAll + public static void before() { + assumeTrue(TestEnv.checkTestEnabled()); + } + + @BeforeEach + public void setUp() { + TOSV2 tosSdkClientV2 = + new TOSV2ClientBuilder().build(TestUtility.region(), TestUtility.endpoint(), + new StaticCredentials(ENV_ACCESS_KEY, ENV_SECRET_KEY)); + try (ByteArrayInputStream stream = new ByteArrayInputStream(TEST_DATA.getBytes())) { + PutObjectInput putObjectInput = + new PutObjectInput().setBucket(TestUtility.bucket()).setKey(TEST_KEY).setContent(stream); + tosSdkClientV2.putObject(putObjectInput); + } catch (IOException e) { + fail(e.getMessage()); + } + } + + @Test + public void testHeadApiRetry() throws IOException { + Configuration conf = new Configuration(); + conf.set(ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key(TOS_SCHEME), + "https://test.tos-cn-beijing.ivolces.com"); + conf.set(TosKeys.FS_TOS_CREDENTIALS_PROVIDER, SimpleCredentialsProvider.NAME); + conf.setBoolean(TosKeys.FS_TOS_DISABLE_CLIENT_CACHE, false); + conf.set(TosKeys.FS_TOS_BUCKET_ACCESS_KEY_ID.key("test"), "ACCESS_KEY"); + conf.set(TosKeys.FS_TOS_BUCKET_SECRET_ACCESS_KEY.key("test"), "SECRET_KEY"); + + DelegationClient tosV2 = new DelegationClientBuilder().bucket("test").conf(conf).build(); + TOSV2 mockClient = mock(TOSV2.class); + tosV2.setClient(mockClient); + tosV2.setMaxRetryTimes(5); + + HeadObjectV2Input input = HeadObjectV2Input.builder().bucket("test").build(); + when(tosV2.headObject(input)).thenThrow( + new TosServerException(HttpStatus.INTERNAL_SERVER_ERROR), + new TosServerException(HttpStatus.TOO_MANY_REQUESTS), + new TosClientException("fake toe", new IOException("fake ioe")), + new TosException(new SocketException("fake msg")), + new TosException(new UnknownHostException("fake msg")), + new TosException(new SSLException("fake msg")), + new TosException(new InterruptedException("fake msg")), + new TosException(new InterruptedException("fake msg"))) + .thenReturn(new HeadObjectV2Output()); + + RuntimeException exception = + assertThrows(RuntimeException.class, () -> tosV2.headObject(input)); + assertTrue(exception instanceof TosException); + assertTrue(exception.getCause() instanceof UnknownHostException); + verify(tosV2.client(), times(5)).headObject(input); + + HeadObjectV2Input inputOneTime = HeadObjectV2Input.builder().bucket("inputOneTime").build(); + HeadObjectV2Output output = new HeadObjectV2Output(); + when(tosV2.headObject(inputOneTime)).thenReturn(output); + HeadObjectV2Output headObject = tosV2.headObject(inputOneTime); + assertEquals(headObject, output); + verify(tosV2.client(), times(1)).headObject(inputOneTime); + tosV2.close(); + + DelegationClient newClient = new DelegationClientBuilder().bucket("test").conf(conf).build(); + mockClient = mock(TOSV2.class); + newClient.setClient(mockClient); + newClient.setMaxRetryTimes(5); + when(newClient.headObject(input)).thenThrow( + new TosClientException("fake toe", new EOFException("fake eof")), + new TosServerException(HttpStatus.INTERNAL_SERVER_ERROR), + new TosServerException(HttpStatus.TOO_MANY_REQUESTS)).thenReturn(new HeadObjectV2Output()); + + exception = assertThrows(RuntimeException.class, () -> newClient.headObject(input)); + assertTrue(exception instanceof TosClientException); + assertTrue(exception.getCause() instanceof EOFException); + verify(newClient.client(), times(1)).headObject(input); + newClient.close(); + } + + @Test + public void testEnableCrcCheck(TestInfo testInfo, TestReporter testReporter) throws IOException { + String bucket = testInfo.getTestMethod().map(method -> method.getName()).orElse("Unknown"); + Configuration conf = new Configuration(); + conf.set(ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key(TOS_SCHEME), + "https://test.tos-cn-beijing.ivolces.com"); + conf.set(TosKeys.FS_TOS_CREDENTIALS_PROVIDER, SimpleCredentialsProvider.NAME); + conf.setBoolean(TosKeys.FS_TOS_DISABLE_CLIENT_CACHE, true); + conf.set(TosKeys.FS_TOS_BUCKET_ACCESS_KEY_ID.key(bucket), "ACCESS_KEY"); + conf.set(TosKeys.FS_TOS_BUCKET_SECRET_ACCESS_KEY.key(bucket), "SECRET_KEY"); + + DelegationClient tosV2 = new DelegationClientBuilder().bucket(bucket).conf(conf).build(); + assertTrue(tosV2.config().isEnableCrc()); + + conf.setBoolean(TosKeys.FS_TOS_CRC_CHECK_ENABLED, false); + tosV2 = new DelegationClientBuilder().bucket(bucket).conf(conf).build(); + assertFalse(tosV2.config().isEnableCrc()); + + tosV2.close(); + } + + @Test + public void testClientCache(TestInfo testInfo, TestReporter testReporter) throws IOException { + String bucket = testInfo.getTestMethod().map(method -> method.getName()).orElse("Unknown"); + Configuration conf = new Configuration(); + conf.set(ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key(TOS_SCHEME), + "https://test.tos-cn-beijing.ivolces.com"); + conf.set(TosKeys.FS_TOS_CREDENTIALS_PROVIDER, SimpleCredentialsProvider.NAME); + conf.setBoolean(TosKeys.FS_TOS_DISABLE_CLIENT_CACHE, false); + conf.set(TosKeys.FS_TOS_BUCKET_ACCESS_KEY_ID.key(bucket), "ACCESS_KEY_A"); + conf.set(TosKeys.FS_TOS_BUCKET_SECRET_ACCESS_KEY.key(bucket), "SECRET_KEY_A"); + + DelegationClient tosV2 = new DelegationClientBuilder().bucket(bucket).conf(conf).build(); + DelegationClient tosV2Cached = new DelegationClientBuilder().bucket(bucket).conf(conf).build(); + assertEquals(tosV2Cached, tosV2, "client must be load in cache"); + assertEquals("ACCESS_KEY_A", tosV2.usedCredential().getAccessKeyId()); + tosV2Cached.close(); + + String newBucket = "new-test-bucket"; + conf.set(TosKeys.FS_TOS_BUCKET_ACCESS_KEY_ID.key(newBucket), "ACCESS_KEY_B"); + conf.set(TosKeys.FS_TOS_BUCKET_SECRET_ACCESS_KEY.key(newBucket), "SECRET_KEY_B"); + DelegationClient changeBucketClient = + new DelegationClientBuilder().bucket(newBucket).conf(conf).build(); + assertNotEquals(changeBucketClient, tosV2, "client should be created entirely new"); + assertEquals("ACCESS_KEY_B", changeBucketClient.usedCredential().getAccessKeyId()); + changeBucketClient.close(); + + conf.setBoolean(TosKeys.FS_TOS_DISABLE_CLIENT_CACHE, true); // disable cache: true + DelegationClient tosV2NotCached = + new DelegationClientBuilder().bucket(bucket).conf(conf).build(); + assertNotEquals(tosV2NotCached, tosV2, "client should be created entirely new"); + assertEquals("ACCESS_KEY_A", tosV2NotCached.usedCredential().getAccessKeyId()); + tosV2NotCached.close(); + + tosV2.close(); + } + + @Test + public void testOverwriteHttpConfig() throws IOException { + Configuration conf = new Configuration(); + conf.set(ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key(TOS_SCHEME), + "https://tos-cn-beijing.ivolces.com"); + conf.set(TosKeys.FS_TOS_CREDENTIALS_PROVIDER, SimpleCredentialsProvider.NAME); + conf.set(TosKeys.FS_TOS_BUCKET_ACCESS_KEY_ID.key("test"), "ACCESS_KEY"); + conf.set(TosKeys.FS_TOS_BUCKET_SECRET_ACCESS_KEY.key("test"), "SECRET_KEY"); + conf.setInt(TosKeys.FS_TOS_HTTP_MAX_CONNECTIONS, 24); + conf.setInt(MAX_RETRY_COUNT_KEY, 24); + conf.setInt(TosKeys.FS_TOS_REQUEST_MAX_RETRY_TIMES, 24); + conf.setBoolean(TosKeys.FS_TOS_DISABLE_CLIENT_CACHE, true); + + DelegationClient tosV2 = new DelegationClientBuilder().bucket("test").conf(conf).build(); + assertEquals("ACCESS_KEY", tosV2.usedCredential().getAccessKeyId()); + assertEquals(24, tosV2.config().getTransportConfig().getMaxConnections(), + "http max connection overwrite to 24 from 1024, must be 24"); + assertEquals(DelegationClientBuilder.DISABLE_TOS_RETRY_VALUE, + tosV2.config().getTransportConfig().getMaxRetryCount(), + "tos maxRetryCount disabled, must be -1"); + assertEquals(24, tosV2.maxRetryTimes(), "maxRetryTimes must be 24"); + assertEquals("https://tos-cn-beijing.ivolces.com", tosV2.config().getEndpoint(), + "endpoint must be equals to https://tos-cn-beijing.ivolces.com"); + + tosV2.close(); + } + + @Test + public void testDynamicRefreshAkSk() throws IOException { + Configuration conf = new Configuration(); + conf.set(ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key(TOS_SCHEME), ENV_ENDPOINT); + conf.set(TosKeys.FS_TOS_CREDENTIALS_PROVIDER, SimpleCredentialsProvider.NAME); + conf.set(TosKeys.FS_TOS_BUCKET_ACCESS_KEY_ID.key(TestUtility.bucket()), ENV_ACCESS_KEY); + conf.set(TosKeys.FS_TOS_BUCKET_SECRET_ACCESS_KEY.key(TestUtility.bucket()), ENV_SECRET_KEY); + conf.setInt(TosKeys.FS_TOS_HTTP_MAX_CONNECTIONS, 24); + conf.setInt(MAX_RETRY_COUNT_KEY, 24); + + TOSV2 tosSdkClientV2 = + new TOSV2ClientBuilder().build(TestUtility.region(), TestUtility.endpoint(), + new StaticCredentials("a", "b")); + DelegationClient delegationClientV2 = + new DelegationClientBuilder().bucket(TestUtility.bucket()).conf(conf).build(); + + ListObjectsV2Input inputV2 = + ListObjectsV2Input.builder().bucket(TestUtility.bucket()).prefix(TEST_KEY).marker("") + .maxKeys(10).build(); + + assertThrows(TosServerException.class, () -> tosSdkClientV2.listObjects(inputV2)); + + tosSdkClientV2.changeCredentials(new StaticCredentials(ENV_ACCESS_KEY, ENV_SECRET_KEY)); + + ListObjectsV2Output tosSdkOutput = tosSdkClientV2.listObjects(inputV2); + ListObjectsV2Output delegateOutput = delegationClientV2.listObjects(inputV2); + int nativeContentSize = + tosSdkOutput.getContents() == null ? -1 : tosSdkOutput.getContents().size(); + int delegateContentSize = + delegateOutput.getContents() == null ? -1 : delegateOutput.getContents().size(); + + assertEquals(nativeContentSize, delegateContentSize, + "delegation client must same as native client"); + assertEquals(ENV_ACCESS_KEY, delegationClientV2.usedCredential().getAccessKeyId()); + + delegationClientV2.close(); + } + + @Test + public void testCreateClientWithEnvironmentCredentials() throws IOException { + Configuration conf = new Configuration(); + conf.set(ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key(TOS_SCHEME), ENV_ENDPOINT); + conf.set(TosKeys.FS_TOS_CREDENTIALS_PROVIDER, EnvironmentCredentialsProvider.NAME); + + DelegationClient tosV2 = + new DelegationClientBuilder().bucket(TestUtility.bucket()).conf(conf).build(); + Credential cred = tosV2.usedCredential(); + + String assertMsg = + String.format("expect %s, but got %s", ENV_ACCESS_KEY, cred.getAccessKeyId()); + assertEquals(cred.getAccessKeyId(), ENV_ACCESS_KEY, assertMsg); + assertMsg = String.format("expect %s, but got %s", ENV_SECRET_KEY, cred.getAccessKeySecret()); + assertEquals(cred.getAccessKeySecret(), ENV_SECRET_KEY, assertMsg); + + tosV2.close(); + } + + @Test + public void testCreateClientWithSimpleCredentials() throws IOException { + Configuration conf = new Configuration(); + conf.set(ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key(TOS_SCHEME), ENV_ENDPOINT); + conf.set(TosKeys.FS_TOS_CREDENTIALS_PROVIDER, SimpleCredentialsProvider.NAME); + conf.set(TosKeys.FS_TOS_BUCKET_ACCESS_KEY_ID.key(TestUtility.bucket()), ENV_ACCESS_KEY); + conf.set(TosKeys.FS_TOS_BUCKET_SECRET_ACCESS_KEY.key(TestUtility.bucket()), ENV_SECRET_KEY); + conf.setInt(TosKeys.FS_TOS_HTTP_MAX_CONNECTIONS, 24); + conf.setInt(MAX_RETRY_COUNT_KEY, 24); + + ListObjectsV2Input input = + ListObjectsV2Input.builder().bucket(TestUtility.bucket()).prefix(TEST_KEY).marker("") + .maxKeys(10).build(); + + TOSV2 v2 = new TOSV2ClientBuilder().build(TestUtility.region(), TestUtility.endpoint(), + new StaticCredentials(ENV_ACCESS_KEY, ENV_SECRET_KEY)); + ListObjectsV2Output outputV2 = v2.listObjects(input); + + DelegationClient tosV2 = + new DelegationClientBuilder().bucket(TestUtility.bucket()).conf(conf).build(); + + ListObjectsV2Output output = tosV2.listObjects(input); + assertEquals(outputV2.getContents().size(), output.getContents().size(), + "delegation client must be same as native client"); + + tosV2.close(); + } + + @Test + public void testCachedConcurrently(TestInfo testInfo, TestReporter testReporter) { + String bucketName = testInfo.getTestMethod().map(method -> method.getName()).orElse("Unknown"); + + Function commonConf = bucket -> { + Configuration conf = new Configuration(); + conf.set(ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key(TOS_SCHEME), ENV_ENDPOINT); + conf.set(TosKeys.FS_TOS_CREDENTIALS_PROVIDER, SimpleCredentialsProvider.NAME); + conf.set(TosKeys.FS_TOS_BUCKET_ACCESS_KEY_ID.key(bucket), ENV_ACCESS_KEY); + conf.set(TosKeys.FS_TOS_BUCKET_SECRET_ACCESS_KEY.key(bucket), ENV_SECRET_KEY); + return conf; + }; + + // enable cache + Function enableCachedConf = bucket -> { + Configuration conf = commonConf.apply(bucket); + conf.setBoolean(TosKeys.FS_TOS_DISABLE_CLIENT_CACHE, false); + return conf; + }; + + ExecutorService es = ThreadPools.newWorkerPool("testCachedConcurrently", 32); + int bucketCount = 5; + int taskCount = 10000; + + AtomicInteger success = new AtomicInteger(0); + AtomicInteger failure = new AtomicInteger(0); + Tasks.foreach(IntStream.range(0, taskCount).boxed().map(i -> bucketName + (i % bucketCount))) + .executeWith(es).run(bucket -> { + try { + Configuration conf = enableCachedConf.apply(bucket); + DelegationClient client = + new DelegationClientBuilder().bucket(bucket).conf(conf).build(); + client.close(); + success.incrementAndGet(); + } catch (Exception e) { + failure.incrementAndGet(); + } + }); + + assertEquals(bucketCount, DelegationClientBuilder.CACHE.size()); + assertEquals(taskCount, success.get()); + assertEquals(0, failure.get()); + + // clear cache + DelegationClientBuilder.CACHE.clear(); + + // disable cache + Function disableCachedConf = bucket -> { + Configuration conf = commonConf.apply(bucket); + conf.setBoolean(TosKeys.FS_TOS_DISABLE_CLIENT_CACHE, true); + return conf; + }; + + success.set(0); + failure.set(0); + Tasks.foreach(IntStream.range(0, taskCount).boxed().map(i -> bucketName + (i % bucketCount))) + .executeWith(es).run(bucket -> { + try { + Configuration conf = disableCachedConf.apply(bucket); + DelegationClient client = + new DelegationClientBuilder().bucket(bucket).conf(conf).build(); + client.close(); + success.incrementAndGet(); + } catch (Exception e) { + failure.incrementAndGet(); + } + }); + + assertTrue(DelegationClientBuilder.CACHE.isEmpty()); + assertEquals(taskCount, success.get()); + assertEquals(0, failure.get()); + + es.shutdown(); + } + + @AfterEach + public void deleteAllTestData() throws IOException { + TOSV2 tosSdkClientV2 = + new TOSV2ClientBuilder().build(TestUtility.region(), TestUtility.endpoint(), + new StaticCredentials(ENV_ACCESS_KEY, ENV_SECRET_KEY)); + tosSdkClientV2.deleteObject( + DeleteObjectInput.builder().bucket(TestUtility.bucket()).key(TEST_KEY).build()); + + tosSdkClientV2.close(); + DelegationClientBuilder.CACHE.clear(); + } + + @Test + public void testRetryableException() { + assertTrue(retryableException(new TosServerException(500))); + assertTrue(retryableException(new TosServerException(501))); + assertTrue(retryableException(new TosServerException(429))); + assertFalse(retryableException(new TosServerException(404))); + + assertTrue(retryableException(new TosException(new SocketException()))); + assertTrue(retryableException(new TosException(new UnknownHostException()))); + assertTrue(retryableException(new TosException(new SSLException("fake ssl")))); + assertTrue(retryableException(new TosException(new SocketTimeoutException()))); + assertTrue(retryableException(new TosException(new InterruptedException()))); + + assertTrue(retryableException(new TosClientException("fake ioe", new IOException()))); + assertFalse(retryableException(new TosClientException("fake eof", new EOFException()))); + + assertTrue(retryableException(new TosServerException(409))); + assertTrue( + retryableException(new TosServerException(409).setEc(TOSErrorCodes.PATH_LOCK_CONFLICT))); + assertFalse( + retryableException(new TosServerException(409).setEc(TOSErrorCodes.DELETE_NON_EMPTY_DIR))); + assertFalse( + retryableException(new TosServerException(409).setEc(TOSErrorCodes.LOCATED_UNDER_A_FILE))); + assertFalse(retryableException( + new TosServerException(409).setEc(TOSErrorCodes.COPY_BETWEEN_DIR_AND_FILE))); + assertFalse(retryableException( + new TosServerException(409).setEc(TOSErrorCodes.RENAME_TO_AN_EXISTED_DIR))); + assertFalse( + retryableException(new TosServerException(409).setEc(TOSErrorCodes.RENAME_TO_SUB_DIR))); + assertFalse(retryableException( + new TosServerException(409).setEc(TOSErrorCodes.RENAME_BETWEEN_DIR_AND_FILE))); + } + + private boolean retryableException(TosException e) { + return isRetryableException(e, + Arrays.asList(TOSErrorCodes.FAST_FAILURE_CONFLICT_ERROR_CODES.split(","))); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestTOSInputStream.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestTOSInputStream.java new file mode 100644 index 0000000000000..48449196e47bb --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestTOSInputStream.java @@ -0,0 +1,153 @@ +/* + * 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.tosfs.object.tos; + +import com.volcengine.tos.internal.util.aborthook.AbortInputStreamHook; +import com.volcengine.tos.model.object.GetObjectBasicOutput; +import com.volcengine.tos.model.object.GetObjectV2Output; +import org.apache.hadoop.fs.tosfs.object.Constants; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.apache.hadoop.thirdparty.com.google.common.io.ByteStreams; +import org.junit.jupiter.api.Test; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; + +import static junit.framework.TestCase.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; + +public class TestTOSInputStream { + + private static final int DATA_SIZE = 1 << 20; + private static final byte[] DATA = TestUtility.rand(DATA_SIZE); + + @Test + public void testForceClose() throws IOException { + TOSInputStream stream = createStream(DATA, 0, DATA_SIZE - 1, 1024); + stream.close(); + assertTrue("Expected force close", cast(stream).isForceClose()); + + stream = createStream(DATA, 0, DATA_SIZE - 1, 1024); + ByteStreams.skipFully(stream, DATA_SIZE - 1024 - 1); + stream.close(); + assertTrue("Expected force close", cast(stream).isForceClose()); + + stream = createStream(DATA, 0, -1, 1024); + stream.close(); + assertTrue("Expected force close", cast(stream).isForceClose()); + + stream = createStream(DATA, 0, -1, 1024); + ByteStreams.skipFully(stream, DATA_SIZE - 1024 - 1); + stream.close(); + assertTrue("Expected force close", cast(stream).isForceClose()); + + stream = createStream(DATA, 0, -1, 1024); + ByteStreams.skipFully(stream, DATA_SIZE - 1024); + stream.close(); + assertTrue("Expected force close", cast(stream).isForceClose()); + } + + @Test + public void testClose() throws IOException { + TOSInputStream stream = createStream(DATA, 0, DATA_SIZE - 1, DATA_SIZE); + stream.close(); + assertFalse(cast(stream).isForceClose(), "Expected close by skipping bytes"); + + stream = createStream(DATA, 0, DATA_SIZE - 1, 1024); + ByteStreams.skipFully(stream, DATA_SIZE - 1024); + stream.close(); + assertFalse(cast(stream).isForceClose(), "Expected close by skipping bytes"); + + stream = createStream(DATA, 0, DATA_SIZE - 1, 1024); + ByteStreams.skipFully(stream, DATA_SIZE - 1023); + stream.close(); + assertFalse(cast(stream).isForceClose(), "Expected close by skipping bytes"); + + stream = createStream(DATA, 0, -1, DATA_SIZE + 1); + stream.close(); + assertFalse(cast(stream).isForceClose(), "Expected close by skipping bytes"); + + stream = createStream(DATA, 0, -1, 1024); + ByteStreams.skipFully(stream, DATA_SIZE - 1023); + stream.close(); + assertFalse(cast(stream).isForceClose(), "Expected close by skipping bytes"); + + stream = createStream(DATA, 0, -1, 1024); + ByteStreams.skipFully(stream, DATA_SIZE); + stream.close(); + assertFalse(cast(stream).isForceClose(), "Expected close by skipping bytes"); + } + + private TestInputStream cast(TOSInputStream stream) throws IOException { + InputStream content = stream.getObjectOutput().verifiedContent(Constants.MAGIC_CHECKSUM); + assertTrue("Not a TestInputStream", content instanceof TestInputStream); + return (TestInputStream) content; + } + + private TOSInputStream createStream(byte[] data, long startOff, long endOff, long maxDrainSize) + throws IOException { + TestInputStream stream = + new TestInputStream(data, (int) startOff, (int) (data.length - startOff)); + GetObjectV2Output output = new GetObjectV2Output(new GetObjectBasicOutput(), stream).setHook( + new ForceCloseHook(stream)); + + return new TOSInputStream(new GetObjectOutput(output, Constants.MAGIC_CHECKSUM), startOff, + endOff, maxDrainSize, Constants.MAGIC_CHECKSUM); + } + + private final static class TestInputStream extends ByteArrayInputStream { + // -1 means call close() + // 0 means neither call close() nor forceClose() + // 1 means call forceClose() + private int cloeState = 0; + + private TestInputStream(byte[] buf, int off, int len) { + super(buf, off, len); + } + + @Override + public void close() { + cloeState = -1; + } + + public void forceClose() { + cloeState = 1; + } + + boolean isForceClose() { + assertTrue("Neither call close() nor forceClose()", cloeState == -1 || cloeState == 1); + return cloeState == 1; + } + } + + private final static class ForceCloseHook implements AbortInputStreamHook { + private final TestInputStream in; + + private ForceCloseHook(TestInputStream in) { + this.in = in; + } + + @Override public void abort() { + if (in != null) { + in.forceClose(); + } + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestTOSObjectStorage.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestTOSObjectStorage.java new file mode 100644 index 0000000000000..282d39e204d57 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestTOSObjectStorage.java @@ -0,0 +1,302 @@ +/* + * 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.tosfs.object.tos; + +import com.volcengine.tos.internal.model.CRC64Checksum; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.fs.tosfs.common.Bytes; +import org.apache.hadoop.fs.tosfs.conf.TosKeys; +import org.apache.hadoop.fs.tosfs.object.ChecksumType; +import org.apache.hadoop.fs.tosfs.object.Constants; +import org.apache.hadoop.fs.tosfs.object.MultipartUpload; +import org.apache.hadoop.fs.tosfs.object.ObjectInfo; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectStorageFactory; +import org.apache.hadoop.fs.tosfs.object.Part; +import org.apache.hadoop.fs.tosfs.object.exceptions.NotAppendableException; +import org.apache.hadoop.fs.tosfs.object.request.ListObjectsRequest; +import org.apache.hadoop.fs.tosfs.object.response.ListObjectsResponse; +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.apache.hadoop.fs.tosfs.util.UUIDUtils; +import org.apache.hadoop.util.PureJavaCrc32C; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.ByteArrayInputStream; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Stream; +import java.util.zip.Checksum; + +import static org.apache.hadoop.fs.tosfs.object.tos.TOS.TOS_SCHEME; +import static org.junit.Assert.assertThrows; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assumptions.assumeFalse; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + +public class TestTOSObjectStorage { + public static Stream provideArguments() { + assumeTrue(TestEnv.checkTestEnabled()); + + List values = new ArrayList<>(); + + Configuration conf = new Configuration(); + conf.set(TosKeys.FS_TOS_CHECKSUM_TYPE, ChecksumType.CRC64ECMA.name()); + values.add(Arguments.of( + ObjectStorageFactory.createWithPrefix(String.format("tos-%s/", UUIDUtils.random()), + TOS_SCHEME, TestUtility.bucket(), conf), + new CRC64Checksum(), + ChecksumType.CRC64ECMA)); + + conf = new Configuration(); + conf.set(TosKeys.FS_TOS_CHECKSUM_TYPE, ChecksumType.CRC32C.name()); + values.add(Arguments.of( + ObjectStorageFactory.createWithPrefix(String.format("tos-%s/", UUIDUtils.random()), + TOS_SCHEME, TestUtility.bucket(), conf), + new PureJavaCrc32C(), + ChecksumType.CRC32C)); + + return values.stream(); + } + + private ObjectStorage tos; + private ChecksumType type; + + private void setEnv(ObjectStorage objectStore, ChecksumType csType) { + this.tos = objectStore; + this.type = csType; + } + + @AfterEach + public void tearDown() throws Exception { + CommonUtils.runQuietly(() -> tos.deleteAll("")); + for (MultipartUpload upload : tos.listUploads("")) { + tos.abortMultipartUpload(upload.key(), upload.uploadId()); + } + tos.close(); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testHeadObj(ObjectStorage objectStore, Checksum ckmer, ChecksumType csType) { + setEnv(objectStore, csType); + + String key = "testPutChecksum"; + byte[] data = TestUtility.rand(1024); + ckmer.update(data, 0, data.length); + assertEquals(ckmer.getValue(), parseChecksum(objectStore.put(key, data))); + + ObjectInfo objInfo = objectStore.head(key); + assertEquals(ckmer.getValue(), parseChecksum(objInfo.checksum())); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testGetFileStatus(ObjectStorage objectStore, Checksum ckmer, ChecksumType csType) { + setEnv(objectStore, csType); + assumeFalse(objectStore.bucket().isDirectory()); + + Configuration conf = new Configuration(objectStore.conf()); + conf.setBoolean(TosKeys.FS_TOS_GET_FILE_STATUS_ENABLED, true); + objectStore.initialize(conf, objectStore.bucket().name()); + + String key = "testFileStatus"; + byte[] data = TestUtility.rand(256); + byte[] checksum = objectStore.put(key, data); + + ObjectInfo obj1 = objectStore.objectStatus(key); + assertArrayEquals(checksum, obj1.checksum()); + assertEquals(key, obj1.key()); + assertEquals(obj1, objectStore.head(key)); + + ObjectInfo obj2 = objectStore.objectStatus(key + "/"); + assertNull(obj2); + + String dirKey = "testDirStatus/"; + checksum = objectStore.put(dirKey, new byte[0]); + + ObjectInfo obj3 = objectStore.objectStatus("testDirStatus"); + assertArrayEquals(checksum, obj3.checksum()); + assertEquals(dirKey, obj3.key()); + assertEquals(obj3, objectStore.head(dirKey)); + assertNull(objectStore.head("testDirStatus")); + ObjectInfo obj4 = objectStore.objectStatus(dirKey); + assertArrayEquals(checksum, obj4.checksum()); + assertEquals(dirKey, obj4.key()); + assertEquals(obj4, objectStore.head(dirKey)); + + String prefix = "testPrefix/"; + objectStore.put(prefix + "subfile", data); + ObjectInfo obj5 = objectStore.objectStatus(prefix); + assertEquals(prefix, obj5.key()); + assertArrayEquals(Constants.MAGIC_CHECKSUM, obj5.checksum()); + assertNull(objectStore.head(prefix)); + ObjectInfo obj6 = objectStore.objectStatus("testPrefix"); + assertEquals(prefix, obj6.key()); + assertArrayEquals(Constants.MAGIC_CHECKSUM, obj6.checksum()); + assertNull(objectStore.head("testPrefix")); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testObjectStatus(ObjectStorage objectStore, Checksum checksum, ChecksumType csType) { + setEnv(objectStore, csType); + assumeFalse(objectStore.bucket().isDirectory()); + + String key = "testObjectStatus"; + byte[] data = TestUtility.rand(1024); + checksum.update(data, 0, data.length); + assertEquals(checksum.getValue(), parseChecksum(objectStore.put(key, data))); + + ObjectInfo objInfo = objectStore.objectStatus(key); + assertEquals(checksum.getValue(), parseChecksum(objInfo.checksum())); + + objInfo = objectStore.head(key); + assertEquals(checksum.getValue(), parseChecksum(objInfo.checksum())); + + String dir = key + "/"; + objectStore.put(dir, new byte[0]); + objInfo = objectStore.objectStatus(dir); + assertEquals(Constants.MAGIC_CHECKSUM, objInfo.checksum()); + + objInfo = objectStore.head(dir); + assertEquals(Constants.MAGIC_CHECKSUM, objInfo.checksum()); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testListObjs(ObjectStorage objectStore, Checksum checksum, ChecksumType csType) { + setEnv(objectStore, csType); + + String key = "testListObjs"; + byte[] data = TestUtility.rand(1024); + checksum.update(data, 0, data.length); + for (int i = 0; i < 5; i++) { + assertEquals(checksum.getValue(), parseChecksum(objectStore.put(key, data))); + } + + ListObjectsRequest request = + ListObjectsRequest.builder().prefix(key).startAfter(null).maxKeys(-1).delimiter("/") + .build(); + Iterator iter = objectStore.list(request).iterator(); + while (iter.hasNext()) { + List objs = iter.next().objects(); + for (ObjectInfo obj : objs) { + assertEquals(checksum.getValue(), parseChecksum(obj.checksum())); + } + } + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testPutChecksum(ObjectStorage objectStore, Checksum checksum, ChecksumType csType) { + setEnv(objectStore, csType); + + String key = "testPutChecksum"; + byte[] data = TestUtility.rand(1024); + checksum.update(data, 0, data.length); + + byte[] checksumStr = objectStore.put(key, data); + + assertEquals(checksum.getValue(), parseChecksum(checksumStr)); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testMPUChecksum(ObjectStorage objectStore, Checksum checksum, ChecksumType csType) { + setEnv(objectStore, csType); + + int partNumber = 2; + String key = "testMPUChecksum"; + MultipartUpload mpu = objectStore.createMultipartUpload(key); + byte[] data = TestUtility.rand(mpu.minPartSize() * partNumber); + checksum.update(data, 0, data.length); + + List parts = new ArrayList<>(); + for (int i = 0; i < partNumber; i++) { + final int index = i; + Part part = objectStore.uploadPart(key, mpu.uploadId(), index + 1, + () -> new ByteArrayInputStream(data, index * mpu.minPartSize(), mpu.minPartSize()), + mpu.minPartSize()); + parts.add(part); + } + + byte[] checksumStr = objectStore.completeUpload(key, mpu.uploadId(), parts); + assertEquals(checksum.getValue(), parseChecksum(checksumStr)); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testAppendable(ObjectStorage objectStore, Checksum checksum, ChecksumType csType) { + setEnv(objectStore, csType); + assumeFalse(objectStore.bucket().isDirectory()); + + // Test create object with append then append. + byte[] data = TestUtility.rand(256); + String prefix = "a/testAppendable/"; + String key = prefix + "object.txt"; + objectStore.append(key, data); + + objectStore.append(key, new byte[0]); + + // Test create object with put then append. + data = TestUtility.rand(256); + objectStore.put(key, data); + + assertThrows("Expect not appendable.", NotAppendableException.class, + () -> objectStore.append(key, new byte[0])); + + objectStore.delete(key); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testDirectoryBucketAppendable(ObjectStorage objectStore, Checksum checksum, + ChecksumType csType) { + setEnv(objectStore, csType); + assumeTrue(objectStore.bucket().isDirectory()); + + byte[] data = TestUtility.rand(256); + String prefix = "a/testAppendable/"; + String key = prefix + "object.txt"; + objectStore.put(key, data); + + objectStore.append(key, new byte[1024]); + + objectStore.delete(key); + } + + private long parseChecksum(byte[] checksum) { + switch (type) { + case CRC32C: + case CRC64ECMA: + return Bytes.toLong(checksum); + default: + throw new IllegalArgumentException( + String.format("Checksum type %s is not supported by TOS.", type.name())); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestTOSRetryPolicy.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestTOSRetryPolicy.java new file mode 100644 index 0000000000000..2f841fce83896 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestTOSRetryPolicy.java @@ -0,0 +1,210 @@ +/* + * 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.tosfs.object.tos; + +import com.volcengine.tos.TOSV2; +import com.volcengine.tos.TosException; +import com.volcengine.tos.TosServerException; +import com.volcengine.tos.comm.HttpStatus; +import com.volcengine.tos.model.RequestInfo; +import com.volcengine.tos.model.object.PutObjectOutput; +import com.volcengine.tos.model.object.UploadPartV2Output; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.conf.TosKeys; +import org.apache.hadoop.fs.tosfs.object.InputStreamProvider; +import org.apache.hadoop.fs.tosfs.object.ObjectStorageFactory; +import org.apache.hadoop.fs.tosfs.object.Part; +import org.apache.hadoop.fs.tosfs.object.tos.auth.SimpleCredentialsProvider; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.io.InputStream; +import java.net.SocketException; +import java.net.UnknownHostException; +import java.util.HashMap; +import java.util.Map; +import javax.net.ssl.SSLException; + +import static org.apache.hadoop.fs.tosfs.object.tos.TOS.TOS_SCHEME; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assumptions.assumeTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class TestTOSRetryPolicy { + + private final String retryKey = "retryKey.txt"; + private TOSV2 tosClient; + private DelegationClient client; + + @BeforeAll + public static void before() { + assumeTrue(TestEnv.checkTestEnabled()); + } + + @BeforeEach + public void setUp() { + client = createRetryableDelegationClient(); + tosClient = mock(TOSV2.class); + client.setClient(tosClient); + } + + @AfterEach + public void tearDown() throws IOException { + tosClient.close(); + client.close(); + } + + private DelegationClient createRetryableDelegationClient() { + Configuration conf = new Configuration(); + conf.set(ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key(TOS_SCHEME), + "https://tos-cn-beijing.ivolces.com"); + conf.set(TosKeys.FS_TOS_CREDENTIALS_PROVIDER, SimpleCredentialsProvider.NAME); + conf.setBoolean(TosKeys.FS_TOS_DISABLE_CLIENT_CACHE, true); + conf.set(TosKeys.FS_TOS_ACCESS_KEY_ID, "ACCESS_KEY"); + conf.set(TosKeys.FS_TOS_SECRET_ACCESS_KEY, "SECRET_KEY"); + return new DelegationClientBuilder().bucket("test").conf(conf).build(); + } + + @Test + public void testShouldThrowExceptionAfterRunOut5RetryTimesIfNoRetryConfigSet() + throws IOException { + TOS storage = + (TOS) ObjectStorageFactory.create(TOS_SCHEME, TestUtility.bucket(), new Configuration()); + storage.setClient(client); + client.setMaxRetryTimes(5); + + PutObjectOutput response = mock(PutObjectOutput.class); + InputStreamProvider streamProvider = mock(InputStreamProvider.class); + + when(tosClient.putObject(any())).thenThrow( + new TosServerException(HttpStatus.INTERNAL_SERVER_ERROR), + new TosServerException(HttpStatus.TOO_MANY_REQUESTS), + new TosException(new SocketException("fake msg")), + new TosException(new UnknownHostException("fake msg")), + new TosException(new SSLException("fake msg")), + new TosException(new InterruptedException("fake msg")), + new TosException(new InterruptedException("fake msg"))).thenReturn(response); + + // after run out retry times, should throw exception + RuntimeException exception = + assertThrows(RuntimeException.class, () -> storage.put(retryKey, streamProvider, 0)); + assertTrue(exception instanceof TosException); + assertTrue(exception.getCause() instanceof SSLException); + + // the newStream method of stream provider should be called 5 times + verify(streamProvider, times(5)).newStream(); + + storage.close(); + } + + @Test + public void testShouldReturnResultAfterRetry8TimesIfConfigured10TimesRetry() + throws IOException { + TOS storage = + (TOS) ObjectStorageFactory.create(TOS_SCHEME, TestUtility.bucket(), new Configuration()); + DelegationClient delegationClient = createRetryableDelegationClient(); + delegationClient.setClient(tosClient); + delegationClient.setMaxRetryTimes(10); + storage.setClient(delegationClient); + + UploadPartV2Output response = new UploadPartV2Output().setPartNumber(1).setEtag("etag"); + + InputStream in = mock(InputStream.class); + InputStreamProvider streamProvider = mock(InputStreamProvider.class); + when(streamProvider.newStream()).thenReturn(in); + + when(tosClient.uploadPart(any())).thenThrow( + new TosServerException(HttpStatus.INTERNAL_SERVER_ERROR), + new TosServerException(HttpStatus.TOO_MANY_REQUESTS), + new TosException(new SocketException("fake msg")), + new TosException(new UnknownHostException("fake msg")), + new TosException(new SSLException("fake msg")), + new TosException(new InterruptedException("fake msg")), + new TosException(new InterruptedException("fake msg"))).thenReturn(response); + + // after run out retry times, should throw exception + Part part = storage.uploadPart(retryKey, "uploadId", 1, streamProvider, 0); + assertEquals(1, part.num()); + assertEquals("etag", part.eTag()); + + // the newStream method of stream provider should be called 8 times + verify(streamProvider, times(8)).newStream(); + + storage.close(); + } + + @Test + public void testShouldReturnResultIfRetry3TimesSucceed() throws IOException { + TOS storage = + (TOS) ObjectStorageFactory.create(TOS_SCHEME, TestUtility.bucket(), new Configuration()); + storage.setClient(client); + + PutObjectOutput response = mock(PutObjectOutput.class); + InputStreamProvider streamProvider = mock(InputStreamProvider.class); + + RequestInfo requestInfo = mock(RequestInfo.class); + Map header = new HashMap<>(); + when(response.getRequestInfo()).thenReturn(requestInfo); + when(requestInfo.getHeader()).thenReturn(header); + + when(tosClient.putObject(any())).thenThrow( + new TosServerException(HttpStatus.INTERNAL_SERVER_ERROR), + new TosServerException(HttpStatus.TOO_MANY_REQUESTS)).thenReturn(response); + + storage.put(retryKey, streamProvider, 0); + // the newStream method of stream provider should be called 3 times + verify(streamProvider, times(3)).newStream(); + + storage.close(); + } + + @Test + public void testShouldNotRetryIfThrowUnRetryException() throws IOException { + TOS storage = + (TOS) ObjectStorageFactory.create(TOS_SCHEME, TestUtility.bucket(), new Configuration()); + storage.setClient(client); + + InputStreamProvider streamProvider = mock(InputStreamProvider.class); + + when(tosClient.putObject(any())).thenThrow( + new TosException(new NullPointerException("fake msg."))); + + RuntimeException exception = + assertThrows(RuntimeException.class, () -> storage.put(retryKey, streamProvider, 0)); + assertTrue(exception instanceof TosException); + assertTrue(exception.getCause() instanceof NullPointerException); + + // the newStream method of stream provider should be only called once. + verify(streamProvider, times(1)).newStream(); + + storage.close(); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/auth/TestAbstractCredentialsProvider.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/auth/TestAbstractCredentialsProvider.java new file mode 100644 index 0000000000000..cbdbc7e4f7b19 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/auth/TestAbstractCredentialsProvider.java @@ -0,0 +1,60 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.object.tos.auth; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.object.tos.TOS; +import org.apache.hadoop.fs.tosfs.util.TestUtility; + +public abstract class TestAbstractCredentialsProvider { + private String envAccessKeyId; + private String envSecretAccessKey; + private String envSessionToken; + + protected Configuration getConf() { + return new Configuration(); + } + + protected void saveOsCredEnv() { + if (StringUtils.isNotEmpty(System.getenv(TOS.ENV_TOS_ACCESS_KEY_ID))) { + envAccessKeyId = System.getenv(TOS.ENV_TOS_ACCESS_KEY_ID); + } + + if (StringUtils.isNotEmpty(System.getenv(TOS.ENV_TOS_SECRET_ACCESS_KEY))) { + envSecretAccessKey = System.getenv(TOS.ENV_TOS_SECRET_ACCESS_KEY); + } + + if (StringUtils.isNotEmpty(System.getenv(TOS.ENV_TOS_SESSION_TOKEN))) { + envSessionToken = System.getenv(TOS.ENV_TOS_SESSION_TOKEN); + } + } + + protected void resetOsCredEnv() { + resetOsCredEnv(TOS.ENV_TOS_ACCESS_KEY_ID, envAccessKeyId); + resetOsCredEnv(TOS.ENV_TOS_SECRET_ACCESS_KEY, envSecretAccessKey); + resetOsCredEnv(TOS.ENV_TOS_SESSION_TOKEN, envSessionToken); + } + + private void resetOsCredEnv(String key, String value) { + if (StringUtils.isNotEmpty(value)) { + TestUtility.setSystemEnv(key, value); + } else { + TestUtility.removeSystemEnv(key); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/auth/TestDefaultCredentialsProviderChain.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/auth/TestDefaultCredentialsProviderChain.java new file mode 100644 index 0000000000000..cc235da58241a --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/auth/TestDefaultCredentialsProviderChain.java @@ -0,0 +1,209 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.object.tos.auth; + +import com.volcengine.tos.TosException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.conf.TosKeys; +import org.apache.hadoop.fs.tosfs.object.tos.TOS; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import static org.apache.hadoop.fs.tosfs.util.TestUtility.removeSystemEnv; +import static org.apache.hadoop.fs.tosfs.util.TestUtility.setSystemEnv; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestDefaultCredentialsProviderChain extends TestAbstractCredentialsProvider { + + private static final String MOCK_TEST_AK = "AK"; + private static final String MOCK_TEST_SK = "SK"; + private static final String MOCK_TEST_TST_TOKEN = "STS_TOKEN"; + + private static final String MOCK_TEST_AK_WITH_BUCKET = "AK_WITH_BUCKET"; + private static final String MOCK_TEST_SK_WITH_BUCKET = "SK_WITH_BUCKET"; + private static final String MOCK_TEST_STS_TOKEN_WITH_BUCKET = "STS_TOKEN_WITH_BUCKET"; + + private static final String MOCK_TEST_ENV_AK = "ENV_AK"; + private static final String MOCK_TEST_ENV_SK = "ENV_SK"; + private static final String MOCK_TEST_ENV_STS_TOKEN = "ENV_STS_TOKEN"; + + private static final String MOCK_TEST_BUCKET = "test"; + private static final String MOCK_TEST_ROLE_NAME = "roleName"; + private static final String MOCK_PATH = "/volcstack/latest/iam/security_credentials/"; + private static final String API_ENDPOINT = MOCK_PATH + MOCK_TEST_ROLE_NAME; + private static final String EXPIRED_TIME_PATTERN = "yyyy-MM-dd'T'HH:mm:ssXXX"; + + @Override + public Configuration getConf() { + Configuration conf = new Configuration(); + conf.set(TosKeys.FS_TOS_BUCKET_ACCESS_KEY_ID.key("test"), MOCK_TEST_AK_WITH_BUCKET); + conf.set(TosKeys.FS_TOS_BUCKET_SECRET_ACCESS_KEY.key("test"), MOCK_TEST_SK_WITH_BUCKET); + conf.set(TosKeys.FS_TOS_BUCKET_SESSION_TOKEN.key("test"), MOCK_TEST_STS_TOKEN_WITH_BUCKET); + conf.set(TosKeys.FS_TOS_ACCESS_KEY_ID, MOCK_TEST_AK); + conf.set(TosKeys.FS_TOS_SECRET_ACCESS_KEY, MOCK_TEST_SK); + conf.set(TosKeys.FS_TOS_SESSION_TOKEN, MOCK_TEST_TST_TOKEN); + return conf; + } + + @BeforeEach + public void setUp() { + saveOsCredEnv(); + } + + @Test + public void testLoadCredFromEnvProvider() { + Configuration conf = getConf(); + setSystemEnv(TOS.ENV_TOS_ACCESS_KEY_ID, MOCK_TEST_ENV_AK); + setSystemEnv(TOS.ENV_TOS_SECRET_ACCESS_KEY, MOCK_TEST_ENV_SK); + setSystemEnv(TOS.ENV_TOS_SESSION_TOKEN, MOCK_TEST_ENV_STS_TOKEN); + DefaultCredentialsProviderChain chain = new DefaultCredentialsProviderChain(); + chain.initialize(conf, null); + + assertEquals(chain.credential().getAccessKeyId(), + MOCK_TEST_ENV_AK, String.format("expect %s", MOCK_TEST_ENV_AK)); + assertEquals(chain.credential().getAccessKeySecret(), MOCK_TEST_ENV_SK, + String.format("expect %s", MOCK_TEST_ENV_SK)); + assertEquals(chain.credential().getSecurityToken(), MOCK_TEST_ENV_STS_TOKEN, + String.format("expect %s", MOCK_TEST_ENV_STS_TOKEN)); + assertTrue(chain.lastUsedProvider() instanceof EnvironmentCredentialsProvider); + } + + @Test + public void testLoadCredFromSimpleProviderWithBucket() { + Configuration conf = getConf(); + removeSystemEnv(TOS.ENV_TOS_ACCESS_KEY_ID); + removeSystemEnv(TOS.ENV_TOS_SECRET_ACCESS_KEY); + removeSystemEnv(TOS.ENV_TOS_SESSION_TOKEN); + DefaultCredentialsProviderChain chain = new DefaultCredentialsProviderChain(); + chain.initialize(conf, MOCK_TEST_BUCKET); + + assertEquals(chain.credential().getAccessKeyId(), MOCK_TEST_AK_WITH_BUCKET, + String.format("expect %s", MOCK_TEST_AK_WITH_BUCKET)); + assertEquals(chain.credential().getAccessKeySecret(), MOCK_TEST_SK_WITH_BUCKET, + String.format("expect %s", MOCK_TEST_SK_WITH_BUCKET)); + assertEquals(chain.credential().getSecurityToken(), MOCK_TEST_STS_TOKEN_WITH_BUCKET, + String.format("expect %s", MOCK_TEST_STS_TOKEN_WITH_BUCKET)); + assertTrue(chain.lastUsedProvider() instanceof SimpleCredentialsProvider); + } + + @Test + public void testLoadCredFromSimpleProvider() { + Configuration conf = getConf(); + removeSystemEnv(TOS.ENV_TOS_ACCESS_KEY_ID); + removeSystemEnv(TOS.ENV_TOS_SECRET_ACCESS_KEY); + DefaultCredentialsProviderChain chain = new DefaultCredentialsProviderChain(); + chain.initialize(conf, "test-bucket"); + + assertEquals(chain.credential().getAccessKeyId(), MOCK_TEST_AK, + String.format("expect %s", MOCK_TEST_AK)); + assertEquals(chain.credential().getAccessKeySecret(), MOCK_TEST_SK, + String.format("expect %s", MOCK_TEST_SK)); + assertTrue(chain.lastUsedProvider() instanceof SimpleCredentialsProvider); + } + + @Test + public void testNotFoundAnyProvider() { + removeSystemEnv(TOS.ENV_TOS_ACCESS_KEY_ID); + removeSystemEnv(TOS.ENV_TOS_SECRET_ACCESS_KEY); + DefaultCredentialsProviderChain chain = new DefaultCredentialsProviderChain(); + chain.initialize(new Configuration(), MOCK_TEST_BUCKET); + assertThrows(RuntimeException.class, chain::credential); + } + + @AfterEach + public void after() { + resetOsCredEnv(); + } + + @Test + public void testShouldReturnAKSKFollowByProviderSequence() { + setSystemEnv(TOS.ENV_TOS_ACCESS_KEY_ID, "ENV_ACCESS_KEY"); + setSystemEnv(TOS.ENV_TOS_SECRET_ACCESS_KEY, "ENV_SECRET_KEY"); + + // use the simple credential provider at first. + String providerClassesStr = SimpleCredentialsProvider.class.getName() + ',' + + EnvironmentCredentialsProvider.class.getName(); + + Configuration conf = new Configuration(); + conf.set(TosKeys.FS_TOS_CUSTOM_CREDENTIAL_PROVIDER_CLASSES, providerClassesStr); + conf.set(TosKeys.FS_TOS_ACCESS_KEY_ID, MOCK_TEST_AK); + conf.set(TosKeys.FS_TOS_SECRET_ACCESS_KEY, MOCK_TEST_SK); + conf.set(TosKeys.FS_TOS_SESSION_TOKEN, MOCK_TEST_TST_TOKEN); + + DefaultCredentialsProviderChain provider = new DefaultCredentialsProviderChain(); + provider.initialize(conf, MOCK_TEST_BUCKET); + + ExpireableCredential cred = provider.createCredential(); + assertEquals(MOCK_TEST_AK, cred.getAccessKeyId()); + assertEquals(MOCK_TEST_SK, cred.getAccessKeySecret()); + + assertFalse(cred.isExpired()); + + // use the env credential provider at first. + providerClassesStr = EnvironmentCredentialsProvider.class.getName() + ',' + + SimpleCredentialsProvider.class.getName(); + conf.set(TosKeys.FS_TOS_CUSTOM_CREDENTIAL_PROVIDER_CLASSES, providerClassesStr); + + provider = new DefaultCredentialsProviderChain(); + provider.initialize(conf, MOCK_TEST_BUCKET); + cred = provider.createCredential(); + assertEquals("ENV_ACCESS_KEY", cred.getAccessKeyId()); + assertEquals("ENV_SECRET_KEY", cred.getAccessKeySecret()); + assertFalse(cred.isExpired()); + + removeSystemEnv(TOS.ENV_TOS_ACCESS_KEY_ID); + removeSystemEnv(TOS.ENV_TOS_SECRET_ACCESS_KEY); + } + + @Test + public void testShouldThrowExceptionWhenCustomClassNotFound() { + Configuration conf = new Configuration(); + conf.set(TosKeys.FS_TOS_CUSTOM_CREDENTIAL_PROVIDER_CLASSES, + SimpleCredentialsProvider.class.getName() + "NotExist"); + + DefaultCredentialsProviderChain provider = new DefaultCredentialsProviderChain(); + TosException tosException = + assertThrows(TosException.class, () -> provider.initialize(conf, null)); + assertTrue(tosException.getCause() instanceof ClassNotFoundException); + } + + @Test + public void testShouldThrowExceptionIfNoDefaultConstructorFound() { + Configuration conf = new Configuration(); + conf.set(TosKeys.FS_TOS_CUSTOM_CREDENTIAL_PROVIDER_CLASSES, + TestCredentialProviderNoDefaultConstructor.class.getName()); + DefaultCredentialsProviderChain provider = new DefaultCredentialsProviderChain(); + RuntimeException exception = + assertThrows(RuntimeException.class, () -> provider.initialize(conf, null)); + assertTrue(exception.getMessage().contains("java.lang.NoSuchMethodException")); + } + + static class TestCredentialProviderNoDefaultConstructor extends AbstractCredentialsProvider { + + TestCredentialProviderNoDefaultConstructor(String fake) { + } + + @Override + protected ExpireableCredential createCredential() { + return null; + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/auth/TestEnvironmentCredentialsProvider.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/auth/TestEnvironmentCredentialsProvider.java new file mode 100644 index 0000000000000..d5d9addc9d6f9 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/auth/TestEnvironmentCredentialsProvider.java @@ -0,0 +1,67 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.object.tos.auth; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.object.tos.TOS; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; + +public class TestEnvironmentCredentialsProvider extends TestAbstractCredentialsProvider { + + @BeforeEach + public void setUp() { + saveOsCredEnv(); + } + + @Test + public void testLoadAkSkFromEnvProvider() { + TestUtility.setSystemEnv(TOS.ENV_TOS_ACCESS_KEY_ID, "AccessKeyId"); + TestUtility.setSystemEnv(TOS.ENV_TOS_SECRET_ACCESS_KEY, "SecretAccessKey"); + + EnvironmentCredentialsProvider provider = new EnvironmentCredentialsProvider(); + provider.initialize(new Configuration(), null); + + ExpireableCredential oldCred = provider.credential(); + assertEquals(oldCred.getAccessKeyId(), "AccessKeyId", "provider ak must be equals to env ak"); + assertEquals(oldCred.getAccessKeySecret(), "SecretAccessKey", + "provider sk must be equals to env sk"); + + TestUtility.setSystemEnv(TOS.ENV_TOS_ACCESS_KEY_ID, "newAccessKeyId"); + TestUtility.setSystemEnv(TOS.ENV_TOS_SECRET_ACCESS_KEY, "newSecretAccessKey"); + TestUtility.setSystemEnv(TOS.ENV_TOS_SESSION_TOKEN, "newSessionToken"); + + assertFalse(oldCred.isExpired()); + + ExpireableCredential newCred = provider.credential(); + assertEquals(newCred.getAccessKeyId(), "AccessKeyId", "provider ak must be equals to env ak"); + assertEquals(newCred.getAccessKeySecret(), "SecretAccessKey", + "provider sk must be equals to env sk"); + + assertFalse(newCred.isExpired()); + } + + @AfterEach + public void resetEnv() { + resetOsCredEnv(); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/auth/TestSimpleCredentialsProvider.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/auth/TestSimpleCredentialsProvider.java new file mode 100644 index 0000000000000..6e3dc8825a776 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/auth/TestSimpleCredentialsProvider.java @@ -0,0 +1,76 @@ +/* + * ByteDance Volcengine EMR, Copyright 2022. + * + * Licensed 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.tosfs.object.tos.auth; + +import com.volcengine.tos.auth.Credential; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.tosfs.conf.TosKeys; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestSimpleCredentialsProvider extends TestAbstractCredentialsProvider { + + @Test + public void testStaticCredentials() { + Configuration conf = new Configuration(); + conf.set(TosKeys.FS_TOS_ACCESS_KEY_ID, "ACCESS_KEY"); + conf.set(TosKeys.FS_TOS_SECRET_ACCESS_KEY, "SECRET_KEY"); + conf.set(TosKeys.FS_TOS_SESSION_TOKEN, "STS_TOKEN"); + SimpleCredentialsProvider provider = new SimpleCredentialsProvider(); + provider.initialize(conf, "test"); + Credential credentials = provider.credential(); + assertEquals("ACCESS_KEY", credentials.getAccessKeyId(), "access key must be ACCESS_KEY"); + assertEquals("SECRET_KEY", credentials.getAccessKeySecret(), "secret key must be SECRET_KEY"); + assertEquals("STS_TOKEN", credentials.getSecurityToken(), "sts token must be STS_TOKEN"); + } + + @Test + public void testStaticCredentialsWithBucket() { + Configuration conf = new Configuration(); + conf.set(TosKeys.FS_TOS_BUCKET_ACCESS_KEY_ID.key("test"), "ACCESS_KEY"); + conf.set(TosKeys.FS_TOS_BUCKET_SECRET_ACCESS_KEY.key("test"), "SECRET_KEY"); + conf.set(TosKeys.FS_TOS_BUCKET_SESSION_TOKEN.key("test"), "STS_TOKEN"); + SimpleCredentialsProvider provider = new SimpleCredentialsProvider(); + provider.initialize(conf, "test"); + Credential credentials = provider.credential(); + assertEquals("ACCESS_KEY", credentials.getAccessKeyId(), "access key must be ACCESS_KEY"); + assertEquals("SECRET_KEY", credentials.getAccessKeySecret(), "secret key must be SECRET_KEY"); + assertEquals("STS_TOKEN", credentials.getSecurityToken(), "sts token must be STS_TOKEN"); + } + + @Test + public void testStaticCredentialsWithPriority() { + Configuration conf = new Configuration(); + conf.set(TosKeys.FS_TOS_ACCESS_KEY_ID, "ACCESS_KEY"); + conf.set(TosKeys.FS_TOS_SECRET_ACCESS_KEY, "SECRET_KEY"); + conf.set(TosKeys.FS_TOS_SESSION_TOKEN, "STS_TOKEN"); + conf.set(TosKeys.FS_TOS_BUCKET_ACCESS_KEY_ID.key("test"), "ACCESS_KEY_BUCKET"); + conf.set(TosKeys.FS_TOS_BUCKET_SECRET_ACCESS_KEY.key("test"), "SECRET_KEY_BUCKET"); + conf.set(TosKeys.FS_TOS_BUCKET_SESSION_TOKEN.key("test"), "STS_TOKEN_BUCKET"); + + SimpleCredentialsProvider provider = new SimpleCredentialsProvider(); + provider.initialize(conf, "test"); + Credential credentials = provider.credential(); + assertEquals("ACCESS_KEY_BUCKET", credentials.getAccessKeyId(), + "access key must be ACCESS_KEY_BUCKET"); + assertEquals("SECRET_KEY_BUCKET", credentials.getAccessKeySecret(), + "secret key must be SECRET_KEY_BUCKET"); + assertEquals("STS_TOKEN_BUCKET", credentials.getSecurityToken(), + "sts token must be STS_TOKEN_BUCKET"); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestBaseFsOps.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestBaseFsOps.java new file mode 100644 index 0000000000000..3048f05eef30b --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestBaseFsOps.java @@ -0,0 +1,306 @@ +/* + * 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.tosfs.ops; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException; +import org.apache.hadoop.fs.tosfs.RawFileStatus; +import org.apache.hadoop.fs.tosfs.object.ObjectInfo; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectUtils; +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.IOException; +import java.io.InputStream; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public abstract class TestBaseFsOps implements TestBaseOps { + + private ObjectStorage storage; + + @Override + public ObjectStorage storage() { + return storage; + } + + private void setStorage(ObjectStorage storage) { + this.storage = storage; + } + + @AfterEach + public void tearDown() { + CommonUtils.runQuietly(() -> storage.deleteAll("")); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testDeleteFile(ObjectStorage store, FsOps fsOps) throws IOException { + setStorage(store); + Path path = new Path("/a/b"); + touchFile(path, TestUtility.rand(8)); + assertFileExist(path); + + fsOps.deleteFile(path); + assertFileDoesNotExist(path); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testDeleteEmptyDir(ObjectStorage store, FsOps fsOps) throws IOException { + setStorage(store); + Path path = new Path("/a/b/"); + mkdir(path); + + fsOps.deleteDir(path, false); + assertDirDoesNotExist(path); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testDeleteNonEmptyDir(ObjectStorage store, FsOps fsOps) throws IOException { + setStorage(store); + Path dirPath = new Path("/a/b/"); + Path subDirPath = new Path("/a/b/c/"); + Path filePath = new Path("/a/b/file.txt"); + mkdir(dirPath); + mkdir(subDirPath); + touchFile(filePath, new byte[10]); + + assertThrows(PathIsNotEmptyDirectoryException.class, () -> fsOps.deleteDir(dirPath, false)); + assertDirExist(dirPath); + assertDirExist(subDirPath); + assertFileExist(filePath); + + fsOps.deleteDir(dirPath, true); + assertDirDoesNotExist(dirPath); + assertDirDoesNotExist(subDirPath); + assertFileDoesNotExist(filePath); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testCreateDirRecursive(ObjectStorage store, FsOps fsOps) throws IOException { + setStorage(store); + Path path = new Path("/aa/bb/cc"); + String key = ObjectUtils.pathToKey(path, true); + String parentKey = ObjectUtils.pathToKey(path.getParent(), true); + String grandparents = ObjectUtils.pathToKey(path.getParent().getParent(), true); + + assertDirDoesNotExist(parentKey); + assertDirDoesNotExist(grandparents); + + fsOps.mkdirs(path); + assertDirExist(key); + assertDirExist(parentKey); + assertDirExist(grandparents); + + store.delete(key); + assertDirExist(parentKey); + assertDirExist(grandparents); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testListEmptyDir(ObjectStorage store, FsOps fsOps) { + setStorage(store); + Path dir = path("testListEmptyDir"); + mkdir(dir); + + assertFalse(listDir(fsOps, dir, false).iterator().hasNext()); + assertFalse(listDir(fsOps, dir, true).iterator().hasNext()); + assertTrue(fsOps.isEmptyDirectory(dir)); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testListNonExistDir(ObjectStorage store, FsOps fsOps) { + setStorage(store); + Path dir = path("testListNonExistDir"); + assertDirDoesNotExist(dir); + + assertFalse(listDir(fsOps, dir, false).iterator().hasNext()); + assertFalse(listDir(fsOps, dir, false).iterator().hasNext()); + assertTrue(fsOps.isEmptyDirectory(dir)); + } + + private Iterable listDir(FsOps fsOps, Path dir, boolean recursive) { + return fsOps.listDir(dir, recursive, s -> true); + } + + private Iterable listFiles(FsOps fsOps, Path dir, boolean recursive) { + return fsOps.listDir(dir, recursive, s -> !ObjectInfo.isDir(s)); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testListAFileViaListDir(ObjectStorage store, FsOps fsOps) { + setStorage(store); + Path file = new Path("testListFileViaListDir"); + touchFile(file, TestUtility.rand(8)); + assertFalse(listDir(fsOps, file, false).iterator().hasNext()); + assertFalse(listDir(fsOps, file, true).iterator().hasNext()); + + Path nonExistFile = new Path("testListFileViaListDir-nonExist"); + assertFileDoesNotExist(nonExistFile); + assertFalse(listDir(fsOps, nonExistFile, false).iterator().hasNext()); + assertFalse(listDir(fsOps, nonExistFile, true).iterator().hasNext()); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testListFiles(ObjectStorage store, FsOps fsOps) { + setStorage(store); + Path dir = path("testListEmptyFiles"); + mkdir(dir); + + assertFalse(listFiles(fsOps, dir, false).iterator().hasNext()); + assertFalse(listFiles(fsOps, dir, true).iterator().hasNext()); + + mkdir(new Path(dir, "subDir")); + assertFalse(listFiles(fsOps, dir, false).iterator().hasNext()); + assertFalse(listFiles(fsOps, dir, true).iterator().hasNext()); + + RawFileStatus subDir = listDir(fsOps, dir, false).iterator().next(); + assertFalse(subDir.isFile()); + assertEquals("subDir", subDir.getPath().getName()); + + ObjectInfo fileObj = touchFile(new Path(dir, "subFile"), TestUtility.rand(8)); + RawFileStatus subFile = listFiles(fsOps, dir, false).iterator().next(); + assertArrayEquals(fileObj.checksum(), subFile.checksum()); + assertTrue(subFile.isFile()); + + assertFalse(fsOps.isEmptyDirectory(dir)); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testRecursiveList(ObjectStorage store, FsOps fsOps) { + setStorage(store); + Path root = path("root"); + Path file1 = path("root", "file1"); + Path file2 = path("root", "afile2"); + Path dir1 = path("root", "dir1"); + Path file3 = path("root", "dir1", "file3"); + + mkdir(root); + mkdir(dir1); + touchFile(file1, TestUtility.rand(8)); + touchFile(file2, TestUtility.rand(8)); + touchFile(file3, TestUtility.rand(8)); + + // List result is in sorted lexicographical order if recursive is false + Assertions.assertThat(listDir(fsOps, root, false)) + .hasSize(3) + .extracting(f -> f.getPath().getName()) + .contains("afile2", "dir1", "file1"); + + // List result is in sorted lexicographical order if recursive is false + Assertions.assertThat(listFiles(fsOps, root, false)) + .hasSize(2) + .extracting(f -> f.getPath().getName()) + .contains("afile2", "file1"); + + // listDir with recursive=true doesn't guarantee the return result in a sorted order + Assertions.assertThat(listDir(fsOps, root, true)) + .hasSize(4) + .extracting(f -> f.getPath().getName()) + .containsExactlyInAnyOrder("afile2", "dir1", "file1", "file3"); + + // listFiles with recursive=true doesn't guarantee the return result in a sorted order + Assertions.assertThat(listFiles(fsOps, root, true)) + .hasSize(3) + .extracting(f -> f.getPath().getName()) + .containsExactlyInAnyOrder("afile2", "file1", "file3"); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testRenameFile(ObjectStorage store, FsOps fsOps) throws IOException { + setStorage(store); + Path renameSrc = path("renameSrc"); + Path renameDest = path("renameDst"); + + int dataSize = 1024 * 1024; + String filename = String.format("%sMB.txt", dataSize >> 20); + Path srcFile = new Path(renameSrc, filename); + byte[] data = writeData(srcFile, dataSize); + Path dstFile = new Path(renameDest, filename); + + // The dest file and dest parent don't exist. + assertFileExist(srcFile); + assertDirDoesNotExist(renameDest); + assertFileDoesNotExist(dstFile); + + fsOps.renameFile(srcFile, dstFile, data.length); + assertFileDoesNotExist(srcFile); + assertDirExist(renameSrc); + assertFileExist(dstFile); + + try (InputStream in = store.get(ObjectUtils.pathToKey(dstFile)).stream()) { + assertArrayEquals(data, IOUtils.toByteArray(in)); + } + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testRenameDir(ObjectStorage store, FsOps fsOps) throws IOException { + setStorage(store); + Path renameSrc = path("renameSrc"); + Path renameDest = path("renameDst"); + + mkdir(renameSrc); + int dataSize = 1024 * 1024; + String filename = String.format("%sMB.txt", dataSize >> 20); + Path srcFile = new Path(renameSrc, filename); + Path dstFile = new Path(renameDest, filename); + byte[] data = writeData(srcFile, dataSize); + + assertFileExist(srcFile); + assertFileDoesNotExist(dstFile); + assertDirExist(renameSrc); + assertDirDoesNotExist(renameDest); + + fsOps.renameDir(renameSrc, renameDest); + assertFileDoesNotExist(srcFile); + assertDirDoesNotExist(renameSrc); + assertFileExist(dstFile); + assertDirExist(renameDest); + + try (InputStream in = store.get(ObjectUtils.pathToKey(dstFile)).stream()) { + assertArrayEquals(data, IOUtils.toByteArray(in)); + } + } + + private byte[] writeData(Path path, int size) { + byte[] data = TestUtility.rand(size); + touchFile(path, data); + return data; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestBaseOps.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestBaseOps.java new file mode 100644 index 0000000000000..90e31cbc47bdd --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestBaseOps.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.tosfs.ops; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.object.ObjectInfo; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectUtils; +import org.apache.hadoop.thirdparty.com.google.common.base.Joiner; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; + +public interface TestBaseOps { + + default Path path(String... keys) { + return new Path(String.format("/%s", Joiner.on("/").join(keys))); + } + + default void assertFileExist(Path file) { + assertNotNull(ObjectUtils.pathToKey(file)); + } + + default void assertFileDoesNotExist(String key) { + assertNull(storage().head(key)); + } + + default void assertFileDoesNotExist(Path file) { + assertFileDoesNotExist(ObjectUtils.pathToKey(file)); + } + + default void assertDirExist(String key) { + assertNotNull(storage().head(key)); + } + + default void assertDirExist(Path path) { + assertDirExist(ObjectUtils.pathToKey(path, true)); + } + + default void assertDirDoesNotExist(String key) { + assertNull(storage().head(key)); + } + + default void assertDirDoesNotExist(Path path) { + assertDirDoesNotExist(ObjectUtils.pathToKey(path, true)); + } + + default void mkdir(Path path) { + storage().put(ObjectUtils.pathToKey(path, true), new byte[0]); + assertDirExist(path); + } + + default ObjectInfo touchFile(Path path, byte[] data) { + byte[] checksum = storage().put(ObjectUtils.pathToKey(path), data); + ObjectInfo obj = storage().head(ObjectUtils.pathToKey(path)); + assertArrayEquals(checksum, obj.checksum()); + return obj; + } + + ObjectStorage storage(); +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestDefaultFsOps.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestDefaultFsOps.java new file mode 100644 index 0000000000000..e587c71c61ab9 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestDefaultFsOps.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.tosfs.ops; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.RawFileStatus; +import org.apache.hadoop.fs.tosfs.RawFileSystem; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.fs.tosfs.common.ThreadPools; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectStorageFactory; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.apache.hadoop.fs.tosfs.util.UUIDUtils; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.params.provider.Arguments; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.stream.Stream; + +import static org.apache.hadoop.fs.tosfs.object.tos.TOS.TOS_SCHEME; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + +public class TestDefaultFsOps extends TestBaseFsOps { + private static ExecutorService threadPool; + + static Stream provideArguments() { + // Case1: direct rename. + List values = new ArrayList<>(); + Configuration directRenameConf = new Configuration(); + directRenameConf.setBoolean(ConfKeys.FS_OBJECT_RENAME_ENABLED.key("tos"), true); + directRenameConf.setBoolean(ConfKeys.FS_ASYNC_CREATE_MISSED_PARENT.key("tos"), false); + + ObjectStorage storage0 = + ObjectStorageFactory.createWithPrefix(String.format("tos-%s/", UUIDUtils.random()), + TOS_SCHEME, TestUtility.bucket(), directRenameConf); + values.add(Arguments.of( + storage0, + new DefaultFsOps(storage0, directRenameConf, threadPool, obj -> { + long modifiedTime = RawFileSystem.dateToLong(obj.mtime()); + String path = + String.format("%s://%s/%s", storage0.scheme(), storage0.bucket().name(), obj.key()); + return new RawFileStatus(obj.size(), obj.isDir(), 0, modifiedTime, new Path(path), "fake", + obj.checksum()); + }))); + + // Case2: copied rename. + Configuration copiedRenameConf = new Configuration(); + copiedRenameConf.setLong(ConfKeys.FS_MULTIPART_COPY_THRESHOLD.key("tos"), 1L << 20); + copiedRenameConf.setBoolean(ConfKeys.FS_ASYNC_CREATE_MISSED_PARENT.key("tos"), false); + + ObjectStorage storage1 = + ObjectStorageFactory.createWithPrefix(String.format("tos-%s/", UUIDUtils.random()), + TOS_SCHEME, TestUtility.bucket(), copiedRenameConf); + values.add(Arguments.of( + storage1, + new DefaultFsOps(storage1, copiedRenameConf, threadPool, obj -> { + long modifiedTime = RawFileSystem.dateToLong(obj.mtime()); + String path = + String.format("%s://%s/%s", storage1.scheme(), storage1.bucket().name(), obj.key()); + return new RawFileStatus(obj.size(), obj.isDir(), 0, modifiedTime, new Path(path), "fake", + obj.checksum()); + }))); + + return values.stream(); + } + + @BeforeAll + public static void beforeClass() { + assumeTrue(TestEnv.checkTestEnabled()); + threadPool = ThreadPools.newWorkerPool("TestDefaultFsHelper-pool"); + } + + @AfterAll + public static void afterClass() { + if (!TestEnv.checkTestEnabled()) { + return; + } + + if (!threadPool.isShutdown()) { + threadPool.shutdown(); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestDirectoryFsOps.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestDirectoryFsOps.java new file mode 100644 index 0000000000000..eaac49ef73e90 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestDirectoryFsOps.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.tosfs.ops; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.RawFileStatus; +import org.apache.hadoop.fs.tosfs.RawFileSystem; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.fs.tosfs.object.DirectoryStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectStorageFactory; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.apache.hadoop.fs.tosfs.util.UUIDUtils; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.params.provider.Arguments; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Stream; + +import static org.apache.hadoop.fs.tosfs.object.tos.TOS.TOS_SCHEME; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + +// TODO change to directory bucket configuration. +public class TestDirectoryFsOps extends TestBaseFsOps { + + public static Stream provideArguments() { + List values = new ArrayList<>(); + + ObjectStorage storage = + ObjectStorageFactory.createWithPrefix(String.format("tos-%s/", UUIDUtils.random()), + TOS_SCHEME, TestUtility.bucket(), new Configuration()); + values.add(Arguments.of(storage, new DirectoryFsOps((DirectoryStorage) storage, obj -> { + long modifiedTime = RawFileSystem.dateToLong(obj.mtime()); + String path = + String.format("%s://%s/%s", storage.scheme(), storage.bucket().name(), obj.key()); + return new RawFileStatus(obj.size(), obj.isDir(), 0, modifiedTime, new Path(path), "fake", + obj.checksum()); + }))); + + return values.stream(); + } + + @BeforeAll + public static void beforeClass() { + assumeTrue(TestEnv.checkTestEnabled()); + } + + @Override + public void testRenameDir(ObjectStorage store, FsOps fsOps) { + // Will remove this test case once test environment support + assumeTrue(store.bucket().isDirectory()); + } + + @Override + public void testRenameFile(ObjectStorage store, FsOps fsOps) { + // Will remove this test case once test environment support + assumeTrue(store.bucket().isDirectory()); + } + + @Override + public void testCreateDirRecursive(ObjectStorage store, FsOps fsOps) { + // Will remove this test case once test environment support + assumeTrue(store.bucket().isDirectory()); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestRenameOp.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestRenameOp.java new file mode 100644 index 0000000000000..159e6f8760147 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestRenameOp.java @@ -0,0 +1,198 @@ +/* + * 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.tosfs.ops; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.tosfs.TestEnv; +import org.apache.hadoop.fs.tosfs.common.ThreadPools; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectUtils; +import org.apache.hadoop.fs.tosfs.object.Part; +import org.apache.hadoop.fs.tosfs.util.CommonUtils; +import org.apache.hadoop.fs.tosfs.util.TempFiles; +import org.apache.hadoop.fs.tosfs.util.TestUtility; +import org.apache.hadoop.thirdparty.com.google.common.collect.Maps; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assumptions.assumeFalse; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + +public class TestRenameOp implements TestBaseOps { + private static final String FILE_STORE_ROOT = TempFiles.newTempDir("TestRenameOp"); + + private ObjectStorage storage; + private ExecutorService renamePool; + + static Stream provideArguments() { + assumeTrue(TestEnv.checkTestEnabled()); + + List values = new ArrayList<>(); + List storages = TestUtility.createTestObjectStorage(FILE_STORE_ROOT); + for (ObjectStorage store : storages) { + values.add(Arguments.of(store)); + } + return values.stream(); + } + + @Override + public ObjectStorage storage() { + return storage; + } + + private void setStorage(ObjectStorage storage) { + this.storage = storage; + } + + @BeforeEach + public void prepare() { + this.renamePool = ThreadPools.newWorkerPool("renamePool"); + } + + @AfterEach + public void tearDown() { + CommonUtils.runQuietly(() -> storage().deleteAll("")); + CommonUtils.runQuietly(renamePool::shutdown); + } + + @AfterAll + public static void afterClass() { + CommonUtils.runQuietly(() -> TempFiles.deleteDir(FILE_STORE_ROOT)); + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testRenameFileDirectly(ObjectStorage store) throws IOException { + setStorage(store); + + Configuration conf = new Configuration(); + conf.setLong(ConfKeys.FS_MULTIPART_COPY_THRESHOLD.key(store.scheme()), 1L << 20); + ExtendedRenameOp operation = new ExtendedRenameOp(conf, store, renamePool); + + Path renameSrc = path("renameSrc"); + Path renameDest = path("renameDst"); + + int dataSize = 1024 * 1024; + String filename = String.format("%sMB.txt", dataSize >> 20); + Path srcFile = new Path(renameSrc, filename); + Path dstFile = new Path(renameDest, filename); + byte[] data = writeData(srcFile, dataSize); + mkdir(renameDest); + + assertFileExist(srcFile); + assertFileDoesNotExist(dstFile); + assertDirExist(renameDest); + + operation.renameFile(srcFile, dstFile, data.length); + assertFileDoesNotExist(srcFile); + assertFileExist(dstFile); + Map> uploadInfos = operation.uploadInfos; + assertEquals(0, uploadInfos.size(), + "use put method when rename file, upload info's size should be 0"); + + try (InputStream in = store.get(ObjectUtils.pathToKey(dstFile)).stream()) { + assertArrayEquals(data, IOUtils.toByteArray(in)); + } + } + + @ParameterizedTest + @MethodSource("provideArguments") + public void testRenameFileByUploadParts(ObjectStorage store) throws IOException { + setStorage(store); + + assumeFalse(store.bucket().isDirectory()); + Configuration conf = new Configuration(); + conf.setLong(ConfKeys.FS_MULTIPART_COPY_THRESHOLD.key(store.scheme()), 1L << 20); + ExtendedRenameOp operation = new ExtendedRenameOp(conf, store, renamePool); + + Path renameSrc = path("renameSrc"); + Path renameDest = path("renameDst"); + + int dataSize = 10 * 1024 * 1024; + String filename = String.format("%sMB.txt", dataSize >> 20); + Path srcFile = new Path(renameSrc, filename); + Path dstFile = new Path(renameDest, filename); + byte[] data = writeData(srcFile, dataSize); + mkdir(renameDest); + + assertFileExist(srcFile); + assertFileDoesNotExist(dstFile); + assertDirExist(renameDest); + + operation.renameFile(srcFile, dstFile, data.length); + assertFileDoesNotExist(srcFile); + assertFileExist(dstFile); + Map> uploadInfos = operation.uploadInfos; + assertTrue(uploadInfos.size() != 0, + "use upload parts method when rename file, upload info's size should not be 0"); + List parts = uploadInfos.get(ObjectUtils.pathToKey(dstFile)); + assertNotNull(parts, + "use upload parts method when rename file, upload info should not be null"); + assertTrue(parts.size() >= 2, "use upload parts method when rename file," + + " the num of upload parts should be greater than or equal to 2"); + long fileLength = parts.stream().mapToLong(Part::size).sum(); + assertEquals(dataSize, fileLength); + + try (InputStream in = store.get(ObjectUtils.pathToKey(dstFile)).stream()) { + assertArrayEquals(data, IOUtils.toByteArray(in)); + } + } + + private byte[] writeData(Path path, int size) { + byte[] data = TestUtility.rand(size); + touchFile(path, data); + return data; + } + + static class ExtendedRenameOp extends RenameOp { + private Map> uploadInfos = Maps.newHashMap(); + + ExtendedRenameOp(Configuration conf, ObjectStorage storage, ExecutorService pool) { + super(conf, storage, pool); + } + + @Override + protected void finishUpload(String key, String uploadId, List uploadParts) { + super.finishUpload(key, uploadId, uploadParts); + if (!uploadInfos.isEmpty()) { + uploadInfos.clear(); + } + uploadInfos.put(key, uploadParts); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TempFiles.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TempFiles.java new file mode 100644 index 0000000000000..d4191de66a7cc --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TempFiles.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.tosfs.util; + +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.util.Lists; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.List; + +public final class TempFiles implements Closeable { + private final List files = Lists.newArrayList(); + private final List dirs = Lists.newArrayList(); + + private TempFiles() { + } + + public static TempFiles of() { + return new TempFiles(); + } + + public String newFile() { + String p = newTempFile(); + files.add(p); + return p; + } + + public String newDir() { + return newDir(null); + } + + public String newDir(String prefix) { + String p = newTempDir(prefix); + dirs.add(p); + return p; + } + + @Override + public void close() { + files.forEach(file -> CommonUtils.runQuietly(() -> TempFiles.deleteFile(file))); + files.clear(); + dirs.forEach(dir -> CommonUtils.runQuietly(() -> TempFiles.deleteDir(dir))); + dirs.clear(); + } + + public static String newTempFile() { + return String.join(File.pathSeparator, newTempDir(), UUIDUtils.random()); + } + + public static String newTempDir() { + return newTempDir(null); + } + + public static String newTempDir(String prefix) { + try { + return Files.createTempDirectory(prefix).toFile().getAbsolutePath(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + public static void deleteFile(String path) { + try { + Files.deleteIfExists(Paths.get(path)); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + public static void deleteDir(String path) { + try { + FileUtils.deleteDirectory(new File(path)); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestFSUtils.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestFSUtils.java new file mode 100644 index 0000000000000..cdf32997777d2 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestFSUtils.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.tosfs.util; + +import org.apache.hadoop.conf.Configuration; +import org.junit.jupiter.api.Test; + +import java.net.URI; +import java.net.URISyntaxException; + +import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + +public class TestFSUtils { + @Test + public void testNormalizeURI() throws URISyntaxException { + URI uri = new URI("tos://abc/dir/key"); + URI normalizeURI = FSUtils.normalizeURI(uri, new Configuration()); + assertEquals("tos", normalizeURI.getScheme()); + assertEquals("abc", normalizeURI.getAuthority()); + assertEquals("abc", normalizeURI.getHost()); + assertEquals("/dir/key", normalizeURI.getPath()); + + uri = new URI("/abc/dir/key"); + normalizeURI = FSUtils.normalizeURI(uri, new Configuration()); + assertNull(uri.getScheme()); + assertEquals("file", normalizeURI.getScheme()); + assertNull(uri.getAuthority()); + assertNull(normalizeURI.getAuthority()); + assertEquals("/abc/dir/key", uri.getPath()); + assertEquals("/", normalizeURI.getPath()); + + uri = new URI("tos:///abc/dir/key"); + normalizeURI = FSUtils.normalizeURI(uri, new Configuration()); + assertEquals("tos", uri.getScheme()); + assertNull(uri.getAuthority()); + assertEquals("/abc/dir/key", uri.getPath()); + assertEquals("tos", normalizeURI.getScheme()); + assertNull(normalizeURI.getAuthority()); + assertEquals("/abc/dir/key", normalizeURI.getPath()); + + Configuration conf = new Configuration(); + conf.set(FS_DEFAULT_NAME_KEY, "tos://bucket/"); + normalizeURI = FSUtils.normalizeURI(uri, conf); + assertEquals("tos", normalizeURI.getScheme()); + assertEquals("bucket", normalizeURI.getAuthority()); + assertEquals("/", normalizeURI.getPath()); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestIterables.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestIterables.java new file mode 100644 index 0000000000000..16ba414af075c --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestIterables.java @@ -0,0 +1,165 @@ +/* + * 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.tosfs.util; + +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; +import java.util.function.Predicate; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestIterables { + + @Test + public void testTransform() { + List list = Arrays.asList(1, 2, 3, 4, 5); + Function transform = i -> i + 10; + Iterator iter = Iterables.transform(list, transform).iterator(); + + for (int i = 0; i < 5; i++) { + assertTrue(iter.hasNext()); + int value = iter.next(); + assertEquals(10 + i + 1, value); + } + assertFalse(iter.hasNext()); + } + + @Test + public void testTransformEmptyIterable() { + List list = Arrays.asList(); + Function transform = i -> i + 10; + Iterator iter = Iterables.transform(list, transform).iterator(); + + assertFalse(iter.hasNext()); + } + + @Test + public void testFilter() { + // Filter odd elements. + List list = Arrays.asList(1, 2, 3, 4, 5); + Predicate filter = i -> (i % 2) == 0; + Iterator iter = Iterables.filter(list, filter).iterator(); + + for (int i = 0; i < 2; i++) { + assertTrue(iter.hasNext()); + int value = iter.next(); + assertEquals((i + 1) * 2, value); + } + assertFalse(iter.hasNext()); + + // Ignore all elements. + filter = i -> false; + iter = Iterables.filter(list, filter).iterator(); + assertFalse(iter.hasNext()); + } + + @Test + public void testFilterEmptyIterable() { + List list = Arrays.asList(); + Predicate filter = i -> (i % 2) == 0; + Iterator iter = Iterables.filter(list, filter).iterator(); + + assertFalse(iter.hasNext()); + } + + // Full iterators. + @Test + public void testConcatFullIterators() { + List expectedList = new ArrayList<>(); + List> iterList = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + List list = new ArrayList<>(); + for (int j = 0; j < 10; j++) { + list.add(i * 10 + j); + expectedList.add(i * 10 + j); + } + iterList.add(list); + } + + verifyConcat(expectedList.iterator(), iterList); + } + + // Empty iterators. + @Test + public void testConcatEmptyIterators() { + List expectedList = new ArrayList<>(); + List> iterList = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + iterList.add(Collections.emptyList()); + } + + verifyConcat(expectedList.iterator(), iterList); + } + + // Mix full and empty iterators. + @Test + public void testConcatMixFullAndEmptyIterators() { + List expectedList = new ArrayList<>(); + List> iterList = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + List list = new ArrayList<>(); + for (int j = 0; j < 10; j++) { + list.add(i * 10 + j); + expectedList.add(i * 10 + j); + } + iterList.add(list); + iterList.add(Collections.emptyList()); + iterList.add(Collections.emptyList()); + } + + verifyConcat(expectedList.iterator(), iterList); + } + + // Invalid iterators. + @Test + public void testConcatNullMetaIterator() { + assertThrows(NullPointerException.class, () -> verifyConcat(Collections.emptyIterator(), null), + "Expect null verification error."); + } + + // Concat null iterators. + @Test + public void testConcatNullElementIterators() { + List> list = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + list.add(() -> null); + } + verifyConcat(Collections.emptyIterator(), list); + } + + private void verifyConcat(Iterator expectedValues, Iterable> metaIter) { + Iterator iter = Iterables.concat(metaIter).iterator(); + while (expectedValues.hasNext()) { + assertTrue(iter.hasNext()); + T v1 = expectedValues.next(); + T v2 = iter.next(); + assertEquals(v1, v2); + } + assertFalse(iter.hasNext()); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestLazyReload.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestLazyReload.java new file mode 100644 index 0000000000000..c4c0d9ffb4ddf --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestLazyReload.java @@ -0,0 +1,95 @@ +/* + * 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.tosfs.util; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Iterator; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestLazyReload { + @Test + public void testLoadWithFilterCondition() { + LazyReload integers = new LazyReload<>(() -> { + Iterator source = Arrays.asList(1, 3, 5, 2, 4, 6).iterator(); + return buf -> { + if (!source.hasNext()) { + return true; + } + + int pollCnt = 2; + while (source.hasNext() && pollCnt-- > 0) { + Integer item = source.next(); + if (item % 2 == 0) { + buf.add(item); + } + } + + return !source.hasNext(); + }; + }); + + Iterator iterator = integers.iterator(); + assertTrue(iterator.hasNext()); + assertEquals(2, (int) iterator.next()); + assertEquals(4, (int) iterator.next()); + assertEquals(6, (int) iterator.next()); + assertFalse(iterator.hasNext()); + } + + @Test + public void testLoadResultIsIdempotent() { + LazyReload integers = new LazyReload<>(() -> { + Iterator source = Arrays.asList(1, 3, 5, 2, 4, 6).iterator(); + return buf -> { + if (!source.hasNext()) { + return true; + } + + int pollCnt = 2; + while (source.hasNext() && pollCnt-- > 0) { + Integer item = source.next(); + buf.add(item); + } + + return !source.hasNext(); + }; + }); + Iterator iterator1 = integers.iterator(); + Iterator iterator2 = integers.iterator(); + + assertEquals(1, (int) iterator1.next()); + assertEquals(1, (int) iterator2.next()); + assertEquals(3, (int) iterator1.next()); + assertEquals(3, (int) iterator2.next()); + assertEquals(5, (int) iterator1.next()); + assertEquals(5, (int) iterator2.next()); + + assertEquals(2, (int) iterator1.next()); + assertEquals(4, (int) iterator1.next()); + assertEquals(6, (int) iterator1.next()); + assertEquals(2, (int) iterator2.next()); + assertEquals(4, (int) iterator2.next()); + assertEquals(6, (int) iterator2.next()); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestRange.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestRange.java new file mode 100644 index 0000000000000..08fcae1e20c7a --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestRange.java @@ -0,0 +1,87 @@ +/* + * 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.tosfs.util; + +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestRange { + + @Test + public void testInclude() { + Object[][] inputs = new Object[][]{ + new Object[]{Range.of(0, 0), 0L, false}, + new Object[]{Range.of(0, 1), 0L, true}, + new Object[]{Range.of(1, 1), 0L, false}, + new Object[]{Range.of(1, 1), 1L, true}, + new Object[]{Range.of(1, 1), 2L, false}, + new Object[]{Range.of(1, 99), 0L, false}, + new Object[]{Range.of(1, 99), 1L, true}, + new Object[]{Range.of(1, 99), 99L, true}, + new Object[]{Range.of(1, 99), 100L, false} + }; + + for (Object[] input : inputs) { + Range r = (Range) input[0]; + long pos = (long) input[1]; + boolean expected = (boolean) input[2]; + + assertEquals(expected, r.include(pos)); + } + } + + @Test + public void testOverlap() { + Object[][] inputs = new Object[][]{ + new Object[]{Range.of(0, 0), Range.of(0, 0), false}, + new Object[]{Range.of(0, 1), Range.of(0, 1), true}, + new Object[]{Range.of(0, 1), Range.of(1, 0), false}, + new Object[]{Range.of(0, 1), Range.of(1, 1), false}, + new Object[]{Range.of(0, 2), Range.of(1, 1), true}, + new Object[]{Range.of(0, 2), Range.of(0, 1), true}, + new Object[]{Range.of(0, 2), Range.of(1, 2), true}, + new Object[]{Range.of(0, 2), Range.of(2, 0), false}, + new Object[]{Range.of(0, 2), Range.of(2, 1), false}, + new Object[]{Range.of(5, 9), Range.of(0, 5), false}, + new Object[]{Range.of(5, 9), Range.of(0, 6), true} + }; + + for (Object[] input : inputs) { + Range l = (Range) input[0]; + Range r = (Range) input[1]; + boolean expect = (boolean) input[2]; + + assertEquals(expect, l.overlap(r)); + } + } + + @Test + public void testSplit() { + assertEquals(Range.split(10, 3), + ImmutableList.of(Range.of(0, 3), Range.of(3, 3), Range.of(6, 3), Range.of(9, 1))); + assertEquals(Range.split(10, 5), + ImmutableList.of(Range.of(0, 5), Range.of(5, 5))); + assertEquals(Range.split(10, 12), + ImmutableList.of(Range.of(0, 10))); + assertEquals(Range.split(2, 1), + ImmutableList.of(Range.of(0, 1), Range.of(1, 1))); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestUtility.java b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestUtility.java new file mode 100644 index 0000000000000..89b0493feff4b --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestUtility.java @@ -0,0 +1,168 @@ +/* + * 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.tosfs.util; + +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.tosfs.conf.ConfKeys; +import org.apache.hadoop.fs.tosfs.object.ObjectStorage; +import org.apache.hadoop.fs.tosfs.object.ObjectStorageFactory; +import org.apache.hadoop.fs.tosfs.object.tos.TOS; +import org.apache.hadoop.util.Lists; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.net.URI; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Random; + +public final class TestUtility { + private static final String ENV_TOS_BUCKET = "TOS_BUCKET"; + private static final String ENV_TEST_SCHEME = "TEST_SCHEME"; + private static final Random RND = new Random(System.currentTimeMillis()); + + private TestUtility() { + } + + public static byte[] rand(int size) { + byte[] buffer = new byte[size]; + RND.nextBytes(buffer); + return buffer; + } + + public static int randInt(int bound) { + return RND.nextInt(bound); + } + + public static String randomWithChinese() { + return RandomStringUtils.random(10, 0x4e00, 0x9fa5, false, false); + } + + public static String createUniquePath(String scheme) { + String bucket = bucket(); + if (bucket != null) { + return String.format("%s://%s/%s-%s/", scheme, bucket, scheme, UUIDUtils.random()); + } else { + throw new IllegalStateException("OS test bucket is not available"); + } + } + + public static String defaultFs() { + return String.format("%s://%s/", scheme(), bucket()); + } + + public static String scheme() { + return ParseUtils.envAsString(ENV_TEST_SCHEME, "tos"); + } + + public static String bucket() { + String bucket = ParseUtils.envAsString(ENV_TOS_BUCKET); + if (bucket != null) { + return bucket; + } + + // Parse from endpoint if it is formatted like http[s]://..xxx.com + String endpoint = ParseUtils.envAsString(TOS.ENV_TOS_ENDPOINT); + if (endpoint != null) { + for (String scheme : Lists.newArrayList("http://", "https://")) { + if (endpoint.startsWith(scheme)) { + endpoint = endpoint.substring(scheme.length()); + } + } + + String[] elements = endpoint.split("\\."); + if (elements.length == 4) { + return elements[0]; + } + } + throw new RuntimeException( + "Cannot decide the bucket name for object storage with scheme 'tos'"); + } + + public static String region() { + return TOSClientContextUtils.parseRegion(ParseUtils.envAsString(TOS.ENV_TOS_ENDPOINT)); + } + + public static String endpoint() { + return ParseUtils.envAsString(TOS.ENV_TOS_ENDPOINT); + } + + @SuppressWarnings("unchecked") + public static void setSystemEnv(String key, String value) { + try { + Map env = System.getenv(); + Class cl = env.getClass(); + Field field = cl.getDeclaredField("m"); + field.setAccessible(true); + Map writableEnv = (Map) field.get(env); + writableEnv.put(key, value); + } catch (Exception e) { + throw new IllegalStateException("Failed to set environment variable", e); + } + } + + @SuppressWarnings("unchecked") + public static void removeSystemEnv(String key) { + try { + Map env = System.getenv(); + Class cl = env.getClass(); + Field field = cl.getDeclaredField("m"); + field.setAccessible(true); + Map writableEnv = (Map) field.get(env); + writableEnv.remove(key); + } catch (Exception e) { + throw new IllegalStateException( + String.format("Failed to remove environment variable: %s", key), e); + } + } + + public static FileContext createTestFileContext(Configuration conf) throws IOException { + URI testURI = URI.create(defaultFs()); + return FileContext.getFileContext(testURI, conf); + } + + private static ObjectStorage generalBucketObjectStorage() { + Configuration conf = new Configuration(); + String endpoint = ParseUtils.envAsString(TOS.ENV_TOS_ENDPOINT, ""); + if (!StringUtils.isEmpty(endpoint)) { + conf.set(ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key(scheme()), endpoint); + } + + return ObjectStorageFactory.createWithPrefix( + String.format("%s-%s/", scheme(), UUIDUtils.random()), scheme(), bucket(), conf); + } + + public static List createTestObjectStorage(String fileStoreRoot) { + List storages = new ArrayList<>(); + + // 1. FileStore + Configuration fileStoreConf = new Configuration(); + fileStoreConf.set(ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key("filestore"), fileStoreRoot); + storages.add(ObjectStorageFactory.create("filestore", TestUtility.bucket(), fileStoreConf)); + + // 2. General Bucket + storages.add(generalBucketObjectStorage()); + + return storages; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/resources/contract/tos.xml b/hadoop-cloud-storage-project/hadoop-tos/src/test/resources/contract/tos.xml new file mode 100644 index 0000000000000..54f53bde1e11f --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/resources/contract/tos.xml @@ -0,0 +1,113 @@ + + + fs.contract.test.fs.tos + tos://{your_bucket}/ + + + + fs.contract.test.random-seek-count + 10 + + + + fs.contract.is-blobstore + true + + + + fs.contract.create-visibility-delayed + true + + + + fs.contract.is-case-sensitive + true + + + + fs.contract.rename-returns-false-if-source-missing + true + + + + fs.contract.rename-returns-false-if-dest-exists + true + + + + fs.contract.supports-append + false + + + + fs.contract.supports-atomic-directory-delete + false + + + + fs.contract.supports-atomic-rename + false + + + + fs.contract.supports-block-locality + false + + + + fs.contract.supports-concat + false + + + + fs.contract.supports-unbuffer + true + + + + fs.contract.rename-creates-dest-dirs + true + + + + + fs.contract.test.root-tests-enabled + false + + + + fs.contract.supports-getfilestatus + true + + + + fs.contract.supports-seek + true + + + + fs.contract.supports-seek-on-closed-file + true + + + + fs.contract.rejects-seek-past-eof + true + + + + fs.contract.supports-strict-exceptions + true + + + + fs.contract.supports-multipartuploader + true + + + + fs.contract.supports-unix-permissions + false + + + diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/resources/core-site.xml b/hadoop-cloud-storage-project/hadoop-tos/src/test/resources/core-site.xml new file mode 100644 index 0000000000000..c4829ba490767 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/resources/core-site.xml @@ -0,0 +1,59 @@ + + + + + + + hadoop.tmp.dir + target/build/test + A base for other temporary directories. + true + + + + + hadoop.security.authentication + simple + + + + fs.tos.impl + org.apache.hadoop.fs.tosfs.TosFileSystem + + + fs.AbstractFileSystem.tos.impl + org.apache.hadoop.fs.tosfs.TosFS + + + fs.tos.impl.disable.cache + true + + + fs.filestore.impl + org.apache.hadoop.fs.tosfs.RawFileSystem + + + fs.AbstractFileSystem.filestore.impl + org.apache.hadoop.fs.tosfs.RawFS + + + fs.filestore.impl.disable.cache + true + + diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/test/resources/log4j.properties b/hadoop-cloud-storage-project/hadoop-tos/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..c671ccce5120c --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-tos/src/test/resources/log4j.properties @@ -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. +# +# log4j configuration used during build and unit tests + +log4j.rootLogger=info,stdout +log4j.threshold=ALL +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n diff --git a/hadoop-cloud-storage-project/pom.xml b/hadoop-cloud-storage-project/pom.xml index 5746e01896b81..036dfa3a699a9 100644 --- a/hadoop-cloud-storage-project/pom.xml +++ b/hadoop-cloud-storage-project/pom.xml @@ -33,6 +33,7 @@ hadoop-cloud-storage hadoop-cos hadoop-huaweicloud + hadoop-tos diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 1c046529a0efb..463c05a78122b 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -777,6 +777,12 @@ ${hadoop.version} + + org.apache.hadoop + hadoop-tos + ${hadoop.version} + + org.apache.hadoop hadoop-kms diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml index 43c505e11cb68..6c1f9b0e38fea 100644 --- a/hadoop-project/src/site/site.xml +++ b/hadoop-project/src/site/site.xml @@ -180,6 +180,7 @@ href="hadoop-azure-datalake/index.html"/> +