Browse Source

HADOOP-19236. Incorporate VolcanoEngine Cloud TOS File System Implementation. (#7504)

Contributed by: ZhengHu, SunXin, XianyinXin, Rascal Wu, FangBo, Yuanzhihuan.

Signed-off-by: jinglun <jinglun@apache.org>
Jinglun 1 month ago
parent
commit
5c10e0d458
100 changed files with 13246 additions and 0 deletions
  1. 5 0
      hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml
  2. 89 0
      hadoop-cloud-storage-project/hadoop-tos/dev-support/findbugs-exclude.xml
  3. 183 0
      hadoop-cloud-storage-project/hadoop-tos/pom.xml
  4. 47 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFS.java
  5. 67 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFSUtils.java
  6. 74 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFileStatus.java
  7. 770 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFileSystem.java
  8. 30 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawLocatedFileStatus.java
  9. 67 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/TosChecksum.java
  10. 47 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/TosFS.java
  11. 43 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/TosFileSystem.java
  12. 45 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/CommitContext.java
  13. 370 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/CommitUtils.java
  14. 510 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/Committer.java
  15. 33 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/CommitterFactory.java
  16. 151 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/MagicOutputStream.java
  17. 179 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/Pending.java
  18. 123 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/PendingSet.java
  19. 233 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/SuccessData.java
  20. 189 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/mapred/Committer.java
  21. 26 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/mapred/package-info.java
  22. 43 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/PendingOps.java
  23. 40 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/PendingOpsFactory.java
  24. 57 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/RawPendingOps.java
  25. 26 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/ops/package-info.java
  26. 26 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/commit/package-info.java
  27. 103 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/Bytes.java
  28. 148 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/Chain.java
  29. 592 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/Tasks.java
  30. 140 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/ThreadPools.java
  31. 26 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/common/package-info.java
  32. 32 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/ArgumentKey.java
  33. 156 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/ConfKeys.java
  34. 41 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/FileStoreKeys.java
  35. 276 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/TosKeys.java
  36. 26 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/package-info.java
  37. 91 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/BucketInfo.java
  38. 37 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ChecksumInfo.java
  39. 41 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ChecksumType.java
  40. 32 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/Constants.java
  41. 50 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/DirectoryStorage.java
  42. 648 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/FileStore.java
  43. 34 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/InputStreamProvider.java
  44. 102 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/MultipartUpload.java
  45. 27 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectConstants.java
  46. 52 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectContent.java
  47. 117 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectInfo.java
  48. 233 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectMultiRangeInputStream.java
  49. 343 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectOutputStream.java
  50. 199 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectRangeInputStream.java
  51. 372 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorage.java
  52. 73 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorageFactory.java
  53. 92 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectUtils.java
  54. 78 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/Part.java
  55. 254 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/PrefixStorage.java
  56. 34 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/ChecksumMismatchException.java
  57. 31 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/InvalidObjectKeyException.java
  58. 25 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/NotAppendableException.java
  59. 26 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/package-info.java
  60. 26 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/package-info.java
  61. 85 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/ListObjectsRequest.java
  62. 26 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/package-info.java
  63. 43 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/ListObjectsResponse.java
  64. 26 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/package-info.java
  65. 177 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/FileStagingPart.java
  66. 79 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/StagingPart.java
  67. 23 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/State.java
  68. 26 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/package-info.java
  69. 137 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/ChainTOSInputStream.java
  70. 1255 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClient.java
  71. 185 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClientBuilder.java
  72. 60 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/GetObjectOutput.java
  73. 1038 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOS.java
  74. 53 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOSErrorCodes.java
  75. 121 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOSInputStream.java
  76. 119 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOSUtils.java
  77. 78 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TosObjectInfo.java
  78. 79 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/AbstractCredentialsProvider.java
  79. 35 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/CredentialsProvider.java
  80. 101 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/DefaultCredentialsProviderChain.java
  81. 34 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/EnvironmentCredentialsProvider.java
  82. 67 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/ExpireableCredential.java
  83. 58 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/SimpleCredentialsProvider.java
  84. 26 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/auth/package-info.java
  85. 26 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/package-info.java
  86. 189 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/DefaultFsOps.java
  87. 109 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/DirectoryFsOps.java
  88. 91 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/FsOps.java
  89. 230 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/RenameOp.java
  90. 26 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/package-info.java
  91. 26 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/package-info.java
  92. 46 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/CommonUtils.java
  93. 67 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/FSUtils.java
  94. 28 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/FuseUtils.java
  95. 128 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Iterables.java
  96. 46 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/JsonCodec.java
  97. 35 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/LazyReload.java
  98. 69 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/LazyReloadIter.java
  99. 65 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/ParseUtils.java
  100. 104 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/util/Range.java

+ 5 - 0
hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml

@@ -125,5 +125,10 @@
       <artifactId>hadoop-huaweicloud</artifactId>
       <scope>compile</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-tos</artifactId>
+      <scope>compile</scope>
+    </dependency>
   </dependencies>
 </project>

+ 89 - 0
hadoop-cloud-storage-project/hadoop-tos/dev-support/findbugs-exclude.xml

@@ -0,0 +1,89 @@
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+<FindBugsFilter>
+  <Match>
+    <Class name="org.apache.hadoop.fs.tosfs.object.ObjectOutputStream"/>
+    <or>
+      <Bug pattern="IS2_INCONSISTENT_SYNC" />
+      <Bug pattern="NP_NULL_ON_SOME_PATH" />
+    </or>
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.fs.tosfs.conf.TosKeys"/>
+    <Bug pattern="MS_MUTABLE_ARRAY" />
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.fs.tosfs.object.Constants"/>
+    <Bug pattern="MS_MUTABLE_ARRAY" />
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.fs.tosfs.RawFileStatus"/>
+    <or>
+      <Bug pattern="EI_EXPOSE_REP" />
+      <Bug pattern="EI_EXPOSE_REP2" />
+    </or>
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.fs.tosfs.TosChecksum"/>
+    <or>
+      <Bug pattern="EI_EXPOSE_REP" />
+      <Bug pattern="EI_EXPOSE_REP2" />
+    </or>
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.fs.tosfs.object.ObjectContent"/>
+    <or>
+      <Bug pattern="EI_EXPOSE_REP" />
+      <Bug pattern="EI_EXPOSE_REP2" />
+    </or>
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.fs.tosfs.object.ObjectInfo"/>
+    <or>
+      <Bug pattern="EI_EXPOSE_REP" />
+      <Bug pattern="EI_EXPOSE_REP2" />
+    </or>
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.fs.tosfs.object.ObjectMultiRangeInputStream"/>
+    <or>
+      <Bug pattern="EI_EXPOSE_REP" />
+      <Bug pattern="EI_EXPOSE_REP2" />
+    </or>
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.fs.tosfs.object.ObjectRangeInputStream"/>
+    <or>
+      <Bug pattern="EI_EXPOSE_REP" />
+      <Bug pattern="EI_EXPOSE_REP2" />
+    </or>
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.fs.tosfs.object.tos.GetObjectOutput"/>
+    <or>
+      <Bug pattern="EI_EXPOSE_REP" />
+      <Bug pattern="EI_EXPOSE_REP2" />
+    </or>
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.fs.tosfs.object.tos.ChainTOSInputStream"/>
+    <Bug pattern="EI_EXPOSE_REP" />
+  </Match>
+  <Match>
+    <Bug pattern="ICAST_QUESTIONABLE_UNSIGNED_RIGHT_SHIFT" />
+  </Match>
+</FindBugsFilter>

+ 183 - 0
hadoop-cloud-storage-project/hadoop-tos/pom.xml

@@ -0,0 +1,183 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>3.5.0-SNAPSHOT</version>
+    <relativePath>../../hadoop-project</relativePath>
+  </parent>
+
+  <artifactId>hadoop-tos</artifactId>
+  <version>3.5.0-SNAPSHOT</version>
+  <name>Apache Hadoop Volcano Engine Services support</name>
+  <description>
+    This module contains code to support integration with Volcano Engine TOS.
+    It also declares the dependencies needed to work with Volcano Engine services.
+  </description>
+  <packaging>jar</packaging>
+
+  <properties>
+    <file.encoding>UTF-8</file.encoding>
+    <ve-tos-java-sdk.version>2.8.9</ve-tos-java-sdk.version>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.volcengine</groupId>
+      <artifactId>ve-tos-java-sdk-hadoop</artifactId>
+      <version>${ve-tos-java-sdk.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.jetbrains</groupId>
+          <artifactId>annotations</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-annotations</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-databind</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-core</artifactId>
+        </exclusion>
+      </exclusions>
+
+    </dependency>
+
+    <!-- Test dependencies -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    <!-- Artifacts needed to bring up a Mini MR Yarn cluster-->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-examples</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-distcp</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-distcp</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+
+    <dependency>
+      <groupId>org.assertj</groupId>
+      <artifactId>assertj-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <version>4.11.0</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <parallel>classes</parallel>
+          <threadCount>1</threadCount>
+          <perCoreThreadCount>true</perCoreThreadCount>
+          <rerunFailingTestsCount>2</rerunFailingTestsCount>
+          <forkCount>8</forkCount>
+          <argLine>-Xmx2048m</argLine>
+        </configuration>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <createDependencyReducedPom>true</createDependencyReducedPom>
+              <filters>
+                <filter>
+                  <artifact>*:*</artifact>
+                  <excludes>
+                    <exclude>META-INF/*.SF</exclude>
+                    <exclude>META-INF/*.DSA</exclude>
+                    <exclude>META-INF/*.RSA</exclude>
+                  </excludes>
+                </filter>
+              </filters>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
+        <groupId>com.github.spotbugs</groupId>
+        <artifactId>spotbugs-maven-plugin</artifactId>
+        <configuration>
+          <xmlOutput>true</xmlOutput>
+          <excludeFilterFile>${basedir}/dev-support/findbugs-exclude.xml
+          </excludeFilterFile>
+          <effort>Max</effort>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+</project>

+ 47 - 0
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.
+ * <pre>
+ *  fs.AbstractFileSystem.{scheme}.impl=io.proton.fs.RawFS.
+ * </pre>
+ */
+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;
+  }
+}

+ 67 - 0
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;
+  }
+}

+ 74 - 0
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);
+  }
+}

+ 770 - 0
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<LocatedFileStatus> 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<LocatedFileStatus> 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<CreateFlag> 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.
+   *
+   * <ul>
+   *   <li>Return false if src doesn't exist</li>
+   *   <li>Return false if src is root</li>
+   *   <li>Return false if dst path is under src path, e.g. rename('/a/b', '/a/b/c')</li>
+   *   <li>Return false if dst path already exists</li>
+   *   <li>Return true if rename('/a/b', '/a/b') and 'b' is an existed file</li>
+   *   <li>Return true if rename('/a/b', '/a') and 'a' is an existed dir,
+   *   fs will rename '/a/b' to '/a/b' internally</li>
+   *   <li>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.</li>
+   * </ul>
+   *
+   * @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<FileStatus> srcStatusFuture = taskThreadPool.submit(() -> checkAndGetSrcStatus(src));
+    Future<Path> 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<RawFileStatus> listStatus(Path f, boolean recursive) throws IOException {
+    Path path = makeQualified(f);
+    // Assuming path is a dir at first.
+    Iterator<RawFileStatus> 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<FileStatus> 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<FileStatus>() {
+      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<XAttrSetFlag> 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<String, String> 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<String, byte[]> 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<String, String> 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<String, byte[]> 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<String, byte[]> getXAttrs(Path path, List<String> names) throws IOException {
+    Map<String, byte[]> xAttrs = getXAttrs(path);
+    xAttrs.keySet().retainAll(names);
+    if (xAttrs.size() == names.size()) {
+      return xAttrs;
+    } else {
+      List<String> badNames = names.stream().filter(n -> !xAttrs.containsKey(n)).collect(
+          Collectors.toList());
+      throw new IOException("Attributes with name " + badNames + " are not found.");
+    }
+  }
+
+  @Override
+  public List<String> 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<String, String> existedTags = storage.getTags(key);
+      if (existedTags.remove(name) != null) {
+        storage.putTags(key, existedTags);
+      }
+    }
+  }
+
+  private void validateXAttrFlag(String xAttrName, boolean xAttrExists, EnumSet<XAttrSetFlag> 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.");
+      }
+    }
+  }
+}

+ 30 - 0
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);
+  }
+}

+ 67 - 0
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);
+  }
+}

+ 47 - 0
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;
+  }
+}

+ 43 - 0
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);
+  }
+}

+ 45 - 0
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<FileStatus> pendingSets;
+  // It will be accessed in multi-threads, please access it in a thread-safe context.
+  private final List<String> destKeys;
+
+  public CommitContext(List<FileStatus> pendingSets) {
+    this.pendingSets = pendingSets;
+    this.destKeys = Lists.newArrayList();
+  }
+
+  public List<FileStatus> pendingSets() {
+    return pendingSets;
+  }
+
+  public synchronized void addDestKey(String destKey) {
+    destKeys.add(destKey);
+  }
+
+  public synchronized List<String> destKeys() {
+    return destKeys;
+  }
+}

+ 370 - 0
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. <br>
+   * 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. <br>
+   * 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. <br>
+   * 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. <br>
+   * 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<FileStatus> listPendingFiles(FileSystem fs, Path dir) throws IOException {
+    List<FileStatus> 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<LocatedFileStatus> 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<String> 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<String> 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 <T extends Exception> void triggerError(Supplier<T> error, String stage) throws T {
+    if (testMode && errorStage.contains(stage)) {
+      throw error.get();
+    }
+  }
+}

+ 510 - 0
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<FileStatus> 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<String> 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<FileStatus> pendingFiles = CommitUtils.listPendingFiles(destFs, taskAttemptBasePath);
+      // Use the thread-safe collection to collect the pending list.
+      List<Pending> 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<FileStatus> 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<MultipartUpload> 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();
+  }
+}

+ 33 - 0
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);
+  }
+}

+ 151 - 0
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<String> splits = Lists.newArrayList(magicKey.split("/"));
+
+    // Break the full splits list into three collections: <parentSplits>, __magic, <childrenSplits>
+    int magicIndex = splits.indexOf(CommitUtils.MAGIC);
+    Preconditions.checkArgument(magicIndex >= 0, "Cannot locate %s in path %s", CommitUtils.MAGIC,
+        magicPath);
+    List<String> parentSplits = splits.subList(0, magicIndex);
+    List<String> 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<String> destSplits = Lists.newArrayList(parentSplits);
+    if (childrenSplits.contains(CommitUtils.BASE)) {
+      // Break the <childrenDir> into three collections: <baseParentSplits>, __base,
+      // <baseChildrenSplits>, and add all <baseChildrenSplits> into the destination splits.
+      int baseIndex = childrenSplits.indexOf(CommitUtils.BASE);
+      Preconditions.checkArgument(baseIndex >= 0, "Cannot locate %s in path %s", CommitUtils.BASE,
+          magicPath);
+      List<String> 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 <childrenSplits> 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<Part> 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<String> splits = Arrays.stream(path.split("/"))
+        .filter(StringUtils::isNoneEmpty)
+        .collect(Collectors.toList());
+    return splits.contains(CommitUtils.MAGIC) && !isInternalFile(p);
+  }
+}

+ 179 - 0
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<Pending> CODEC = new JsonCodec<>(Pending.class);
+
+  private String bucket;
+  private String destKey;
+  private String uploadId;
+  private long length;
+  private long createdTimestamp;
+  private List<Part> 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<Part> 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<Part> 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<Part> 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<Part> 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);
+    }
+  }
+}

+ 123 - 0
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<PendingSet> CODEC = new JsonCodec<>(PendingSet.class);
+
+  private String jobId;
+  private List<Pending> pendings;
+  private Map<String, String> 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<Pending> pendings) {
+    this.jobId = jobId;
+    this.pendings = Lists.newArrayList(pendings);
+    this.extraData = Maps.newHashMap();
+  }
+
+  public PendingSet addAll(Iterable<Pending> 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<Pending> commits() {
+    return pendings;
+  }
+
+  public Map<String, String> 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);
+  }
+}

+ 233 - 0
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<SuccessData> 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<String> filenames = new ArrayList<>();
+
+  // Diagnostics information.
+  private final Map<String, String> 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<String, String> diagnostics() {
+    return diagnostics;
+  }
+
+  public List<String> 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<String> 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<String> newFileNamesInput) {
+      if (newFileNamesInput != null) {
+        Iterables.addAll(this.filenames, newFileNamesInput);
+      }
+      return this;
+    }
+
+    public SuccessData build() {
+      return new SuccessData(this);
+    }
+  }
+}

+ 189 - 0
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;
+  }
+}

+ 26 - 0
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;

+ 43 - 0
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);
+}

+ 40 - 0
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);
+    }
+  }
+}

+ 57 - 0
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());
+  }
+}

+ 26 - 0
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;

+ 26 - 0
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;

+ 103 - 0
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);
+  }
+}

+ 148 - 0
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<T extends Closeable> implements Closeable {
+  private static final Logger LOG = LoggerFactory.getLogger(Chain.class);
+
+  private final List<IOException> suppressed = Lists.newArrayList();
+  private final Queue<ItemFactory<T>> queue;
+  private final Predicate<IOException> shouldContinue;
+  private T curItem;
+
+  private Chain(Deque<ItemFactory<T>> queue, Predicate<IOException> shouldContinue) {
+    this.queue = queue;
+    this.shouldContinue = shouldContinue;
+    this.curItem = null;
+  }
+
+  public <R> R run(Task<T, R> 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<T> 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 extends Closeable> {
+    T newItem() throws IOException;
+  }
+
+  public interface Task<T extends Closeable, R> {
+    R run(T call) throws IOException;
+  }
+
+  public static class Builder<T extends Closeable> {
+    private final Deque<ItemFactory<T>> factories = Queues.newArrayDeque();
+    private Predicate<IOException> shouldContinue;
+
+    public Builder<T> addFirst(ItemFactory<T> factory) {
+      factories.addFirst(factory);
+      return this;
+    }
+
+    public Builder<T> addLast(ItemFactory<T> factory) {
+      factories.addLast(factory);
+      return this;
+    }
+
+    public Builder<T> shouldContinue(Predicate<IOException> continueCondition) {
+      this.shouldContinue = continueCondition;
+      return this;
+    }
+
+    public Chain<T> build() throws IOException {
+      Chain<T> chain = new Chain<>(factories, shouldContinue);
+
+      // Do nothing in the chain task to initialize the first item.
+      chain.run(item -> null);
+      return chain;
+    }
+  }
+
+  public static <T extends Closeable> Builder<T> builder() {
+    return new Builder<>();
+  }
+}

+ 592 - 0
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<I, E extends Exception> {
+    void run(I item, Exception exception) throws E;
+  }
+
+  public interface Task<I, E extends Exception> {
+    void run(I item) throws E;
+  }
+
+  public static class Builder<I> {
+    private final Iterable<I> items;
+    private ExecutorService service = null;
+    private FailureTask<I, ?> onFailure = null;
+    private boolean stopOnFailure = false;
+    private boolean throwFailureWhenFinished = true;
+    private Task<I, ?> revertTask = null;
+    private boolean stopRevertsOnFailure = false;
+    private Task<I, ?> abortTask = null;
+    private boolean stopAbortsOnFailure = false;
+
+    // retry settings
+    private List<Class<? extends Exception>> stopRetryExceptions =
+        Lists.newArrayList(UnrecoverableException.class);
+    private List<Class<? extends Exception>> onlyRetryExceptions = null;
+    private Predicate<Exception> 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<I> items) {
+      this.items = items;
+    }
+
+    public Builder<I> executeWith(ExecutorService svc) {
+      this.service = svc;
+      return this;
+    }
+
+    public Builder<I> onFailure(FailureTask<I, ?> task) {
+      this.onFailure = task;
+      return this;
+    }
+
+    public Builder<I> stopOnFailure() {
+      this.stopOnFailure = true;
+      return this;
+    }
+
+    public Builder<I> throwFailureWhenFinished() {
+      this.throwFailureWhenFinished = true;
+      return this;
+    }
+
+    public Builder<I> throwFailureWhenFinished(boolean throwWhenFinished) {
+      this.throwFailureWhenFinished = throwWhenFinished;
+      return this;
+    }
+
+    public Builder<I> suppressFailureWhenFinished() {
+      this.throwFailureWhenFinished = false;
+      return this;
+    }
+
+    public Builder<I> revertWith(Task<I, ?> task) {
+      this.revertTask = task;
+      return this;
+    }
+
+    public Builder<I> stopRevertsOnFailure() {
+      this.stopRevertsOnFailure = true;
+      return this;
+    }
+
+    public Builder<I> abortWith(Task<I, ?> task) {
+      this.abortTask = task;
+      return this;
+    }
+
+    public Builder<I> stopAbortsOnFailure() {
+      this.stopAbortsOnFailure = true;
+      return this;
+    }
+
+    @SafeVarargs public final Builder<I> stopRetryOn(Class<? extends Exception>... exceptions) {
+      stopRetryExceptions.addAll(Arrays.asList(exceptions));
+      return this;
+    }
+
+    public Builder<I> shouldRetryTest(Predicate<Exception> shouldRetry) {
+      this.shouldRetryPredicate = shouldRetry;
+      return this;
+    }
+
+    public Builder<I> noRetry() {
+      this.maxAttempts = 1;
+      return this;
+    }
+
+    public Builder<I> retry(int nTimes) {
+      this.maxAttempts = nTimes + 1;
+      return this;
+    }
+
+    public Builder<I> onlyRetryOn(Class<? extends Exception> exception) {
+      this.onlyRetryExceptions = Collections.singletonList(exception);
+      return this;
+    }
+
+    @SafeVarargs public final Builder<I> onlyRetryOn(Class<? extends Exception>... exceptions) {
+      this.onlyRetryExceptions = Lists.newArrayList(exceptions);
+      return this;
+    }
+
+    public Builder<I> 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<I, RuntimeException> task) {
+      return run(task, RuntimeException.class);
+    }
+
+    public <E extends Exception> boolean run(Task<I, E> task, Class<E> exceptionClass) throws E {
+      if (service != null) {
+        return runParallel(task, exceptionClass);
+      } else {
+        return runSingleThreaded(task, exceptionClass);
+      }
+    }
+
+    private <E extends Exception> boolean runSingleThreaded(
+        Task<I, E> task, Class<E> exceptionClass) throws E {
+      List<I> succeeded = Lists.newArrayList();
+      List<Throwable> exceptions = Lists.newArrayList();
+
+      Iterator<I> 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 <E extends Exception> boolean runParallel(
+        final Task<I, E> task, Class<E> exceptionClass) throws E {
+      final Queue<I> succeeded = new ConcurrentLinkedQueue<>();
+      final Queue<Throwable> exceptions = new ConcurrentLinkedQueue<>();
+      final AtomicBoolean taskFailed = new AtomicBoolean(false);
+      final AtomicBoolean abortFailed = new AtomicBoolean(false);
+      final AtomicBoolean revertFailed = new AtomicBoolean(false);
+
+      List<Future<?>> 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 <E extends Exception> void runTaskWithRetry(
+        Task<I, E> 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<? extends Exception> 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<? extends Exception> 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<Throwable> waitFor(
+      Collection<Future<?>> futures) {
+    while (true) {
+      int numFinished = 0;
+      for (Future<?> future : futures) {
+        if (future.isDone()) {
+          numFinished += 1;
+        }
+      }
+
+      if (numFinished == futures.size()) {
+        List<Throwable> 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<Integer> {
+    private final int size;
+
+    Range(int size) {
+      this.size = size;
+    }
+
+    @Override
+    public Iterator<Integer> iterator() {
+      return new Iterator<Integer>() {
+        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<Integer> range(int upTo) {
+    return new Builder<>(new Range(upTo));
+  }
+
+  public static <I> Builder<I> foreach(Iterator<I> items) {
+    return new Builder<>(() -> items);
+  }
+
+  public static <I> Builder<I> foreach(Iterable<I> items) {
+    return new Builder<>(items);
+  }
+
+  @SafeVarargs public static <I> Builder<I> foreach(I... items) {
+    return new Builder<>(Arrays.asList(items));
+  }
+
+  public static <I> Builder<I> foreach(Stream<I> items) {
+    return new Builder<>(items::iterator);
+  }
+
+  private static <E extends Exception> void throwOne(Collection<Throwable> exceptions,
+      Class<E> allowedException) throws E {
+    Iterator<Throwable> iter = exceptions.iterator();
+    Throwable exception = iter.next();
+    Class<? extends Throwable> exceptionClass = exception.getClass();
+
+    while (iter.hasNext()) {
+      Throwable other = iter.next();
+      if (!exceptionClass.isInstance(other)) {
+        exception.addSuppressed(other);
+      }
+    }
+
+    castAndThrow(exception, allowedException);
+  }
+
+  public static <E extends Exception> void castAndThrow(
+      Throwable exception, Class<E> 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);
+  }
+}

+ 140 - 0
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.
+   * <p>
+   * 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;
+  }
+}

+ 26 - 0
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;

+ 32 - 0
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);
+  }
+}

+ 156 - 0
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);
+    }
+  }
+}

+ 41 - 0
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();
+}

+ 276 - 0
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();
+}

+ 26 - 0
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;

+ 91 - 0
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.
+ * <p>
+ * 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.
+ * <p>
+ * And also there are some general difference between general purpose bucket and directory bucket.
+ * <ul>
+ *   <li>Directory bucket treats the object end with '/' as the directory during creating object.
+ *   </li>
+ *   <li>TOS directory bucket will create missed parent dir during create object automatically,
+ *   but general purpose bucket only create one object.</li>
+ *   <li>Directory bucket doesn't allow create any object under a file, but general purpose bucket
+ *   haven't this constraint.</li>
+ *   <li>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</li>
+ *   <li>TOS directory bucket provides atomic rename/delete dir abilities</li>
+ * </ul>
+ */
+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();
+  }
+}

+ 37 - 0
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;
+  }
+}

+ 41 - 0
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;
+  }
+}

+ 32 - 0
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 = "/";
+}

+ 50 - 0
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<ObjectInfo> 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);
+}

+ 648 - 0
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<String> batchDelete(List<String> keys) {
+    Preconditions.checkArgument(keys.size() <= MAX_DELETE_OBJECTS_COUNT,
+        "The batch delete object count should <= %s", MAX_DELETE_OBJECTS_COUNT);
+    List<String> 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<ObjectInfo> 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<ListObjectsResponse> list(ListObjectsRequest request) {
+    try (Stream<Path> stream = Files.walk(Paths.get(root))) {
+      List<ObjectInfo> 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<ObjectInfo> objects) {
+    int retSize = Math.min(limit, objects.size());
+
+    if (Strings.isNullOrEmpty(delimiter)) {
+      // the response only contains objects
+      List<ObjectInfo> retObjs = objects.subList(0, retSize);
+      return new ListObjectsResponse(retObjs, Collections.emptyList());
+    } else {
+      // the response only contains objects and common prefixes
+      Set<String> commonPrefixes = new TreeSet<>();
+      List<ObjectInfo> 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<Part> 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<Integer> 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<Integer> listPartNums(File uploadDir) {
+    try (Stream<Path> 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<MultipartUpload> listUploads(String prefix) {
+    Path stagingDir = Paths.get(root, STAGING_DIR);
+    if (!Files.exists(stagingDir)) {
+      return Collections.emptyList();
+    }
+    try (Stream<Path> 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<ObjectInfo> 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<ObjectInfo> list(Stream<Path> 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 {
+  }
+}

+ 34 - 0
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.
+ * <p>
+ * 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();
+}

+ 102 - 0
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<MultipartUpload> {
+  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);
+    }
+  }
+}

+ 27 - 0
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() {
+  }
+}

+ 52 - 0
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;
+  }
+}

+ 117 - 0
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);
+  }
+}

+ 233 - 0
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;
+  }
+}

+ 343 - 0
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<File> stagingDirs;
+  private final List<StagingPart> stagingParts = Lists.newArrayList();
+
+  // For multipart uploads.
+  private final AtomicInteger partNumGetter = new AtomicInteger(0);
+  private MultipartUpload multipartUpload = null;
+  private final List<CompletableFuture<Part>> 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<File> 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<File> 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<Part> 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<Part> result =
+                asyncUploadPart(stagingPart, partNumGetter.incrementAndGet());
+            results.add(result);
+          }
+        }
+      }
+    }
+  }
+
+  private CompletableFuture<Part> 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<Part> 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<Part> 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<Part> 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<Part> result = asyncUploadEmptyPart(partNumGetter.incrementAndGet());
+        results.add(result);
+      } else if (curPart != null) {
+        curPart.complete();
+        // Submit the last part to upload thread pool.
+        CompletableFuture<Part> 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<StagingPart> stagingParts() {
+    return stagingParts;
+  }
+
+  public String destKey() {
+    return destKey;
+  }
+
+  public MultipartUpload upload() {
+    return multipartUpload;
+  }
+
+  private void deleteStagingPart(List<StagingPart> 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;
+  }
+}

+ 199 - 0
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;
+  }
+}

+ 372 - 0
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.
+   * <p>
+   * The first one wins if there are concurrent appends.
+   * <p>
+   * 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<String> batchDelete(List<String> 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:
+   * <ul>
+   *   <li>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</li>
+   *   <li>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/'</li>
+   * </ul>
+   *
+   * @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<ListObjectsResponse> 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<ObjectInfo> 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<ListObjectsResponse> 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<ObjectInfo> 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<Part> 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<MultipartUpload> 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<String, String> 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<String, String> 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.
+   * <p>
+   * There are three kinds of implementations:
+   * <ul>
+   *   <li>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</li>
+   *   <li>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.</li>
+   *   <li>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.</li>
+   * </ul>
+   *
+   * @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();
+}

+ 73 - 0
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);
+    }
+  }
+}

+ 92 - 0
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<ObjectInfo> objects,
+      int batchSize) {
+    List<String> 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<String> keys) {
+    List<String> 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);
+  }
+}

+ 78 - 0
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();
+  }
+}

+ 254 - 0
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<String> batchDelete(List<String> 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<ObjectInfo> objects = response.objects().stream()
+        .map(this::removePrefix)
+        .collect(Collectors.toList());
+    List<String> commonPrefixKeys = response.commonPrefixes().stream()
+        .map(this::removePrefix)
+        .collect(Collectors.toList());
+    return new ListObjectsResponse(objects, commonPrefixKeys);
+  }
+
+  @Override
+  public Iterable<ListObjectsResponse> 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<Part> 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<MultipartUpload> 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<String, String> newTags) {
+    storage.putTags(prefix + key, newTags);
+  }
+
+  @Override
+  public Map<String, String> 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<ObjectInfo> 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);
+  }
+}

+ 34 - 0
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)));
+  }
+}

+ 31 - 0
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);
+  }
+}

+ 25 - 0
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);
+  }
+}

+ 26 - 0
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;

+ 26 - 0
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;

+ 85 - 0
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);
+    }
+  }
+}

+ 26 - 0
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;

+ 43 - 0
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<ObjectInfo> objects;
+  private final List<String> commonPrefixes;
+
+  public ListObjectsResponse(
+      List<ObjectInfo> objects,
+      List<String> commonPrefixes) {
+    this.objects = objects;
+    this.commonPrefixes = commonPrefixes;
+  }
+
+  public List<ObjectInfo> objects() {
+    return objects;
+  }
+
+  public List<String> commonPrefixes() {
+    return commonPrefixes;
+  }
+}

+ 26 - 0
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;

+ 177 - 0
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;
+    }
+  }
+}

+ 79 - 0
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();
+}

+ 23 - 0
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
+}

+ 26 - 0
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;

+ 137 - 0
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<TOSInputStream> 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<TOSInputStream> createChain() {
+    Chain.Builder<TOSInputStream> builder = Chain.<TOSInputStream>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;
+  }
+}

+ 1255 - 0
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<String> nonRetryable409ErrorCodes;
+
+  protected DelegationClient(TOSClientConfiguration configuration, int maxRetryTimes,
+      List<String> 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> T retry(Callable<T> 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<String> 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<String> nonRetryable409ErrorCodes) {
+    return e instanceof TosServerException
+        && e.getStatusCode() == HttpStatus.CONFLICT
+        && isRetryableTosConflictException((TosServerException) e, nonRetryable409ErrorCodes);
+  }
+
+  private static boolean isRetryableTosConflictException(TosServerException e,
+      List<String> nonRetryableCodes) {
+    String errorCode = e.getEc();
+    return StringUtils.isEmpty(errorCode) || !nonRetryableCodes.contains(errorCode);
+  }
+}

+ 185 - 0
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<String, DelegationClient> 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<String> 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<String> nonRetryable409ErrorCodes) {
+      super(configuration, maxRetryTimes, nonRetryable409ErrorCodes);
+    }
+
+    @Override
+    public void close() {
+      // do nothing as this client may be shared by multiple upper-layer instances
+    }
+  }
+}

+ 60 - 0
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();
+  }
+}

+ 1038 - 0
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<String> 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<ObjectInfo> 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<ObjectInfo> bfsListDir(String key) {
+    return new LazyReload<>(() -> {
+      final Deque<String> dirQueue = new LinkedList<>();
+      AtomicReference<String> continueToken = new AtomicReference<>("");
+      AtomicReference<String> 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<ObjectInfo> innerListDir(String key, String startAfter, int limit) {
+    return new LazyReload<>(() -> {
+      AtomicReference<String> 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<String> batchDelete(List<String> keys) {
+    checkAvailableClient();
+    int totalKeyCnt = keys.size();
+
+    Preconditions.checkArgument(totalKeyCnt <= maxDeleteObjectsCount,
+        "The batch delete object count should <= %s", maxDeleteObjectsCount);
+
+
+    List<DeleteError> 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<ObjectInfo> objects = listAll(prefix, "");
+      ObjectUtils.deleteAllObjects(this, objects,
+          conf.getInt(ConfKeys.FS_BATCH_DELETE_SIZE.key(scheme()),
+              ConfKeys.FS_BATCH_DELETE_SIZE_DEFAULT));
+    }
+  }
+
+  private List<DeleteError> innerBatchDelete(List<String> keys) {
+    List<ObjectTobeDeleted> 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<DeleteError> 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<String> deleteErrorKeys(List<DeleteError> errorKeys) {
+    List<String> 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<String, String> 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<ListObjectsResponse> list(ListObjectsRequest req) {
+    return new LazyReload<>(() -> {
+      AtomicReference<String> 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<ObjectInfo> objects = listObjectsOutputToObjectInfos(response);
+        List<String> 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<String> listObjectsOutputToCommonPrefixes(ListObjectsType2Output listObjectsOutput) {
+    if (listObjectsOutput.getCommonPrefixes() == null) {
+      return Lists.newArrayList();
+    }
+
+    return listObjectsOutput.getCommonPrefixes()
+        .stream()
+        .map(ListedCommonPrefix::getPrefix)
+        .collect(Collectors.toList());
+  }
+
+  private List<ObjectInfo> 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<Part> uploadParts) {
+    checkAvailableClient();
+    List<UploadedPartV2> 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<MultipartUpload> listUploads(String prefix) {
+    checkAvailableClient();
+    return new LazyReload<>(() -> {
+      AtomicReference<String> nextKeyMarker = new AtomicReference<>("");
+      AtomicReference<String> 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<String, String> newTags) {
+    checkAvailableClient();
+    List<Tag> 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<String, String> getTags(String key) {
+    Map<String, String> result = new HashMap<>();
+    for (Tag tag : getObjectTaggingList(key)) {
+      result.put(tag.getKey(), tag.getValue());
+    }
+    return result;
+  }
+
+  private List<Tag> 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<Tag> 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)}. <br>
+   *
+   * The detail behavior are as follows:
+   * <ul>
+   *   <li>Assume object 'a/b' exists in TOS, getFileStatus("a/b") will get object('a/b') succeed,
+   *   getFileStatus("a/b/") will get 404.</li>
+   *   <li>Assume object 'a/b/' exists in TOS, both getFileStatus("a/b") & getFileStatus("a/b/")
+   *   will get object('a/b/') succeed </li>
+   *   <li>Assume object 'a/b/c' exists in TOS, both getFileStatus("a/b") & getFileStatus("a/b/")
+   *   will get object('a/b/') succeed.</li>
+   * </ul>
+   * <p>
+   * And the following is the logic of {@link TOSV2#getFileStatus(GetFileStatusInput)}: <br>
+   * Step 1: Head the specified key, if the head operation is successful, the response is filled
+   * with the actual object. <br>
+   * Step 2: Append the key with the suffix '/' to perform list operation, if the list operation is
+   * successful, the response is filled with the <strong>first object from the listing results
+   * </strong>; if there are no objects, return 404. <br>
+   *
+   * @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<ObjectInfo> 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();
+  }
+}

+ 53 - 0
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();
+}

+ 121 - 0
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);
+      }
+    }
+  }
+}

+ 119 - 0
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<ChecksumType, String> 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<String, String> 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<String, String> headers) {
+    String header = CHECKSUM_HEADER.get(ChecksumType.CRC64ECMA);
+    return headers.get(header);
+  }
+
+  public static boolean appendable(Map<String, String> headers) {
+    String value = headers.get(OBJECT_TYPE_KEY);
+    return APPENDABLE_TYPE_VALUE.equals(value);
+  }
+}

+ 78 - 0
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();
+  }
+}

+ 79 - 0
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();
+}

+ 35 - 0
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);
+}

+ 101 - 0
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<AbstractCredentialsProvider> 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<Exception> 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;
+  }
+}

+ 34 - 0
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
+    );
+  }
+}

+ 67 - 0
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;
+  }
+}

+ 58 - 0
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);
+  }
+}

+ 26 - 0
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;

+ 26 - 0
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;

+ 189 - 0
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<ObjectInfo, RawFileStatus> objMapper;
+  private final RenameOp renameOp;
+  private final boolean asyncCreateParentDir;
+
+  public DefaultFsOps(
+      ObjectStorage storage,
+      Configuration conf,
+      ExecutorService taskThreadPool,
+      Function<ObjectInfo, RawFileStatus> 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<RawFileStatus> listDir(Path dir, boolean recursive,
+      Predicate<String> 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<ObjectInfo> asObjectInfo(Iterable<ListObjectsResponse> listResponses,
+      Predicate<String> filter) {
+    Iterable<List<ObjectInfo>> results = Iterables.transform(listResponses, listResp -> {
+      List<ObjectInfo> 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()));
+  }
+}

+ 109 - 0
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<ObjectInfo, RawFileStatus> objMapper;
+
+  public DirectoryFsOps(DirectoryStorage storage, Function<ObjectInfo, RawFileStatus> 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<RawFileStatus> listDir(Path dir, boolean recursive,
+      Predicate<String> postFilter) {
+    String key = ObjectUtils.pathToKey(dir, true);
+    Iterable<ObjectInfo> 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]);
+  }
+}

+ 91 - 0
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<RawFileStatus> listDir(Path dir, boolean recursive, Predicate<String> 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;
+}

+ 230 - 0
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<ObjectInfo> 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<CompletableFuture<Part>> 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<Part> result = asyncUploadPartCopy(srcKey, multipartUpload,
+            partNumGetter.incrementAndGet(), start, end);
+        results.add(result);
+      }
+
+      // Waiting for all the upload parts to be finished.
+      List<Part> 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<Part> uploadParts) {
+    storage.completeUpload(key, uploadId, uploadParts);
+  }
+
+  private CompletableFuture<Part> 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]);
+  }
+}

+ 26 - 0
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;

+ 26 - 0
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;

+ 46 - 0
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;
+  }
+}

+ 67 - 0
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();
+    }
+  }
+}

+ 28 - 0
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);
+  }
+}

+ 128 - 0
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 <F, T> Iterable<T> transform(final Iterable<F> fromIterable,
+      final Function<? super F, ? extends T> function) {
+    Preconditions.checkNotNull(fromIterable);
+    Preconditions.checkNotNull(function);
+    return () -> new Iterator<T>() {
+      private Iterator<F> iterator = fromIterable.iterator();
+
+      @Override
+      public boolean hasNext() {
+        return iterator.hasNext();
+      }
+
+      @Override
+      public T next() {
+        return function.apply(iterator.next());
+      }
+    };
+  }
+
+  public static <T> Iterable<T> filter(final Iterable<T> unfiltered,
+      final Predicate<? super T> predicate) {
+    Preconditions.checkNotNull(unfiltered);
+    Preconditions.checkNotNull(predicate);
+    return () -> new Iterator<T>() {
+      private Iterator<T> 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 <T extends @Nullable Object> Iterable<T> concat(
+      Iterable<? extends Iterable<? extends T>> inputs) {
+    return () -> new ConcatenatedIterator<>(inputs.iterator());
+  }
+
+  private static class ConcatenatedIterator<T> implements Iterator<T> {
+    // Iterators is the iterator of iterables.
+    private final Iterator<? extends Iterable<? extends T>> iterators;
+    private Iterator<? extends T> curIter;
+
+    ConcatenatedIterator(Iterator<? extends Iterable<? extends T>> 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");
+    }
+  }
+}

+ 46 - 0
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<T> {
+  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<T> clazz;
+
+  public JsonCodec(Class<T> 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);
+  }
+}

+ 35 - 0
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<T> implements Iterable<T> {
+  private final Supplier<Reload<T>> reload;
+
+  public LazyReload(Supplier<Reload<T>> reload) {
+    this.reload = reload;
+  }
+
+  @Override
+  public Iterator<T> iterator() {
+    return new LazyReloadIter<>(reload.get());
+  }
+}

+ 69 - 0
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<T> implements Iterator<T> {
+  private final List<T> buf = Lists.newArrayList();
+  private final Reload<T> reload;
+  private int cur = 0;
+  private boolean exhausted = false;
+
+  public LazyReloadIter(Reload<T> 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();
+    }
+  }
+}

+ 65 - 0
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));
+    }
+  }
+}

+ 104 - 0
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<Range> 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<Range> 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;
+  }
+}

Some files were not shown because too many files changed in this diff