Ver Fonte

Integration of TOS: Add RawFileSystem unit tests.

lijinglun há 6 meses atrás
pai
commit
25879366d0
41 ficheiros alterados com 3893 adições e 113 exclusões
  1. 14 1
      hadoop-cloud-storage-project/hadoop-tos/pom.xml
  2. 9 9
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFileSystem.java
  3. 38 6
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/ConfKeys.java
  4. 3 0
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/FileStoreKeys.java
  5. 0 34
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/TosKeys.java
  6. 1 1
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/Constants.java
  7. 3 2
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/FileStore.java
  8. 6 5
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectOutputStream.java
  9. 1 1
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClientBuilder.java
  10. 6 5
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOS.java
  11. 3 2
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/DefaultFsOps.java
  12. 8 6
      hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/RenameOp.java
  13. 40 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestRawFSUtils.java
  14. 55 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestRawFileSystem.java
  15. 124 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestTosChecksum.java
  16. 127 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestChecksum.java
  17. 30 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestCreate.java
  18. 60 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestDelete.java
  19. 36 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestDistCp.java
  20. 159 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestGetFileStatus.java
  21. 30 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestMkdir.java
  22. 65 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestOpen.java
  23. 270 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestRename.java
  24. 28 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestRootDir.java
  25. 28 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestSeek.java
  26. 28 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestUnbuffer.java
  27. 168 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestXAttr.java
  28. 53 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TosContract.java
  29. 213 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestDirectoryStorage.java
  30. 4 4
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectOutputStream.java
  31. 1407 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectStorage.java
  32. 4 15
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestTOSObjectStorage.java
  33. 274 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestBaseFsOps.java
  34. 96 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestBaseOps.java
  35. 78 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestDefaultFsOps.java
  36. 62 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestDirectoryFsOps.java
  37. 171 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestRenameOp.java
  38. 0 22
      hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestUtility.java
  39. 113 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/resources/contract/tos.xml
  40. 55 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/resources/core-site.xml
  41. 23 0
      hadoop-cloud-storage-project/hadoop-tos/src/test/resources/log4j.properties

+ 14 - 1
hadoop-cloud-storage-project/hadoop-tos/pom.xml

@@ -34,7 +34,7 @@
 
   <properties>
     <file.encoding>UTF-8</file.encoding>
-    <ve-tos-java-sdk.version>2.8.2</ve-tos-java-sdk.version>
+    <ve-tos-java-sdk.version>2.8.6</ve-tos-java-sdk.version>
   </properties>
 
   <dependencies>
@@ -88,6 +88,19 @@
       <artifactId>mockito-core</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>
   </dependencies>
 
   <build>

+ 9 - 9
hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/RawFileSystem.java

@@ -98,7 +98,6 @@ public class RawFileSystem extends FileSystem {
   private static final long DFS_BLOCK_SIZE_DEFAULT = 128 << 20;
 
   private String scheme;
-  private Configuration protonConf;
   private String username;
   private Path workingDir;
   private URI uri;
@@ -171,17 +170,17 @@ public class RawFileSystem extends FileSystem {
 
     if (MagicOutputStream.isMagic(path)) {
       return new FSDataOutputStream(
-          new MagicOutputStream(this, storage, uploadThreadPool, protonConf, makeQualified(path)), null);
+          new MagicOutputStream(this, storage, uploadThreadPool, getConf(), makeQualified(path)), null);
     } else {
       ObjectOutputStream out =
-          new ObjectOutputStream(storage, uploadThreadPool, protonConf, makeQualified(path), true);
+          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. For more details please see: https://code.byted.org/emr/proton/issues/825
         out.close();
-        out = new ObjectOutputStream(storage, uploadThreadPool, protonConf, makeQualified(path), true);
+        out = new ObjectOutputStream(storage, uploadThreadPool, getConf(), makeQualified(path), true);
       }
 
       return new FSDataOutputStream(out, null);
@@ -598,23 +597,23 @@ public class RawFileSystem extends FileSystem {
     this.workingDir = new Path("/user", username).makeQualified(uri, null);
     this.uri = URI.create(scheme + "://" + uri.getAuthority());
     this.bucket = this.uri.getAuthority();
-    this.storage = ObjectStorageFactory.create(scheme, bucket, protonConf);
+    this.storage = ObjectStorageFactory.create(scheme, bucket, getConf());
     if (storage.bucket() == null) {
       throw new FileNotFoundException(String.format("Bucket: %s not found.", uri.getAuthority()));
     }
 
     int taskThreadPoolSize =
-        protonConf.getInt(ConfKeys.TASK_THREAD_POOL_SIZE, ConfKeys.TASK_THREAD_POOL_SIZE_DEFAULT);
+        getConf().getInt(ConfKeys.TASK_THREAD_POOL_SIZE, ConfKeys.TASK_THREAD_POOL_SIZE_DEFAULT);
     this.taskThreadPool = ThreadPools.newWorkerPool(TASK_THREAD_POOL_PREFIX, taskThreadPoolSize);
 
-    int uploadThreadPoolSize = protonConf.getInt(ConfKeys.MULTIPART_THREAD_POOL_SIZE,
+    int uploadThreadPoolSize = getConf().getInt(ConfKeys.MULTIPART_THREAD_POOL_SIZE,
         ConfKeys.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, protonConf, taskThreadPool, this::objectToFileStatus);
+      fsOps = new DefaultFsOps(storage, getConf(), taskThreadPool, this::objectToFileStatus);
     }
   }
 
@@ -648,7 +647,8 @@ public class RawFileSystem extends FileSystem {
       // Compatible with HDFS
       throw new FileNotFoundException(String.format("Path is not a file, %s", f));
     }
-    if (!protonConf.getBoolean(ConfKeys.CHECKSUM_ENABLED, ConfKeys.CHECKSUM_ENABLED_DEFAULT)) {
+    if (!getConf().getBoolean(ConfKeys.CHECKSUM_ENABLED.key(storage.scheme()),
+        ConfKeys.CHECKSUM_ENABLED_DEFAULT)) {
       return null;
     }
 

+ 38 - 6
hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/ConfKeys.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs.tosfs.conf;
 
+import org.apache.hadoop.fs.tosfs.object.ChecksumType;
 import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 
 public class ConfKeys {
@@ -52,7 +53,7 @@ public class ConfKeys {
   /**
    * The multipart upload part size of the given object storage, e.g. fs.tos.multipart.size.
    */
-  public static final String MULTIPART_SIZE = "fs.tos.multipart.size";
+  public static final ArgumentKey MULTIPART_SIZE = new ArgumentKey("fs.%s.multipart.size");
   public static final long MULTIPART_SIZE_DEFAULT = 8L << 20;
 
   /**
@@ -60,7 +61,8 @@ public class ConfKeys {
    * 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 String MULTIPART_COPY_THRESHOLD = "fs.tos.multipart.copy-threshold";
+  public static final ArgumentKey MULTIPART_COPY_THRESHOLD =
+      new ArgumentKey("fs.%s.multipart.copy-threshold");
   public static final long MULTIPART_COPY_THRESHOLD_DEFAULT = 5L << 20;
 
   /**
@@ -68,7 +70,8 @@ public class ConfKeys {
    * 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 String MULTIPART_THRESHOLD = "fs.tos.multipart.threshold";
+  public static final ArgumentKey MULTIPART_THRESHOLD =
+      new ArgumentKey("fs.%s.multipart.threshold");
   public static final long MULTIPART_THRESHOLD_DEFAULT = 10 << 20;
 
   /**
@@ -97,7 +100,8 @@ public class ConfKeys {
   /**
    * True to create the missed parent dir asynchronously during deleting or renaming a file or dir.
    */
-  public static final String ASYNC_CREATE_MISSED_PARENT = "fs.tos.missed.parent.dir.async-create";
+  public static final ArgumentKey ASYNC_CREATE_MISSED_PARENT =
+      new ArgumentKey("fs.%s.missed.parent.dir.async-create");
   public static final boolean ASYNC_CREATE_MISSED_PARENT_DEFAULT = true;
 
   /**
@@ -108,7 +112,7 @@ public class ConfKeys {
    * If you are using TOS, you have to send putBucketRename request before sending rename request,
    * otherwise MethodNotAllowed exception will be thrown.
    */
-  public static final String OBJECT_RENAME_ENABLED = "fs.tos.rename.enabled";
+  public static final ArgumentKey OBJECT_RENAME_ENABLED = new ArgumentKey("fs.%s.rename.enabled");
   public static final boolean OBJECT_RENAME_ENABLED_DEFAULT = false;
 
   /**
@@ -133,13 +137,41 @@ public class ConfKeys {
   public static final int MULTIPART_THREAD_POOL_SIZE_DEFAULT =
       Math.max(2, Runtime.getRuntime().availableProcessors());
 
+  /**
+   * 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 TOS_GET_FILE_STATUS_ENABLED = "fs.tos.get-file-status.enabled";
+  public static final boolean TOS_GET_FILE_STATUS_ENABLED_DEFAULT = true;
+
   /**
    * The toggle indicates whether enable checksum during getting file status for the given object.
    * E.g. fs.tos.checksum.enabled
    */
-  public static final String CHECKSUM_ENABLED = "fs.tos.checksum.enabled";
+  public static final ArgumentKey CHECKSUM_ENABLED = new ArgumentKey("fs.%s.checksum.enabled");
   public static final boolean CHECKSUM_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 TOS_CHECKSUM_ALGORITHM = "fs.tos.checksum-algorithm";
+  public static final String TOS_CHECKSUM_ALGORITHM_DEFAULT = "PROTON-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 TOS_CHECKSUM_TYPE = "fs.tos.checksum-type";
+  public static final String TOS_CHECKSUM_TYPE_DEFAULT = ChecksumType.CRC64ECMA.name();
+
   public static String defaultDir(String basename) {
     String tmpdir = System.getProperty("java.io.tmpdir");
     Preconditions.checkNotNull(tmpdir, "System property 'java.io.tmpdir' cannot be null");

+ 3 - 0
hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/FileStoreKeys.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.fs.tosfs.conf;
 
+import org.apache.hadoop.fs.tosfs.object.ChecksumType;
+
 public class FileStoreKeys {
 
   /**
@@ -37,6 +39,7 @@ public class FileStoreKeys {
    * 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();
 
   /**
    * The batch size of deleting multiple objects per request for the given object storage.

+ 0 - 34
hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/conf/TosKeys.java

@@ -26,12 +26,6 @@ import java.util.List;
 
 public class TosKeys {
 
-  /**
-   * Tos object storage endpoint to connect to, which should include both region and object domain
-   * name.
-   */
-  public static final String FS_TOS_ENDPOINT = "fs.tos.endpoint";
-
   /**
    * The accessKey key to access the tos object storage.
    */
@@ -149,18 +143,6 @@ public class TosKeys {
   public static final String FS_ASYNC_CREATE_MISSED_PARENT = "fs.tos.missed.parent.dir.async-create";
   public static final boolean FS_ASYNC_CREATE_MISSED_PARENT_DEFAULT = false;
 
-  /**
-   * 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;
-
   /**
    * 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
@@ -185,22 +167,6 @@ public class TosKeys {
   public static final String FS_TOS_USER_AGENT_PREFIX = "fs.tos.user.agent.prefix";
   public static final String FS_TOS_USER_AGENT_PREFIX_DEFAULT = "EMR";
 
-  /**
-   * 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 = "PROTON-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();
-
   // TOS common keys.
   /**
    * The threshold indicates whether reuse the socket connection to optimize read performance during

+ 1 - 1
hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/Constants.java

@@ -22,7 +22,7 @@ public class Constants {
   private Constants() {
   }
 
-  public static final String PROTON = "Proton";
+  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.

+ 3 - 2
hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/FileStore.java

@@ -62,6 +62,7 @@ import static org.apache.hadoop.fs.tosfs.conf.FileStoreKeys.FS_FILESTORE_BATCH_D
 import static org.apache.hadoop.fs.tosfs.conf.FileStoreKeys.FS_FILESTORE_BATCH_DELETE_SIZE_DEFAULT;
 import static org.apache.hadoop.fs.tosfs.conf.FileStoreKeys.FS_FILESTORE_CHECKSUM_ALGORITHM;
 import static org.apache.hadoop.fs.tosfs.conf.FileStoreKeys.FS_FILESTORE_CHECKSUM_TYPE;
+import static org.apache.hadoop.fs.tosfs.conf.FileStoreKeys.FS_FILESTORE_CHECKSUM_TYPE_DEFAULT;
 import static org.apache.hadoop.fs.tosfs.conf.FileStoreKeys.FS_FILESTORE_ENDPOINT;
 
 public class FileStore implements ObjectStorage {
@@ -112,8 +113,8 @@ public class FileStore implements ObjectStorage {
     LOG.debug("the root path is: {}", this.root);
 
     String algorithm = conf.get(FS_FILESTORE_CHECKSUM_ALGORITHM);
-    ChecksumType checksumType =
-        ChecksumType.valueOf(conf.get(FS_FILESTORE_CHECKSUM_TYPE).toUpperCase());
+    ChecksumType checksumType = ChecksumType.valueOf(
+        conf.get(FS_FILESTORE_CHECKSUM_TYPE, 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);

+ 6 - 5
hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectOutputStream.java

@@ -73,9 +73,10 @@ public class ObjectOutputStream extends OutputStream {
     this.destScheme = dest.toUri().getScheme();
     this.totalWroteSize = 0;
     this.destKey = createDestKey(dest);
-    this.multiUploadThreshold =
-        conf.getLong(ConfKeys.MULTIPART_THRESHOLD, ConfKeys.MULTIPART_THRESHOLD_DEFAULT);
-    this.byteSizePerPart = conf.getLong(ConfKeys.MULTIPART_SIZE, ConfKeys.MULTIPART_SIZE_DEFAULT);
+    this.multiUploadThreshold = conf.getLong(ConfKeys.MULTIPART_THRESHOLD.key(destScheme),
+        ConfKeys.MULTIPART_THRESHOLD_DEFAULT);
+    this.byteSizePerPart =
+        conf.getLong(ConfKeys.MULTIPART_SIZE.key(destScheme), ConfKeys.MULTIPART_SIZE_DEFAULT);
     this.stagingBufferSize = conf.getInt(ConfKeys.MULTIPART_STAGING_BUFFER_SIZE,
         ConfKeys.MULTIPART_STAGING_BUFFER_SIZE_DEFAULT);
     this.allowPut = allowPut;
@@ -172,8 +173,8 @@ public class ObjectOutputStream extends OutputStream {
         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.MULTIPART_THRESHOLD.format(destScheme));
+                + " please check configure key %s.", byteSizePerPart, multipartUpload.minPartSize(),
+            ConfKeys.MULTIPART_THRESHOLD.key(destScheme));
 
         // Upload the accumulated staging files whose length >= byteSizePerPart.
         for (StagingPart stagingPart : stagingParts) {

+ 1 - 1
hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClientBuilder.java

@@ -88,7 +88,7 @@ public class DelegationClientBuilder {
         .transportConfig(createTransportConfig(conf))
         .userAgentProductName(conf.get(
             TosKeys.FS_TOS_USER_AGENT_PREFIX, TosKeys.FS_TOS_USER_AGENT_PREFIX_DEFAULT))
-        .userAgentSoftName(Constants.PROTON)
+        .userAgentSoftName(Constants.TOS_FS)
         .userAgentSoftVersion(VersionInfo.getVersion())
         .build();
 

+ 6 - 5
hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOS.java

@@ -181,11 +181,12 @@ public class TOS implements DirectoryStorage {
         conf.getInt(TosKeys.FS_TOS_LIST_OBJECTS_COUNT, TosKeys.FS_TOS_LIST_OBJECTS_COUNT_DEFAULT);
     maxInputStreamRetries = conf.getInt(TosKeys.FS_TOS_MAX_READ_OBJECT_RETRIES,
         TosKeys.FS_TOS_MAX_READ_OBJECT_RETRIES_DEFAULT);
-    defaultAcl = TypeConverter.convertACLType(TosKeys.FS_TOS_ACL_DEFAULT);
+    defaultAcl = TypeConverter.convertACLType(conf.get(TosKeys.FS_TOS_ACL_DEFAULT));
 
-    String algorithm = conf.get(TosKeys.FS_TOS_CHECKSUM_ALGORITHM);
+    String algorithm =
+        conf.get(ConfKeys.TOS_CHECKSUM_ALGORITHM, ConfKeys.TOS_CHECKSUM_ALGORITHM_DEFAULT);
     ChecksumType checksumType = ChecksumType.valueOf(
-        conf.get(TosKeys.FS_TOS_CHECKSUM_TYPE, TosKeys.FS_TOS_CHECKSUM_TYPE_DEFAULT).toUpperCase());
+        conf.get(ConfKeys.TOS_CHECKSUM_TYPE, ConfKeys.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);
@@ -1004,8 +1005,8 @@ public class TOS implements DirectoryStorage {
   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)) {
+    } else if (conf.getBoolean(ConfKeys.TOS_GET_FILE_STATUS_ENABLED,
+        ConfKeys.TOS_GET_FILE_STATUS_ENABLED_DEFAULT)) {
       return getFileStatus(key);
     } else {
       ObjectInfo obj = head(key);

+ 3 - 2
hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/DefaultFsOps.java

@@ -62,8 +62,9 @@ public class DefaultFsOps implements FsOps {
     this.taskThreadPool = taskThreadPool;
     this.objMapper = objMapper;
     this.renameOp = new RenameOp(conf, storage, taskThreadPool);
-    this.asyncCreateParentDir = conf.getBoolean(ConfKeys.ASYNC_CREATE_MISSED_PARENT,
-        ConfKeys.ASYNC_CREATE_MISSED_PARENT_DEFAULT);
+    this.asyncCreateParentDir =
+        conf.getBoolean(ConfKeys.ASYNC_CREATE_MISSED_PARENT.key(storage.scheme()),
+            ConfKeys.ASYNC_CREATE_MISSED_PARENT_DEFAULT);
   }
 
   @Override

+ 8 - 6
hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/ops/RenameOp.java

@@ -56,8 +56,8 @@ public class RenameOp {
     this.conf = conf;
     this.storage = storage;
     this.renamePool = taskThreadPool;
-    this.renameObjectEnabled =
-        conf.getBoolean(ConfKeys.OBJECT_RENAME_ENABLED, ConfKeys.OBJECT_RENAME_ENABLED_DEFAULT);
+    this.renameObjectEnabled = conf.getBoolean(ConfKeys.OBJECT_RENAME_ENABLED.key(storage.scheme()),
+        ConfKeys.OBJECT_RENAME_ENABLED_DEFAULT);
   }
 
   public void renameDir(Path src, Path dst) {
@@ -155,9 +155,11 @@ public class RenameOp {
   }
 
   private void copyFile(String srcKey, String dstKey, long srcSize) throws IOException {
-    long byteSizePerPart = conf.getLong(ConfKeys.MULTIPART_SIZE, ConfKeys.MULTIPART_SIZE_DEFAULT);
+    long byteSizePerPart = conf.getLong(ConfKeys.MULTIPART_SIZE.key(storage.scheme()),
+        ConfKeys.MULTIPART_SIZE_DEFAULT);
     long multiPartCopyThreshold =
-        conf.getLong(ConfKeys.MULTIPART_COPY_THRESHOLD, ConfKeys.MULTIPART_COPY_THRESHOLD_DEFAULT);
+        conf.getLong(ConfKeys.MULTIPART_COPY_THRESHOLD.key(storage.scheme()),
+            ConfKeys.MULTIPART_COPY_THRESHOLD_DEFAULT);
     if (srcSize > multiPartCopyThreshold) {
       uploadPartCopy(srcKey, srcSize, dstKey, byteSizePerPart);
     } else {
@@ -170,8 +172,8 @@ public class RenameOp {
     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.MULTIPART_SIZE.format(storage.scheme()));
+              + " please check configure key %s.", byteSizePerPart, multipartUpload.minPartSize(),
+          ConfKeys.MULTIPART_SIZE.key(storage.scheme()));
 
       AtomicInteger partNumGetter = new AtomicInteger(0);
       List<CompletableFuture<Part>> results = Lists.newArrayList();

+ 40 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestRawFSUtils.java

@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestRawFSUtils {
+
+  @Test
+  public void testIsAncestor() {
+    Assert.assertTrue(RawFSUtils.inSubtree("/", "/"));
+    Assert.assertTrue(RawFSUtils.inSubtree("/", "/a"));
+    Assert.assertTrue(RawFSUtils.inSubtree("/a", "/a"));
+    Assert.assertFalse(RawFSUtils.inSubtree("/a", "/"));
+    Assert.assertTrue(RawFSUtils.inSubtree("/", "/a/b/c"));
+    Assert.assertFalse(RawFSUtils.inSubtree("/a/b/c", "/"));
+    Assert.assertTrue(RawFSUtils.inSubtree("/", "/a/b/c.txt"));
+    Assert.assertFalse(RawFSUtils.inSubtree("/a/b/c.txt", "/"));
+    Assert.assertTrue(RawFSUtils.inSubtree("/a/b/", "/a/b"));
+    Assert.assertTrue(RawFSUtils.inSubtree("/a/b/", "/a/b/c"));
+    Assert.assertFalse(RawFSUtils.inSubtree("/a/b/c", "/a/b"));
+  }
+}

+ 55 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestRawFileSystem.java

@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import static org.junit.Assert.assertThrows;
+
+public class TestRawFileSystem {
+  @Test
+  public void testInitializeFileSystem() throws URISyntaxException, IOException {
+    Configuration conf = new Configuration();
+    try (RawFileSystem fs = new RawFileSystem()) {
+      fs.initialize(new URI("filestore://bucket_a/a/b/c"), conf);
+      Assert.assertEquals("bucket_a", fs.bucket());
+
+      fs.initialize(new URI("filestore://bucket-/a/b/c"), conf);
+      Assert.assertEquals("bucket-", fs.bucket());
+
+      fs.initialize(new URI("filestore://-bucket/a/b/c"), conf);
+      Assert.assertEquals("-bucket", fs.bucket());
+    }
+  }
+
+  @Test
+  public void testBucketNotExist() {
+    Configuration conf = new Configuration();
+    RawFileSystem fs = new RawFileSystem();
+    assertThrows("Bucket doesn't exist.", FileNotFoundException.class,
+        () -> fs.initialize(new URI("tos://not-exist-bucket/"), conf));
+  }
+}

+ 124 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/TestTosChecksum.java

@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.tosfs.conf.ConfKeys;
+import org.apache.hadoop.fs.tosfs.conf.FileStoreKeys;
+import org.apache.hadoop.fs.tosfs.conf.TosKeys;
+import org.apache.hadoop.fs.tosfs.object.ChecksumType;
+import org.apache.hadoop.fs.tosfs.object.ObjectStorage;
+import org.apache.hadoop.fs.tosfs.object.ObjectStorageFactory;
+import org.apache.hadoop.fs.tosfs.util.TempFiles;
+import org.apache.hadoop.fs.tosfs.util.TestUtility;
+import org.apache.hadoop.fs.tosfs.util.UUIDUtils;
+import org.junit.After;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hadoop.fs.tosfs.object.tos.TOS.TOS_SCHEME;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+@RunWith(Parameterized.class)
+public class TestTosChecksum {
+  private static final String FILE_STORE_ROOT = TempFiles.newTempDir("TestProtonChecksum");
+  private static final String ALGORITHM_NAME = "mock-algorithm";
+  private static final String PREFIX = UUIDUtils.random();
+
+  @Parameterized.Parameters(name = "checksumType = {0}, conf = {1}, uri = {2}, objectStorage = {3}")
+  public static Iterable<Object[]> createStorage() throws URISyntaxException {
+    return createTestObjectStorage(FILE_STORE_ROOT);
+  }
+
+  public static Iterable<Object[]> createTestObjectStorage(String fileStoreRoot)
+      throws URISyntaxException {
+    List<Object[]> list = new ArrayList<>();
+
+    // The 1st argument.
+    Configuration fileStoreConf = new Configuration();
+    fileStoreConf.set(FileStoreKeys.FS_FILESTORE_CHECKSUM_ALGORITHM, ALGORITHM_NAME);
+    fileStoreConf.set(FileStoreKeys.FS_FILESTORE_CHECKSUM_TYPE, ChecksumType.MD5.name());
+    fileStoreConf.set(FileStoreKeys.FS_FILESTORE_ENDPOINT, fileStoreRoot);
+
+    URI uri0 = new URI("filestore://" + TestUtility.bucket() + "/");
+    Object[] objs = new Object[] { ChecksumType.MD5, fileStoreConf, uri0,
+        ObjectStorageFactory.create(uri0.getScheme(), uri0.getAuthority(), fileStoreConf) };
+    list.add(objs);
+
+    // The 2nd argument.
+    Configuration tosConf = new Configuration();
+    tosConf.set(ConfKeys.TOS_CHECKSUM_ALGORITHM, ALGORITHM_NAME);
+    tosConf.set(ConfKeys.TOS_CHECKSUM_TYPE, ChecksumType.CRC32C.name());
+
+    URI uri1 = new URI(TOS_SCHEME + "://" + TestUtility.bucket() + "/");
+    objs = new Object[] { ChecksumType.CRC32C, tosConf, uri1,
+        ObjectStorageFactory.create(uri1.getScheme(), uri1.getAuthority(), tosConf) };
+    list.add(objs);
+
+    return list;
+  }
+
+  @After
+  public void tearDown() {
+    objectStorage.deleteAll(PREFIX);
+  }
+
+  private ChecksumType type;
+  private Configuration conf;
+  private URI uri;
+  private ObjectStorage objectStorage;
+
+  public TestTosChecksum(ChecksumType type, Configuration conf, URI uri,
+      ObjectStorage objectStorage) {
+    this.type = type;
+    this.conf = conf;
+    this.uri = uri;
+    this.objectStorage = objectStorage;
+  }
+
+  @Test
+  public void testChecksumInfo() {
+    assertEquals(ALGORITHM_NAME, objectStorage.checksumInfo().algorithm());
+    assertEquals(type, objectStorage.checksumInfo().checksumType());
+  }
+
+  @Test
+  public void testFileChecksum() throws Exception {
+    try (RawFileSystem fs = new RawFileSystem()) {
+      fs.initialize(uri, conf);
+      Path file = new Path("/" + PREFIX, "testFileChecksum");
+      fs.create(file).close();
+      FileChecksum checksum = fs.getFileChecksum(file, Long.MAX_VALUE);
+      assertEquals(ALGORITHM_NAME, checksum.getAlgorithmName());
+
+      String key = file.toString().substring(1);
+      byte[] checksumData = objectStorage.head(key).checksum();
+      assertArrayEquals(checksumData, checksum.getBytes());
+    }
+  }
+}

+ 127 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestChecksum.java

@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.tosfs.RawFileSystem;
+import org.apache.hadoop.fs.tosfs.conf.ConfKeys;
+import org.apache.hadoop.fs.tosfs.object.ObjectUtils;
+import org.apache.hadoop.fs.tosfs.util.TestUtility;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+
+public class TestChecksum extends AbstractFSContractTestBase {
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new TosContract(conf);
+  }
+
+  private Path testCreateNewFile(String fileName, byte[] data, boolean useBuilder) throws IOException {
+    describe("Foundational 'create a file' test, using builder API=" + useBuilder);
+    Path path = path(fileName, useBuilder);
+
+    writeDataset(getFileSystem(), path, data, data.length, 1024 * 1024, false, useBuilder);
+    ContractTestUtils.verifyFileContents(getFileSystem(), path, data);
+
+    return path;
+  }
+
+  private Path path(String filepath, boolean useBuilder) throws IOException {
+    return super.path(filepath + (useBuilder ? "" : "-builder"));
+  }
+
+  @Test
+  public void testCheckSumWithSimplePut() throws IOException {
+    byte[] data = dataset(256, 'a', 'z');
+    Path path1 = testCreateNewFile("file1", data, true);
+    Path path2 = testCreateNewFile("file2", data, true);
+    Path path3 = testCreateNewFile("file3", dataset(512, 'a', 'z'), true);
+
+    FileChecksum expected = getFileSystem().getFileChecksum(path1);
+    assertEquals("Checksum value should be same among objects with same content",
+        expected, getFileSystem().getFileChecksum(path2));
+    assertEquals("Checksum value should be same among multiple call for same object",
+        expected, getFileSystem().getFileChecksum(path1));
+    assertNotEquals("Checksum value should be different for different objects with different content",
+        expected, getFileSystem().getFileChecksum(path3));
+
+    Path renamed = path("renamed");
+    getFileSystem().rename(path1, renamed);
+    assertEquals("Checksum value should not change after rename",
+        expected, getFileSystem().getFileChecksum(renamed));
+  }
+
+  @Test
+  public void testCheckSumShouldSameViaPutAndMPU() throws IOException {
+    byte[] data = TestUtility.rand(11 << 20);
+
+    // simple put
+    Path singleFile = path("singleFile");
+    RawFileSystem fs = (RawFileSystem) getFileSystem();
+    fs.storage().put(ObjectUtils.pathToKey(singleFile), data);
+
+    // MPU upload data, the default threshold is 10MB
+    Path mpuFile = testCreateNewFile("mpuFile", data, true);
+
+    assertEquals(fs.getFileChecksum(singleFile), fs.getFileChecksum(mpuFile));
+  }
+
+  @Test
+  public void testDisableCheckSum() throws IOException {
+    Path path1 = testCreateNewFile("file1", dataset(256, 'a', 'z'), true);
+    Path path2 = testCreateNewFile("file2", dataset(512, 'a', 'z'), true);
+    assertNotEquals(getFileSystem().getFileChecksum(path1), getFileSystem().getFileChecksum(path2));
+
+    // disable checksum
+    Configuration newConf = new Configuration(getFileSystem().getConf());
+    newConf.setBoolean(ConfKeys.CHECKSUM_ENABLED.key("tos"), false);
+    FileSystem newFS = FileSystem.get(newConf);
+
+    assertEquals(newFS.getFileChecksum(path1), newFS.getFileChecksum(path2));
+  }
+
+  @Test
+  public void testGetDirChecksum() throws IOException {
+    FileSystem fs = getFileSystem();
+
+    Path dir1 = path("dir1", true);
+    Path dir2 = path("dir2", true);
+    assertPathDoesNotExist("directory already exists", dir1);
+    assertPathDoesNotExist("directory already exists", dir2);
+    fs.mkdirs(dir1);
+
+    assertThrows("Path is not a file", FileNotFoundException.class,
+        () -> getFileSystem().getFileChecksum(dir1));
+    assertThrows("No such file or directory", FileNotFoundException.class,
+        () -> getFileSystem().getFileChecksum(dir2));
+
+    assertDeleted(dir1, false);
+  }
+}

+ 30 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestCreate.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.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestCreate extends AbstractContractCreateTest {
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new TosContract(conf);
+  }
+}

+ 60 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestDelete.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.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.tosfs.object.ObjectTestUtils.assertDirExist;
+import static org.apache.hadoop.fs.tosfs.object.ObjectTestUtils.assertObjectNotExist;
+
+public class TestDelete extends AbstractContractDeleteTest {
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new TosContract(conf);
+  }
+
+  @Test
+  public void testParentDirCreatedAfterDeleteSubChildren() throws IOException {
+    Path path = path("testParentDirCreatedAfterDeleteSubChildren/");
+    Path file1 = new Path(path, "f1");
+    Path file2 = new Path(path, "f2");
+    ContractTestUtils.writeTextFile(getFileSystem(), file1,
+        "the first file", true);
+    ContractTestUtils.writeTextFile(getFileSystem(), file2,
+        "the second file", true);
+    assertPathExists("file1 not created", file1);
+    assertPathExists("file1 not created", file2);
+
+    assertObjectNotExist(path, false);
+    assertObjectNotExist(path, true);
+
+    assertDeleted(file1, false);
+    assertPathExists("parent path should exist", path);
+
+    assertObjectNotExist(path, false);
+    assertDirExist(path);
+  }
+}

+ 36 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestDistCp.java

@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
+import org.junit.Test;
+
+public class TestDistCp extends AbstractContractDistCpTest {
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new TosContract(conf);
+  }
+
+  // ignore this test case as there is intermittent IllegalStateException issue
+  @Test
+  public void testDistCpWithIterator() {
+  }
+}

+ 159 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestGetFileStatus.java

@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.tosfs.RawFileStatus;
+import org.apache.hadoop.fs.tosfs.conf.ConfKeys;
+import org.apache.hadoop.fs.tosfs.object.Constants;
+import org.apache.hadoop.fs.tosfs.util.UUIDUtils;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+
+@RunWith(Parameterized.class)
+public class TestGetFileStatus extends AbstractContractGetFileStatusTest {
+
+  private final boolean getFileStatusEnabled;
+
+  @Parameterized.Parameters(name = "getFileStatusEnabled={0}")
+  public static List<Boolean> createParameters() {
+    return Arrays.asList(false, true);
+  }
+
+  public TestGetFileStatus(boolean getFileStatusEnabled) {
+    this.getFileStatusEnabled = getFileStatusEnabled;
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    conf.setBoolean(ConfKeys.TOS_GET_FILE_STATUS_ENABLED, getFileStatusEnabled);
+    conf.setBoolean(ConfKeys.ASYNC_CREATE_MISSED_PARENT.key("tos"), false);
+    return new TosContract(conf);
+  }
+
+  @Test
+  public void testDirModificationTimeShouldNotBeZero() throws IOException {
+    FileSystem fs = getFileSystem();
+    Path path = getContract().getTestPath();
+    fs.delete(path, true);
+
+    Path subfolder = path.suffix('/' + this.methodName.getMethodName() + "-" + UUIDUtils.random());
+    mkdirs(subfolder);
+
+    FileStatus fileStatus = fs.getFileStatus(path);
+    assertTrue(fileStatus.getModificationTime() > 0);
+  }
+
+  @Test
+  public void testThrowExceptionWhenListStatusForNonExistPath() {
+    FileSystem fs = getFileSystem();
+    Path path = getContract().getTestPath();
+
+    assertThrows("Path doesn't exist", FileNotFoundException.class,
+        () -> fs.listStatusIterator(new Path(path, "testListStatusForNonExistPath")));
+  }
+
+  @Test
+  public void testPathStatNonexistentFile() {
+    FileSystem fs = getFileSystem();
+    // working dir does not exist.
+    Path file = new Path(getContract().getTestPath(), this.methodName.getMethodName());
+    assertThrows("Path doesn't exist", FileNotFoundException.class, () -> fs.getFileStatus(file));
+  }
+
+  @Test
+  public void testPathStatExistentFile() throws IOException {
+    FileSystem fs = getFileSystem();
+    Path file = new Path(getContract().getTestPath(), this.methodName.getMethodName());
+
+    int size = 1 << 20;
+    byte[] data = dataset(size, 'a', 'z');
+    createFile(fs, file, true, data);
+    FileStatus status = fs.getFileStatus(file);
+    Assert.assertTrue(status.isFile());
+    Assert.assertTrue(status.getModificationTime() > 0);
+    Assert.assertEquals(size, status.getLen());
+  }
+
+  @Test
+  public void testPathStatEmptyDirectory() throws IOException {
+    FileSystem fs = getFileSystem();
+    Path workingPath = new Path(getContract().getTestPath(), this.methodName.getMethodName());
+    mkdirs(workingPath);
+
+    FileStatus dirStatus = fs.getFileStatus(workingPath);
+    Assert.assertTrue(dirStatus.isDirectory());
+    Assert.assertTrue(dirStatus.getModificationTime() > 0);
+    if (dirStatus instanceof RawFileStatus) {
+      Assert.assertArrayEquals(Constants.MAGIC_CHECKSUM, ((RawFileStatus) dirStatus).checksum());
+    }
+  }
+
+  @Test
+  public void testPathStatWhenCreateSubDir() throws IOException {
+    FileSystem fs = getFileSystem();
+    Path workintPath = new Path(getContract().getTestPath(), this.methodName.getMethodName());
+    // create sub directory directly.
+    Path subDir = new Path(workintPath, UUIDUtils.random());
+    mkdirs(subDir);
+    Assert.assertTrue(fs.getFileStatus(subDir).isDirectory());
+
+    // can get FileStatus of working dir.
+    Assert.assertTrue(fs.getFileStatus(workintPath).isDirectory());
+    // delete sub directory.
+    fs.delete(subDir, true);
+    // still cat get FileStatus of working dir.
+    Assert.assertTrue(fs.getFileStatus(workintPath).isDirectory());
+  }
+
+  @Test
+  public void testPathStatDirNotExistButSubFileExist() throws IOException {
+    FileSystem fs = getFileSystem();
+    // working dir does not exist.
+    Path workintPath = new Path(getContract().getTestPath(), this.methodName.getMethodName());
+    assertThrows("Path doesn't exist", FileNotFoundException.class, () -> fs.getFileStatus(workintPath));
+
+    // create sub file in working dir directly.
+    Path file = workintPath.suffix('/' + UUIDUtils.random());
+    touch(fs, file);
+
+    // can get FileStatus of working dir.
+    Assert.assertTrue(fs.getFileStatus(workintPath).isDirectory());
+
+    // delete sub file, will create parent directory.
+    fs.delete(file, false);
+    Assert.assertTrue(fs.getFileStatus(workintPath).isDirectory());
+  }
+}

+ 30 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestMkdir.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.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestMkdir extends AbstractContractMkdirTest {
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new TosContract(conf);
+  }
+}

+ 65 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestOpen.java

@@ -0,0 +1,65 @@
+/*
+ * ByteDance Volcengine EMR, Copyright 2022.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.tosfs.RawFileStatus;
+import org.apache.hadoop.fs.tosfs.RawFileSystem;
+import org.apache.hadoop.fs.tosfs.object.exceptions.ChecksumMismatchException;
+import org.apache.hadoop.fs.tosfs.util.Range;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+
+public class TestOpen extends AbstractContractOpenTest {
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new TosContract(conf);
+  }
+
+  @Test
+  public void testOpenAExpiredFile() throws IOException {
+    Path file = path("testOpenAOutageFile");
+    FileSystem fs = getFileSystem();
+    byte[] data = dataset(256, 'a', 'z');
+    writeDataset(fs, file, data, data.length, 1024 * 1024, true);
+
+    FileStatus fileStatus = fs.getFileStatus(file);
+    if (fs instanceof RawFileSystem) {
+      byte[] expectChecksum = ((RawFileStatus) fileStatus).checksum();
+      FSDataInputStream fsDataInputStream =
+          ((RawFileSystem) fs).open(file, expectChecksum, Range.of(0, Long.MAX_VALUE));
+      fsDataInputStream.close();
+
+      // update the file
+      data = dataset(512, 'a', 'z');
+      writeDataset(fs, file, data, data.length, 1024 * 1024, true);
+
+      FSDataInputStream newStream = ((RawFileSystem) fs).open(file, expectChecksum, Range.of(0, Long.MAX_VALUE));
+      assertThrows("the file is expired", ChecksumMismatchException.class, () -> newStream.read());
+    }
+  }
+}

+ 270 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestRename.java

@@ -0,0 +1,270 @@
+/*
+ * ByteDance Volcengine EMR, Copyright 2022.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.contract;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.tosfs.conf.ConfKeys;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+
+public class TestRename extends AbstractContractRenameTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    // Add follow two keys into hadoop configuration.
+    String defaultScheme = FileSystem.getDefaultUri(conf).getScheme();
+    Configuration newConf = new Configuration(conf);
+    newConf.setLong(ConfKeys.MULTIPART_SIZE.key(defaultScheme), ConfKeys.MULTIPART_SIZE_DEFAULT);
+    newConf.setLong(ConfKeys.MULTIPART_THRESHOLD.key(defaultScheme),
+        ConfKeys.MULTIPART_THRESHOLD_DEFAULT);
+
+    return new TosContract(newConf);
+  }
+
+  @Test
+  public void testSucceedRenameFile() throws IOException {
+    describe("check if source file and dest file exists when succeed to rename");
+    Path renameSrc = path("renameSrc");
+    Path renameDest = path("renameDst");
+    FileSystem fs = getFileSystem();
+    byte[] data = dataset(256, 'a', 'z');
+    writeDataset(fs, renameSrc, data, data.length, 1024 * 1024, true);
+    boolean renamed = rename(renameSrc, renameDest);
+    assertTrue(renamed);
+    assertPathExists("dest file should exist when succeed to rename", renameDest);
+    assertPathDoesNotExist("source file should not exist when succeed to rename", renameSrc);
+  }
+
+  @Test
+  public void testSucceedRenameDir() throws IOException {
+    describe("check if source dir and dest dir exists when succeed to rename");
+    Path renameSrc = path("renameSrc");
+    Path renameDest = path("renameDst");
+    int fileNums = 10;
+    int byteSize = 10 << 20; // trigger multipart upload
+    FileSystem fs = getFileSystem();
+    for (int i = 0; i < fileNums; i++) {
+      byte[] data = dataset(byteSize >> i, 'a', 'z');
+      writeDataset(fs, new Path(renameSrc, String.format("src%02d", i)), data, data.length, 1024 * 1024, true);
+    }
+    boolean renamed = rename(renameSrc, renameDest);
+    assertTrue(renamed);
+    for (int i = 0; i < fileNums; i++) {
+      Path srcFilePath = new Path(renameSrc, String.format("src%02d", i));
+      Path dstFilePath = new Path(renameDest, String.format("src%02d", i));
+      byte[] data = dataset(byteSize >> i, 'a', 'z');
+      assertPathExists("dest file should exist when succeed to rename", dstFilePath);
+      assertPathDoesNotExist("source file should not exist when succeed to rename", srcFilePath);
+      try (InputStream is = fs.open(dstFilePath)) {
+        assertArrayEquals(data, IOUtils.toByteArray(is));
+      }
+    }
+  }
+
+  @Test
+  public void testFailedRename() throws IOException {
+    describe("check if source file and dest file exists when failed to rename");
+    Path renameSrc = path("src/renameSrc");
+    Path renameDest = path("src/renameSrc/renameDst");
+    FileSystem fs = getFileSystem();
+    byte[] data = dataset(256, 'a', 'z');
+    writeDataset(fs, renameSrc, data, data.length, 1024 * 1024, true);
+    boolean renamed;
+    try {
+      renamed = rename(renameSrc, renameDest);
+    } catch (IOException e) {
+      renamed = false;
+    }
+    assertFalse(renamed);
+    assertPathExists("source file should exist when failed to rename", renameSrc);
+    assertPathDoesNotExist("dest file should not exist when failed to rename", renameDest);
+  }
+
+  @Test
+  public void testRenameSmallFile() throws IOException {
+    testRenameFileByPut(1 << 20);
+    testRenameFileByPut(3 << 20);
+  }
+
+  @Test
+  public void testRenameLargeFile() throws IOException {
+    testRenameFileByUploadParts(16 << 20);
+    testRenameFileByUploadParts(10 << 20);
+  }
+
+  @Test
+  public void testRenameDirWithSubFileAndSubDir() throws IOException {
+    FileSystem fs = getFileSystem();
+
+    Path renameSrc = path("dir/renameSrc");
+    Path renameDest = path("dir/renameDst");
+    int size = 1024;
+    byte[] data = dataset(size, 'a', 'z');
+    String fileName = "file.txt";
+    writeDataset(fs, new Path(renameSrc, fileName), data, data.length, 1024, true);
+
+    String dirName = "dir";
+    Path dirPath = new Path(renameSrc, dirName);
+    mkdirs(dirPath);
+    assertPathExists("source dir should exist", dirPath);
+
+    boolean renamed = fs.rename(renameSrc, renameDest);
+
+    assertTrue(renamed);
+    Path srcFilePath = new Path(renameSrc, fileName);
+    Path dstFilePath = new Path(renameDest, fileName);
+    assertPathExists("dest file should exist when succeed to rename", dstFilePath);
+    assertPathDoesNotExist("source file should not exist when succeed to rename", srcFilePath);
+
+    assertPathExists("dest dir should exist when succeed to rename", new Path(renameDest, dirName));
+    assertPathDoesNotExist("source dir should not exist when succeed to rename", new Path(renameSrc, dirName));
+
+    ContractTestUtils.cleanup("TEARDOWN", fs, getContract().getTestPath());
+  }
+
+  public void testRenameFileByPut(int size) throws IOException {
+    describe("check if use put method when rename file");
+    Path renameSrc = path("renameSrc");
+    Path renameDest = path("renameDst");
+    FileSystem fs = getFileSystem();
+
+    byte[] data = dataset(size, 'a', 'z');
+    String fileName = String.format("%sMB.txt", size >> 20);
+    writeDataset(fs, new Path(renameSrc, fileName), data, data.length, 1024 * 1024, true);
+    boolean renamed = fs.rename(renameSrc, renameDest);
+
+    assertTrue(renamed);
+    Path srcFilePath = new Path(renameSrc, fileName);
+    Path dstFilePath = new Path(renameDest, fileName);
+    assertPathExists("dest file should exist when succeed to rename", dstFilePath);
+    assertPathDoesNotExist("source file should not exist when succeed to rename", srcFilePath);
+
+    assertPathExists("dest src should exist when succeed to rename", renameDest);
+    assertPathDoesNotExist("source src should not exist when succeed to rename", renameSrc);
+
+    try (InputStream is = fs.open(dstFilePath)) {
+      assertArrayEquals(data, IOUtils.toByteArray(is));
+    }
+    ContractTestUtils.cleanup("TEARDOWN", fs, getContract().getTestPath());
+  }
+
+  @Test
+  public void testCreateParentDirAfterRenameSubFile() throws IOException {
+    FileSystem fs = getFileSystem();
+
+    Path srcDir = path("srcDir");
+    Path destDir = path("destDir");
+
+    assertPathDoesNotExist("Src dir should not exist", srcDir);
+    assertPathDoesNotExist("Dest dir should not exist", destDir);
+    int size = 1 << 20;
+    byte[] data = dataset(size, 'a', 'z');
+    String fileName = String.format("%sMB.txt", size >> 20);
+    Path srcFile = new Path(srcDir, fileName);
+    Path destFile = new Path(destDir, fileName);
+    writeDataset(fs, srcFile, data, data.length, 1024 * 1024, true);
+
+    assertPathExists("Src file should exist", srcFile);
+    assertPathExists("Src dir should exist", srcDir);
+
+    mkdirs(destDir);
+    assertPathExists("Dest dir should exist", destDir);
+
+    boolean renamed = fs.rename(srcFile, destFile);
+    assertTrue(renamed);
+
+    assertPathExists("Dest file should exist", destFile);
+    assertPathExists("Dest dir should exist", destDir);
+    assertPathDoesNotExist("Src file should not exist", srcFile);
+    assertPathExists("Src dir should exist", srcDir);
+  }
+
+  @Test
+  public void testCreateParentDirAfterRenameSubDir() throws IOException {
+    FileSystem fs = getFileSystem();
+
+    Path srcDir = path("srcDir");
+    Path destDir = path("destDir");
+
+    assertPathDoesNotExist("Src dir should not exist", srcDir);
+    assertPathDoesNotExist("Dest dir should not exist", destDir);
+
+    String subDirName = String.format("subDir");
+    Path srcSubDir = new Path(srcDir, subDirName);
+    Path destDestDir = new Path(destDir, subDirName);
+    mkdirs(srcSubDir);
+
+    assertPathExists("Src sub dir should exist", srcSubDir);
+    assertPathExists("Src dir should exist", srcDir);
+
+    mkdirs(destDir);
+    assertPathExists("Dest dir should exist", destDir);
+
+    boolean renamed = fs.rename(srcSubDir, destDestDir);
+    assertTrue(renamed);
+
+    assertPathExists("Dest sub dir should exist", destDestDir);
+    assertPathExists("Dest dir should exist", destDir);
+    assertPathDoesNotExist("Src sub dir should not exist", srcSubDir);
+    assertPathExists("Src sub dir should exist", srcDir);
+  }
+
+  public void testRenameFileByUploadParts(int size) throws IOException {
+    describe("check if use upload parts method when rename file");
+    Path renameSrc = path("renameSrc");
+    Path renameDest = path("renameDst");
+    FileSystem fs = getFileSystem();
+
+    byte[] data = dataset(size, 'a', 'z');
+    String fileName = String.format("%sMB.txt", size >> 20);
+    writeDataset(fs, new Path(renameSrc, fileName), data, data.length, 1024 * 1024, true);
+    boolean renamed = fs.rename(renameSrc, renameDest);
+
+    assertTrue(renamed);
+    Path srcFilePath = new Path(renameSrc, fileName);
+    Path dstFilePath = new Path(renameDest, fileName);
+    assertPathExists("dest file should exist when succeed to rename", dstFilePath);
+    assertPathDoesNotExist("source file should not exist when succeed to rename", srcFilePath);
+
+    try (InputStream is = fs.open(dstFilePath)) {
+      assertArrayEquals(data, IOUtils.toByteArray(is));
+    }
+    ContractTestUtils.cleanup("TEARDOWN", fs, getContract().getTestPath());
+  }
+
+  @Ignore
+  @Test
+  public void testRenameFileUnderFileSubdir() {
+  }
+
+  @Ignore
+  @Test
+  public void testRenameFileUnderFile() {
+  }
+}

+ 28 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestRootDir.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.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestRootDir extends AbstractContractRootDirectoryTest {
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new TosContract(conf);
+  }
+}

+ 28 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestSeek.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.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestSeek extends AbstractContractSeekTest {
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new TosContract(conf);
+  }
+}

+ 28 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestUnbuffer.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.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractUnbufferTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestUnbuffer extends AbstractContractUnbufferTest {
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new TosContract(conf);
+  }
+}

+ 168 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TestXAttr.java

@@ -0,0 +1,168 @@
+/*
+ * ByteDance Volcengine EMR, Copyright 2022.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
+import org.apache.hadoop.fs.tosfs.common.Bytes;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+
+public class TestXAttr extends AbstractFSContractTestBase {
+  private static final String XATTR_NAME = "xAttrName";
+  private static final byte[] XATTR_VALUE = "xAttrValue".getBytes();
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new TosContract(conf);
+  }
+
+  @Test
+  public void testGetNonExistedXAttr() throws Exception {
+    FileSystem fs = getFileSystem();
+    Path path = path("testSetAndGet/file");
+    fs.create(path).close();
+
+    fs.setXAttr(path, XATTR_NAME, XATTR_VALUE);
+    assertThrows("Not found.", IOException.class,
+        () -> fs.getXAttr(path, "non-exist"));
+    assertThrows("Not found.", IOException.class,
+        () -> fs.getXAttrs(path, Arrays.asList("non-exist")));
+    assertThrows("Not found.", IOException.class,
+        () -> fs.getXAttrs(path, Arrays.asList("non-exist", XATTR_NAME)));
+  }
+
+  @Test
+  public void testSetAndGetWhenPathNotExist() throws Exception {
+    FileSystem fs = getFileSystem();
+    Path path = path("testXAttrWhenPathNotExist/file");
+    fs.delete(path);
+
+    assertThrows("No such file", FileNotFoundException.class,
+        () -> fs.setXAttr(path, XATTR_NAME, XATTR_VALUE));
+    assertThrows("No such file", FileNotFoundException.class,
+        () -> fs.getXAttrs(path));
+    assertThrows("No such file", FileNotFoundException.class,
+        () -> fs.removeXAttr(path, "name"));
+  }
+
+  @Test
+  public void testSetAndGet() throws Exception {
+    FileSystem fs = getFileSystem();
+    Path path = path("testSetAndGet/file");
+    fs.create(path).close();
+
+    fs.setXAttr(path, XATTR_NAME, XATTR_VALUE);
+    assertArrayEquals(XATTR_VALUE, fs.getXAttr(path, XATTR_NAME));
+  }
+
+  @Test
+  public void testSetAndGetNonExistedObject() throws Exception {
+    FileSystem fs = getFileSystem();
+    Path path = path("testSetAndGetOnNonExistedObject/dir-0/dir-1/file");
+    fs.create(path).close();
+
+    Path nonExistedPath = path.getParent().getParent();
+    fs.setXAttr(nonExistedPath, XATTR_NAME, XATTR_VALUE);
+    assertThrows("Not found.", IOException.class,
+        () -> fs.getXAttr(nonExistedPath, XATTR_NAME));
+  }
+
+  @Test
+  public void testSetAndGetOnExistedObjectDir() throws Exception {
+    FileSystem fs = getFileSystem();
+    Path path = path("testSetAndGetOnDir/dir-0/dir-1");
+    fs.mkdirs(path);
+
+    fs.setXAttr(path, XATTR_NAME, XATTR_VALUE);
+    assertThrows("Not found.", IOException.class,
+        () -> fs.getXAttr(path, XATTR_NAME));
+  }
+
+  @Test
+  public void testGetAndListAll() throws Exception {
+    FileSystem fs = getFileSystem();
+    Path path = path("testGetAndListAll/file");
+    fs.create(path).close();
+
+    int size = 10;
+    for (int i = 0; i < size; i++) {
+      fs.setXAttr(path, XATTR_NAME + i, Bytes.toBytes("VALUE" + i));
+    }
+
+    Map<String, byte[]> result = fs.getXAttrs(path);
+    assertEquals(size, result.size());
+    for (int i = 0; i < size; i++) {
+      assertEquals("VALUE" + i, Bytes.toString(result.get(XATTR_NAME + i)));
+    }
+
+    List<String> names = fs.listXAttrs(path);
+    assertEquals(size, names.size());
+    for (int i = 0; i < size; i++) {
+      assertTrue(names.contains(XATTR_NAME + i));
+    }
+  }
+
+  @Test
+  public void testRemove() throws Exception {
+    FileSystem fs = getFileSystem();
+    Path path = path("testRemove/file");
+    fs.create(path).close();
+
+    int size = 10;
+    for (int i = 0; i < size; i++) {
+      fs.setXAttr(path, XATTR_NAME + i, Bytes.toBytes("VALUE" + i));
+    }
+
+    for (int i = 0; i < size; i++) {
+      fs.removeXAttr(path, XATTR_NAME + i);
+      String name = XATTR_NAME + i;
+      assertThrows("Not found.", IOException.class,
+          () -> fs.getXAttr(path, name));
+      assertEquals(size - 1 - i, fs.listXAttrs(path).size());
+    }
+  }
+
+  @Test
+  public void testXAttrFlag() throws Exception {
+    FileSystem fs = getFileSystem();
+    Path path = path("testXAttrFlag/file");
+    fs.create(path).close();
+
+    String key = XATTR_NAME;
+    byte[] value = XATTR_VALUE;
+    assertThrows("The CREATE flag must be specified", IOException.class,
+        () -> fs.setXAttr(path, key, value, EnumSet.of(XAttrSetFlag.REPLACE)));
+    fs.setXAttr(path, key, value, EnumSet.of(XAttrSetFlag.CREATE));
+    assertArrayEquals(value, fs.getXAttr(path, key));
+
+    byte[] newValue = Bytes.toBytes("new value");
+    assertThrows("The REPLACE flag must be specified", IOException.class,
+        () -> fs.setXAttr(path, key, newValue, EnumSet.of(XAttrSetFlag.CREATE)));
+    fs.setXAttr(path, key, newValue, EnumSet.of(XAttrSetFlag.REPLACE));
+    assertArrayEquals(newValue, fs.getXAttr(path, key));
+  }
+}

+ 53 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/contract/TosContract.java

@@ -0,0 +1,53 @@
+/*
+ * ByteDance Volcengine EMR, Copyright 2022.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.contract;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
+import org.apache.hadoop.fs.tosfs.util.TestUtility;
+import org.apache.hadoop.fs.tosfs.util.UUIDUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TosContract extends AbstractBondedFSContract {
+  private static final Logger LOG = LoggerFactory.getLogger(TosContract.class);
+  private final String testDir;
+
+  public TosContract(Configuration conf) {
+    super(conf);
+    addConfResource("contract/tos.xml");
+    // Set the correct contract test path if there is a provided bucket name from environment.
+    if (StringUtils.isNoneEmpty(TestUtility.bucket())) {
+      conf.set("fs.contract.test.fs.tos", String.format("tos://%s/", TestUtility.bucket()));
+    }
+
+    testDir = "/test-" + UUIDUtils.random();
+  }
+
+  @Override
+  public String getScheme() {
+    return "tos";
+  }
+
+  @Override
+  public Path getTestPath() {
+    LOG.info("the test dir is: {}", testDir);
+    return new Path(testDir);
+  }
+}

+ 213 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestDirectoryStorage.java

@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.conf.TosKeys;
+import org.apache.hadoop.fs.tosfs.util.CommonUtils;
+import org.apache.hadoop.fs.tosfs.util.TestUtility;
+import org.apache.hadoop.fs.tosfs.util.UUIDUtils;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import static org.apache.hadoop.fs.tosfs.util.TestUtility.scheme;
+
+public class TestDirectoryStorage {
+  private final ObjectStorage storage;
+
+  public TestDirectoryStorage() {
+    Configuration conf = new Configuration();
+    storage =
+        ObjectStorageFactory.createWithPrefix(String.format("%s-%s/", scheme(), UUIDUtils.random()),
+            scheme(), TestUtility.bucket(), conf);
+  }
+
+  @After
+  public void tearDown() {
+    CommonUtils.runQuietly(() -> storage.deleteAll(""));
+    for (MultipartUpload upload : storage.listUploads("")) {
+      storage.abortMultipartUpload(upload.key(), upload.uploadId());
+    }
+  }
+
+  @Test
+  public void testListEmptyDir() {
+    String key = "testListEmptyDir/";
+    mkdir(key);
+    Assert.assertNotNull(directoryStorage().head(key));
+
+    Assert.assertFalse(directoryStorage().listDir(key, false).iterator().hasNext());
+    Assert.assertFalse(directoryStorage().listDir(key, false).iterator().hasNext());
+    Assert.assertTrue(directoryStorage().isEmptyDir(key));
+  }
+
+  @Test
+  public void testListNonExistDir() {
+    String key = "testListNonExistDir/";
+    Assert.assertNull(directoryStorage().head(key));
+
+    Assert.assertFalse(directoryStorage().listDir(key, false).iterator().hasNext());
+    Assert.assertFalse(directoryStorage().listDir(key, false).iterator().hasNext());
+    Assert.assertTrue(directoryStorage().isEmptyDir(key));
+  }
+
+  @Test
+  public void testRecursiveList() {
+    String root = "root/";
+    String file1 = "root/file1";
+    String file2 = "root/afile2";
+    String dir1 = "root/dir1/";
+    String file3 = "root/dir1/file3";
+
+    mkdir(root);
+    mkdir(dir1);
+    touchFile(file1, TestUtility.rand(8));
+    touchFile(file2, TestUtility.rand(8));
+    touchFile(file3, TestUtility.rand(8));
+
+    Assertions.assertThat(directoryStorage().listDir(root, false))
+        .hasSize(3)
+        .extracting(ObjectInfo::key)
+        .contains(dir1, file1, file2);
+
+    Assertions.assertThat(directoryStorage().listDir(root, true))
+        .hasSize(4)
+        .extracting(ObjectInfo::key)
+        .contains(dir1, file1, file2, file3);
+  }
+
+  @Test
+  public void testRecursiveListWithSmallBatch() {
+    Configuration conf = new Configuration(directoryStorage().conf());
+    conf.setInt(TosKeys.FS_TOS_LIST_OBJECTS_COUNT, 5);
+    directoryStorage().initialize(conf, directoryStorage().bucket().name());
+
+    String root = "root/";
+    mkdir(root);
+
+    // Create 2 files start with 'a', 2 sub dirs start with 'b', 2 files start with 'c'
+    for (int i = 1; i <= 2; i++) {
+      touchFile("root/a-file-" + i, TestUtility.rand(8));
+      mkdir("root/b-dir-" + i + "/");
+      touchFile("root/c-file-" + i, TestUtility.rand(8));
+    }
+
+    // Create two files under each sub dirs.
+    for (int j = 1; j <= 2; j++) {
+      touchFile(String.format("root/b-dir-%d/file1", j), TestUtility.rand(8));
+      touchFile(String.format("root/b-dir-%d/file2", j), TestUtility.rand(8));
+    }
+
+    Assertions.assertThat(directoryStorage().listDir(root, false))
+        .hasSize(6)
+        .extracting(ObjectInfo::key)
+        .contains(
+            "root/a-file-1", "root/a-file-2",
+            "root/b-dir-1/", "root/b-dir-2/",
+            "root/c-file-1", "root/c-file-2");
+
+    Assertions.assertThat(directoryStorage().listDir(root, true))
+        .hasSize(10)
+        .extracting(ObjectInfo::key)
+        .contains(
+            "root/a-file-1", "root/a-file-2",
+            "root/b-dir-1/", "root/b-dir-1/file1", "root/b-dir-1/file2",
+            "root/b-dir-2/", "root/b-dir-2/file1", "root/b-dir-2/file2",
+            "root/c-file-1", "root/c-file-2");
+  }
+
+  @Test
+  public void testRecursiveListRoot() {
+    String root = "root/";
+    String dir1 = "root/dir1/";
+    mkdir(root);
+    mkdir(dir1);
+
+    Assertions.assertThat(directoryStorage().listDir("", true))
+        .hasSize(2)
+        .extracting(ObjectInfo::key)
+        .contains("root/", "root/dir1/");
+  }
+
+  @Test
+  public void testDeleteEmptyDir() {
+    String dir = "a/b/";
+    mkdir(dir);
+
+    directoryStorage().deleteDir(dir, false);
+    Assert.assertNull(directoryStorage().head(dir));
+  }
+
+  @Test
+  public void testDeleteNonEmptyDir() {
+    String dir = "a/b/";
+    String subDir = "a/b/c/";
+    String file = "a/b/file.txt";
+    mkdir(dir);
+    mkdir(subDir);
+    touchFile(file, new byte[10]);
+
+    Assert.assertThrows(RuntimeException.class, () -> directoryStorage().deleteDir(dir, false));
+    Assert.assertNotNull(directoryStorage().head(dir));
+    Assert.assertNotNull(directoryStorage().head(subDir));
+    Assert.assertNotNull(directoryStorage().head(file));
+
+    directoryStorage().deleteDir(dir, true);
+    Assert.assertNull(directoryStorage().head(dir));
+    Assert.assertNull(directoryStorage().head(subDir));
+    Assert.assertNull(directoryStorage().head(file));
+  }
+
+  @Test
+  public void testRecursiveDeleteDirViaTosSDK() {
+    Configuration conf = new Configuration(directoryStorage().conf());
+    conf.setBoolean(TosKeys.FS_TOS_RMR_CLIENT_ENABLE, true);
+    directoryStorage().initialize(conf, directoryStorage().bucket().name());
+
+    testDeleteNonEmptyDir();
+  }
+
+  // TOS doesn't enable recursive delete in server side currently.
+  @Ignore
+  @Test
+  public void testAtomicDeleteDir() {
+    Configuration conf = new Configuration(directoryStorage().conf());
+    conf.setBoolean(TosKeys.FS_TOS_RMR_SERVER_ENABLED, true);
+    directoryStorage().initialize(conf, directoryStorage().bucket().name());
+
+    testDeleteNonEmptyDir();
+  }
+
+  private void touchFile(String key, byte[] data) {
+    directoryStorage().put(key, data);
+  }
+
+  private void mkdir(String key) {
+    directoryStorage().put(key, new byte[0]);
+  }
+
+  private DirectoryStorage directoryStorage() {
+    Assume.assumeTrue(storage.bucket().isDirectory());
+    return (DirectoryStorage) storage;
+  }
+}

+ 4 - 4
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectOutputStream.java

@@ -172,7 +172,7 @@ public class TestObjectOutputStream extends ObjectStorageTestBase {
 
   public void testWrite(int uploadPartSize, int len) throws IOException {
     Configuration newConf = new Configuration(protonConf);
-    newConf.setLong(ConfKeys.MULTIPART_SIZE.format(FSUtils.scheme(conf, testDir.toUri())),
+    newConf.setLong(ConfKeys.MULTIPART_SIZE.key(FSUtils.scheme(conf, testDir.toUri())),
         uploadPartSize);
 
     Path outPath = path(len + ".txt");
@@ -208,7 +208,7 @@ public class TestObjectOutputStream extends ObjectStorageTestBase {
   public void testParallelWriteOneOutPutStreamImpl(int partSize, int epochs, int batchSize)
       throws IOException, ExecutionException, InterruptedException {
     Configuration newConf = new Configuration(protonConf);
-    newConf.setLong(ConfKeys.MULTIPART_SIZE.format(FSUtils.scheme(conf, testDir.toUri())),
+    newConf.setLong(ConfKeys.MULTIPART_SIZE.key(FSUtils.scheme(conf, testDir.toUri())),
         partSize);
 
     String file = String.format("%d-%d-%d-testParallelWriteOneOutPutStream.txt", partSize, epochs, batchSize);
@@ -284,8 +284,8 @@ public class TestObjectOutputStream extends ObjectStorageTestBase {
 
   private void testMultipartThreshold(int partSize, int multipartThreshold, int dataSize) throws IOException {
     Configuration newConf = new Configuration(protonConf);
-    newConf.setLong(ConfKeys.MULTIPART_SIZE.format(scheme), partSize);
-    newConf.setLong(ConfKeys.MULTIPART_THRESHOLD.format(scheme), multipartThreshold);
+    newConf.setLong(ConfKeys.MULTIPART_SIZE.key(scheme), partSize);
+    newConf.setLong(ConfKeys.MULTIPART_THRESHOLD.key(scheme), multipartThreshold);
     Path outPath = path(String.format("threshold-%d-%d-%d.txt", partSize, multipartThreshold, dataSize));
 
     byte[] data = TestUtility.rand(dataSize);

+ 1407 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/TestObjectStorage.java

@@ -0,0 +1,1407 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object;
+
+import com.volcengine.tos.TosServerException;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.tosfs.conf.TosKeys;
+import org.apache.hadoop.fs.tosfs.object.exceptions.InvalidObjectKeyException;
+import org.apache.hadoop.fs.tosfs.object.exceptions.NotAppendableException;
+import org.apache.hadoop.fs.tosfs.object.request.ListObjectsRequest;
+import org.apache.hadoop.fs.tosfs.object.response.ListObjectsResponse;
+import org.apache.hadoop.fs.tosfs.util.CommonUtils;
+import org.apache.hadoop.fs.tosfs.util.TempFiles;
+import org.apache.hadoop.fs.tosfs.util.TestUtility;
+import org.apache.hadoop.fs.tosfs.util.UUIDUtils;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Maps;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.StreamSupport;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class TestObjectStorage {
+  private static final String FILE_STORE_ROOT = TempFiles.newTempDir("TestObjectStorage");
+  private final ObjectStorage storage;
+
+  public TestObjectStorage(ObjectStorage storage) {
+    this.storage = storage;
+  }
+
+  @Parameterized.Parameters(name = "ObjectStorage = {0}")
+  public static List<ObjectStorage> createStorage() {
+    return TestUtility.createTestObjectStorage(FILE_STORE_ROOT);
+  }
+
+  @After
+  public void tearDown() {
+    CommonUtils.runQuietly(() -> storage.deleteAll(""));
+    for (MultipartUpload upload : storage.listUploads("")) {
+      storage.abortMultipartUpload(upload.key(), upload.uploadId());
+    }
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    CommonUtils.runQuietly(() -> TempFiles.deleteDir(FILE_STORE_ROOT));
+  }
+
+  @Test
+  public void testHeadNonExistObject() {
+    assertNull(storage.head("a/b/c.txt"));
+
+    byte[] data = TestUtility.rand(256);
+    storage.put("a/b/c.txt", data);
+    assertNotNull(storage.head("a/b/c.txt"));
+
+    assertNull(storage.head("a/b/c/non-exits"));
+    if (storage.bucket().isDirectory()) {
+      assertThrows(InvalidObjectKeyException.class, () -> storage.head("a/b/c.txt/non-exits"));
+    } else {
+      assertNull(storage.head("a/b/c.txt/non-exits"));
+    }
+  }
+
+  @Test
+  public void testHeadExistObject() {
+    byte[] data = TestUtility.rand(256);
+    String key = "testHeadExistObject.txt";
+    storage.put(key, data);
+
+    ObjectInfo obj = storage.head(key);
+    Assert.assertEquals(key, obj.key());
+    Assert.assertFalse(obj.isDir());
+    if (storage.bucket().isDirectory()) {
+      assertThrows(InvalidObjectKeyException.class, () -> storage.head(key + "/"));
+    } else {
+      Assert.assertNull(storage.head(key + "/"));
+    }
+
+    String dirKey = "testHeadExistObject/";
+    storage.put(dirKey, new byte[0]);
+    obj = storage.head(dirKey);
+    Assert.assertEquals(dirKey, obj.key());
+    Assert.assertTrue(obj.isDir());
+
+    if (storage.bucket().isDirectory()) {
+      obj = storage.head("testHeadExistObject");
+      Assert.assertEquals("testHeadExistObject", obj.key());
+      Assert.assertTrue(obj.isDir());
+    } else {
+      Assert.assertNull(storage.head("testHeadExistObject"));
+    }
+  }
+
+  @Test
+  public void testGetAndDeleteNonExistFile() {
+    // ensure file is not exist
+    assertNull(storage.head("a/b/c.txt"));
+
+    assertThrows(RuntimeException.class, () -> storage.get("a/b/c.txt", 0, 0));
+    assertThrows(RuntimeException.class, () -> storage.get("a/b/c.txt", 0, 1));
+
+    // Allow to delete a non-exist object.
+    storage.delete("a/b/c.txt");
+  }
+
+  @Test
+  public void testPutAndDeleteFileWithEmptyKey() {
+    assertThrows(RuntimeException.class, () -> storage.put("", new byte[0]));
+    assertThrows(RuntimeException.class, () -> storage.put(null, new byte[0]));
+    assertThrows(RuntimeException.class, () -> storage.delete(null));
+    assertThrows(RuntimeException.class, () -> storage.head(""));
+    assertThrows(RuntimeException.class, () -> storage.head(null));
+    assertThrows(RuntimeException.class, () -> getStream(""));
+    assertThrows(RuntimeException.class, () -> getStream(null));
+  }
+
+  @Test
+  public void testPutObjectButContentLengthDisMatch() throws IOException {
+    byte[] data = TestUtility.rand(256);
+    String key = "a/truncated.txt";
+
+    // The final object data will be truncated if content length is smaller.
+    byte[] checksum = storage.put(key, () -> new ByteArrayInputStream(data), 200);
+    assertArrayEquals(Arrays.copyOfRange(data, 0, 200), IOUtils.toByteArray(getStream(key)));
+    ObjectInfo info = storage.head(key);
+    assertEquals(key, info.key());
+    assertEquals(200, info.size());
+    assertArrayEquals(checksum, info.checksum());
+
+    // Will create object failed is the content length is bigger.
+    assertThrows(RuntimeException.class, () -> storage.put(key, () -> new ByteArrayInputStream(data), 300));
+  }
+
+  private InputStream getStream(String key) {
+    return storage.get(key).stream();
+  }
+
+  @Test
+  public void testPutAndGetFile() throws IOException {
+    byte[] data = TestUtility.rand(256);
+    String key = "a/test.txt";
+    byte[] checksum = storage.put(key, data);
+    assertArrayEquals(data, IOUtils.toByteArray(getStream(key)));
+
+    if (storage.bucket().isDirectory()) {
+      // Directory bucket will create missed parent dir.
+      assertArrayEquals(new byte[0], IOUtils.toByteArray(getStream("a")));
+      assertArrayEquals(new byte[0], IOUtils.toByteArray(getStream("a/")));
+    } else {
+      assertNull(storage.head("a"));
+      assertNull(storage.head("a/"));
+    }
+
+    ObjectInfo info = storage.head(key);
+    assertEquals(key, info.key());
+    assertEquals(data.length, info.size());
+    assertArrayEquals(checksum, info.checksum());
+
+    ObjectContent content = storage.get(key);
+    assertArrayEquals(info.checksum(), content.checksum());
+    assertArrayEquals(data, IOUtils.toByteArray(content.stream()));
+
+    assertArrayEquals(data, IOUtils.toByteArray(getStream(key, 0, -1)));
+    assertThrows("offset is negative", RuntimeException.class, () -> storage.get(key, -1, -1));
+    assertThrows("path not found or resource type is invalid",
+        RuntimeException.class, () -> storage.get(key + "/", 0, -1));
+
+    assertArrayEquals(data, IOUtils.toByteArray(getStream(key, 0, 256)));
+    assertArrayEquals(data, IOUtils.toByteArray(getStream(key, 0, 512)));
+
+    byte[] secondHalfData = Arrays.copyOfRange(data, 128, 256);
+    assertArrayEquals(secondHalfData, IOUtils.toByteArray(getStream(key, 128, -1)));
+    assertArrayEquals(secondHalfData, IOUtils.toByteArray(getStream(key, 128, 256)));
+    assertArrayEquals(secondHalfData, IOUtils.toByteArray(getStream(key, 128, 257)));
+    assertArrayEquals(new byte[0], IOUtils.toByteArray(getStream(key, 128, 0)));
+
+    ObjectContent partContent = storage.get(key, 8, 32);
+    assertArrayEquals(info.checksum(), partContent.checksum());
+    assertArrayEquals(Arrays.copyOfRange(data, 8, 40),
+        IOUtils.toByteArray(partContent.stream()));
+    assertArrayEquals(data, IOUtils.toByteArray(getStream(key)));
+
+    assertThrows("offset is bigger than object length", RuntimeException.class, () -> storage.get(key, 257, 8));
+    assertArrayEquals(new byte[0], IOUtils.toByteArray(getStream(key, 256, 8)));
+
+    assertArrayEquals(new byte[0], IOUtils.toByteArray(getStream(key, 0, 0)));
+    assertArrayEquals(new byte[0], IOUtils.toByteArray(getStream(key, 1, 0)));
+
+
+    // assert the original data is not changed during random get request
+    assertArrayEquals(data, IOUtils.toByteArray(getStream(key)));
+
+    storage.delete(key);
+    assertNull(storage.head(key));
+  }
+
+  @Test
+  public void testAppendAndGetFile() throws Exception {
+    String key = "a/testAppendAndGetFile.txt";
+
+    // Append zero bytes.
+    assertThrows("Append non-existed object with zero byte is not supported.",
+        NotAppendableException.class, () -> storage.append(key, new byte[0]));
+
+    // Append 256 bytes.
+    byte[] data = TestUtility.rand(256);
+    byte[] checksum = storage.append(key, data);
+    assertArrayEquals(data, IOUtils.toByteArray(getStream(key)));
+
+    // Append zero bytes.
+    byte[] newChecksum = storage.append(key, new byte[0]);
+    assertArrayEquals(checksum, newChecksum);
+    checksum = newChecksum;
+
+    // Append one byte.
+    newChecksum = storage.append(key, new byte[1]);
+    assertFalse(Arrays.equals(checksum, newChecksum));
+    assertArrayEquals(newChecksum, storage.head(key).checksum());
+    checksum = newChecksum;
+
+    // Append 1024 byte.
+    data = TestUtility.rand(1024);
+    newChecksum = storage.append(key, data);
+    assertFalse(Arrays.equals(checksum, newChecksum));
+    assertArrayEquals(newChecksum, storage.head(key).checksum());
+
+    storage.delete(key);
+  }
+
+  @Test
+  public void testAppendLengthNotMatch() {
+    byte[] data = TestUtility.rand(256);
+    String key = "a/testAppendLengthNotMatch.txt";
+    storage.append(key, () -> new ByteArrayInputStream(data), 128);
+    assertEquals(128, storage.head(key).size());
+
+    assertThrows("Expect unexpected end of stream error.", RuntimeException.class,
+        () -> storage.append(key, () -> new ByteArrayInputStream(data), 1024));
+  }
+
+  @Test
+  public void testHeadAndListAndObjectStatusShouldGetSameObjectInfo() {
+    String key = "testHeadAndListObjectCheckSum.txt";
+    byte[] data = TestUtility.rand(256);
+    byte[] checksum = storage.put(key, data);
+
+    ObjectInfo obj = storage.head(key);
+    assertEquals(obj, storage.objectStatus(key));
+    if (!storage.bucket().isDirectory()) {
+      List<ObjectInfo> objects = toList(storage.list(key, null, 1));
+      assertEquals(1, objects.size());
+      assertEquals(obj, objects.get(0));
+      assertArrayEquals(checksum, objects.get(0).checksum());
+    }
+
+
+    key = "testHeadAndListObjectCheckSum/";
+    checksum = storage.put(key, new byte[0]);
+    obj = storage.head(key);
+    assertEquals(obj, storage.objectStatus(key));
+    if (!storage.bucket().isDirectory()) {
+      List<ObjectInfo> objects = toList(storage.list(key, null, 1));
+      assertEquals(1, objects.size());
+      assertEquals(obj, objects.get(0));
+      assertArrayEquals(checksum, objects.get(0).checksum());
+    }
+  }
+
+  @Test
+  public void testObjectStatus() {
+    // test get file status
+    String key = "a/b/testObjectStatus.txt";
+    byte[] data = TestUtility.rand(256);
+    byte[] checksum = storage.put(key, data);
+
+    ObjectInfo obj = storage.head(key);
+    assertArrayEquals(checksum, obj.checksum());
+    Assert.assertEquals(obj, storage.objectStatus(key));
+
+    if (storage.bucket().isDirectory()) {
+      assertThrows(InvalidObjectKeyException.class, () -> storage.head(key + "/"));
+      assertThrows(InvalidObjectKeyException.class, () -> storage.objectStatus(key + "/"));
+    } else {
+      Assert.assertNull(storage.head(key + "/"));
+      Assert.assertNull(storage.objectStatus(key + "/"));
+    }
+
+    // test get dir status
+    String dirKey = "a/b/dir/";
+    checksum = storage.put(dirKey, new byte[0]);
+    obj = storage.head(dirKey);
+    Assert.assertEquals(Constants.MAGIC_CHECKSUM, checksum);
+    assertArrayEquals(Constants.MAGIC_CHECKSUM, checksum);
+    assertArrayEquals(checksum, obj.checksum());
+    Assert.assertTrue(obj.isDir());
+    Assert.assertEquals(dirKey, obj.key());
+    Assert.assertEquals(obj, storage.objectStatus(dirKey));
+
+    if (storage.bucket().isDirectory()) {
+      Assert.assertNotNull(storage.head("a/b/dir"));
+      Assert.assertEquals("a/b/dir", storage.objectStatus("a/b/dir").key());
+    } else {
+      Assert.assertNull(storage.head("a/b/dir"));
+      Assert.assertEquals(dirKey, storage.objectStatus("a/b/dir").key());
+    }
+
+    // test get dir status of prefix
+    String prefix = "a/b/";
+    obj = storage.objectStatus(prefix);
+    Assert.assertEquals(prefix, obj.key());
+    Assert.assertEquals(Constants.MAGIC_CHECKSUM, obj.checksum());
+    Assert.assertTrue(obj.isDir());
+
+    if (storage.bucket().isDirectory()) {
+      Assert.assertEquals(obj, storage.head(prefix));
+      Assert.assertEquals("a/b", storage.objectStatus("a/b").key());
+    } else {
+      Assert.assertNull(storage.head(prefix));
+      Assert.assertEquals(prefix, storage.objectStatus("a/b").key());
+    }
+  }
+
+  @Test
+  public void testPutAndGetDirectory() throws IOException {
+    String key = "a/b/";
+    byte[] data = new byte[0];
+    storage.put(key, data);
+
+    ObjectInfo info = storage.head(key);
+    assertEquals(key, info.key());
+    assertEquals(data.length, info.size());
+    assertArrayEquals(data, IOUtils.toByteArray(getStream(key)));
+    assertArrayEquals(data, IOUtils.toByteArray(getStream(key, 0, 256)));
+
+    // test create the same dir again
+    storage.put(key, data);
+
+    storage.delete(key);
+    assertNull(storage.head(key));
+  }
+
+  @Test
+  public void testOverwriteFile() throws IOException {
+    String key = "a/test.txt";
+    byte[] data1 = TestUtility.rand(256);
+    byte[] data2 = TestUtility.rand(128);
+
+    storage.put(key, data1);
+    assertArrayEquals(data1, IOUtils.toByteArray(getStream(key, 0, -1)));
+
+    storage.put(key, data2);
+    assertArrayEquals(data2, IOUtils.toByteArray(getStream(key, 0, -1)));
+
+    storage.delete(key);
+    assertNull(storage.head(key));
+  }
+
+  @Test
+  public void testListObjectsWithEmptyDelimiters() {
+    // Directory bucket only supports list with delimiter = '/' currently.
+    Assume.assumeFalse(storage.bucket().isDirectory());
+    String key1 = "a/b/c/d";
+    String key2 = "a/b";
+
+    byte[] data = TestUtility.rand(256);
+    for (int i = 0; i < 10; i++) {
+      storage.put(String.format("%s/file-%d.txt", key1, i), data);
+      storage.put(String.format("%s/file-%d.txt", key2, i), data);
+    }
+
+    // list 100 objects under 'a/', there are total 20 objects.
+    ListObjectsResponse response = list("a/", "", 100, "");
+    assertEquals(20, response.objects().size());
+    assertEquals(0, response.commonPrefixes().size());
+    assertEquals("a/b/c/d/file-0.txt", response.objects().get(0).key());
+    assertEquals("a/b/file-9.txt", response.objects().get(19).key());
+
+    // list 20 objects and there only have 20 objects under 'a/'
+    response = list("a/", "", 20, "");
+    assertEquals(20, response.objects().size());
+    assertEquals(0, response.commonPrefixes().size());
+    assertEquals("a/b/c/d/file-0.txt", response.objects().get(0).key());
+    assertEquals("a/b/file-9.txt", response.objects().get(19).key());
+
+    // list the top 10 objects among 20 objects
+    response = list("a/", "", 10, "");
+    assertEquals(10, response.objects().size());
+    assertEquals(0, response.commonPrefixes().size());
+    assertEquals("a/b/c/d/file-0.txt", response.objects().get(0).key());
+    assertEquals("a/b/c/d/file-9.txt", response.objects().get(9).key());
+
+    // list the next 5 objects behind a/b/c/d/file-9.txt among 20 objects
+    response = list("a/", "a/b/c/d/file-9.txt", 5, "");
+    assertEquals(5, response.objects().size());
+    assertEquals(0, response.commonPrefixes().size());
+    assertEquals("a/b/file-0.txt", response.objects().get(0).key());
+    assertEquals("a/b/file-4.txt", response.objects().get(4).key());
+
+    // list the next 10 objects behind a/b/c/d/file-9.txt among 20 objects
+    response = list("a/", "a/b/c/d/file-9.txt", 10, "");
+    assertEquals(10, response.objects().size());
+    assertEquals(0, response.commonPrefixes().size());
+    assertEquals("a/b/file-0.txt", response.objects().get(0).key());
+    assertEquals("a/b/file-9.txt", response.objects().get(9).key());
+  }
+
+  @Test
+  public void testListEmptyDirWithSlashDelimiter() {
+    String key = "a/b/";
+    storage.put(key, new byte[0]);
+
+    ListObjectsResponse response = list(key, null, 10, "/");
+    assertEquals(1, response.objects().size());
+    assertEquals(0, response.commonPrefixes().size());
+    assertEquals("a/b/", response.objects().get(0).key());
+
+    response = list(key, key, 10, "/");
+    assertEquals(0, response.objects().size());
+    assertEquals(0, response.commonPrefixes().size());
+  }
+
+  @Test
+  public void testDeleteMultipleKeys() {
+    String prefix = "a/b";
+    byte[] data = TestUtility.rand(256);
+
+    List<String> keys = Lists.newArrayList();
+    for (int i = 0; i < 50; i++) {
+      String existingKey = String.format("%s/existing-file-%d.txt", prefix, i);
+      storage.put(existingKey, data);
+      keys.add(existingKey);
+
+      String unExistingKey = String.format("%s/unExisting-file-%d.txt", prefix, i);
+      keys.add(unExistingKey);
+    }
+
+    List<String> failedKeys = storage.batchDelete(keys);
+
+    for (String key : failedKeys) {
+      Assert.assertNotNull(storage.head(key));
+    }
+
+    for (String key : keys) {
+      if (!failedKeys.contains(key)) {
+        Assert.assertNull(storage.head(key));
+      }
+    }
+
+    assertThrows("The deleted keys size should be <= 1000", IllegalArgumentException.class,
+        () -> storage.batchDelete(
+            IntStream.range(0, 1001).mapToObj(String::valueOf).collect(Collectors.toList())));
+  }
+
+  @Test
+  public void testListObjectsWithEmptyMarkers() {
+    String key1 = "a/b/c/d";
+    String key2 = "a/b";
+    String key3 = "a1/b1";
+
+    // create the folder to compatible with directory bucket.
+    storage.put("a/", new byte[0]);
+    storage.put("a/b/", new byte[0]);
+    storage.put("a/b/c/", new byte[0]);
+    storage.put("a/b/c/d/", new byte[0]);
+    storage.put("a1/", new byte[0]);
+    storage.put("a1/b1/", new byte[0]);
+
+    byte[] data = TestUtility.rand(256);
+    for (int i = 0; i < 10; i++) {
+      storage.put(String.format("%s/file-%d.txt", key1, i), data);
+      storage.put(String.format("%s/file-%d.txt", key2, i), data);
+      storage.put(String.format("%s/file-%d.txt", key3, i), data);
+    }
+
+    // group objects by '/' under 'a/'
+    ListObjectsResponse response = list("a/", null, 100, "/");
+    assertEquals(1, response.objects().size());
+    assertEquals("a/", response.objects().get(0).key());
+    assertEquals(1, response.commonPrefixes().size());
+    assertEquals("a/b/", response.commonPrefixes().get(0));
+
+    response = list("a", null, 100, "/");
+    assertEquals(0, response.objects().size());
+    assertEquals(2, response.commonPrefixes().size());
+    assertEquals("a/", response.commonPrefixes().get(0));
+    assertEquals("a1/", response.commonPrefixes().get(1));
+
+    // group objects by '/' under 'a/b/' and group objects by 'b/' under 'a', they are same
+    response = list("a/b/", null, 100, "/");
+    assertEquals(11, response.objects().size());
+    assertEquals(1, response.commonPrefixes().size());
+    assertEquals("a/b/c/", response.commonPrefixes().get(0));
+    assertEquals("a/b/", response.objects().get(0).key());
+    assertEquals("a/b/file-0.txt", response.objects().get(1).key());
+    assertEquals("a/b/file-9.txt", response.objects().get(10).key());
+
+    response = list("a/b", null, 100, "/");
+    assertEquals(0, response.objects().size());
+    assertEquals(1, response.commonPrefixes().size());
+    assertEquals("a/b/", response.commonPrefixes().get(0));
+
+    if (!storage.bucket().isDirectory()) {
+      // Directory bucket only supports list with delimiter = '/' currently.
+      response = list("a", null, 100, "b/");
+      assertEquals(13, response.objects().size());
+      assertEquals(1, response.commonPrefixes().size());
+      assertEquals("a/b/", response.commonPrefixes().get(0));
+      assertEquals("a/", response.objects().get(0).key());
+      assertEquals("a1/", response.objects().get(1).key());
+      assertEquals("a1/b1/", response.objects().get(2).key());
+      assertEquals("a1/b1/file-0.txt", response.objects().get(3).key());
+      assertEquals("a1/b1/file-9.txt", response.objects().get(12).key());
+
+      response = list("a/", null, 100, "b/");
+      assertEquals(1, response.objects().size());
+      assertEquals(1, response.commonPrefixes().size());
+      assertEquals("a/b/", response.commonPrefixes().get(0));
+      assertEquals("a/", response.objects().get(0).key());
+    }
+
+    // group objects by different delimiter under 'a/b/c/d/' or 'a/b/c/d'
+    response = list("a/b/c/d/", null, 100, "/");
+    assertEquals(11, response.objects().size());
+    assertEquals(0, response.commonPrefixes().size());
+    assertEquals("a/b/c/d/", response.objects().get(0).key());
+
+    response = list("a/b/c/d/", null, 5, "/");
+    assertEquals(5, response.objects().size());
+    assertEquals(0, response.commonPrefixes().size());
+    assertEquals("a/b/c/d/", response.objects().get(0).key());
+
+    response = list("a/b/c/d", null, 100, "/");
+    assertEquals(0, response.objects().size());
+    assertEquals(1, response.commonPrefixes().size());
+    assertEquals("a/b/c/d/", response.commonPrefixes().get(0));
+  }
+
+  @Test
+  public void testListObjectWithLimitObjectAndCommonPrefixes() {
+    String key1 = "a/b/c/d";
+    String key2 = "a/b";
+    String key3 = "a1/b1";
+
+    byte[] data = TestUtility.rand(256);
+    for (int i = 0; i < 10; i++) {
+      storage.put(String.format("%s/file-%d.txt", key1, i), data);
+      storage.put(String.format("%s/file-%d.txt", key2, i), data);
+      storage.put(String.format("%s/file-%d.txt", key3, i), data);
+    }
+
+    List<String> dirKeys = Lists.newArrayList("a/b/d/", "a/b/e/", "a/b/f/", "a/b/g/");
+    for (String key : dirKeys) {
+      storage.put(key, new byte[0]);
+    }
+
+    // group objects by '/' under 'a/b/', and limit top 5 objects among 10 objects and 1 common prefix
+    ListObjectsResponse response = list("a/b/", "a/b/", 5, "/");
+    assertEquals(1, response.objects().size());
+    assertEquals(4, response.commonPrefixes().size());
+    assertEquals("a/b/c/", response.commonPrefixes().get(0));
+    assertEquals("a/b/d/", response.commonPrefixes().get(1));
+    assertEquals("a/b/e/", response.commonPrefixes().get(2));
+    assertEquals("a/b/f/", response.commonPrefixes().get(3));
+    assertEquals("a/b/file-0.txt", response.objects().get(0).key());
+
+    response = list("a/b/", "a/b/", 14, "/");
+    assertEquals(10, response.objects().size());
+    assertEquals(4, response.commonPrefixes().size());
+    assertEquals("a/b/c/", response.commonPrefixes().get(0));
+    assertEquals("a/b/d/", response.commonPrefixes().get(1));
+    assertEquals("a/b/e/", response.commonPrefixes().get(2));
+    assertEquals("a/b/f/", response.commonPrefixes().get(3));
+    assertEquals("a/b/file-0.txt", response.objects().get(0).key());
+    assertEquals("a/b/file-9.txt", response.objects().get(9).key());
+
+    response = list("a/b/", "a/b/", 15, "/");
+    assertEquals(10, response.objects().size());
+    assertEquals(5, response.commonPrefixes().size());
+    assertEquals("a/b/c/", response.commonPrefixes().get(0));
+    assertEquals("a/b/d/", response.commonPrefixes().get(1));
+    assertEquals("a/b/e/", response.commonPrefixes().get(2));
+    assertEquals("a/b/f/", response.commonPrefixes().get(3));
+    assertEquals("a/b/g/", response.commonPrefixes().get(4));
+    assertEquals("a/b/file-0.txt", response.objects().get(0).key());
+    assertEquals("a/b/file-9.txt", response.objects().get(9).key());
+
+    // a/b/h-file-0.txt is behind from a/b/g/
+    storage.put("a/b/h-file-0.txt", data);
+    response = list("a/b/", "a/b/", 15, "/");
+    assertEquals(10, response.objects().size());
+    assertEquals(5, response.commonPrefixes().size());
+    assertEquals("a/b/c/", response.commonPrefixes().get(0));
+    assertEquals("a/b/d/", response.commonPrefixes().get(1));
+    assertEquals("a/b/e/", response.commonPrefixes().get(2));
+    assertEquals("a/b/f/", response.commonPrefixes().get(3));
+    assertEquals("a/b/g/", response.commonPrefixes().get(4));
+    assertEquals("a/b/file-0.txt", response.objects().get(0).key());
+    assertEquals("a/b/file-9.txt", response.objects().get(9).key());
+
+    response = list("a/b/", "a/b/", 20, "/");
+    assertEquals(11, response.objects().size());
+    assertEquals(5, response.commonPrefixes().size());
+    assertEquals("a/b/c/", response.commonPrefixes().get(0));
+    assertEquals("a/b/d/", response.commonPrefixes().get(1));
+    assertEquals("a/b/e/", response.commonPrefixes().get(2));
+    assertEquals("a/b/f/", response.commonPrefixes().get(3));
+    assertEquals("a/b/g/", response.commonPrefixes().get(4));
+    assertEquals("a/b/file-0.txt", response.objects().get(0).key());
+    assertEquals("a/b/h-file-0.txt", response.objects().get(10).key());
+
+    response = list("a/b/", "a/b/", 1, "/");
+    assertEquals(0, response.objects().size());
+    assertEquals(1, response.commonPrefixes().size());
+    assertEquals("a/b/c/", response.commonPrefixes().get(0));
+
+    response = list("a/b/", "a/b/", 2, "/");
+    assertEquals(0, response.objects().size());
+    assertEquals(2, response.commonPrefixes().size());
+    assertEquals("a/b/c/", response.commonPrefixes().get(0));
+    assertEquals("a/b/d/", response.commonPrefixes().get(1));
+  }
+
+  @Test
+  public void testListedIteratorIsIdempotent() {
+    String key1 = "a/b/c/d";
+
+    byte[] data = TestUtility.rand(256);
+    for (int i = 0; i < 10; i++) {
+      storage.put(String.format("%s/file-%d.txt", key1, i), data);
+    }
+
+    Iterable<ObjectInfo> res;
+    if (storage.bucket().isDirectory()) {
+      res = ((DirectoryStorage) storage).listDir("a/b/c/d/", true);
+    } else {
+      res = storage.list("a/b/c/d/", "a/b/c/d/", 10);
+    }
+    Iterator<ObjectInfo> batch1 = res.iterator();
+    Iterator<ObjectInfo> batch2 = res.iterator();
+
+    for (int i = 0; i < 10; i++) {
+      assertTrue(batch1.hasNext());
+      ObjectInfo obj = batch1.next();
+      assertEquals(String.format("a/b/c/d/file-%d.txt", i), obj.key());
+    }
+    assertFalse(batch1.hasNext());
+
+    for (int i = 0; i < 10; i++) {
+      assertTrue(batch2.hasNext());
+      ObjectInfo obj = batch2.next();
+      assertEquals(String.format("a/b/c/d/file-%d.txt", i), obj.key());
+    }
+    assertFalse(batch2.hasNext());
+  }
+
+  @Test
+  public void testListObjectsWithSmallBatch() {
+    Assume.assumeFalse(storage.bucket().isDirectory());
+    String key1 = "a/b/c/d/";
+
+    byte[] data = TestUtility.rand(256);
+    for (int i = 0; i < 10; i++) {
+      storage.put(String.format("%sfile-%d.txt", key1, i), data);
+    }
+
+    // change list object count
+    Configuration newConf = new Configuration(storage.conf());
+    newConf.setInt(TosKeys.FS_TOS_LIST_OBJECTS_COUNT, 5);
+    storage.initialize(newConf, storage.bucket().name());
+
+    List<Integer> maxKeys = Arrays.asList(5, 10, 9, 20, -1);
+    for (int maxKey : maxKeys) {
+      Iterator<ObjectInfo> objs = storage.list(key1, key1, maxKey).iterator();
+      int end = Math.min(maxKey == -1 ? 10 : maxKey, 10);
+      for (int i = 0; i < end; i++) {
+        assertTrue(objs.hasNext());
+        ObjectInfo obj = objs.next();
+        assertEquals(String.format("a/b/c/d/file-%d.txt", i), obj.key());
+      }
+      assertFalse(objs.hasNext());
+    }
+
+    // reset list object count
+    newConf = new Configuration(storage.conf());
+    newConf.setInt(TosKeys.FS_TOS_LIST_OBJECTS_COUNT, 1000);
+    storage.initialize(newConf, storage.bucket().name());
+  }
+
+  @Test
+  public void testListObjectsWithSpecificDelimiters() {
+    Assume.assumeFalse(storage.bucket().isDirectory());
+    String key1 = "a/b/c/d";
+    String key2 = "a/b";
+    String key3 = "a1/b1";
+
+    byte[] data = TestUtility.rand(256);
+    for (int i = 0; i < 10; i++) {
+      storage.put(String.format("%s/file-%d.txt", key1, i), data);
+      storage.put(String.format("%s/file-%d.txt", key2, i), data);
+      storage.put(String.format("%s/file-%d.txt", key3, i), data);
+    }
+
+    ListObjectsResponse response = list("a", "", 11, "b/");
+    assertEquals(10, response.objects().size());
+    assertEquals(1, response.commonPrefixes().size());
+    assertEquals("a/b/", response.commonPrefixes().get(0));
+    assertEquals("a1/b1/file-0.txt", response.objects().get(0).key());
+    assertEquals("a1/b1/file-9.txt", response.objects().get(9).key());
+
+    response = list("a", "", 5, "b/");
+    assertEquals(4, response.objects().size());
+    assertEquals(1, response.commonPrefixes().size());
+    assertEquals("a/b/", response.commonPrefixes().get(0));
+    assertEquals("a1/b1/file-0.txt", response.objects().get(0).key());
+    assertEquals("a1/b1/file-3.txt", response.objects().get(3).key());
+
+    response = list("a", "a1/b1/file-3.txt", 5, "b/");
+    assertEquals(5, response.objects().size());
+    assertEquals(0, response.commonPrefixes().size());
+    assertEquals("a1/b1/file-4.txt", response.objects().get(0).key());
+    assertEquals("a1/b1/file-8.txt", response.objects().get(4).key());
+
+    response = list("a", "a1/b1/file-3.txt", 6, "b/");
+    assertEquals(6, response.objects().size());
+    assertEquals(0, response.commonPrefixes().size());
+    assertEquals("a1/b1/file-4.txt", response.objects().get(0).key());
+    assertEquals("a1/b1/file-9.txt", response.objects().get(5).key());
+
+    response = list("a", "a/b/file-3.txt", 5, "b/");
+    assertEquals(4, response.objects().size());
+    assertEquals(1, response.commonPrefixes().size());
+    assertEquals("a/b/", response.commonPrefixes().get(0));
+    assertEquals("a1/b1/file-0.txt", response.objects().get(0).key());
+    assertEquals("a1/b1/file-3.txt", response.objects().get(3).key());
+
+    response = list("a", "a/b/file-3.txt", 10, "b/");
+    assertEquals(9, response.objects().size());
+    assertEquals(1, response.commonPrefixes().size());
+    assertEquals("a/b/", response.commonPrefixes().get(0));
+    assertEquals("a1/b1/file-0.txt", response.objects().get(0).key());
+    assertEquals("a1/b1/file-8.txt", response.objects().get(8).key());
+
+    response = list("a", "a/b/file-3.txt", 11, "b/");
+    assertEquals(10, response.objects().size());
+    assertEquals(1, response.commonPrefixes().size());
+    assertEquals("a/b/", response.commonPrefixes().get(0));
+    assertEquals("a1/b1/file-0.txt", response.objects().get(0).key());
+    assertEquals("a1/b1/file-9.txt", response.objects().get(9).key());
+
+    response = list("a", "a/b/", 1, "b/");
+    assertEquals(1, response.objects().size());
+    assertEquals(0, response.commonPrefixes().size());
+
+    response = list("a/b/c/d", "", 100, "/file");
+    assertEquals(0, response.objects().size());
+    assertEquals(1, response.commonPrefixes().size());
+    assertEquals("a/b/c/d/file", response.commonPrefixes().get(0));
+
+    response = list("a/b/c/d/", "", 100, "file");
+    assertEquals(0, response.objects().size());
+    assertEquals(1, response.commonPrefixes().size());
+    assertEquals("a/b/c/d/file", response.commonPrefixes().get(0));
+
+
+    // group objects by different delimiter under 'a1' or 'a1/'
+    response = list("a1", "", 100, "");
+    assertEquals(10, response.objects().size());
+    assertEquals(0, response.commonPrefixes().size());
+
+    response = list("a1", "", 100, "/");
+    assertEquals(0, response.objects().size());
+    assertEquals(1, response.commonPrefixes().size());
+    assertEquals("a1/", response.commonPrefixes().get(0));
+
+    response = list("a1/", "", 100, "/");
+    assertEquals(0, response.objects().size());
+    assertEquals(1, response.commonPrefixes().size());
+    assertEquals("a1/b1/", response.commonPrefixes().get(0));
+
+    response = list("a1/", "", 1, "/");
+    assertEquals(0, response.objects().size());
+    assertEquals(1, response.commonPrefixes().size());
+    assertEquals("a1/b1/", response.commonPrefixes().get(0));
+
+    // group objects by non-exist delimiter under 'a1' or 'a1/'
+    response = list("a1", "", 100, "non-exist");
+    assertEquals(10, response.objects().size());
+    assertEquals(0, response.commonPrefixes().size());
+
+    response = list("a1/", "", 100, "non-exist");
+    assertEquals(10, response.objects().size());
+    assertEquals(0, response.commonPrefixes().size());
+
+    // test the sequent of common prefixes
+    response = list("a", "", 100, "b");
+    assertEquals(0, response.objects().size());
+    assertEquals(2, response.commonPrefixes().size());
+    assertEquals("a/b", response.commonPrefixes().get(0));
+    assertEquals("a1/b", response.commonPrefixes().get(1));
+  }
+
+  @Test
+  public void testOverwriteDirectoryWithAFile() throws IOException {
+    String dirKey = "a/b/";
+    String key = "a/b";
+    storage.delete("a/");
+
+    byte[] data1 = new byte[0];
+    byte[] data2 = TestUtility.rand(128);
+
+    storage.put(dirKey, data1);
+    assertArrayEquals(data1, IOUtils.toByteArray(getStream(dirKey, 0, 256)));
+
+    if (!storage.bucket().isDirectory()) {
+      // Directory bucket doesn't allow overwrote if the resource type is changed.
+      storage.put(key, data2);
+      assertArrayEquals(data2, IOUtils.toByteArray(getStream(key, 0, 256)));
+    }
+
+    storage.delete(key);
+    storage.delete(dirKey);
+    assertNull(storage.head(key));
+    assertNull(storage.head(dirKey));
+  }
+
+  private InputStream getStream(String key, long off, long limit) {
+    return storage.get(key, off, limit).stream();
+  }
+
+  @Test
+  public void testDeleteNonEmptyDir() throws IOException {
+    storage.put("a/", new byte[0]);
+    storage.put("a/b/", new byte[0]);
+    assertArrayEquals(new byte[0], IOUtils.toByteArray(getStream("a/b/", 0, 256)));
+
+    ListObjectsResponse response = list("a/b/", "a/b/", 100, "/");
+    assertEquals(0, response.objects().size());
+    assertEquals(0, response.commonPrefixes().size());
+
+    if (!storage.bucket().isDirectory()) {
+      // Directory bucket only supports list with delimiter = '/'.
+      response = list("a/b/", "a/b/", 100, null);
+      assertEquals(0, response.objects().size());
+      assertEquals(0, response.commonPrefixes().size());
+    }
+
+    storage.delete("a/b/");
+    assertNull(storage.head("a/b/"));
+    assertNull(storage.head("a/b"));
+    assertNotNull(storage.head("a/"));
+  }
+
+  @Test
+  public void testRecursiveDelete() {
+    storage.put("a/", new byte[0]);
+    storage.put("a/b/", new byte[0]);
+    storage.put("a/b/c1/", new byte[0]);
+    storage.put("a/b/c2/", new byte[0]);
+    storage.put("a/b/c3/", new byte[0]);
+    assertNotNull(storage.head("a/"));
+    assertNotNull(storage.head("a/b/"));
+    assertNotNull(storage.head("a/b/c1/"));
+    assertNotNull(storage.head("a/b/c2/"));
+    assertNotNull(storage.head("a/b/c3/"));
+
+    storage.delete("a/b/c3/");
+    assertNull(storage.head("a/b/c3/"));
+
+    storage.deleteAll("");
+    assertNull(storage.head("a/b/c1/"));
+    assertNull(storage.head("a/b/c2/"));
+    assertNull(storage.head("a/b/"));
+    assertNull(storage.head("a/"));
+  }
+
+  @Test
+  public void testListObjectKeys() {
+    Assume.assumeFalse(storage.bucket().isDirectory());
+    byte[] dirBytes = new byte[0];
+    byte[] fileBytes = TestUtility.rand(128);
+    storage.put("a/b1/", dirBytes);
+    storage.put("a/b2/c0/", dirBytes);
+    storage.put("a/b2/c1/d1.txt", fileBytes);
+    storage.put("a/b2/c1/e1.txt", fileBytes);
+    storage.put("a/b2/c2.txt", fileBytes);
+
+    // list single dir
+    List<ObjectInfo> ret = toList(storage.list("a/b1", "", 10));
+    assertEquals(1, ret.size());
+    assertEquals("a/b1/", ret.get(0).key());
+    assertEquals(0, ret.get(0).size());
+
+    ret = toList(storage.list("a/b1/", "", 10));
+    assertEquals(1, ret.size());
+    assertEquals("a/b1/", ret.get(0).key());
+    assertEquals(0, ret.get(0).size());
+
+    // list single file
+    ret = toList(storage.list("a/b2/c1/d1.txt", "", 10));
+    assertEquals(1, ret.size());
+    assertEquals("a/b2/c1/d1.txt", ret.get(0).key());
+    assertEquals(fileBytes.length, ret.get(0).size());
+
+    // list multiple files & dirs
+    ret = toList(storage.list("a/b2", "", 10));
+    assertEquals(4, ret.size());
+    assertEquals("a/b2/c0/", ret.get(0).key());
+    assertEquals("a/b2/c1/d1.txt", ret.get(1).key());
+    assertEquals("a/b2/c1/e1.txt", ret.get(2).key());
+    assertEquals("a/b2/c2.txt", ret.get(3).key());
+    assertEquals(dirBytes.length, ret.get(0).size());
+
+    // list single file with marker
+    ret = toList(storage.list("a/b2", "a/b2/c1/e1.txt", 10));
+    assertEquals(1, ret.size());
+    assertEquals("a/b2/c2.txt", ret.get(0).key());
+    assertEquals(fileBytes.length, ret.get(0).size());
+
+    // list multiple files with marker
+    ret = toList(storage.list("a/b2", "a/b2/c1/", 10));
+    assertEquals(3, ret.size());
+    assertEquals("a/b2/c1/d1.txt", ret.get(0).key());
+    assertEquals("a/b2/c1/e1.txt", ret.get(1).key());
+    assertEquals("a/b2/c2.txt", ret.get(2).key());
+    assertEquals(fileBytes.length, ret.get(0).size());
+
+    // list multiple files & dirs with part path as prefix
+    ret = toList(storage.list("a/b2/c", "", 10));
+    assertEquals(4, ret.size());
+    assertEquals("a/b2/c0/", ret.get(0).key());
+    assertEquals("a/b2/c1/d1.txt", ret.get(1).key());
+    assertEquals("a/b2/c1/e1.txt", ret.get(2).key());
+    assertEquals("a/b2/c2.txt", ret.get(3).key());
+    assertEquals(dirBytes.length, ret.get(0).size());
+
+    ret = toList(storage.list("a/b2/c", "", 2));
+    assertEquals(2, ret.size());
+    assertEquals("a/b2/c0/", ret.get(0).key());
+
+    ret = toList(storage.list("a/b2/c1/d1.", "", 10));
+    assertEquals(1, ret.size());
+    assertEquals("a/b2/c1/d1.txt", ret.get(0).key());
+    assertEquals(fileBytes.length, ret.get(0).size());
+  }
+
+  @Test
+  public void testListAllObjectKeys() {
+    Assume.assumeFalse(storage.bucket().isDirectory());
+    byte[] dirBytes = new byte[0];
+    byte[] fileBytes = TestUtility.rand(128);
+    storage.put("a/b1/", dirBytes);
+    storage.put("a/b2/c0/", dirBytes);
+    storage.put("a/b2/c1/d1.txt", fileBytes);
+    storage.put("a/b2/c1/e1.txt", fileBytes);
+    storage.put("a/b2/c2.txt", dirBytes);
+
+    // list single dir
+    List<ObjectInfo> ret = Lists.newArrayList(storage.listAll("a/b1", ""));
+    assertEquals(1, ret.size());
+    assertEquals("a/b1/", ret.get(0).key());
+    assertEquals(0, ret.get(0).size());
+
+    // list single file
+    ret = Lists.newArrayList(storage.listAll("a/b2/c1/d1.txt", ""));
+    assertEquals(1, ret.size());
+    assertEquals("a/b2/c1/d1.txt", ret.get(0).key());
+    assertEquals(fileBytes.length, ret.get(0).size());
+
+    // list multiple files & dirs
+    ret = Lists.newArrayList(storage.listAll("a/b2", ""));
+    assertEquals(4, ret.size());
+    assertEquals("a/b2/c0/", ret.get(0).key());
+    assertEquals("a/b2/c1/d1.txt", ret.get(1).key());
+    assertEquals("a/b2/c1/e1.txt", ret.get(2).key());
+    assertEquals("a/b2/c2.txt", ret.get(3).key());
+    assertEquals(dirBytes.length, ret.get(0).size());
+
+    // list multiple files & dirs with part path as prefix
+    ret = Lists.newArrayList(storage.listAll("a/b2/c", ""));
+    assertEquals(4, ret.size());
+    assertEquals("a/b2/c0/", ret.get(0).key());
+    assertEquals("a/b2/c1/d1.txt", ret.get(1).key());
+    assertEquals("a/b2/c1/e1.txt", ret.get(2).key());
+    assertEquals("a/b2/c2.txt", ret.get(3).key());
+    assertEquals(dirBytes.length, ret.get(0).size());
+  }
+
+  @Test
+  public void testListEmptyKeys() {
+    if (storage.bucket().isDirectory()) {
+      assertEquals(0, Lists.newArrayList(((DirectoryStorage) storage).listDir("not-exist", true)).size());
+    } else {
+      assertEquals(0, Lists.newArrayList(storage.list("not-exist", "", 2)).size());
+    }
+  }
+
+  @Test
+  public void testMultiUploadEmptyFile() {
+    String key = "a/b/empty.txt";
+    MultipartUpload upload = storage.createMultipartUpload(key);
+    assertThrows(Exception.class, () -> storage.completeUpload(key, upload.uploadId(), Lists.newArrayList()));
+  }
+
+  @Test
+  public void testMultiUploadZeroByte() throws IOException {
+    String key = "a/b/zero.txt";
+    MultipartUpload upload = storage.createMultipartUpload(key);
+    Part part = storage.uploadPart(key, upload.uploadId(), 1, () -> new ByteArrayInputStream(new byte[0]), 0);
+    storage.completeUpload(key, upload.uploadId(), Lists.newArrayList(part));
+    assertArrayEquals(ObjectTestUtils.EMPTY_BYTES, IOUtils.toByteArray(getStream(key)));
+  }
+
+  @Test
+  public void testMultiUploadFile() throws IOException {
+    String key1 = "a/b/c/e.txt";
+    String uploadId1 = storage.createMultipartUpload(key1).uploadId();
+    assertNotEquals(uploadId1, "");
+
+    byte[] dataset = multipleUpload(key1, uploadId1, 2, true);
+    assertArrayEquals(dataset, IOUtils.toByteArray(getStream(key1)));
+
+    String key2 = "a/b/e/e.txt";
+    String uploadId2 = storage.createMultipartUpload(key2).uploadId();
+    assertNotEquals(uploadId2, "");
+
+    dataset = multipleUpload(key2, uploadId2, 3, true);
+    assertArrayEquals(dataset, IOUtils.toByteArray(getStream(key2)));
+  }
+
+  @Test
+  public void testPutAndCompleteMPUWithSameContent() throws IOException {
+    String mpu = "a/b/mpu.txt";
+    String put = "a/b/put.txt";
+    byte[] dataset = TestUtility.rand(11 << 20);
+    byte[] checksum = multipleUpload(mpu, dataset);
+
+    storage.put(put, dataset);
+
+    ObjectInfo mputObj = storage.head(mpu);
+    ObjectInfo putObj = storage.head(put);
+    assertArrayEquals(checksum, mputObj.checksum());
+    assertArrayEquals(checksum, putObj.checksum());
+
+    if (!storage.bucket().isDirectory()) {
+      List<ObjectInfo> objectInfo = toList(storage.list(mpu, null, 10));
+      Assert.assertEquals(mputObj, objectInfo.get(0));
+    }
+  }
+
+  /*@Test
+  public void testMultiUpload11M() throws IOException {
+    byte[] dataset = TestUtility.rand(11 << 20);
+    multipleUpload("a11/b/c.txt", dataset);
+  }
+
+  @Test
+  public void testMultiUpload101M() throws IOException {
+    byte[] dataset = TestUtility.rand(101 << 20);
+    multipleUpload("a101/b/c.txt", dataset);
+  }*/
+
+  @Test
+  public void testListUploads() {
+    String key1 = "a/b/c/e.txt";
+    String uploadId1 = storage.createMultipartUpload(key1).uploadId();
+    assertNotEquals(uploadId1, "");
+    multipleUpload(key1, uploadId1, 2, false);
+
+    String key2 = "a/b/e/e.txt";
+    String uploadId2 = storage.createMultipartUpload(key2).uploadId();
+    assertNotEquals(uploadId2, "");
+    multipleUpload(key2, uploadId2, 3, false);
+
+    Iterable<MultipartUpload> iterable = storage.listUploads("");
+    List<MultipartUpload> uploads = Lists.newArrayList(iterable.iterator());
+    assertEquals(2, uploads.size());
+    assertEquals(key1, uploads.get(0).key());
+    assertEquals(uploadId1, uploads.get(0).uploadId());
+    assertEquals(key2, uploads.get(1).key());
+    assertEquals(uploadId2, uploads.get(1).uploadId());
+
+    // check iterator is idempotent
+    uploads = Lists.newArrayList(iterable.iterator());
+    assertEquals(2, uploads.size());
+    assertEquals(key1, uploads.get(0).key());
+    assertEquals(uploadId1, uploads.get(0).uploadId());
+    assertEquals(key2, uploads.get(1).key());
+    assertEquals(uploadId2, uploads.get(1).uploadId());
+
+    uploads = Lists.newArrayList(storage.listUploads("a/b/"));
+    assertEquals(2, uploads.size());
+    assertEquals(key1, uploads.get(0).key());
+    assertEquals(uploadId1, uploads.get(0).uploadId());
+    assertEquals(key2, uploads.get(1).key());
+    assertEquals(uploadId2, uploads.get(1).uploadId());
+
+    uploads = Lists.newArrayList(storage.listUploads("a/b/c/"));
+    assertEquals(1, uploads.size());
+    assertEquals(key1, uploads.get(0).key());
+    assertEquals(uploadId1, uploads.get(0).uploadId());
+
+    storage.abortMultipartUpload(key1, uploadId1);
+    storage.abortMultipartUpload(key2, uploadId2);
+    assertEquals(0, Lists.newArrayList((storage.listUploads("a/b/"))).size());
+  }
+
+  private byte[] multipleUpload(String key, String uploadId, int partCnt, boolean completeUpload) {
+    int partSize = 5 * 1024 * 1024;
+    byte[] dataset = new byte[partCnt * partSize];
+    byte[] partData = TestUtility.rand(partSize);
+    try {
+      int offset = 0;
+      List<Part> parts = new ArrayList<>();
+      for (int i = 1; i <= partCnt; i++) {
+        Part part = storage.uploadPart(key, uploadId, i, () -> new ByteArrayInputStream(partData), partData.length);
+        parts.add(part);
+        System.arraycopy(partData, 0, dataset, offset, partData.length);
+        offset += partData.length;
+      }
+      if (completeUpload) {
+        storage.completeUpload(key, uploadId, parts);
+      }
+    } catch (RuntimeException e) {
+      storage.abortMultipartUpload(key, uploadId);
+    }
+    return dataset;
+  }
+
+  private byte[] multipleUpload(String key, byte[] dataset) throws IOException {
+    int partSize = 5 * 1024 * 1024;
+    int partCnt = (int) Math.ceil((double) dataset.length / partSize);
+
+    String uploadId = storage.createMultipartUpload(key).uploadId();
+    assertNotEquals(uploadId, "");
+
+    try {
+      List<Part> parts = new ArrayList<>();
+      for (int i = 0; i < partCnt; i++) {
+        int start = i * partSize;
+        int end = Math.min(dataset.length, start + partSize);
+        byte[] partData = Arrays.copyOfRange(dataset, start, end);
+
+        Part part = storage.uploadPart(key, uploadId, i + 1, () -> new ByteArrayInputStream(partData), partData.length);
+
+        assertEquals(DigestUtils.md5Hex(partData), part.eTag().replace("\"", ""));
+        parts.add(part);
+      }
+
+      byte[] checksum = storage.completeUpload(key, uploadId, parts);
+      assertArrayEquals(dataset, IOUtils.toByteArray(getStream(key)));
+
+      return checksum;
+    } catch (IOException | RuntimeException e) {
+      storage.abortMultipartUpload(key, uploadId);
+      throw e;
+    }
+  }
+
+  @Test
+  public void testUploadPartCopy10MB() {
+    String srcKey = "src10MB.txt";
+    String dstKey = "dst10MB.txt";
+    testUploadPartCopy(srcKey, dstKey, 10 << 20); // 10MB
+  }
+
+  @Test
+  public void testUploadPartCopy100MB() {
+    String srcKey = "src100MB.txt";
+    String dstKey = "dst100MB.txt";
+    testUploadPartCopy(srcKey, dstKey, 100 << 20);// 100MB
+  }
+
+  @Test
+  public void testUploadPartCopy65MB() {
+    String srcKey = "src65MB.txt";
+    String dstKey = "dst65MB.txt";
+    testUploadPartCopy(srcKey, dstKey, 65 << 20);// 65MB
+  }
+
+  private void testUploadPartCopy(String srcKey, String key, int fileSize) {
+    MultipartUpload srcMultipartUpload = storage.createMultipartUpload(srcKey);
+    long partSize = 5 << 20;
+    int partCnt = (int) (fileSize / partSize + (fileSize % partSize == 0 ? 0 : 1));
+    byte[] data = multipleUpload(srcMultipartUpload.key(), srcMultipartUpload.uploadId(), partCnt, true);
+    MultipartUpload dstMultipartUpload = storage.createMultipartUpload(key);
+    long copyPartRangeStart = 0L;
+    List<Part> results = Lists.newArrayList();
+    try {
+      for (int i = 0; i < partCnt; i++) {
+        Part result = storage.uploadPartCopy(srcKey, key, dstMultipartUpload.uploadId(), i + 1,
+            copyPartRangeStart, Math.min(copyPartRangeStart + partSize, fileSize) - 1);
+        results.add(result);
+        copyPartRangeStart += partSize;
+      }
+      storage.completeUpload(key, dstMultipartUpload.uploadId(), results);
+      assertArrayEquals(data, IOUtils.toByteArray(getStream(key)));
+    } catch (Exception e) {
+      storage.abortMultipartUpload(key, dstMultipartUpload.uploadId());
+    }
+  }
+
+  @Test
+  public void testCopy0MB() throws IOException {
+    String srcKey = "src0MB.txt";
+    String dstKey = "dst0MB.txt";
+    testCopy(srcKey, dstKey, 0);
+  }
+
+  @Test
+  public void testCopy5MB() throws IOException {
+    String srcKey = "src5MB.txt";
+    String dstKey = "dst5MB.txt";
+    testCopy(srcKey, dstKey, 5 << 20);
+  }
+
+  @Test
+  public void testCopy10MB() throws IOException {
+    String srcKey = "src10MB.txt";
+    String dstKey = "dst10MB.txt";
+    testCopy(srcKey, dstKey, 10 << 20);
+  }
+
+  @Test
+  public void testRename() throws IOException {
+    String srcKey = "src.txt";
+    String dstKey = "dst.txt";
+
+    // Rename source to a un-exist object
+    renameObject(srcKey, dstKey, 256);
+    renameObject(srcKey, dstKey, 0);
+
+    // Overwrite an existing object
+    renameObjectWhenDestExist(srcKey, dstKey, 256, 0);
+    renameObjectWhenDestExist(srcKey, dstKey, 0, 256);
+
+    assertNull(storage.head(srcKey));
+    assertThrows("Source key not found", RuntimeException.class,
+        () -> storage.rename(srcKey, dstKey));
+
+    assertThrows("Cannot rename to the same object", RuntimeException.class,
+        () -> renameObject(srcKey, srcKey, 256));
+  }
+
+  private void renameObjectWhenDestExist(String srcKey, String dstKey, int srcSize, int destSize) throws IOException {
+    byte[] dstData = new byte[destSize];
+    storage.put(dstKey, dstData, 0, destSize);
+    assertArrayEquals(dstData, IOUtils.toByteArray(getStream(dstKey)));
+
+    renameObject(srcKey, dstKey, srcSize);
+  }
+
+  private void renameObject(String srcKey, String dstKey, int fileSize) throws IOException {
+    byte[] data = new byte[fileSize];
+    storage.put(srcKey, data, 0, fileSize);
+    assertArrayEquals(data, IOUtils.toByteArray(getStream(srcKey)));
+
+    storage.rename(srcKey, dstKey);
+    assertArrayEquals(data, IOUtils.toByteArray(getStream(dstKey)));
+    assertNull(storage.head(srcKey));
+
+    storage.delete(dstKey);
+    assertNull(storage.head(dstKey));
+  }
+
+  private void testCopy(String srcKey, String dstKey, int fileSize) throws IOException {
+    byte[] data = new byte[fileSize];
+    storage.put(srcKey, data, 0, fileSize);
+    storage.copy(srcKey, dstKey);
+    assertArrayEquals(data, IOUtils.toByteArray(getStream(dstKey)));
+  }
+
+  private ListObjectsResponse list(String prefix, String startAfter, int limit, String delimiter) {
+    Preconditions.checkArgument(limit <= 1000, "Cannot list more than 1000 objects.");
+    ListObjectsRequest request = ListObjectsRequest.builder()
+        .prefix(prefix)
+        .startAfter(startAfter)
+        .maxKeys(limit)
+        .delimiter(delimiter)
+        .build();
+    Iterator<ListObjectsResponse> iterator = storage.list(request).iterator();
+    if (iterator.hasNext()) {
+      return iterator.next();
+    } else {
+      return new ListObjectsResponse(new ArrayList<>(), new ArrayList<>());
+    }
+  }
+
+  private static <T> List<T> toList(final Iterable<T> iterable) {
+    return StreamSupport.stream(iterable.spliterator(), false)
+        .collect(Collectors.toList());
+  }
+
+  @Test
+  public void testObjectTagging() {
+    Assume.assumeFalse(storage.bucket().isDirectory());
+    if (storage instanceof FileStore) {
+      return;
+    }
+
+    // create key.
+    String key = "ObjectTagging";
+    String tagPrefix = "tag" + UUIDUtils.random() + "_";
+    String valuePrefix = "value" + UUIDUtils.random() + "_";
+    storage.put(key, new byte[0], 0, 0);
+
+    Map<String, String> tagsMap = new HashMap<>();
+    for (int i = 0; i < 10; i++) {
+      tagsMap.put(tagPrefix + i, valuePrefix + i);
+    }
+
+    // 1. put and get when key exists.
+    storage.putTags(key, tagsMap);
+    Map<String, String> tags = storage.getTags(key);
+    assertEquals(10, tags.keySet().size());
+    assertTrue(Maps.difference(tagsMap, tags).areEqual());
+
+    // 2. put and get when key doesn't exist.
+    assertThrows("NoSuchKey", TosServerException.class, () -> storage.putTags("non-exist-key", tagsMap));
+    assertThrows("doesn't exist", TosServerException.class, () -> storage.getTags("non-exist-key"));
+
+    // 3. tag threshold.
+    Map<String, String> bigMap = new HashMap<>(tagsMap);
+    bigMap.put(tagPrefix + 11, valuePrefix + 11);
+    assertThrows("exceed limit of 10", RuntimeException.class, () -> storage.putTags(key, bigMap));
+
+    // 4. put tag with null tagName.
+    Map<String, String> nullKeyTag = new HashMap<>();
+    nullKeyTag.put(null, "some value");
+    assertThrows("TagKey you have provided is invalid", TosServerException.class,
+        () -> storage.putTags(key, nullKeyTag));
+
+    // 5. put tag with null value.
+    Map<String, String> nullValueTag = new HashMap<>();
+    nullValueTag.put("some-key", null);
+    storage.putTags(key, nullValueTag);
+    assertNull(storage.getTags(key).get("some-key"));
+
+    // 6. remove tags.
+    Map<String, String> emptyTag = new HashMap<>();
+    storage.putTags(key, emptyTag);
+    assertEquals(0, storage.getTags(key).size());
+  }
+
+  @Test
+  public void testObjectChecksum() throws IOException {
+    byte[] data = TestUtility.rand(256);
+    String key = "a/truncated.txt";
+
+    // Read object at the end offset.
+    byte[] checksum = storage.put(key, () -> new ByteArrayInputStream(data), 200);
+    ObjectContent objContent = storage.get(key, 200, -1);
+    objContent.stream().close();
+    assertArrayEquals(checksum, objContent.checksum());
+
+    // Read empty object.
+    checksum = storage.put(key, () -> new ByteArrayInputStream(new byte[0]), 0);
+    objContent = storage.get(key, 0, -1);
+    objContent.stream().close();
+    assertArrayEquals(checksum, objContent.checksum());
+  }
+}

+ 4 - 15
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/object/tos/TestTOSObjectStorage.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.fs.tosfs.object.tos;
 import com.volcengine.tos.internal.model.CRC64Checksum;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.tosfs.common.Bytes;
+import org.apache.hadoop.fs.tosfs.conf.ConfKeys;
 import org.apache.hadoop.fs.tosfs.conf.TosKeys;
 import org.apache.hadoop.fs.tosfs.object.ChecksumType;
 import org.apache.hadoop.fs.tosfs.object.Constants;
@@ -50,7 +51,6 @@ import java.util.List;
 import java.util.zip.Checksum;
 
 import static org.apache.hadoop.fs.tosfs.object.tos.TOS.TOS_SCHEME;
-import static org.apache.hadoop.fs.tosfs.util.TestUtility.directoryBucketObjectStorage;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThrows;
 
@@ -72,28 +72,17 @@ public class TestTOSObjectStorage {
     List<Object[]> values = new ArrayList<>();
 
     Configuration conf = new Configuration();
-    conf.set(TosKeys.FS_TOS_CHECKSUM_TYPE, ChecksumType.CRC64ECMA.name());
+    conf.set(ConfKeys.TOS_CHECKSUM_TYPE, ChecksumType.CRC64ECMA.name());
     values.add(new Object[] {
         ObjectStorageFactory.createWithPrefix(String.format("tos-%s/", UUIDUtils.random()),
             TOS_SCHEME, TestUtility.bucket(), conf), new CRC64Checksum(), ChecksumType.CRC64ECMA });
 
     conf = new Configuration();
-    conf.set(TosKeys.FS_TOS_CHECKSUM_TYPE, ChecksumType.CRC32C.name());
+    conf.set(ConfKeys.TOS_CHECKSUM_TYPE, ChecksumType.CRC32C.name());
     values.add(new Object[] {
         ObjectStorageFactory.createWithPrefix(String.format("tos-%s/", UUIDUtils.random()),
             TOS_SCHEME, TestUtility.bucket(), conf), new PureJavaCrc32C(), ChecksumType.CRC32C });
 
-    conf = new Configuration();
-    conf.set(TosKeys.FS_TOS_CHECKSUM_TYPE, ChecksumType.CRC64ECMA.name());
-    ObjectStorage directoryObjectStorage = directoryBucketObjectStorage(conf);
-    values.add(
-        new Object[] { directoryObjectStorage, new CRC64Checksum(), ChecksumType.CRC64ECMA });
-
-    conf = new Configuration();
-    conf.set(TosKeys.FS_TOS_CHECKSUM_TYPE, ChecksumType.CRC32C.name());
-    directoryObjectStorage = directoryBucketObjectStorage(conf);
-    values.add(new Object[] { directoryObjectStorage, new PureJavaCrc32C(), ChecksumType.CRC32C });
-
     return values;
   }
 
@@ -124,7 +113,7 @@ public class TestTOSObjectStorage {
     Assume.assumeFalse(tos.bucket().isDirectory());
 
     Configuration conf = new Configuration(tos.conf());
-    conf.setBoolean(TosKeys.FS_TOS_GET_FILE_STATUS_ENABLED, true);
+    conf.setBoolean(ConfKeys.TOS_GET_FILE_STATUS_ENABLED, true);
     tos.initialize(conf, tos.bucket().name());
 
     String key = "testFileStatus";

+ 274 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestBaseFsOps.java

@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.ops;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
+import org.apache.hadoop.fs.tosfs.RawFileStatus;
+import org.apache.hadoop.fs.tosfs.object.ObjectInfo;
+import org.apache.hadoop.fs.tosfs.object.ObjectStorage;
+import org.apache.hadoop.fs.tosfs.object.ObjectUtils;
+import org.apache.hadoop.fs.tosfs.util.CommonUtils;
+import org.apache.hadoop.fs.tosfs.util.TestUtility;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import static org.junit.Assert.assertArrayEquals;
+
+public abstract class TestBaseFsOps extends TestBaseOps {
+  public TestBaseFsOps(ObjectStorage storage) {
+    super(storage);
+  }
+
+  public abstract FsOps fsOps();
+
+  @After
+  public void tearDown() {
+    CommonUtils.runQuietly(() -> storage.deleteAll(""));
+  }
+
+  @Test
+  public void testDeleteFile() throws IOException {
+    Path path = new Path("/a/b");
+    touchFile(path, TestUtility.rand(8));
+    assertFileExist(path);
+
+    fsOps().deleteFile(path);
+    assertFileDoesNotExist(path);
+  }
+
+  @Test
+  public void testDeleteEmptyDir() throws IOException {
+    Path path = new Path("/a/b/");
+    mkdir(path);
+
+    fsOps().deleteDir(path, false);
+    assertDirDoesNotExist(path);
+  }
+
+  @Test
+  public void testDeleteNonEmptyDir() throws IOException {
+    Path dirPath = new Path("/a/b/");
+    Path subDirPath = new Path("/a/b/c/");
+    Path filePath = new Path("/a/b/file.txt");
+    mkdir(dirPath);
+    mkdir(subDirPath);
+    touchFile(filePath, new byte[10]);
+
+    Assert.assertThrows(PathIsNotEmptyDirectoryException.class, () -> fsOps().deleteDir(dirPath, false));
+    assertDirExist(dirPath);
+    assertDirExist(subDirPath);
+    assertFileExist(filePath);
+
+    fsOps().deleteDir(dirPath, true);
+    assertDirDoesNotExist(dirPath);
+    assertDirDoesNotExist(subDirPath);
+    assertFileDoesNotExist(filePath);
+  }
+
+  @Test
+  public void testCreateDirRecursive() throws IOException {
+    Path path = new Path("/aa/bb/cc");
+    String key = ObjectUtils.pathToKey(path, true);
+    String parentKey = ObjectUtils.pathToKey(path.getParent(), true);
+    String grandparents = ObjectUtils.pathToKey(path.getParent().getParent(), true);
+
+    assertDirDoesNotExist(parentKey);
+    assertDirDoesNotExist(grandparents);
+
+    fsOps().mkdirs(path);
+    assertDirExist(key);
+    assertDirExist(parentKey);
+    assertDirExist(grandparents);
+
+    storage.delete(key);
+    assertDirExist(parentKey);
+    assertDirExist(grandparents);
+  }
+
+  @Test
+  public void testListEmptyDir() {
+    Path dir = path("testListEmptyDir");
+    mkdir(dir);
+
+    Assert.assertFalse(listDir(dir, false).iterator().hasNext());
+    Assert.assertFalse(listDir(dir, true).iterator().hasNext());
+    Assert.assertTrue(fsOps().isEmptyDirectory(dir));
+  }
+
+  @Test
+  public void testListNonExistDir() {
+    Path dir = path("testListNonExistDir");
+    assertDirDoesNotExist(dir);
+
+    Assert.assertFalse(listDir(dir, false).iterator().hasNext());
+    Assert.assertFalse(listDir(dir, false).iterator().hasNext());
+    Assert.assertTrue(fsOps().isEmptyDirectory(dir));
+  }
+
+  private Iterable<RawFileStatus> listDir(Path dir, boolean recursive) {
+    return fsOps().listDir(dir, recursive, s -> true);
+  }
+
+  private Iterable<RawFileStatus> listFiles(Path dir, boolean recursive) {
+    return fsOps().listDir(dir, recursive, s -> !ObjectInfo.isDir(s));
+  }
+
+  @Test
+  public void testListAFileViaListDir() {
+    Path file = new Path("testListFileViaListDir");
+    touchFile(file, TestUtility.rand(8));
+    Assert.assertFalse(listDir(file, false).iterator().hasNext());
+    Assert.assertFalse(listDir(file, true).iterator().hasNext());
+
+    Path nonExistFile = new Path("testListFileViaListDir-nonExist");
+    assertFileDoesNotExist(nonExistFile);
+    Assert.assertFalse(listDir(nonExistFile, false).iterator().hasNext());
+    Assert.assertFalse(listDir(nonExistFile, true).iterator().hasNext());
+  }
+
+  @Test
+  public void testListFiles() {
+    Path dir = path("testListEmptyFiles");
+    mkdir(dir);
+
+    Assert.assertFalse(listFiles(dir, false).iterator().hasNext());
+    Assert.assertFalse(listFiles(dir, true).iterator().hasNext());
+
+    mkdir(new Path(dir, "subDir"));
+    Assert.assertFalse(listFiles(dir, false).iterator().hasNext());
+    Assert.assertFalse(listFiles(dir, true).iterator().hasNext());
+
+    RawFileStatus subDir = listDir(dir, false).iterator().next();
+    Assert.assertFalse(subDir.isFile());
+    Assert.assertEquals("subDir", subDir.getPath().getName());
+
+    ObjectInfo fileObj = touchFile(new Path(dir, "subFile"), TestUtility.rand(8));
+    RawFileStatus subFile = listFiles(dir, false).iterator().next();
+    assertArrayEquals(fileObj.checksum(), subFile.checksum());
+    Assert.assertTrue(subFile.isFile());
+
+    Assert.assertFalse(fsOps().isEmptyDirectory(dir));
+  }
+
+  @Test
+  public void testRecursiveList() {
+    Path root = path("root");
+    Path file1 = path("root", "file1");
+    Path file2 = path("root", "afile2");
+    Path dir1 = path("root", "dir1");
+    Path file3 = path("root", "dir1", "file3");
+
+    mkdir(root);
+    mkdir(dir1);
+    touchFile(file1, TestUtility.rand(8));
+    touchFile(file2, TestUtility.rand(8));
+    touchFile(file3, TestUtility.rand(8));
+
+    // List result is in sorted lexicographical order if recursive is false
+    Assertions.assertThat(listDir(root, false))
+        .hasSize(3)
+        .extracting(f -> f.getPath().getName())
+        .contains("afile2", "dir1", "file1");
+
+    // List result is in sorted lexicographical order if recursive is false
+    Assertions.assertThat(listFiles(root, false))
+        .hasSize(2)
+        .extracting(f -> f.getPath().getName())
+        .contains("afile2", "file1");
+
+    // listDir with recursive=true doesn't guarantee the return result in a sorted order
+    Assertions.assertThat(listDir(root, true))
+        .hasSize(4)
+        .extracting(f -> f.getPath().getName())
+        .containsExactlyInAnyOrder("afile2", "dir1", "file1", "file3");
+
+    // listFiles with recursive=true doesn't guarantee the return result in a sorted order
+    Assertions.assertThat(listFiles(root, true))
+        .hasSize(3)
+        .extracting(f -> f.getPath().getName())
+        .containsExactlyInAnyOrder("afile2", "file1", "file3");
+  }
+
+  @Test
+  public void testRenameFile() throws IOException {
+    Path renameSrc = path("renameSrc");
+    Path renameDest = path("renameDst");
+
+    int dataSize = 1024 * 1024;
+    String filename = String.format("%sMB.txt", dataSize >> 20);
+    Path srcFile = new Path(renameSrc, filename);
+    byte[] data = writeData(srcFile, dataSize);
+    Path dstFile = new Path(renameDest, filename);
+
+    // The dest file and dest parent don't exist.
+    assertFileExist(srcFile);
+    assertDirDoesNotExist(renameDest);
+    assertFileDoesNotExist(dstFile);
+
+    fsOps().renameFile(srcFile, dstFile, data.length);
+    assertFileDoesNotExist(srcFile);
+    assertDirExist(renameSrc);
+    assertFileExist(dstFile);
+
+    try (InputStream in = storage.get(ObjectUtils.pathToKey(dstFile)).stream()) {
+      assertArrayEquals(data, IOUtils.toByteArray(in));
+    }
+  }
+
+  @Test
+  public void testRenameDir() throws IOException {
+    Path renameSrc = path("renameSrc");
+    Path renameDest = path("renameDst");
+
+    mkdir(renameSrc);
+    int dataSize = 1024 * 1024;
+    String filename = String.format("%sMB.txt", dataSize >> 20);
+    Path srcFile = new Path(renameSrc, filename);
+    Path dstFile = new Path(renameDest, filename);
+    byte[] data = writeData(srcFile, dataSize);
+
+    assertFileExist(srcFile);
+    assertFileDoesNotExist(dstFile);
+    assertDirExist(renameSrc);
+    assertDirDoesNotExist(renameDest);
+
+    fsOps().renameDir(renameSrc, renameDest);
+    assertFileDoesNotExist(srcFile);
+    assertDirDoesNotExist(renameSrc);
+    assertFileExist(dstFile);
+    assertDirExist(renameDest);
+
+    try (InputStream in = storage.get(ObjectUtils.pathToKey(dstFile)).stream()) {
+      assertArrayEquals(data, IOUtils.toByteArray(in));
+    }
+  }
+
+  private byte[] writeData(Path path, int size) {
+    byte[] data = TestUtility.rand(size);
+    touchFile(path, data);
+    return data;
+  }
+}

+ 96 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestBaseOps.java

@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.ops;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.tosfs.RawFileStatus;
+import org.apache.hadoop.fs.tosfs.RawFileSystem;
+import org.apache.hadoop.fs.tosfs.object.ObjectInfo;
+import org.apache.hadoop.fs.tosfs.object.ObjectStorage;
+import org.apache.hadoop.fs.tosfs.object.ObjectUtils;
+import org.apache.hadoop.thirdparty.com.google.common.base.Joiner;
+import org.junit.Assert;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+public abstract class TestBaseOps {
+  protected ObjectStorage storage;
+
+  public TestBaseOps(ObjectStorage storage) {
+    this.storage = storage;
+  }
+
+  public Path path(String... keys) {
+    return new Path(String.format("/%s", Joiner.on("/").join(keys)));
+  }
+
+  public void assertFileExist(Path file) {
+    assertNotNull(ObjectUtils.pathToKey(file));
+  }
+
+  public void assertFileDoesNotExist(String key) {
+    assertNull(storage.head(key));
+  }
+
+  public void assertFileDoesNotExist(Path file) {
+    assertFileDoesNotExist(ObjectUtils.pathToKey(file));
+  }
+
+  public void assertDirExist(String key) {
+    assertNotNull(storage.head(key));
+  }
+
+  public void assertDirExist(Path path) {
+    assertDirExist(ObjectUtils.pathToKey(path, true));
+  }
+
+  public void assertDirDoesNotExist(String key) {
+    assertNull(storage.head(key));
+  }
+
+  public void assertDirDoesNotExist(Path path) {
+    assertDirDoesNotExist(ObjectUtils.pathToKey(path, true));
+  }
+
+  public void mkdir(Path path) {
+    storage.put(ObjectUtils.pathToKey(path, true), new byte[0]);
+    assertDirExist(path);
+  }
+
+  public ObjectInfo touchFile(Path path, byte[] data) {
+    byte[] checksum = storage.put(ObjectUtils.pathToKey(path), data);
+    ObjectInfo obj = storage.head(ObjectUtils.pathToKey(path));
+    Assert.assertArrayEquals(checksum, obj.checksum());
+    return obj;
+  }
+
+  public FileStatus getFileStatus(Path path) {
+    String key = ObjectUtils.pathToKey(path);
+    return toFileStatus(storage.objectStatus(key));
+  }
+
+  public RawFileStatus toFileStatus(ObjectInfo obj) {
+    long modifiedTime = RawFileSystem.dateToLong(obj.mtime());
+    String path = String.format("%s://%s/%s", storage.scheme(), storage.bucket().name(), obj.key());
+    return new RawFileStatus(obj.size(), obj.isDir(), 0, modifiedTime, new Path(path), "fake", obj.checksum());
+  }
+}
+

+ 78 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestDefaultFsOps.java

@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.ops;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.tosfs.common.ThreadPools;
+import org.apache.hadoop.fs.tosfs.conf.ConfKeys;
+import org.apache.hadoop.fs.tosfs.object.ObjectStorageFactory;
+import org.apache.hadoop.fs.tosfs.util.TestUtility;
+import org.apache.hadoop.fs.tosfs.util.UUIDUtils;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+
+import static org.apache.hadoop.fs.tosfs.object.tos.TOS.TOS_SCHEME;
+
+@RunWith(Parameterized.class)
+public class TestDefaultFsOps extends TestBaseFsOps {
+  private static ExecutorService threadPool;
+  private final FsOps fsOps;
+
+  public TestDefaultFsOps(Configuration conf) {
+    super(ObjectStorageFactory.createWithPrefix(
+        String.format("tos-%s/", UUIDUtils.random()), TOS_SCHEME, TestUtility.bucket(), conf));
+
+    this.fsOps = new DefaultFsOps(storage, new Configuration(conf), threadPool, this::toFileStatus);
+  }
+
+  @Parameterized.Parameters(name = "conf = {0}")
+  public static List<Configuration> createConf() {
+    Configuration directRenameConf = new Configuration();
+    directRenameConf.setBoolean(ConfKeys.OBJECT_RENAME_ENABLED.key("tos"), true);
+    directRenameConf.setBoolean(ConfKeys.ASYNC_CREATE_MISSED_PARENT.key("tos"), false);
+
+    Configuration copiedRenameConf = new Configuration();
+    copiedRenameConf.setLong(ConfKeys.MULTIPART_COPY_THRESHOLD.key("tos"), 1L << 20);
+    copiedRenameConf.setBoolean(ConfKeys.ASYNC_CREATE_MISSED_PARENT.key("tos"), false);
+    return Lists.newArrayList(directRenameConf, copiedRenameConf);
+  }
+
+  @BeforeClass
+  public static void beforeClass() {
+    threadPool = ThreadPools.newWorkerPool("TestDefaultFsHelper-pool");
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    if (!threadPool.isShutdown()) {
+      threadPool.shutdown();
+    }
+  }
+
+  @Override
+  public FsOps fsOps() {
+    return fsOps;
+  }
+}

+ 62 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestDirectoryFsOps.java

@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.ops;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.tosfs.object.DirectoryStorage;
+import org.apache.hadoop.fs.tosfs.object.ObjectStorageFactory;
+import org.apache.hadoop.fs.tosfs.util.TestUtility;
+import org.apache.hadoop.fs.tosfs.util.UUIDUtils;
+import org.junit.Assume;
+
+import static org.apache.hadoop.fs.tosfs.object.tos.TOS.TOS_SCHEME;
+
+// TODO change to directory bucket configuration.
+public class TestDirectoryFsOps extends TestBaseFsOps {
+  private final FsOps fsOps;
+
+  public TestDirectoryFsOps() {
+    super(ObjectStorageFactory.createWithPrefix(
+        String.format("tos-%s/", UUIDUtils.random()), TOS_SCHEME, TestUtility.bucket(), new Configuration()));
+    this.fsOps = new DirectoryFsOps((DirectoryStorage) storage, this::toFileStatus);
+  }
+
+  @Override
+  public void testRenameDir() {
+    // Will remove this test case once test environment support
+    Assume.assumeTrue(storage.bucket().isDirectory());
+  }
+
+  @Override
+  public void testRenameFile() {
+    // Will remove this test case once test environment support
+    Assume.assumeTrue(storage.bucket().isDirectory());
+  }
+
+  @Override
+  public FsOps fsOps() {
+    return fsOps;
+  }
+
+  @Override
+  public void testCreateDirRecursive() {
+    // Will remove this test case once test environment support
+    Assume.assumeTrue(storage.bucket().isDirectory());
+  }
+}

+ 171 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/ops/TestRenameOp.java

@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.ops;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.tosfs.common.ThreadPools;
+import org.apache.hadoop.fs.tosfs.conf.ConfKeys;
+import org.apache.hadoop.fs.tosfs.object.ObjectStorage;
+import org.apache.hadoop.fs.tosfs.object.ObjectUtils;
+import org.apache.hadoop.fs.tosfs.object.Part;
+import org.apache.hadoop.fs.tosfs.util.CommonUtils;
+import org.apache.hadoop.fs.tosfs.util.TempFiles;
+import org.apache.hadoop.fs.tosfs.util.TestUtility;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Maps;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class TestRenameOp extends TestBaseOps {
+  private static final String FILE_STORE_ROOT = TempFiles.newTempDir("TestRenameOp");
+
+  private final ExecutorService renamePool;
+  private ExtendedRenameOp operation;
+
+  public TestRenameOp(ObjectStorage storage) {
+    super(storage);
+    this.renamePool = ThreadPools.newWorkerPool("renamePool");
+  }
+
+  @Parameterized.Parameters
+  public static List<ObjectStorage> createStorage() {
+    return TestUtility.createTestObjectStorage(FILE_STORE_ROOT);
+  }
+
+  @After
+  public void tearDown() {
+    CommonUtils.runQuietly(() -> storage.deleteAll(""));
+    CommonUtils.runQuietly(renamePool::shutdown);
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    CommonUtils.runQuietly(() -> TempFiles.deleteDir(FILE_STORE_ROOT));
+  }
+
+  @Test
+  public void testRenameFileDirectly() throws IOException {
+    Configuration conf = new Configuration();
+    conf.setLong(ConfKeys.MULTIPART_COPY_THRESHOLD.key(storage.scheme()), 1L << 20);
+    operation = new ExtendedRenameOp(conf, storage, renamePool);
+
+    Path renameSrc = path("renameSrc");
+    Path renameDest = path("renameDst");
+
+    int dataSize = 1024 * 1024;
+    String filename = String.format("%sMB.txt", dataSize >> 20);
+    Path srcFile = new Path(renameSrc, filename);
+    Path dstFile = new Path(renameDest, filename);
+    byte[] data = writeData(srcFile, dataSize);
+    mkdir(renameDest);
+
+    assertFileExist(srcFile);
+    assertFileDoesNotExist(dstFile);
+    assertDirExist(renameDest);
+
+    operation.renameFile(srcFile, dstFile, data.length);
+    assertFileDoesNotExist(srcFile);
+    assertFileExist(dstFile);
+    Map<String, List<Part>> uploadInfos = operation.uploadInfos;
+    assertEquals("use put method when rename file, upload info's size should be 0", 0, uploadInfos.size());
+
+    try (InputStream in = storage.get(ObjectUtils.pathToKey(dstFile)).stream()) {
+      assertArrayEquals(data, IOUtils.toByteArray(in));
+    }
+  }
+
+  @Test
+  public void testRenameFileByUploadParts() throws IOException {
+    Assume.assumeFalse(storage.bucket().isDirectory());
+    Configuration conf = new Configuration();
+    conf.setLong(ConfKeys.MULTIPART_COPY_THRESHOLD.key(storage.scheme()), 1L << 20);
+    operation = new ExtendedRenameOp(conf, storage, renamePool);
+
+    Path renameSrc = path("renameSrc");
+    Path renameDest = path("renameDst");
+
+    int dataSize = 10 * 1024 * 1024;
+    String filename = String.format("%sMB.txt", dataSize >> 20);
+    Path srcFile = new Path(renameSrc, filename);
+    Path dstFile = new Path(renameDest, filename);
+    byte[] data = writeData(srcFile, dataSize);
+    mkdir(renameDest);
+
+    assertFileExist(srcFile);
+    assertFileDoesNotExist(dstFile);
+    assertDirExist(renameDest);
+
+    operation.renameFile(srcFile, dstFile, data.length);
+    assertFileDoesNotExist(srcFile);
+    assertFileExist(dstFile);
+    Map<String, List<Part>> uploadInfos = operation.uploadInfos;
+    assertTrue("use upload parts method when rename file, upload info's size should not be 0",
+        uploadInfos.size() != 0);
+    List<Part> parts = uploadInfos.get(ObjectUtils.pathToKey(dstFile));
+    assertNotNull("use upload parts method when rename file, upload info should not be null", parts);
+    assertTrue("use upload parts method when rename file, the num of upload parts should be greater than or equal" +
+        " to 2", parts.size() >= 2);
+    long fileLength = parts.stream().mapToLong(Part::size).sum();
+    assertEquals(dataSize, fileLength);
+
+    try (InputStream in = storage.get(ObjectUtils.pathToKey(dstFile)).stream()) {
+      assertArrayEquals(data, IOUtils.toByteArray(in));
+    }
+  }
+
+  private byte[] writeData(Path path, int size) {
+    byte[] data = TestUtility.rand(size);
+    touchFile(path, data);
+    return data;
+  }
+
+  static class ExtendedRenameOp extends RenameOp {
+    public Map<String, List<Part>> uploadInfos = Maps.newHashMap();
+
+    ExtendedRenameOp(Configuration conf, ObjectStorage storage, ExecutorService pool) {
+      super(conf, storage, pool);
+    }
+
+    @Override
+    protected void finishUpload(String key, String uploadId, List<Part> uploadParts) {
+      super.finishUpload(key, uploadId, uploadParts);
+      if (!uploadInfos.isEmpty()) {
+        uploadInfos.clear();
+      }
+      uploadInfos.put(key, uploadParts);
+    }
+  }
+}

+ 0 - 22
hadoop-cloud-storage-project/hadoop-tos/src/test/java/org/apache/hadoop/fs/tosfs/util/TestUtility.java

@@ -39,8 +39,6 @@ import java.util.Random;
 public class TestUtility {
   private static final String ENV_TOS_BUCKET = "TOS_BUCKET";
   private static final String ENV_TEST_SCHEME = "TEST_SCHEME";
-  private static final String ENV_DIRECTORY_BUCKET = "DIRECTORY_BUCKET";
-  private static final String ENV_DIRECTORY_BUCKET_ENDPOINT = "DIRECTORY_BUCKET_ENDPOINT";
   private static final Random RND = new Random(System.currentTimeMillis());
 
   private TestUtility() {
@@ -152,20 +150,6 @@ public class TestUtility {
         String.format("%s-%s/", scheme(), UUIDUtils.random()), scheme(), bucket(), conf);
   }
 
-  public static ObjectStorage directoryBucketObjectStorage(Configuration conf) {
-    String bucket = ParseUtils.envAsString(ENV_DIRECTORY_BUCKET);
-    if (StringUtils.isEmpty(bucket)) {
-      return null;
-    } else {
-      String endpoint = ParseUtils.envAsString(ENV_DIRECTORY_BUCKET_ENDPOINT, "");
-      if (!StringUtils.isEmpty(endpoint)) {
-        conf.set(ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key(scheme()), endpoint);
-      }
-      return ObjectStorageFactory.createWithPrefix(
-          String.format("%s-%s/", scheme(), UUIDUtils.random()), scheme(), bucket, conf);
-    }
-  }
-
   public static List<ObjectStorage> createTestObjectStorage(String fileStoreRoot) {
     List<ObjectStorage> storages = new ArrayList<>();
 
@@ -177,12 +161,6 @@ public class TestUtility {
     // 2. General Bucket
     storages.add(generalBucketObjectStorage());
 
-    // 3. Directory Bucket is optional
-    ObjectStorage directoryObjectStorage = directoryBucketObjectStorage(new Configuration());
-    if (directoryObjectStorage != null) {
-      storages.add(directoryObjectStorage);
-    }
-
     return storages;
   }
 }

+ 113 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/resources/contract/tos.xml

@@ -0,0 +1,113 @@
+<configuration>
+  <property>
+    <name>fs.contract.test.fs.tos</name>
+    <value>tos://proton-ci/</value>
+  </property>
+
+  <property>
+    <name>fs.contract.test.random-seek-count</name>
+    <value>10</value>
+  </property>
+
+  <property>
+    <name>fs.contract.is-blobstore</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.create-visibility-delayed</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.is-case-sensitive</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.rename-returns-false-if-source-missing</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.rename-returns-false-if-dest-exists</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-append</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-atomic-directory-delete</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-atomic-rename</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-block-locality</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-concat</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-unbuffer</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.rename-creates-dest-dirs</name>
+    <value>true</value>
+  </property>
+
+  <!--be careful that the root test cases will clean whole bucket-->
+  <property>
+    <name>fs.contract.test.root-tests-enabled</name>
+    <value>false</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-getfilestatus</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-seek</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-seek-on-closed-file</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.rejects-seek-past-eof</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-strict-exceptions</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-multipartuploader</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-unix-permissions</name>
+    <value>false</value>
+  </property>
+
+</configuration>

+ 55 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/resources/core-site.xml

@@ -0,0 +1,55 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  ~  Licensed to the Apache Software Foundation (ASF) under one
+  ~  or more contributor license agreements.  See the NOTICE file
+  ~  distributed with this work for additional information
+  ~  regarding copyright ownership.  The ASF licenses this file
+  ~  to you under the Apache License, Version 2.0 (the
+  ~  "License"); you may not use this file except in compliance
+  ~  with the License.  You may obtain a copy of the License at
+  ~
+  ~      http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~  Unless required by applicable law or agreed to in writing, software
+  ~  distributed under the License is distributed on an "AS IS" BASIS,
+  ~  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~  See the License for the specific language governing permissions and
+  ~  limitations under the License.
+  -->
+<configuration>
+
+  <property>
+    <name>hadoop.tmp.dir</name>
+    <value>target/build/test</value>
+    <description>A base for other temporary directories.</description>
+    <final>true</final>
+  </property>
+
+  <!-- Turn security off for tests by default -->
+  <property>
+    <name>hadoop.security.authentication</name>
+    <value>simple</value>
+  </property>
+
+  <property>
+    <name>fs.tos.impl</name>
+    <value>org.apache.hadoop.fs.tosfs.RawFileSystem</value>
+  </property>
+  <property>
+    <name>fs.AbstractFileSystem.tos.impl</name>
+    <value>org.apache.hadoop.fs.tosfs.RawFS</value>
+  </property>
+  <property>
+    <name>fs.filestore.impl</name>
+    <value>org.apache.hadoop.fs.tosfs.RawFileSystem</value>
+  </property>
+  <property>
+    <name>fs.AbstractFileSystem.filestore.impl</name>
+    <value>org.apache.hadoop.fs.tosfs.RawFS</value>
+  </property>
+  <property>
+    <name>fs.filestore.impl.disable.cache</name>
+    <value>true</value>
+  </property>
+</configuration>

+ 23 - 0
hadoop-cloud-storage-project/hadoop-tos/src/test/resources/log4j.properties

@@ -0,0 +1,23 @@
+#
+#   Licensed to the Apache Software Foundation (ASF) under one or more
+#   contributor license agreements.  See the NOTICE file distributed with
+#   this work for additional information regarding copyright ownership.
+#   The ASF licenses this file to You under the Apache License, Version 2.0
+#   (the "License"); you may not use this file except in compliance with
+#   the License.  You may obtain a copy of the License at
+#
+#       http://www.apache.org/licenses/LICENSE-2.0
+#
+#   Unless required by applicable law or agreed to in writing, software
+#   distributed under the License is distributed on an "AS IS" BASIS,
+#   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#   See the License for the specific language governing permissions and
+#   limitations under the License.
+#
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n