stream, String prefix, String startAfter) {
+    return stream
+        .filter(p -> {
+          String absolutePath = p.toFile().getAbsolutePath();
+          return !Objects.equals(key(absolutePath), "") &&
+              decode(key(absolutePath)).startsWith(prefix)
+              && !absolutePath.contains(STAGING_DIR)
+              && filter(decode(key(absolutePath)), startAfter);
+        })
+        .map(this::toObjectInfo)
+        .sorted(Comparator.comparing(ObjectInfo::key));
+  }
+
+  private boolean filter(String key, String startAfter) {
+    if (Strings.isNullOrEmpty(startAfter)) {
+      return true;
+    } else {
+      return key.compareTo(startAfter) > 0;
+    }
+  }
+
+  private ObjectInfo toObjectInfo(Path path) {
+    File file = path.toFile();
+    String key = decode(key(file.getAbsolutePath()));
+    return new ObjectInfo(key, file.length(), new Date(file.lastModified()),
+        getFileChecksum(path));
+  }
+
+  private Path path(String key) {
+    return Paths.get(root, key);
+  }
+
+  private String key(String path) {
+    if (path.length() < root.length()) {
+      // root = path + "/"
+      return "";
+    }
+    return path.substring(root.length());
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/InputStreamProvider.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/InputStreamProvider.java
new file mode 100644
index 0000000000000..98e654f99b5e0
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/InputStreamProvider.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object;
+
+import java.io.InputStream;
+
+/**
+ * Provides the content stream of a request.
+ * 
+ * Each call to the {@link #newStream()} method must result in a stream
+ * whose position is at the beginning of the content.
+ * Implementations may return a new stream or the same stream for each call.
+ * If returning a new stream, the implementation must ensure to {@code close()}
+ * and free any resources acquired by the previous stream.
+ */
+public interface InputStreamProvider {
+  InputStream newStream();
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/MultipartUpload.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/MultipartUpload.java
new file mode 100644
index 0000000000000..e7c89f1f8279b
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/MultipartUpload.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.MoreObjects;
+
+import java.util.Objects;
+
+public class MultipartUpload implements Comparable {
+  private final String key;
+  private final String uploadId;
+  private final int minPartSize;
+  private final int maxPartCount;
+
+  public MultipartUpload(String key, String uploadId, int minPartSize, int maxPartCount) {
+    this.key = key;
+    this.uploadId = uploadId;
+    this.minPartSize = minPartSize;
+    this.maxPartCount = maxPartCount;
+  }
+
+  public String key() {
+    return key;
+  }
+
+  public String uploadId() {
+    return uploadId;
+  }
+
+  public int minPartSize() {
+    return minPartSize;
+  }
+
+  public int maxPartCount() {
+    return maxPartCount;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    } else if (!(o instanceof MultipartUpload)) {
+      return false;
+    }
+
+    MultipartUpload that = (MultipartUpload) o;
+    if (!Objects.equals(key, that.key)) {
+      return false;
+    }
+    if (!Objects.equals(uploadId, that.uploadId)) {
+      return false;
+    }
+    if (!Objects.equals(minPartSize, that.minPartSize)) {
+      return false;
+    }
+    return Objects.equals(maxPartCount, that.maxPartCount);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(key, uploadId, minPartSize, maxPartCount);
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("key", key)
+        .add("uploadId", uploadId)
+        .add("minPartSize", minPartSize)
+        .add("maxPartCount", maxPartCount)
+        .toString();
+  }
+
+  @Override
+  public int compareTo(MultipartUpload o) {
+    if (this == o) {
+      return 0;
+    } else if (o == null) {
+      return 1;
+    } else if (this.key.compareTo(o.key) == 0) {
+      return this.uploadId.compareTo(o.uploadId);
+    } else {
+      return this.key.compareTo(o.key);
+    }
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectConstants.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectConstants.java
new file mode 100644
index 0000000000000..fb42b0091bc63
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectConstants.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object;
+
+public final class ObjectConstants {
+  public static final int MIN_PART_SIZE = 5 * 1024 * 1024;
+  public static final int MAX_PART_COUNT = 10000;
+
+  private ObjectConstants() {
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectContent.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectContent.java
new file mode 100644
index 0000000000000..6961fd83bb7bb
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectContent.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object;
+
+import org.apache.hadoop.fs.tosfs.object.exceptions.ChecksumMismatchException;
+import org.apache.hadoop.fs.tosfs.util.CommonUtils;
+
+import java.io.InputStream;
+import java.util.Arrays;
+
+public class ObjectContent {
+  private final byte[] checksum;
+  private final InputStream stream;
+
+  public ObjectContent(byte[] checksum, InputStream stream) {
+    this.checksum = checksum;
+    this.stream = stream;
+  }
+
+  public InputStream stream() {
+    return stream;
+  }
+
+  public InputStream verifiedStream(byte[] expectedChecksum) throws ChecksumMismatchException {
+    if (!Arrays.equals(expectedChecksum, checksum)) {
+      CommonUtils.runQuietly(stream::close);
+      throw new ChecksumMismatchException(expectedChecksum, checksum);
+    }
+
+    return stream;
+  }
+
+  public byte[] checksum() {
+    return checksum;
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectInfo.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectInfo.java
new file mode 100644
index 0000000000000..08f38ecbdd961
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectInfo.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.MoreObjects;
+import org.apache.hadoop.util.StringUtils;
+
+import java.util.Arrays;
+import java.util.Date;
+import java.util.Objects;
+
+import static org.apache.hadoop.util.Preconditions.checkArgument;
+
+public class ObjectInfo {
+  private final String key;
+  private final long size;
+  private final Date mtime;
+  private final boolean isDir;
+  private final byte[] checksum;
+
+  public ObjectInfo(String key, long size, Date mtime, byte[] checksum) {
+    this(key, size, mtime, checksum, ObjectInfo.isDir(key));
+  }
+
+  public ObjectInfo(String key, long size, Date mtime, byte[] checksum, boolean isDir) {
+    checkArgument(key != null, "Key is null");
+    checkArgument(size >= 0, "The size of key(%s) is negative", key);
+    checkArgument(mtime != null, "The modified time of key(%s) null.", key);
+    this.key = key;
+    this.size = size;
+    this.mtime = mtime;
+    this.isDir = isDir;
+    // checksum can be null since some object storage might not support checksum.
+    this.checksum = checksum == null || isDir ? Constants.MAGIC_CHECKSUM : checksum;
+  }
+
+  public String key() {
+    return key;
+  }
+
+  /**
+   * The size of directory object is 0.
+   *
+   * @return the size of object.
+   */
+  public long size() {
+    return isDir ? 0 : size;
+  }
+
+  public Date mtime() {
+    return mtime;
+  }
+
+  /**
+   * @return {@link Constants#MAGIC_CHECKSUM} if the object is a dir or the object storage
+   * doesn't support the given checksum type.
+   */
+  public byte[] checksum() {
+    return checksum;
+  }
+
+  public boolean isDir() {
+    return isDir;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    } else if (!(o instanceof ObjectInfo)) {
+      return false;
+    }
+
+    ObjectInfo that = (ObjectInfo) o;
+    return Objects.equals(key, that.key)
+        && Objects.equals(size, that.size)
+        && Objects.equals(mtime, that.mtime)
+        && Arrays.equals(checksum, that.checksum)
+        && Objects.equals(isDir, that.isDir);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(key, size, mtime, Arrays.hashCode(checksum), isDir);
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("key", key)
+        .add("size", size)
+        .add("mtime", mtime)
+        .add("checksum", StringUtils.byteToHexString(checksum))
+        .add("isDir", isDir)
+        .toString();
+  }
+
+  public static boolean isDir(String key) {
+    return key.endsWith(Constants.SLASH);
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectMultiRangeInputStream.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectMultiRangeInputStream.java
new file mode 100644
index 0000000000000..4a9357e46bf75
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectMultiRangeInputStream.java
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object;
+
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.tosfs.util.CommonUtils;
+import org.apache.hadoop.fs.tosfs.util.FSUtils;
+import org.apache.hadoop.fs.tosfs.util.Range;
+import org.apache.hadoop.thirdparty.com.google.common.primitives.Ints;
+import org.apache.hadoop.util.Preconditions;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class ObjectMultiRangeInputStream extends FSInputStream {
+  private final AtomicBoolean closed = new AtomicBoolean(false);
+  private final ExecutorService threadPool;
+  private final ObjectStorage storage;
+  private final String objectKey;
+  private final long contentLength;
+  private final long rangeSize;
+
+  private volatile ObjectRangeInputStream stream;
+  private volatile long nextPos = 0;
+  private volatile long currPos = 0;
+  // All range streams should have same checksum.
+  private final byte[] checksum;
+
+  public ObjectMultiRangeInputStream(
+      ExecutorService threadPool,
+      ObjectStorage storage,
+      Path path,
+      long contentLength,
+      long rangeSize,
+      byte[] checksum) {
+    this(threadPool, storage, ObjectUtils.pathToKey(path), contentLength, rangeSize, checksum);
+  }
+
+  public ObjectMultiRangeInputStream(
+      ExecutorService threadPool,
+      ObjectStorage storage,
+      String objectKey,
+      long contentLength,
+      long rangeSize,
+      byte[] checksum) {
+    this.threadPool = threadPool;
+    this.storage = storage;
+    this.objectKey = objectKey;
+    this.contentLength = contentLength;
+    this.rangeSize = rangeSize;
+    this.checksum = checksum;
+
+    Preconditions.checkNotNull(checksum, "Checksum should not be null.");
+  }
+
+  @Override
+  public synchronized void seek(long pos) throws IOException {
+    if (pos < 0) {
+      throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK + " " + pos);
+    }
+
+    if (contentLength <= 0) {
+      return;
+    }
+
+    nextPos = pos;
+  }
+
+  @Override
+  public synchronized long getPos() {
+    return nextPos;
+  }
+
+  @Override
+  public synchronized boolean seekToNewSource(long targetPos) throws IOException {
+    checkNotClosed();
+    return false;
+  }
+
+  @Override
+  public synchronized int read() throws IOException {
+    byte[] buf = new byte[1];
+    int n = read(buf, 0, buf.length);
+    if (n < 0) {
+      return -1;
+    } else {
+      return buf[0] & 0xFF;
+    }
+  }
+
+  @Override
+  public synchronized int read(byte[] buffer, int offset, int length) throws IOException {
+    checkNotClosed();
+    FSUtils.checkReadParameters(buffer, offset, length);
+    if (length == 0) {
+      return 0;
+    }
+
+    int total = 0;
+    while (total < length) {
+      if (contentLength == 0 || nextPos >= contentLength) {
+        return total == 0 ? -1 : total;
+      }
+
+      seekStream();
+      int n = stream.read(buffer, offset, length - total);
+      if (n < 0) {
+        return total == 0 ? -1 : total;
+      }
+
+      total += n;
+      offset += n;
+      currPos += n;
+      nextPos += n;
+    }
+
+    return total;
+  }
+
+  @Override
+  public int read(long position, byte[] buffer, int offset, int length) throws IOException {
+    checkNotClosed();
+    // Check the arguments, according to the HDFS contract.
+    if (position < 0) {
+      throw new EOFException("position is negative");
+    }
+    FSUtils.checkReadParameters(buffer, offset, length);
+    if (length == 0) {
+      return 0;
+    }
+
+    if (contentLength == 0 || position >= contentLength) {
+      return -1;
+    }
+
+    long remaining = contentLength - position;
+    int limit = (remaining >= length) ? length : (int) remaining;
+
+    try (InputStream in = storage.get(objectKey, position, limit).verifiedStream(checksum)) {
+      return in.read(buffer, offset, limit);
+    }
+  }
+
+  private void seekStream() throws IOException {
+    if (stream != null && stream.include(nextPos)) {
+      // Seek to a random position which is still located in the current range of stream.
+      if (nextPos != currPos) {
+        stream.seek(nextPos);
+        currPos = nextPos;
+      }
+      return;
+    }
+
+    // Seek to a position which is located in another range of new stream.
+    currPos = nextPos;
+    openStream();
+  }
+
+  private void openStream() throws IOException {
+    closeStream(true);
+
+    long off = (nextPos / rangeSize) * rangeSize;
+    Range range = Range.of(off, Math.min(contentLength - off, rangeSize));
+    if (nextPos < range.end()) {
+      stream = new ObjectRangeInputStream(storage, objectKey, range, checksum);
+      stream.seek(nextPos);
+    }
+  }
+
+  private void closeStream(boolean asyncClose) throws IOException {
+    if (stream != null) {
+      if (asyncClose) {
+        final ObjectRangeInputStream streamToClose = stream;
+        threadPool.submit(() -> CommonUtils.runQuietly(streamToClose::close));
+      } else {
+        stream.close();
+      }
+      stream = null;
+    }
+  }
+
+  private void checkNotClosed() throws IOException {
+    if (closed.get()) {
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+    }
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    super.close();
+    if (closed.compareAndSet(false, true)) {
+      closeStream(false);
+    }
+  }
+
+  // for test
+  public long nextExpectPos() {
+    return currPos;
+  }
+
+  @Override
+  public synchronized int available() throws IOException {
+    checkNotClosed();
+    return Ints.saturatedCast(contentLength - nextPos);
+  }
+
+  @VisibleForTesting
+  ObjectRangeInputStream stream() {
+    return stream;
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectOutputStream.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectOutputStream.java
new file mode 100644
index 0000000000000..8f86321e2fabc
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectOutputStream.java
@@ -0,0 +1,343 @@
+/*
+ * ByteDance Volcengine EMR, Copyright 2022.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.tosfs.conf.ConfKeys;
+import org.apache.hadoop.fs.tosfs.object.staging.FileStagingPart;
+import org.apache.hadoop.fs.tosfs.object.staging.StagingPart;
+import org.apache.hadoop.fs.tosfs.util.CommonUtils;
+import org.apache.hadoop.fs.tosfs.util.UUIDUtils;
+import org.apache.hadoop.util.Lists;
+import org.apache.hadoop.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.SequenceInputStream;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+public class ObjectOutputStream extends OutputStream {
+  private static final Logger LOG = LoggerFactory.getLogger(ObjectOutputStream.class);
+
+  private final ObjectStorage storage;
+  private final ExecutorService uploadPool;
+  private long totalWroteSize;
+  private final String destKey;
+  private final String destScheme;
+  private final long multiUploadThreshold;
+  private final long byteSizePerPart;
+  private final int stagingBufferSize;
+  private final boolean allowPut;
+  private final List stagingDirs;
+  private final List stagingParts = Lists.newArrayList();
+
+  // For multipart uploads.
+  private final AtomicInteger partNumGetter = new AtomicInteger(0);
+  private MultipartUpload multipartUpload = null;
+  private final List> results = Lists.newArrayList();
+
+  private StagingPart curPart;
+  private final AtomicBoolean closed = new AtomicBoolean(false);
+
+  public ObjectOutputStream(ObjectStorage storage, ExecutorService threadPool, Configuration conf,
+      Path dest, boolean allowPut) {
+    this.storage = storage;
+    this.uploadPool = threadPool;
+    this.destScheme = dest.toUri().getScheme();
+    this.totalWroteSize = 0;
+    this.destKey = createDestKey(dest);
+    this.multiUploadThreshold = conf.getLong(ConfKeys.FS_MULTIPART_THRESHOLD.key(destScheme),
+        ConfKeys.FS_MULTIPART_THRESHOLD_DEFAULT);
+    this.byteSizePerPart = conf.getLong(ConfKeys.FS_MULTIPART_SIZE.key(destScheme),
+        ConfKeys.FS_MULTIPART_SIZE_DEFAULT);
+    this.stagingBufferSize = conf.getInt(ConfKeys.FS_MULTIPART_STAGING_BUFFER_SIZE.key(destScheme),
+        ConfKeys.FS_MULTIPART_STAGING_BUFFER_SIZE_DEFAULT);
+    this.allowPut = allowPut;
+    this.stagingDirs = createStagingDirs(conf, destScheme);
+
+    if (!allowPut) {
+      this.multipartUpload = storage.createMultipartUpload(destKey);
+    }
+  }
+
+  private static List createStagingDirs(Configuration conf, String scheme) {
+    String[] dirs = conf.getStrings(ConfKeys.FS_MULTIPART_STAGING_DIR.key(scheme),
+        ConfKeys.FS_MULTIPART_STAGING_DIR_DEFAULT);
+    Preconditions.checkArgument(dirs != null && dirs.length > 0, "'%s' cannot be an empty list",
+        ConfKeys.FS_MULTIPART_STAGING_DIR.key(scheme));
+
+    List stagingDirs = new ArrayList<>();
+    for (String dir : dirs) {
+      // Create the directory if not exist.
+      File stagingDir = new File(dir);
+      if (!stagingDir.exists() && stagingDir.mkdirs()) {
+        Preconditions.checkArgument(stagingDir.setWritable(true, false),
+            "Failed to change staging dir permission to writable, please check %s with value %s",
+            ConfKeys.FS_MULTIPART_STAGING_DIR.key(scheme), dir);
+        Preconditions.checkArgument(stagingDir.setReadable(true, false),
+            "Failed to change staging dir permission to readable, please check %s with value %s",
+            ConfKeys.FS_MULTIPART_STAGING_DIR.key(scheme), dir);
+      } else {
+        Preconditions.checkArgument(stagingDir.exists(),
+            "Failed to create staging dir, please check %s with value %s",
+            ConfKeys.FS_MULTIPART_STAGING_DIR.key(scheme), dir);
+        Preconditions.checkArgument(stagingDir.isDirectory(),
+            "Staging dir should be a directory, please check %s with value %s",
+            ConfKeys.FS_MULTIPART_STAGING_DIR.key(scheme), dir);
+      }
+      stagingDirs.add(stagingDir);
+    }
+    return stagingDirs;
+  }
+
+  private File chooseStagingDir() {
+    // Choose a random directory from the staging dirs as the candidate staging dir.
+    return stagingDirs.get(ThreadLocalRandom.current().nextInt(stagingDirs.size()));
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    write(new byte[]{(byte) b}, 0, 1);
+  }
+
+  protected String createDestKey(Path dest) {
+    return ObjectUtils.pathToKey(dest);
+  }
+
+  @Override
+  public synchronized void write(byte[] buf, int off, int len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    Preconditions.checkArgument(off >= 0 && off < buf.length,
+        "Invalid offset - off: %s, len: %s, bufferSize: %s", off, len, buf.length);
+    Preconditions.checkArgument(len >= 0 && off + len <= buf.length,
+        "Invalid length - off: %s, len: %s, bufferSize: %s", off, len, buf.length);
+    Preconditions.checkState(!closed.get(), "OutputStream is closed.");
+
+    while (len > 0) {
+      if (curPart == null) {
+        curPart = newStagingPart();
+      }
+
+      Preconditions.checkArgument(curPart.size() <= byteSizePerPart,
+          "Invalid staging size (%s) which is greater than part size (%s)", curPart.size(),
+          byteSizePerPart);
+
+      // size is the remaining length to fill a complete upload part.
+      int size = (int) Math.min(byteSizePerPart - curPart.size(), len);
+      curPart.write(buf, off, size);
+
+      off += size;
+      len -= size;
+      totalWroteSize += size;
+
+      // Switch to the next staging part if current staging part is full.
+      if (curPart.size() >= byteSizePerPart) {
+        curPart.complete();
+
+        // Upload this part if multipart upload was triggered.
+        if (multipartUpload != null) {
+          CompletableFuture result =
+              asyncUploadPart(curPart, partNumGetter.incrementAndGet());
+          results.add(result);
+        }
+
+        // Reset the stagingOut
+        curPart = null;
+      }
+
+      // Trigger the multipart upload when reach the configured threshold.
+      if (multipartUpload == null && totalWroteSize >= multiUploadThreshold) {
+        multipartUpload = storage.createMultipartUpload(destKey);
+        Preconditions.checkState(byteSizePerPart >= multipartUpload.minPartSize(),
+            "Configured upload part size %s must be greater than or equals to the minimal"
+                + " part size %s, please check configure key %s.", byteSizePerPart,
+            multipartUpload.minPartSize(), ConfKeys.FS_MULTIPART_THRESHOLD.key(destScheme));
+
+        // Upload the accumulated staging files whose length >= byteSizePerPart.
+        for (StagingPart stagingPart : stagingParts) {
+          if (stagingPart.size() >= byteSizePerPart) {
+            CompletableFuture result =
+                asyncUploadPart(stagingPart, partNumGetter.incrementAndGet());
+            results.add(result);
+          }
+        }
+      }
+    }
+  }
+
+  private CompletableFuture asyncUploadPart(final StagingPart stagingPart,
+      final int partNum) {
+    final MultipartUpload immutableUpload = multipartUpload;
+    return CompletableFuture.supplyAsync(() -> uploadPart(stagingPart, partNum), uploadPool)
+        .whenComplete((part, err) -> {
+          stagingPart.cleanup();
+          if (err != null) {
+            LOG.error("Failed to upload part, multipartUpload: {}, partNum: {}, stagingPart: {}",
+                immutableUpload, partNum, stagingPart, err);
+          }
+        });
+  }
+
+  private CompletableFuture asyncUploadEmptyPart(final int partNum) {
+    final MultipartUpload immutableUpload = multipartUpload;
+    return CompletableFuture.supplyAsync(
+        () -> storage.uploadPart(
+            immutableUpload.key(),
+            immutableUpload.uploadId(),
+            partNum,
+            () -> new ByteArrayInputStream(new byte[0]),
+            0),
+        uploadPool)
+        .whenComplete((part, err) -> {
+          if (err != null) {
+            LOG.error("Failed to upload empty part, multipartUpload: {}, partNum: {}",
+                immutableUpload, partNum, err);
+          }
+        });
+  }
+
+  private Part uploadPart(StagingPart stagingPart, int partNum) {
+    Preconditions.checkNotNull(storage, "Object storage cannot be null.");
+    Preconditions.checkNotNull(multipartUpload, "Multipart upload is not initialized.");
+    return storage.uploadPart(multipartUpload.key(), multipartUpload.uploadId(),
+        partNum, stagingPart::newIn, stagingPart.size());
+  }
+
+  protected void finishUpload(String key, String uploadId, List parts) throws IOException {
+    storage.completeUpload(key, uploadId, parts);
+  }
+
+  private void simplePut() throws IOException {
+    if (curPart != null) {
+      curPart.complete();
+    }
+    storage.put(
+        destKey,
+        () -> stagingParts()
+            .stream()
+            .map(StagingPart::newIn)
+            .reduce(SequenceInputStream::new)
+            .orElseGet(() -> new ByteArrayInputStream(new byte[0])),
+        stagingParts().stream().mapToLong(StagingPart::size).sum());
+    // Reset the staging output stream.
+    curPart = null;
+  }
+
+  synchronized List waitForPartsUpload() {
+    Preconditions.checkArgument(multipartUpload != null, "Multipart upload cannot be null");
+    Preconditions.checkArgument(!results.isEmpty(), "Upload parts cannot be empty");
+    // Waiting for all the upload parts to be finished.
+    return results.stream()
+        .map(CompletableFuture::join)
+        .sorted(Comparator.comparing(Part::num))
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    if (!closed.compareAndSet(false, true)) {
+      return;
+    }
+
+    try {
+      // Use the simple PUT API if wrote bytes is not reached the multipart threshold.
+      if (multipartUpload == null && allowPut) {
+        simplePut();
+        return;
+      }
+      Preconditions.checkNotNull(multipartUpload,
+          "MultipartUpload cannot be null since allowPut was disabled.");
+
+      // Use multipart upload API to upload those parts.
+      if (totalWroteSize <= 0) {
+        // Write an empty part for this zero-byte file.
+        CompletableFuture result = asyncUploadEmptyPart(partNumGetter.incrementAndGet());
+        results.add(result);
+      } else if (curPart != null) {
+        curPart.complete();
+        // Submit the last part to upload thread pool.
+        CompletableFuture result = asyncUploadPart(curPart, partNumGetter.incrementAndGet());
+        results.add(result);
+        // Reset the staging output stream.
+        curPart = null;
+      }
+
+      // Finish the multipart uploads.
+      finishUpload(multipartUpload.key(), multipartUpload.uploadId(), waitForPartsUpload());
+
+    } catch (Exception e) {
+      LOG.error("Encountering error when closing output stream", e);
+      if (multipartUpload != null) {
+        CommonUtils.runQuietly(
+            () -> storage.abortMultipartUpload(multipartUpload.key(), multipartUpload.uploadId()));
+      }
+      throw e;
+    } finally {
+      // Clear all the staging part.
+      deleteStagingPart(stagingParts);
+    }
+  }
+
+  public long totalWroteSize() {
+    return totalWroteSize;
+  }
+
+  public ObjectStorage storage() {
+    return storage;
+  }
+
+  public List stagingParts() {
+    return stagingParts;
+  }
+
+  public String destKey() {
+    return destKey;
+  }
+
+  public MultipartUpload upload() {
+    return multipartUpload;
+  }
+
+  private void deleteStagingPart(List parts) {
+    for (StagingPart part : parts) {
+      part.cleanup();
+    }
+  }
+
+  private StagingPart newStagingPart() {
+    String stagingPath = String.format("%s/staging-%s.tmp", chooseStagingDir(),
+        UUIDUtils.random());
+    StagingPart part = new FileStagingPart(stagingPath, stagingBufferSize);
+    stagingParts.add(part);
+    return part;
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectRangeInputStream.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectRangeInputStream.java
new file mode 100644
index 0000000000000..8e78dbb03e485
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectRangeInputStream.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object;
+
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.tosfs.util.FSUtils;
+import org.apache.hadoop.fs.tosfs.util.Range;
+import org.apache.hadoop.thirdparty.com.google.common.io.ByteStreams;
+import org.apache.hadoop.thirdparty.com.google.common.primitives.Ints;
+import org.apache.hadoop.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public class ObjectRangeInputStream extends FSInputStream {
+  private static final Logger LOG = LoggerFactory.getLogger(ObjectRangeInputStream.class);
+  private static final int MAX_SKIP_SIZE = 1024 * 1024;
+
+  private final ObjectStorage storage;
+  private final String objectKey;
+  private final Range range;
+  private final byte[] checksum;
+
+  private InputStream stream;
+  private long nextPos;
+  private long currPos;
+  private boolean closed = false;
+
+  public ObjectRangeInputStream(ObjectStorage storage, Path path, Range range, byte[] checksum) {
+    this(storage, ObjectUtils.pathToKey(path), range, checksum);
+  }
+
+  public ObjectRangeInputStream(
+      ObjectStorage storage, String objectKey, Range range, byte[] checksum) {
+    this.storage = storage;
+    this.objectKey = objectKey;
+    this.range = range;
+    this.checksum = checksum;
+
+    this.stream = null;
+    this.nextPos = range.off();
+    this.currPos = nextPos;
+
+    Preconditions.checkNotNull(checksum, "Checksum should not be null.");
+  }
+
+  @Override
+  public int read() throws IOException {
+    byte[] buf = new byte[1];
+    int n = read(buf, 0, buf.length);
+    if (n < 0) {
+      return -1;
+    } else {
+      return buf[0] & 0xFF;
+    }
+  }
+
+  @Override
+  public int read(byte[] buffer, int offset, int length) throws IOException {
+    checkNotClosed();
+    FSUtils.checkReadParameters(buffer, offset, length);
+
+    if (length == 0) {
+      return 0;
+    }
+
+    if (!range.include(nextPos)) {
+      return -1;
+    }
+
+    seekStream();
+
+    int toRead = Math.min(length, Ints.saturatedCast(range.end() - nextPos));
+    int readLen = stream.read(buffer, offset, toRead);
+    if (readLen > 0) {
+      nextPos += readLen;
+      currPos += readLen;
+    }
+    return readLen;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+    closeStream();
+    closed = true;
+  }
+
+  @Override
+  public int read(long position, byte[] buffer, int offset, int length) throws IOException {
+    checkNotClosed();
+
+    FSUtils.checkReadParameters(buffer, offset, length);
+    if (!range.include(position)) {
+      return -1;
+    }
+
+    int toRead = Math.min(length, Ints.saturatedCast(range.end() - position));
+    if (toRead == 0) {
+      return 0;
+    }
+
+    try (InputStream in = openStream(position, toRead)) {
+      return in.read(buffer, offset, toRead);
+    }
+  }
+
+  @Override
+  public void seek(long pos) throws IOException {
+    checkNotClosed();
+    Preconditions.checkArgument(range.include(pos), "Position %s must be in range %s", pos, range);
+    this.nextPos = pos;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    checkNotClosed();
+    return nextPos;
+  }
+
+  @Override
+  public boolean seekToNewSource(long targetPos) throws IOException {
+    checkNotClosed();
+    return false;
+  }
+
+  private void seekStream() throws IOException {
+    // sequential read
+    if (stream != null && nextPos == currPos) {
+      return;
+    }
+
+    // random read
+    if (stream != null && nextPos > currPos) {
+      long skip = nextPos - currPos;
+      // It is not worth skipping because the skip size is too big, or it can't read any bytes
+      // after skip.
+      if (skip < MAX_SKIP_SIZE) {
+        try {
+          ByteStreams.skipFully(stream, skip);
+          currPos = nextPos;
+          return;
+        } catch (IOException ignored) {
+          LOG.warn("Failed to skip {} bytes in stream, will try to reopen the stream", skip);
+        }
+      }
+    }
+
+    currPos = nextPos;
+
+    closeStream();
+    stream = openStream(nextPos, range.end() - nextPos);
+  }
+
+  private InputStream openStream(long offset, long limit) throws IOException {
+    return storage.get(objectKey, offset, limit).verifiedStream(checksum);
+  }
+
+  private void closeStream() throws IOException {
+    if (stream != null) {
+      stream.close();
+    }
+    stream = null;
+  }
+
+  private void checkNotClosed() throws IOException {
+    if (closed) {
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+    }
+  }
+
+  public boolean include(long pos) {
+    return range.include(pos);
+  }
+
+  public Range range() {
+    return range;
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorage.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorage.java
new file mode 100644
index 0000000000000..07187bcd237fc
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorage.java
@@ -0,0 +1,372 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.tosfs.object.exceptions.InvalidObjectKeyException;
+import org.apache.hadoop.fs.tosfs.util.LazyReload;
+import org.apache.hadoop.fs.tosfs.object.exceptions.NotAppendableException;
+import org.apache.hadoop.fs.tosfs.object.request.ListObjectsRequest;
+import org.apache.hadoop.fs.tosfs.object.response.ListObjectsResponse;
+
+import java.io.ByteArrayInputStream;
+import java.io.Closeable;
+import java.io.InputStream;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+public interface ObjectStorage extends Closeable {
+  String EMPTY_DELIMITER = "";
+
+  /**
+   * @return Scheme of the object storage.
+   */
+  String scheme();
+
+  /**
+   * @return null if bucket doesn't exist.
+   */
+  BucketInfo bucket();
+
+  /**
+   * Initialize the Object storage, according to the properties.
+   *
+   * @param conf   to initialize the {@link ObjectStorage}
+   * @param bucket the corresponding bucket name, each object store has one bucket.
+   */
+  void initialize(Configuration conf, String bucket);
+
+  /**
+   * @return storage conf
+   */
+  Configuration conf();
+
+  default ObjectContent get(String key) {
+    return get(key, 0, -1);
+  }
+
+  /**
+   * Get the data for the given object specified by key.
+   * Throw {@link RuntimeException} if object key doesn't exist.
+   * Throw {@link RuntimeException} if object key is null or empty.
+   *
+   * @param key     the object key.
+   * @param offset  the offset to start read.
+   * @param limit   the max length to read.
+   * @return {@link InputStream} to read the object content.
+   */
+  ObjectContent get(String key, long offset, long limit);
+
+  default byte[] put(String key, byte[] data) {
+    return put(key, data, 0, data.length);
+  }
+
+  default byte[] put(String key, byte[] data, int off, int len) {
+    return put(key, () -> new ByteArrayInputStream(data, off, len), len);
+  }
+
+  /**
+   * Put data read from a reader to an object specified by key. The implementation must ensure to
+   * close the stream created by stream provider after finishing stream operation.
+   * Throw {@link RuntimeException} if object key is null or empty.
+   *
+   * @param key            for the object.
+   * @param streamProvider the binary input stream provider that create input stream to write.
+   * @param contentLength  the content length, if the actual data is bigger than content length, the
+   *                       object can be created, but the object data will be truncated to the given
+   *                       content length, if the actual data is smaller than content length, will
+   *                       create object failed with unexpect end of IOException.
+   * @return the checksum of uploaded object
+   */
+  byte[] put(String key, InputStreamProvider streamProvider, long contentLength);
+
+  default byte[] append(String key, byte[] data) {
+    return append(key, data, 0, data.length);
+  }
+
+  default byte[] append(String key, byte[] data, int off, int len) {
+    return append(key, () -> new ByteArrayInputStream(data, off, len), len);
+  }
+
+  /**
+   * Append data read from a reader to an object specified by key. If the object exists, data will
+   * be appended to the tail. Otherwise, the object will be created and data will be written to it.
+   * Content length could be zero if object exists. If the object doesn't exist and content length
+   * is zero, a {@link NotAppendableException} will be thrown.
+   * 
+   * The first one wins if there are concurrent appends.
+   * 
+   * The implementation must ensure to close the stream created by stream provider after finishing
+   * stream operation.
+   * Throw {@link RuntimeException} if object key is null or empty.
+   *
+   * @param key            for the object.
+   * @param streamProvider the binary input stream provider that create input stream to write.
+   * @param contentLength  the appended content length. If the actual appended data is bigger than
+   *                       content length, the object can be appended but the data to append will be
+   *                       truncated to the given content length. If the actual data is smaller than
+   *                       content length, append object will fail with unexpect end IOException.
+   * @return the checksum of appended object.
+   * @throws NotAppendableException if the object already exists and is not appendable, or the
+   *                                object doesn't exist and content length is zero.
+   */
+  byte[] append(String key, InputStreamProvider streamProvider, long contentLength);
+
+  /**
+   * Delete an object.
+   * No exception thrown if the object key doesn't exist.
+   * Throw {@link RuntimeException} if object key is null or empty.
+   *
+   * @param key the given object key to be deleted.
+   */
+  void delete(String key);
+
+  /**
+   * Delete multiple keys. If one key doesn't exist, it will be treated as delete succeed, won't be
+   * included in response list.
+   *
+   * @param keys the given object keys to be deleted
+   * @return the keys delete failed
+   */
+  List batchDelete(List keys);
+
+  /**
+   * Delete all objects with the given prefix(include the prefix if the corresponding object
+   * exists).
+   *
+   * @param prefix the prefix key.
+   */
+  void deleteAll(String prefix);
+
+  /**
+   * Head returns some information about the object or a null if not found.
+   * Throw {@link RuntimeException} if object key is null or empty.
+   * There are some differences between directory bucket and general purpose bucket:
+   * 
+   *   - Assume an file object 'a/b' exists, only head("a/b") will get the meta of object 'a/b'
+   *   for both general purpose bucket and directory bucket+   *
- Assume an dir object 'a/b/' exists, regarding general purpose bucket, only head("a/b/")
+   *   will get the meta of object 'a/b/', but for directory bucket, both head("a/b") and
+   *   head("a/b/") will get the meta of object 'a/b/'+   *
+   *
+   * @param key for the specified object.
+   * @return {@link ObjectInfo}, null if the object does not exist.
+   * @throws InvalidObjectKeyException if the object is locating under an existing file in directory
+   *                                   bucket, which is not allowed.
+   */
+  ObjectInfo head(String key);
+
+  /**
+   * List objects according to the given {@link ListObjectsRequest}.
+   *
+   * @param request {@link ListObjectsRequest}
+   * @return the iterable of {@link ListObjectsResponse} which contains objects and common prefixes
+   */
+  Iterable list(ListObjectsRequest request);
+
+  /**
+   * List limited objects in a given bucket.
+   *
+   * @param prefix     Limits the response to keys that begin with the specified prefix.
+   * @param startAfter StartAfter is where you want the object storage to start listing from.
+   *                   object storage starts listing after this specified key.
+   *                   StartAfter can be any key in the bucket.
+   * @param limit      Limit the maximum number of response objects.
+   * @return {@link ObjectInfo} the object list with matched prefix key
+   */
+  default Iterable list(String prefix, String startAfter, int limit) {
+    ListObjectsRequest request = ListObjectsRequest.builder()
+        .prefix(prefix)
+        .startAfter(startAfter)
+        .maxKeys(limit)
+        .delimiter(EMPTY_DELIMITER)
+        .build();
+
+    return new LazyReload<>(() -> {
+      Iterator iterator = list(request).iterator();
+      return buf -> {
+        if (!iterator.hasNext()) {
+          return true;
+        }
+        buf.addAll(iterator.next().objects());
+
+        return !iterator.hasNext();
+      };
+    });
+  }
+
+  /**
+   * List all objects in a given bucket.
+   *
+   * @param prefix     Limits the response to keys that begin with the specified prefix.
+   * @param startAfter StartAfter is where you want the object storage to start listing from.
+   *                   object storage starts listing after this specified key.
+   *                   StartAfter can be any key in the bucket.
+   * @return {@link ObjectInfo} Iterable to iterate over the objects with matched prefix key
+   *                            and StartAfter
+   */
+  default Iterable listAll(String prefix, String startAfter) {
+    return list(prefix, startAfter, -1);
+  }
+
+  /**
+   * CreateMultipartUpload starts to upload a large object part by part.
+   *
+   * @param key for the specified object.
+   * @return {@link MultipartUpload}.
+   */
+  MultipartUpload createMultipartUpload(String key);
+
+  /**
+   * UploadPart upload a part of an object. The implementation must ensure to close the stream
+   * created by stream provider after finishing stream operation.
+   *
+   * @param key            for the specified object.
+   * @param uploadId       for the multipart upload id.
+   * @param partNum        upload part number.
+   * @param streamProvider the stream provider to provider part stream
+   * @param contentLength  the content length, if the actual data is bigger than content length, the
+   *                       object can be created, but the object data will be truncated to the given
+   *                       content length, if the actual data is smaller than content length, will
+   *                       create object failed with unexpect end of IOException.
+   * @return the uploaded part.
+   */
+  Part uploadPart(String key, String uploadId, int partNum, InputStreamProvider streamProvider,
+      long contentLength);
+
+  /**
+   * Complete the multipart uploads with given object key and upload id.
+   *
+   * @param key         for the specified object.
+   * @param uploadId    id of the multipart upload.
+   * @param uploadParts parts to upload.
+   * @return the checksum of uploaded object
+   */
+  byte[] completeUpload(String key, String uploadId, List uploadParts);
+
+  /**
+   * Abort a multipart upload.
+   *
+   * @param key      object key.
+   * @param uploadId multipart upload Id.
+   */
+  void abortMultipartUpload(String key, String uploadId);
+
+  /**
+   * List multipart uploads under a path.
+   *
+   * @param prefix for uploads to abort.
+   * @return Iterable to iterate over multipart unloads.
+   */
+  Iterable listUploads(String prefix);
+
+  /**
+   * upload part copy with mutipart upload id.
+   *
+   * @param srcKey               source object key
+   * @param dstKey               dest object key
+   * @param uploadId             id of the multipart upload copy
+   * @param partNum              part num of the multipart upload copy
+   * @param copySourceRangeStart copy source range start of source object
+   * @param copySourceRangeEnd   copy source range end of source object
+   * @return {@link Part}.
+   */
+  Part uploadPartCopy(
+      String srcKey, String dstKey, String uploadId, int partNum, long copySourceRangeStart,
+      long copySourceRangeEnd);
+
+  /**
+   * Copy binary content from one object to another object.
+   *
+   * @param srcKey source object key
+   * @param dstKey dest object key
+   */
+  void copy(String srcKey, String dstKey);
+
+  /**
+   * Atomic rename source object to dest object without any data copying.
+   * Will overwrite dest object if dest object exists.
+   *
+   * @param srcKey source object key
+   * @param dstKey dest object key
+   * @throws RuntimeException if rename failed,e.g. srcKey is equal to dstKey or the source object
+   *                          doesn't exist.
+   */
+  void rename(String srcKey, String dstKey);
+
+  /**
+   * Attach tags to specified object. This method will overwrite all existed tags with the new tags.
+   * Remove all existed tags if the new tags are empty. The maximum tags number is 10.
+   *
+   * @param key     the key of the object key.
+   * @param newTags the new tags to put.
+   * @throws RuntimeException if key doesn't exist.
+   */
+  default void putTags(String key, Map newTags) {
+    throw new UnsupportedOperationException(
+        this.getClass().getSimpleName() + " doesn't support putObjectTagging.");
+  }
+
+  /**
+   * Get all attached tags of the object.
+   *
+   * @param key the key of the object.
+   * @return map containing all tags.
+   * @throws RuntimeException if key doesn't exist.
+   */
+  default Map getTags(String key) {
+    throw new UnsupportedOperationException(
+        this.getClass().getSimpleName() + " doesn't support getObjectTagging.");
+  }
+
+  /**
+   * Gets the object status for the given key.
+   * It's different from {@link ObjectStorage#head(String)}, it returns object info if the key
+   * exists or the prefix with value key exists.
+   *
+   * There are three kinds of implementations:
+   * 
+   *   - Uses the headObject API if the object storage support directory bucket and the requested
+   *   bucket is a directory bucket, the object storage will return object directly if the file or
+   *   dir exists, otherwise return null+   *
- Uses getFileStatus API if the object storage support it, e.g. TOS. The object storage
+   *   will return the object directly if the key or prefix exists, otherwise return null.+   *
- If the object storage doesn't support above all cases, you have to try to headObject(key)
+   *   at first, if the object doesn't exist, and then headObject(key + "/") later if the key
+   *   doesn't end with '/', and if neither the new key doesn't exist, and then use listObjects API
+   *   to check whether the prefix/key exist.+   *
+   *
+   * @param key the object
+   * @return object info if the key or prefix exists, otherwise return null.
+   * @throws InvalidObjectKeyException if the object is locating under an existing file in directory
+   *                                   bucket, which is not allowed.
+   */
+  ObjectInfo objectStatus(String key);
+
+  /**
+   * Get the object storage checksum information, including checksum algorithm name,
+   * checksum type, etc.
+   *
+   * @return checksum information of this storage.
+   */
+  ChecksumInfo checksumInfo();
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorageFactory.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorageFactory.java
new file mode 100644
index 0000000000000..09c8a72ab743f
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorageFactory.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.tosfs.conf.ConfKeys;
+import org.apache.hadoop.fs.tosfs.object.tos.TOS;
+import org.apache.hadoop.util.Preconditions;
+
+import java.lang.reflect.InvocationTargetException;
+
+import static org.apache.hadoop.fs.tosfs.conf.ConfKeys.FS_OBJECT_STORAGE_IMPL;
+
+public final class ObjectStorageFactory {
+
+  private static final Configuration DEFAULT_IMPLS = new Configuration();
+
+  static {
+    // Setup default object storage impl for scheme "tos" and "filestore".
+    DEFAULT_IMPLS.set(ConfKeys.FS_OBJECT_STORAGE_IMPL.key("tos"), TOS.class.getName());
+    DEFAULT_IMPLS.set(ConfKeys.FS_OBJECT_STORAGE_IMPL.key("filestore"), FileStore.class.getName());
+  }
+
+  private ObjectStorageFactory() {
+  }
+
+  public static ObjectStorage createWithPrefix(String prefix, String scheme, String bucket,
+      Configuration conf) {
+    ObjectStorage storage = create(scheme, bucket, conf);
+    return new PrefixStorage(storage, prefix);
+  }
+
+  public static ObjectStorage create(String scheme, String bucket, Configuration conf) {
+    Preconditions.checkArgument(StringUtils.isNotEmpty(scheme), "Scheme is null or empty.");
+    Preconditions.checkArgument(StringUtils.isNotEmpty(bucket), "Bucket is null or empty.");
+    Preconditions.checkNotNull(conf, "Conf is null.");
+
+    try {
+      String confKey = FS_OBJECT_STORAGE_IMPL.key(scheme);
+      String impl = conf.get(confKey, DEFAULT_IMPLS.get(confKey));
+
+      Preconditions.checkArgument(StringUtils.isNotEmpty(impl),
+          "Cannot locate the ObjectStorage implementation for scheme '%s'", scheme);
+      ObjectStorage store =
+          (ObjectStorage) Class.forName(impl).getDeclaredConstructor().newInstance();
+      store.initialize(conf, bucket);
+      return store;
+    } catch (ClassNotFoundException |
+             InvocationTargetException |
+             InstantiationException |
+             IllegalAccessException |
+             NoSuchMethodException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectUtils.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectUtils.java
new file mode 100644
index 0000000000000..a0b00f366d963
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectUtils.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.tosfs.util.Range;
+import org.apache.hadoop.thirdparty.com.google.common.base.Joiner;
+import org.apache.hadoop.util.Lists;
+import org.apache.hadoop.util.Preconditions;
+
+import java.util.List;
+
+public final class ObjectUtils {
+  public static final String SLASH = "/";
+
+  private ObjectUtils() {
+  }
+
+  public static Path keyToPath(String key) {
+    return new Path(SLASH + key);
+  }
+
+  public static String path(String key) {
+    return key.startsWith(SLASH) ? key : SLASH + key;
+  }
+
+  public static String pathToKey(Path p) {
+    return pathToKey(p, false);
+  }
+
+  public static String pathToKey(Path p, Boolean isDir) {
+    Preconditions.checkArgument(p != null, "Null path");
+    if (p.toUri().getScheme() != null && p.toUri().getPath().isEmpty()) {
+      return "";
+    }
+    String key = p.toUri().getPath().substring(1);
+    if (isDir && !key.isEmpty()) {
+      return key.endsWith(SLASH) ? key : key + SLASH;
+    }
+    return key;
+  }
+
+  public static void deleteAllObjects(ObjectStorage storage, Iterable objects,
+      int batchSize) {
+    List keysToDelete = Lists.newArrayList();
+    for (ObjectInfo obj : objects) {
+      keysToDelete.add(obj.key());
+
+      if (keysToDelete.size() == batchSize) {
+        batchDelete(storage, keysToDelete);
+        keysToDelete.clear();
+      }
+    }
+
+    if (!keysToDelete.isEmpty()) {
+      batchDelete(storage, keysToDelete);
+    }
+  }
+
+  private static void batchDelete(ObjectStorage storage, List keys) {
+    List failedKeys = storage.batchDelete(keys);
+    if (!failedKeys.isEmpty()) {
+      throw new RuntimeException(String.format("Failed to delete %s objects, detail: %s",
+          failedKeys.size(), Joiner.on(",").join(failedKeys)));
+    }
+  }
+
+  public static Range calculateRange(final long offset, final long limit, final long objSize) {
+    Preconditions.checkArgument(offset >= 0,
+        String.format("offset is a negative number: %s", offset));
+    Preconditions.checkArgument(offset <= objSize,
+        String.format("offset: %s is bigger than object size: %s", offset, objSize));
+    long len = limit < 0 ? objSize - offset : Math.min(objSize - offset, limit);
+    return Range.of(offset, len);
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/Part.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/Part.java
new file mode 100644
index 0000000000000..ace7acaf28a5d
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/Part.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.MoreObjects;
+
+import java.util.Objects;
+
+public class Part {
+  private int num;
+  private long size;
+  private String eTag;
+
+  // No-arg constructor for json serializer, don't use.
+  public Part() {
+  }
+
+  public Part(int num, long size, String eTag) {
+    this.num = num;
+    this.size = size;
+    this.eTag = eTag;
+  }
+
+  public int num() {
+    return num;
+  }
+
+  public long size() {
+    return size;
+  }
+
+  public String eTag() {
+    return eTag;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(num, size, eTag);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    } else if (!(o instanceof Part)) {
+      return false;
+    }
+    Part that = (Part) o;
+    return Objects.equals(num, that.num)
+        && Objects.equals(size, that.size)
+        && Objects.equals(eTag, that.eTag);
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("PartNum", num)
+        .add("PartSize", size)
+        .add("ETag", eTag)
+        .toString();
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/PrefixStorage.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/PrefixStorage.java
new file mode 100644
index 0000000000000..18185bd1efd14
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/PrefixStorage.java
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.tosfs.object.request.ListObjectsRequest;
+import org.apache.hadoop.fs.tosfs.object.response.ListObjectsResponse;
+import org.apache.hadoop.thirdparty.com.google.common.base.Strings;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Iterables;
+import org.apache.hadoop.util.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class PrefixStorage implements DirectoryStorage {
+  private final ObjectStorage storage;
+  private final String prefix;
+
+  public PrefixStorage(ObjectStorage storage, String prefix) {
+    this.storage = storage;
+    this.prefix = prefix;
+  }
+
+  @Override
+  public String scheme() {
+    return storage.scheme();
+  }
+
+  @Override
+  public BucketInfo bucket() {
+    return storage.bucket();
+  }
+
+  @Override
+  public void initialize(Configuration conf, String bucket) {
+    storage.initialize(conf, bucket);
+  }
+
+  @Override
+  public Configuration conf() {
+    return storage.conf();
+  }
+
+  @Override
+  public ObjectContent get(String key, long offset, long limit) {
+    Preconditions.checkArgument(key != null && key.length() > 0,
+        "Object key cannot be null or empty.");
+    return storage.get(prefix + key, offset, limit);
+  }
+
+  @Override
+  public byte[] put(String key, InputStreamProvider streamProvider, long contentLength) {
+    Preconditions.checkArgument(key != null && key.length() > 0,
+        "Object key cannot be null or empty.");
+    return storage.put(prefix + key, streamProvider, contentLength);
+  }
+
+  @Override
+  public byte[] append(String key, InputStreamProvider streamProvider, long contentLength) {
+    Preconditions.checkArgument(key != null && key.length() > 0,
+        "Object key cannot be null or empty.");
+    return storage.append(prefix + key, streamProvider, contentLength);
+  }
+
+  @Override
+  public void delete(String key) {
+    Preconditions.checkArgument(key != null, "Object key cannot be null or empty.");
+    storage.delete(prefix + key);
+  }
+
+  @Override
+  public List batchDelete(List keys) {
+    return storage.batchDelete(keys.stream().map(key -> prefix + key).collect(Collectors.toList()));
+  }
+
+  @Override
+  public void deleteAll(String prefixToDelete) {
+    storage.deleteAll(this.prefix + prefixToDelete);
+  }
+
+  @Override
+  public ObjectInfo head(String key) {
+    Preconditions.checkArgument(key != null && key.length() > 0,
+        "Object key cannot be null or empty.");
+    return removePrefix(storage.head(prefix + key));
+  }
+
+  private ListObjectsResponse removePrefix(ListObjectsResponse response) {
+    List objects = response.objects().stream()
+        .map(this::removePrefix)
+        .collect(Collectors.toList());
+    List commonPrefixKeys = response.commonPrefixes().stream()
+        .map(this::removePrefix)
+        .collect(Collectors.toList());
+    return new ListObjectsResponse(objects, commonPrefixKeys);
+  }
+
+  @Override
+  public Iterable list(ListObjectsRequest request) {
+    String startAfter = Strings.isNullOrEmpty(request.startAfter()) ?
+        request.startAfter() : prefix + request.startAfter();
+
+    ListObjectsRequest newReq = ListObjectsRequest.builder()
+        .prefix(prefix + request.prefix())
+        .startAfter(startAfter)
+        .maxKeys(request.maxKeys())
+        .delimiter(request.delimiter())
+        .build();
+
+    return Iterables.transform(storage.list(newReq), this::removePrefix);
+  }
+
+  @Override
+  public MultipartUpload createMultipartUpload(String key) {
+    Preconditions.checkArgument(key != null && key.length() > 0,
+        "Object key cannot be null or empty.");
+    return removePrefix(storage.createMultipartUpload(prefix + key));
+  }
+
+  @Override
+  public Part uploadPart(
+      String key, String uploadId, int partNum,
+      InputStreamProvider streamProvider, long contentLength) {
+    Preconditions.checkArgument(key != null && key.length() > 0,
+        "Object key cannot be null or empty.");
+    return storage.uploadPart(prefix + key, uploadId, partNum, streamProvider, contentLength);
+  }
+
+  @Override
+  public byte[] completeUpload(String key, String uploadId, List uploadParts) {
+    Preconditions.checkArgument(key != null && key.length() > 0,
+        "Object key cannot be null or empty.");
+    return storage.completeUpload(prefix + key, uploadId, uploadParts);
+  }
+
+  @Override
+  public void abortMultipartUpload(String key, String uploadId) {
+    Preconditions.checkArgument(key != null && key.length() > 0,
+        "Object key cannot be null or empty.");
+    storage.abortMultipartUpload(prefix + key, uploadId);
+  }
+
+  @Override
+  public Iterable listUploads(String keyPrefix) {
+    return Iterables.transform(storage.listUploads(prefix + keyPrefix), this::removePrefix);
+  }
+
+  @Override
+  public Part uploadPartCopy(
+      String srcKey, String dstKey, String uploadId, int partNum, long copySourceRangeStart,
+      long copySourceRangeEnd) {
+    return storage.uploadPartCopy(prefix + srcKey, prefix + dstKey, uploadId, partNum,
+        copySourceRangeStart, copySourceRangeEnd);
+  }
+
+  @Override
+  public void copy(String srcKey, String dstKey) {
+    storage.copy(prefix + srcKey, prefix + dstKey);
+  }
+
+  @Override
+  public void rename(String srcKey, String dstKey) {
+    storage.rename(prefix + srcKey, prefix + dstKey);
+  }
+
+  private ObjectInfo removePrefix(ObjectInfo o) {
+    if (o == null) {
+      return null;
+    }
+    return new ObjectInfo(removePrefix(o.key()), o.size(), o.mtime(), o.checksum(), o.isDir());
+  }
+
+  private MultipartUpload removePrefix(MultipartUpload u) {
+    if (u == null) {
+      return null;
+    }
+    return new MultipartUpload(removePrefix(u.key()), u.uploadId(), u.minPartSize(),
+        u.maxPartCount());
+  }
+
+  private String removePrefix(String key) {
+    if (key == null) {
+      return null;
+    } else if (key.startsWith(prefix)) {
+      return key.substring(prefix.length());
+    } else {
+      return key;
+    }
+  }
+
+  @Override
+  public void putTags(String key, Map newTags) {
+    storage.putTags(prefix + key, newTags);
+  }
+
+  @Override
+  public Map getTags(String key) {
+    return storage.getTags(prefix + key);
+  }
+
+  @Override
+  public ObjectInfo objectStatus(String key) {
+    Preconditions.checkArgument(key != null && !key.isEmpty(),
+        "Object key cannot be null or empty.");
+    return removePrefix(storage.objectStatus(prefix + key));
+  }
+
+  @Override
+  public ChecksumInfo checksumInfo() {
+    return storage.checksumInfo();
+  }
+
+  @Override
+  public void close() throws IOException {
+    storage.close();
+  }
+
+  @Override
+  public Iterable listDir(String key, boolean recursive) {
+    Preconditions.checkArgument(storage instanceof DirectoryStorage);
+    return Iterables.transform(((DirectoryStorage) storage).listDir(prefix + key, recursive),
+        this::removePrefix);
+  }
+
+  @Override
+  public void deleteDir(String key, boolean recursive) {
+    Preconditions.checkArgument(storage instanceof DirectoryStorage);
+    ((DirectoryStorage) storage).deleteDir(prefix + key, recursive);
+  }
+
+  @Override
+  public boolean isEmptyDir(String key) {
+    Preconditions.checkArgument(storage instanceof DirectoryStorage);
+    return ((DirectoryStorage) storage).isEmptyDir(prefix + key);
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/ChecksumMismatchException.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/ChecksumMismatchException.java
new file mode 100644
index 0000000000000..309eebe0fcad6
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/ChecksumMismatchException.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object.exceptions;
+
+import org.apache.hadoop.util.StringUtils;
+
+import java.io.IOException;
+
+public class ChecksumMismatchException extends IOException {
+  public ChecksumMismatchException(String message) {
+    super(message);
+  }
+
+  public ChecksumMismatchException(byte[] expected, byte[] actual) {
+    this(String.format("Expected checksum is %s while actual checksum is %s",
+        StringUtils.byteToHexString(expected), StringUtils.byteToHexString(actual)));
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/InvalidObjectKeyException.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/InvalidObjectKeyException.java
new file mode 100644
index 0000000000000..f9eaf021e28e0
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/InvalidObjectKeyException.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object.exceptions;
+
+/**
+ * Regarding accessing an object in directory bucket, if the object is locating under an existing
+ * file in directory bucket, the {@link InvalidObjectKeyException} will be thrown. E.g. there is a
+ * file object 'a/b/file' exists in directory bucket, the {@link InvalidObjectKeyException} will be
+ * thrown if head object 'a/b/file/c' no matter whether 'c' exists or not.
+ */
+public class InvalidObjectKeyException extends RuntimeException {
+  public InvalidObjectKeyException(Throwable cause) {
+    super(cause);
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/NotAppendableException.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/NotAppendableException.java
new file mode 100644
index 0000000000000..2a7def1176390
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/NotAppendableException.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object.exceptions;
+
+public class NotAppendableException extends RuntimeException {
+  public NotAppendableException(String msg) {
+    super(msg);
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/package-info.java
new file mode 100644
index 0000000000000..c87e1a7eaa031
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Classes for hadoop-tos object.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"})
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.tosfs.object.exceptions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/package-info.java
new file mode 100644
index 0000000000000..7819ecd6c8af0
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Classes for hadoop-tos object.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"})
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.tosfs.object;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/ListObjectsRequest.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/ListObjectsRequest.java
new file mode 100644
index 0000000000000..170426acb07c2
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/ListObjectsRequest.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object.request;
+
+public final class ListObjectsRequest {
+  private final String prefix;
+  private final String startAfter;
+  private final int maxKeys;
+  private final String delimiter;
+
+  private ListObjectsRequest(String prefix, String startAfter, int maxKeys, String delimiter) {
+    this.prefix = prefix;
+    this.startAfter = startAfter;
+    this.maxKeys = maxKeys;
+    this.delimiter = delimiter;
+  }
+
+  public String prefix() {
+    return prefix;
+  }
+
+  public String startAfter() {
+    return startAfter;
+  }
+
+  public int maxKeys() {
+    return maxKeys;
+  }
+
+  public String delimiter() {
+    return delimiter;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static class Builder {
+    private String prefix;
+    private String startAfter;
+    // -1 means list all object keys
+    private int maxKeys = -1;
+    private String delimiter;
+
+    public Builder prefix(String prefixInput) {
+      this.prefix = prefixInput;
+      return this;
+    }
+
+    public Builder startAfter(String startAfterInput) {
+      this.startAfter = startAfterInput;
+      return this;
+    }
+
+    public Builder maxKeys(int maxKeysInput) {
+      this.maxKeys = maxKeysInput;
+      return this;
+    }
+
+    public Builder delimiter(String delimiterInput) {
+      this.delimiter = delimiterInput;
+      return this;
+    }
+
+    public ListObjectsRequest build() {
+      return new ListObjectsRequest(prefix, startAfter, maxKeys, delimiter);
+    }
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/package-info.java
new file mode 100644
index 0000000000000..6e7e52f403950
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Classes for hadoop-tos object request.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"})
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.tosfs.object.request;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/ListObjectsResponse.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/ListObjectsResponse.java
new file mode 100644
index 0000000000000..eb3adf8522ae8
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/ListObjectsResponse.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object.response;
+
+import org.apache.hadoop.fs.tosfs.object.ObjectInfo;
+
+import java.util.List;
+
+public class ListObjectsResponse {
+  private final List objects;
+  private final List commonPrefixes;
+
+  public ListObjectsResponse(
+      List objects,
+      List commonPrefixes) {
+    this.objects = objects;
+    this.commonPrefixes = commonPrefixes;
+  }
+
+  public List objects() {
+    return objects;
+  }
+
+  public List commonPrefixes() {
+    return commonPrefixes;
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/package-info.java
new file mode 100644
index 0000000000000..6678617057243
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Classes for hadoop-tos object response.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"})
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.tosfs.object.response;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/FileStagingPart.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/FileStagingPart.java
new file mode 100644
index 0000000000000..fb39e949febf6
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/FileStagingPart.java
@@ -0,0 +1,177 @@
+/*
+ * ByteDance Volcengine EMR, Copyright 2022.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object.staging;
+
+import org.apache.hadoop.fs.tosfs.util.CommonUtils;
+import org.apache.hadoop.thirdparty.com.google.common.base.MoreObjects;
+import org.apache.hadoop.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UncheckedIOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+public class FileStagingPart implements StagingPart {
+  private static final Logger LOG = LoggerFactory.getLogger(FileStagingPart.class);
+
+  private final Path path;
+  private final int stagingBufferSize;
+  private final StagingFileOutputStream out;
+  private State state = State.WRITABLE;
+
+  public FileStagingPart(String filePath, int stagingBufferSize) {
+    this.path = Paths.get(filePath);
+    this.stagingBufferSize = stagingBufferSize;
+    this.out = new StagingFileOutputStream(path, stagingBufferSize);
+  }
+
+  @Override
+  public synchronized void write(byte[] b, int off, int len) throws IOException {
+    Preconditions.checkState(state == State.WRITABLE,
+        "Cannot write the part since it's not writable now, state: %s", state);
+    out.write(b, off, len);
+  }
+
+  @Override
+  public synchronized void complete() throws IOException {
+    Preconditions.checkState(state == State.WRITABLE,
+        "Cannot complete the part since it's not writable now, state: %s", state);
+    out.close();
+    state = State.READABLE;
+  }
+
+  @Override
+  public synchronized InputStream newIn() {
+    Preconditions.checkState(state == State.READABLE,
+        "Cannot read the part since it's not readable now, state: %s.", state);
+    return out.newIn();
+  }
+
+  @Override
+  public synchronized long size() {
+    return out.size();
+  }
+
+  @Override
+  public synchronized State state() {
+    return state;
+  }
+
+  @Override
+  public synchronized void cleanup() {
+    if (state != State.CLEANED) {
+      try {
+        // Close the stream quietly.
+        CommonUtils.runQuietly(out::close, false);
+
+        // Delete the staging file if exists.
+        Files.deleteIfExists(path);
+      } catch (Exception e) {
+        LOG.error("Failed to delete staging file, stagingFile: {}", path, e);
+      } finally {
+        state = State.CLEANED;
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("path", path)
+        .add("stagingBufferSize", stagingBufferSize)
+        .add("wroteByteSize", size())
+        .toString();
+  }
+
+  private final static class StagingFileOutputStream extends OutputStream {
+    private final Path path;
+    private byte[] buffer;
+    private boolean memBuffered;
+    private int writePos;
+    private OutputStream out;
+
+    private StagingFileOutputStream(Path path, int stagingBufferSize) {
+      this.path = path;
+      this.buffer = new byte[stagingBufferSize];
+      this.memBuffered = true;
+      this.writePos = 0;
+    }
+
+    private int size() {
+      return writePos;
+    }
+
+    public InputStream newIn() {
+      // Just wrap it as a byte array input stream if the staging bytes are still in the in-memory
+      // buffer.
+      if (memBuffered) {
+        return new ByteArrayInputStream(buffer, 0, writePos);
+      }
+
+      // Create a buffered file input stream.
+      try {
+        return new BufferedInputStream(Files.newInputStream(path));
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+      write(new byte[]{(byte) b}, 0, 1);
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+      if (memBuffered && writePos + len > buffer.length) {
+        flushMemToFile();
+      }
+
+      if (memBuffered) {
+        System.arraycopy(b, off, buffer, writePos, len);
+      } else {
+        out.write(b, off, len);
+      }
+
+      writePos += len;
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (out != null) {
+        out.close();
+        out = null;
+      }
+    }
+
+    private void flushMemToFile() throws IOException {
+      // Flush the buffered data to the new file OutputStream.
+      out = new BufferedOutputStream(Files.newOutputStream(path));
+      out.write(buffer, 0, writePos);
+      memBuffered = false;
+      buffer = null;
+    }
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/StagingPart.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/StagingPart.java
new file mode 100644
index 0000000000000..b4fa812397285
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/StagingPart.java
@@ -0,0 +1,79 @@
+/*
+ * ByteDance Volcengine EMR, Copyright 2022.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object.staging;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public interface StagingPart {
+
+  /**
+   * Write bytes into the staging part.
+   *
+   * @param b the buffer to write.
+   * @throws IOException if any IO error.
+   */
+  default void write(byte[] b) throws IOException {
+    write(b, 0, b.length);
+  }
+
+  /**
+   * Write the bytes into the staging part.
+   *
+   * @param b   the buffer to write.
+   * @param off the start offset in buffer.
+   * @param len the length.
+   * @throws IOException if any IO error.
+   */
+  void write(byte[] b, int off, int len) throws IOException;
+
+  /**
+   * Complete the writing process and cannot write more bytes once we've completed this part.
+   *
+   * @throws IOException if any IO error.
+   */
+  void complete() throws IOException;
+
+  /**
+   * The wrote size of staging part.
+   *
+   * @return the staging part size.
+   */
+  long size();
+
+  /**
+   * Access the {@link State} of this part.
+   *
+   * @return the {@link State}.
+   */
+  State state();
+
+  /**
+   * Create a separate new {@link InputStream} to read the staging part data once we've completed
+   * the writing by calling {@link StagingPart#complete()} . Call this method several times will
+   * return many {@link InputStream}s, and remember to close the newly created stream.
+   *
+   * @return a totally new {@link InputStream}.
+   */
+  InputStream newIn();
+
+  /**
+   * Clean all the {@link  StagingPart}'s resources, such as removing temporary file, free the
+   * buffered data etc. it should be idempotent and quiet (without throwing IO error).
+   */
+  void cleanup();
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/State.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/State.java
new file mode 100644
index 0000000000000..418baa6d9b13a
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/State.java
@@ -0,0 +1,23 @@
+/*
+ * ByteDance Volcengine EMR, Copyright 2022.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object.staging;
+
+public enum State {
+  WRITABLE,
+  READABLE,
+  CLEANED
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/package-info.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/package-info.java
new file mode 100644
index 0000000000000..c9c2a9033c40e
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Classes for hadoop-tos object staging part.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN", "Hive"})
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.tosfs.object.staging;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/ChainTOSInputStream.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/ChainTOSInputStream.java
new file mode 100644
index 0000000000000..7790e7c54a33d
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/ChainTOSInputStream.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object.tos;
+
+import org.apache.hadoop.fs.tosfs.common.Chain;
+import org.apache.hadoop.util.Preconditions;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.UncheckedIOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class ChainTOSInputStream extends InputStream {
+  private final Chain chain;
+  private final TOS.GetObjectFactory factory;
+  private final String key;
+  private long curOff;
+  private final long endOff; // range end offset (inclusive)
+  private final long maxDrainByteSize;
+  private final int maxInputStreamRetries;
+
+  private int readBytes;
+  private long skipped;
+  private byte[] objChecksum = null;
+  private final AtomicBoolean closed = new AtomicBoolean(false);
+
+  public ChainTOSInputStream(
+      TOS.GetObjectFactory factory,
+      String key,
+      long startOff,
+      long endOff,
+      long maxDrainByteSize,
+      int maxInputStreamRetries) {
+    this.factory = factory;
+    this.key = key;
+    this.curOff = startOff;
+    this.endOff = endOff;
+    this.maxDrainByteSize = maxDrainByteSize;
+    this.maxInputStreamRetries = maxInputStreamRetries;
+    this.chain = createChain();
+    Preconditions.checkNotNull(objChecksum, "Checksum should not be null.");
+  }
+
+  private Chain createChain() {
+    Chain.Builder builder = Chain.builder()
+        .shouldContinue(e -> !(e instanceof EOFException));
+
+    for (int i = 0; i <= maxInputStreamRetries; i++) {
+      builder.addLast(() -> {
+        GetObjectOutput output = factory.create(key, curOff, endOff);
+
+        // Note: If there are some IO errors occur, the ChainTOSInputStream will create a new
+        // stream in the chain to continue reading object data, we need to record the checksum
+        // during first open object stream, and ensure the checksum of object stream won't be
+        // changed if opening object many times within the lifecycle of the chained stream in case
+        // the underlying object is changed.
+        if (objChecksum == null) {
+          // Init the stream checksum.
+          objChecksum = output.checksum();
+        }
+        return new TOSInputStream(output, curOff, endOff, maxDrainByteSize, objChecksum);
+      });
+    }
+
+    try {
+      return builder.build();
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public long skip(long n) throws IOException {
+    skipped = 0;
+    return chain.run(stream -> {
+      long skip = stream.skip(n - skipped);
+
+      curOff += skip;
+      skipped += skip;
+      return skipped;
+    });
+  }
+
+  @Override
+  public int read() throws IOException {
+    return chain.run(stream -> {
+      int ret = stream.read();
+      curOff++;
+      return ret;
+    });
+  }
+
+  @Override
+  public int available() throws IOException {
+    return chain.run(InputStream::available);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    readBytes = 0;
+    return chain.run(in -> {
+      int read = in.read(b, off + readBytes, len - readBytes);
+
+      readBytes += read;
+      curOff += read;
+      return readBytes;
+    });
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (closed.compareAndSet(false, true)) {
+      chain.close();
+    }
+  }
+
+  public byte[] checksum() {
+    return objChecksum;
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClient.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClient.java
new file mode 100644
index 0000000000000..a1b9f9ce4cb16
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClient.java
@@ -0,0 +1,1255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object.tos;
+
+import com.volcengine.tos.TOSClientConfiguration;
+import com.volcengine.tos.TOSV2ClientBuilder;
+import com.volcengine.tos.TosClientException;
+import com.volcengine.tos.TosException;
+import com.volcengine.tos.TosServerException;
+import com.volcengine.tos.auth.Credential;
+import com.volcengine.tos.auth.Credentials;
+import com.volcengine.tos.TOSV2;
+import com.volcengine.tos.comm.HttpStatus;
+import com.volcengine.tos.comm.common.ACLType;
+import com.volcengine.tos.internal.RequestOptionsBuilder;
+import com.volcengine.tos.model.acl.GetObjectAclOutput;
+import com.volcengine.tos.model.acl.PutObjectAclInput;
+import com.volcengine.tos.model.acl.PutObjectAclOutput;
+import com.volcengine.tos.model.bucket.CreateBucketInput;
+import com.volcengine.tos.model.bucket.CreateBucketOutput;
+import com.volcengine.tos.model.bucket.CreateBucketV2Input;
+import com.volcengine.tos.model.bucket.CreateBucketV2Output;
+import com.volcengine.tos.model.bucket.DeleteBucketCORSInput;
+import com.volcengine.tos.model.bucket.DeleteBucketCORSOutput;
+import com.volcengine.tos.model.bucket.DeleteBucketCustomDomainInput;
+import com.volcengine.tos.model.bucket.DeleteBucketCustomDomainOutput;
+import com.volcengine.tos.model.bucket.DeleteBucketEncryptionInput;
+import com.volcengine.tos.model.bucket.DeleteBucketEncryptionOutput;
+import com.volcengine.tos.model.bucket.DeleteBucketInput;
+import com.volcengine.tos.model.bucket.DeleteBucketInventoryInput;
+import com.volcengine.tos.model.bucket.DeleteBucketInventoryOutput;
+import com.volcengine.tos.model.bucket.DeleteBucketLifecycleInput;
+import com.volcengine.tos.model.bucket.DeleteBucketLifecycleOutput;
+import com.volcengine.tos.model.bucket.DeleteBucketMirrorBackInput;
+import com.volcengine.tos.model.bucket.DeleteBucketMirrorBackOutput;
+import com.volcengine.tos.model.bucket.DeleteBucketOutput;
+import com.volcengine.tos.model.bucket.DeleteBucketPolicyInput;
+import com.volcengine.tos.model.bucket.DeleteBucketPolicyOutput;
+import com.volcengine.tos.model.bucket.DeleteBucketRealTimeLogInput;
+import com.volcengine.tos.model.bucket.DeleteBucketRealTimeLogOutput;
+import com.volcengine.tos.model.bucket.DeleteBucketRenameInput;
+import com.volcengine.tos.model.bucket.DeleteBucketRenameOutput;
+import com.volcengine.tos.model.bucket.DeleteBucketReplicationInput;
+import com.volcengine.tos.model.bucket.DeleteBucketReplicationOutput;
+import com.volcengine.tos.model.bucket.DeleteBucketTaggingInput;
+import com.volcengine.tos.model.bucket.DeleteBucketTaggingOutput;
+import com.volcengine.tos.model.bucket.DeleteBucketWebsiteInput;
+import com.volcengine.tos.model.bucket.DeleteBucketWebsiteOutput;
+import com.volcengine.tos.model.bucket.GetBucketACLInput;
+import com.volcengine.tos.model.bucket.GetBucketACLOutput;
+import com.volcengine.tos.model.bucket.GetBucketCORSInput;
+import com.volcengine.tos.model.bucket.GetBucketCORSOutput;
+import com.volcengine.tos.model.bucket.GetBucketEncryptionInput;
+import com.volcengine.tos.model.bucket.GetBucketEncryptionOutput;
+import com.volcengine.tos.model.bucket.GetBucketInventoryInput;
+import com.volcengine.tos.model.bucket.GetBucketInventoryOutput;
+import com.volcengine.tos.model.bucket.GetBucketLifecycleInput;
+import com.volcengine.tos.model.bucket.GetBucketLifecycleOutput;
+import com.volcengine.tos.model.bucket.GetBucketLocationInput;
+import com.volcengine.tos.model.bucket.GetBucketLocationOutput;
+import com.volcengine.tos.model.bucket.GetBucketMirrorBackInput;
+import com.volcengine.tos.model.bucket.GetBucketMirrorBackOutput;
+import com.volcengine.tos.model.bucket.GetBucketNotificationInput;
+import com.volcengine.tos.model.bucket.GetBucketNotificationOutput;
+import com.volcengine.tos.model.bucket.GetBucketNotificationType2Input;
+import com.volcengine.tos.model.bucket.GetBucketNotificationType2Output;
+import com.volcengine.tos.model.bucket.GetBucketPolicyInput;
+import com.volcengine.tos.model.bucket.GetBucketPolicyOutput;
+import com.volcengine.tos.model.bucket.GetBucketRealTimeLogInput;
+import com.volcengine.tos.model.bucket.GetBucketRealTimeLogOutput;
+import com.volcengine.tos.model.bucket.GetBucketRenameInput;
+import com.volcengine.tos.model.bucket.GetBucketRenameOutput;
+import com.volcengine.tos.model.bucket.GetBucketReplicationInput;
+import com.volcengine.tos.model.bucket.GetBucketReplicationOutput;
+import com.volcengine.tos.model.bucket.GetBucketTaggingInput;
+import com.volcengine.tos.model.bucket.GetBucketTaggingOutput;
+import com.volcengine.tos.model.bucket.GetBucketVersioningInput;
+import com.volcengine.tos.model.bucket.GetBucketVersioningOutput;
+import com.volcengine.tos.model.bucket.GetBucketWebsiteInput;
+import com.volcengine.tos.model.bucket.GetBucketWebsiteOutput;
+import com.volcengine.tos.model.bucket.HeadBucketOutput;
+import com.volcengine.tos.model.bucket.HeadBucketV2Input;
+import com.volcengine.tos.model.bucket.HeadBucketV2Output;
+import com.volcengine.tos.model.bucket.ListBucketCustomDomainInput;
+import com.volcengine.tos.model.bucket.ListBucketCustomDomainOutput;
+import com.volcengine.tos.model.bucket.ListBucketInventoryInput;
+import com.volcengine.tos.model.bucket.ListBucketInventoryOutput;
+import com.volcengine.tos.model.bucket.ListBucketsInput;
+import com.volcengine.tos.model.bucket.ListBucketsOutput;
+import com.volcengine.tos.model.bucket.ListBucketsV2Input;
+import com.volcengine.tos.model.bucket.ListBucketsV2Output;
+import com.volcengine.tos.model.bucket.PutBucketACLInput;
+import com.volcengine.tos.model.bucket.PutBucketACLOutput;
+import com.volcengine.tos.model.bucket.PutBucketCORSInput;
+import com.volcengine.tos.model.bucket.PutBucketCORSOutput;
+import com.volcengine.tos.model.bucket.PutBucketCustomDomainInput;
+import com.volcengine.tos.model.bucket.PutBucketCustomDomainOutput;
+import com.volcengine.tos.model.bucket.PutBucketEncryptionInput;
+import com.volcengine.tos.model.bucket.PutBucketEncryptionOutput;
+import com.volcengine.tos.model.bucket.PutBucketInventoryInput;
+import com.volcengine.tos.model.bucket.PutBucketInventoryOutput;
+import com.volcengine.tos.model.bucket.PutBucketLifecycleInput;
+import com.volcengine.tos.model.bucket.PutBucketLifecycleOutput;
+import com.volcengine.tos.model.bucket.PutBucketMirrorBackInput;
+import com.volcengine.tos.model.bucket.PutBucketMirrorBackOutput;
+import com.volcengine.tos.model.bucket.PutBucketNotificationInput;
+import com.volcengine.tos.model.bucket.PutBucketNotificationOutput;
+import com.volcengine.tos.model.bucket.PutBucketNotificationType2Input;
+import com.volcengine.tos.model.bucket.PutBucketNotificationType2Output;
+import com.volcengine.tos.model.bucket.PutBucketPolicyInput;
+import com.volcengine.tos.model.bucket.PutBucketPolicyOutput;
+import com.volcengine.tos.model.bucket.PutBucketRealTimeLogInput;
+import com.volcengine.tos.model.bucket.PutBucketRealTimeLogOutput;
+import com.volcengine.tos.model.bucket.PutBucketRenameInput;
+import com.volcengine.tos.model.bucket.PutBucketRenameOutput;
+import com.volcengine.tos.model.bucket.PutBucketReplicationInput;
+import com.volcengine.tos.model.bucket.PutBucketReplicationOutput;
+import com.volcengine.tos.model.bucket.PutBucketStorageClassInput;
+import com.volcengine.tos.model.bucket.PutBucketStorageClassOutput;
+import com.volcengine.tos.model.bucket.PutBucketTaggingInput;
+import com.volcengine.tos.model.bucket.PutBucketTaggingOutput;
+import com.volcengine.tos.model.bucket.PutBucketVersioningInput;
+import com.volcengine.tos.model.bucket.PutBucketVersioningOutput;
+import com.volcengine.tos.model.bucket.PutBucketWebsiteInput;
+import com.volcengine.tos.model.bucket.PutBucketWebsiteOutput;
+import com.volcengine.tos.model.object.AbortMultipartUploadInput;
+import com.volcengine.tos.model.object.AbortMultipartUploadOutput;
+import com.volcengine.tos.model.object.AppendObjectInput;
+import com.volcengine.tos.model.object.AppendObjectOutput;
+import com.volcengine.tos.model.object.CompleteMultipartUploadInput;
+import com.volcengine.tos.model.object.CompleteMultipartUploadOutput;
+import com.volcengine.tos.model.object.CompleteMultipartUploadV2Input;
+import com.volcengine.tos.model.object.CompleteMultipartUploadV2Output;
+import com.volcengine.tos.model.object.CopyObjectOutput;
+import com.volcengine.tos.model.object.CopyObjectV2Input;
+import com.volcengine.tos.model.object.CopyObjectV2Output;
+import com.volcengine.tos.model.object.CreateMultipartUploadInput;
+import com.volcengine.tos.model.object.CreateMultipartUploadOutput;
+import com.volcengine.tos.model.object.DeleteMultiObjectsInput;
+import com.volcengine.tos.model.object.DeleteMultiObjectsOutput;
+import com.volcengine.tos.model.object.DeleteMultiObjectsV2Input;
+import com.volcengine.tos.model.object.DeleteMultiObjectsV2Output;
+import com.volcengine.tos.model.object.DeleteObjectInput;
+import com.volcengine.tos.model.object.DeleteObjectOutput;
+import com.volcengine.tos.model.object.DeleteObjectTaggingInput;
+import com.volcengine.tos.model.object.DeleteObjectTaggingOutput;
+import com.volcengine.tos.model.object.DownloadFileInput;
+import com.volcengine.tos.model.object.DownloadFileOutput;
+import com.volcengine.tos.model.object.FetchObjectInput;
+import com.volcengine.tos.model.object.FetchObjectOutput;
+import com.volcengine.tos.model.object.GetFetchTaskInput;
+import com.volcengine.tos.model.object.GetFetchTaskOutput;
+import com.volcengine.tos.model.object.GetFileStatusInput;
+import com.volcengine.tos.model.object.GetFileStatusOutput;
+import com.volcengine.tos.model.object.GetObjectACLV2Input;
+import com.volcengine.tos.model.object.GetObjectACLV2Output;
+import com.volcengine.tos.model.object.GetObjectOutput;
+import com.volcengine.tos.model.object.GetObjectTaggingInput;
+import com.volcengine.tos.model.object.GetObjectTaggingOutput;
+import com.volcengine.tos.model.object.GetObjectToFileInput;
+import com.volcengine.tos.model.object.GetObjectToFileOutput;
+import com.volcengine.tos.model.object.GetObjectV2Input;
+import com.volcengine.tos.model.object.GetObjectV2Output;
+import com.volcengine.tos.model.object.GetSymlinkInput;
+import com.volcengine.tos.model.object.GetSymlinkOutput;
+import com.volcengine.tos.model.object.HeadObjectOutput;
+import com.volcengine.tos.model.object.HeadObjectV2Input;
+import com.volcengine.tos.model.object.HeadObjectV2Output;
+import com.volcengine.tos.model.object.ListMultipartUploadsInput;
+import com.volcengine.tos.model.object.ListMultipartUploadsOutput;
+import com.volcengine.tos.model.object.ListMultipartUploadsV2Input;
+import com.volcengine.tos.model.object.ListMultipartUploadsV2Output;
+import com.volcengine.tos.model.object.ListObjectVersionsInput;
+import com.volcengine.tos.model.object.ListObjectVersionsOutput;
+import com.volcengine.tos.model.object.ListObjectVersionsV2Input;
+import com.volcengine.tos.model.object.ListObjectVersionsV2Output;
+import com.volcengine.tos.model.object.ListObjectsInput;
+import com.volcengine.tos.model.object.ListObjectsOutput;
+import com.volcengine.tos.model.object.ListObjectsType2Input;
+import com.volcengine.tos.model.object.ListObjectsType2Output;
+import com.volcengine.tos.model.object.ListObjectsV2Input;
+import com.volcengine.tos.model.object.ListObjectsV2Output;
+import com.volcengine.tos.model.object.ListPartsInput;
+import com.volcengine.tos.model.object.ListPartsOutput;
+import com.volcengine.tos.model.object.ListUploadedPartsInput;
+import com.volcengine.tos.model.object.ListUploadedPartsOutput;
+import com.volcengine.tos.model.object.ObjectMetaRequestOptions;
+import com.volcengine.tos.model.object.PreSignedPolicyURLInput;
+import com.volcengine.tos.model.object.PreSignedPolicyURLOutput;
+import com.volcengine.tos.model.object.PreSignedPostSignatureInput;
+import com.volcengine.tos.model.object.PreSignedPostSignatureOutput;
+import com.volcengine.tos.model.object.PreSignedURLInput;
+import com.volcengine.tos.model.object.PreSignedURLOutput;
+import com.volcengine.tos.model.object.PreSingedPolicyURLInput;
+import com.volcengine.tos.model.object.PreSingedPolicyURLOutput;
+import com.volcengine.tos.model.object.PutFetchTaskInput;
+import com.volcengine.tos.model.object.PutFetchTaskOutput;
+import com.volcengine.tos.model.object.PutObjectACLInput;
+import com.volcengine.tos.model.object.PutObjectACLOutput;
+import com.volcengine.tos.model.object.PutObjectFromFileInput;
+import com.volcengine.tos.model.object.PutObjectFromFileOutput;
+import com.volcengine.tos.model.object.PutObjectInput;
+import com.volcengine.tos.model.object.PutObjectOutput;
+import com.volcengine.tos.model.object.PutObjectTaggingInput;
+import com.volcengine.tos.model.object.PutObjectTaggingOutput;
+import com.volcengine.tos.model.object.PutSymlinkInput;
+import com.volcengine.tos.model.object.PutSymlinkOutput;
+import com.volcengine.tos.model.object.RenameObjectInput;
+import com.volcengine.tos.model.object.RenameObjectOutput;
+import com.volcengine.tos.model.object.RestoreObjectInput;
+import com.volcengine.tos.model.object.RestoreObjectOutput;
+import com.volcengine.tos.model.object.ResumableCopyObjectInput;
+import com.volcengine.tos.model.object.ResumableCopyObjectOutput;
+import com.volcengine.tos.model.object.SetObjectMetaInput;
+import com.volcengine.tos.model.object.SetObjectMetaOutput;
+import com.volcengine.tos.model.object.SetObjectTimeInput;
+import com.volcengine.tos.model.object.SetObjectTimeOutput;
+import com.volcengine.tos.model.object.UploadFileInput;
+import com.volcengine.tos.model.object.UploadFileOutput;
+import com.volcengine.tos.model.object.UploadFileV2Input;
+import com.volcengine.tos.model.object.UploadFileV2Output;
+import com.volcengine.tos.model.object.UploadPartCopyInput;
+import com.volcengine.tos.model.object.UploadPartCopyOutput;
+import com.volcengine.tos.model.object.UploadPartCopyV2Input;
+import com.volcengine.tos.model.object.UploadPartCopyV2Output;
+import com.volcengine.tos.model.object.UploadPartFromFileInput;
+import com.volcengine.tos.model.object.UploadPartFromFileOutput;
+import com.volcengine.tos.model.object.UploadPartInput;
+import com.volcengine.tos.model.object.UploadPartOutput;
+import com.volcengine.tos.model.object.UploadPartV2Input;
+import com.volcengine.tos.model.object.UploadPartV2Output;
+import com.volcengine.tos.transport.TransportConfig;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.tosfs.object.InputStreamProvider;
+import org.apache.hadoop.fs.tosfs.object.Part;
+import org.apache.hadoop.fs.tosfs.util.RetryableUtils;
+import org.apache.hadoop.thirdparty.com.google.common.base.Throwables;
+import org.apache.hadoop.thirdparty.com.google.common.io.CountingInputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.SocketException;
+import java.net.SocketTimeoutException;
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.Callable;
+import javax.net.ssl.SSLException;
+
+public class DelegationClient implements TOSV2 {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DelegationClient.class);
+
+  private final Credentials provider;
+  private final TOSClientConfiguration config;
+  private int maxRetryTimes;
+  private TOSV2 client;
+  private volatile Credential usedCredential;
+  private final List nonRetryable409ErrorCodes;
+
+  protected DelegationClient(TOSClientConfiguration configuration, int maxRetryTimes,
+      List nonRetryable409ErrorCodes) {
+    this.config = configuration;
+    this.maxRetryTimes = maxRetryTimes;
+    this.provider = configuration.getCredentials();
+    this.usedCredential = provider.credential();
+    this.client = new TOSV2ClientBuilder().build(configuration);
+    this.nonRetryable409ErrorCodes = nonRetryable409ErrorCodes;
+  }
+
+  @VisibleForTesting
+  void setClient(TOSV2 client) {
+    this.client = client;
+  }
+
+  public TOSV2 client() {
+    return client;
+  }
+
+  @VisibleForTesting
+  void setMaxRetryTimes(int maxRetryTimes) {
+    this.maxRetryTimes = maxRetryTimes;
+  }
+
+  public int maxRetryTimes() {
+    return maxRetryTimes;
+  }
+
+  public TOSClientConfiguration config() {
+    return config;
+  }
+
+  public Credential usedCredential() {
+    return usedCredential;
+  }
+
+  @Override
+  public CreateBucketV2Output createBucket(String bucket) throws TosException {
+    return retry(() -> client.createBucket(bucket));
+  }
+
+  @Override
+  public CreateBucketV2Output createBucket(CreateBucketV2Input input) throws TosException {
+    return retry(() -> client.createBucket(input));
+  }
+
+  @Override
+  public HeadBucketV2Output headBucket(HeadBucketV2Input input) throws TosException {
+    return retry(() -> client.headBucket(input));
+  }
+
+  @Override
+  public DeleteBucketOutput deleteBucket(DeleteBucketInput input) throws TosException {
+    return retry(() -> client.deleteBucket(input));
+  }
+
+  @Override
+  public ListBucketsV2Output listBuckets(ListBucketsV2Input input) throws TosException {
+    return retry(() -> client.listBuckets(input));
+  }
+
+  @Override
+  public CreateBucketOutput createBucket(CreateBucketInput input) throws TosException {
+    return retry(() -> client.createBucket(input));
+  }
+
+  @Override
+  public HeadBucketOutput headBucket(String bucket) throws TosException {
+    return retry(() -> client.headBucket(bucket));
+  }
+
+  @Override
+  public DeleteBucketOutput deleteBucket(String bucket) throws TosException {
+    return retry(() -> client.deleteBucket(bucket));
+  }
+
+  @Override
+  public ListBucketsOutput listBuckets(ListBucketsInput input) throws TosException {
+    return retry(() -> client.listBuckets(input));
+  }
+
+  @Override
+  public PutBucketPolicyOutput putBucketPolicy(String bucket, String policy) throws TosException {
+    return retry(() -> client.putBucketPolicy(bucket, policy));
+  }
+
+  @Override
+  public PutBucketPolicyOutput putBucketPolicy(PutBucketPolicyInput input) throws TosException {
+    return retry(() -> client.putBucketPolicy(input));
+  }
+
+  @Override
+  public GetBucketPolicyOutput getBucketPolicy(String bucket) throws TosException {
+    return retry(() -> client.getBucketPolicy(bucket));
+  }
+
+  @Override
+  public GetBucketPolicyOutput getBucketPolicy(GetBucketPolicyInput input) throws TosException {
+    return retry(() -> client.getBucketPolicy(input));
+  }
+
+  @Override
+  public DeleteBucketPolicyOutput deleteBucketPolicy(String bucket) throws TosException {
+    return retry(() -> client.deleteBucketPolicy(bucket));
+  }
+
+  @Override
+  public GetObjectOutput getObject(String bucket, String objectKey,
+      RequestOptionsBuilder... builders) throws TosException {
+    return retry(() -> client.getObject(bucket, objectKey, builders));
+  }
+
+  @Override
+  public HeadObjectOutput headObject(String bucket, String objectKey,
+      RequestOptionsBuilder... builders) throws TosException {
+    return retry(() -> client.headObject(bucket, objectKey, builders));
+  }
+
+  @Override
+  public DeleteObjectOutput deleteObject(String bucket, String objectKey,
+      RequestOptionsBuilder... builders) throws TosException {
+    return retry(() -> client.deleteObject(bucket, objectKey, builders));
+  }
+
+  @Override
+  public DeleteMultiObjectsOutput deleteMultiObjects(
+      String bucket,
+      DeleteMultiObjectsInput input,
+      RequestOptionsBuilder... builders)
+      throws TosException {
+    return retry(() -> client.deleteMultiObjects(bucket, input, builders));
+  }
+
+  @Override
+  public PutObjectOutput putObject(
+      String bucket, String objectKey, InputStream inputStream,
+      RequestOptionsBuilder... builders)
+      throws TosException {
+    throw new UnsupportedOperationException("Not supported");
+  }
+
+  @Override
+  public UploadFileOutput uploadFile(
+      String bucket, UploadFileInput input,
+      RequestOptionsBuilder... builders) throws TosException {
+    return retry(() -> client.uploadFile(bucket, input, builders));
+  }
+
+  @Override
+  public AppendObjectOutput appendObject(
+      String bucket, String objectKey, InputStream content, long offset,
+      RequestOptionsBuilder... builders)
+      throws TosException {
+    throw new UnsupportedOperationException("Not supported");
+  }
+
+  @Override
+  public SetObjectMetaOutput setObjectMeta(String bucket, String objectKey,
+      RequestOptionsBuilder... builders) throws TosException {
+    return retry(() -> client.setObjectMeta(bucket, objectKey, builders));
+  }
+
+  @Override
+  public ListObjectsOutput listObjects(String bucket, ListObjectsInput input) throws TosException {
+    return retry(() -> client.listObjects(bucket, input));
+  }
+
+  @Override
+  public ListObjectVersionsOutput listObjectVersions(String bucket, ListObjectVersionsInput input)
+      throws TosException {
+    return retry(() -> client.listObjectVersions(bucket, input));
+  }
+
+  @Override
+  public CopyObjectOutput copyObject(
+      String bucket, String srcObjectKey, String dstObjectKey,
+      RequestOptionsBuilder... builders)
+      throws TosException {
+    return retry(() -> client.copyObject(bucket, srcObjectKey, dstObjectKey, builders));
+  }
+
+  @Override
+  public CopyObjectOutput copyObjectTo(
+      String bucket, String dstBucket, String dstObjectKey,
+      String srcObjectKey,
+      RequestOptionsBuilder... builders)
+      throws TosException {
+    return retry(() ->
+        client.copyObjectTo(bucket, dstBucket, dstObjectKey, srcObjectKey, builders));
+  }
+
+  @Override
+  public CopyObjectOutput copyObjectFrom(
+      String bucket, String srcBucket, String srcObjectKey, String dstObjectKey,
+      RequestOptionsBuilder... builders)
+      throws TosException {
+    return retry(() ->
+        client.copyObjectFrom(bucket, srcBucket, srcObjectKey, dstObjectKey, builders));
+  }
+
+  @Override
+  public UploadPartCopyOutput uploadPartCopy(
+      String bucket, UploadPartCopyInput input,
+      RequestOptionsBuilder... builders) throws TosException {
+    return retry(() -> client.uploadPartCopy(bucket, input, builders));
+  }
+
+  @Override
+  public PutObjectAclOutput putObjectAcl(String bucket, PutObjectAclInput input)
+      throws TosException {
+    return retry(() -> client.putObjectAcl(bucket, input));
+  }
+
+  @Override
+  public GetObjectAclOutput getObjectAcl(
+      String bucket, String objectKey,
+      RequestOptionsBuilder... builders)
+      throws TosException {
+    return retry(() -> client.getObjectAcl(bucket, objectKey, builders));
+  }
+
+  @Override
+  public CreateMultipartUploadOutput createMultipartUpload(
+      String bucket, String objectKey,
+      RequestOptionsBuilder... builders)
+      throws TosException {
+    return retry(() -> client.createMultipartUpload(bucket, objectKey, builders));
+  }
+
+  @Override
+  public UploadPartOutput uploadPart(
+      String bucket, UploadPartInput input,
+      RequestOptionsBuilder... builders)
+      throws TosException {
+    throw new UnsupportedOperationException("Not supported");
+  }
+
+  @Override
+  public CompleteMultipartUploadOutput completeMultipartUpload(
+      String bucket,
+      CompleteMultipartUploadInput input)
+      throws TosException {
+    return retry(() -> client.completeMultipartUpload(bucket, input));
+  }
+
+  @Override
+  public AbortMultipartUploadOutput abortMultipartUpload(
+      String bucket,
+      AbortMultipartUploadInput input)
+      throws TosException {
+    return retry(() -> client.abortMultipartUpload(bucket, input));
+  }
+
+  @Override
+  public ListUploadedPartsOutput listUploadedParts(
+      String bucket,
+      ListUploadedPartsInput input,
+      RequestOptionsBuilder... builders)
+      throws TosException {
+    return retry(() -> client.listUploadedParts(bucket, input, builders));
+  }
+
+  @Override
+  public ListMultipartUploadsOutput listMultipartUploads(
+      String bucket,
+      ListMultipartUploadsInput input)
+      throws TosException {
+    return retry(() -> client.listMultipartUploads(bucket, input));
+  }
+
+  @Override
+  public String preSignedURL(
+      String httpMethod, String bucket, String objectKey, Duration ttl,
+      RequestOptionsBuilder... builders)
+      throws TosException {
+    return retry(() -> client.preSignedURL(httpMethod, bucket, objectKey, ttl, builders));
+  }
+
+  @Override
+  public DeleteBucketPolicyOutput deleteBucketPolicy(DeleteBucketPolicyInput input)
+      throws TosException {
+    return retry(() -> client.deleteBucketPolicy(input));
+  }
+
+  @Override
+  public PutBucketCORSOutput putBucketCORS(PutBucketCORSInput input)
+      throws TosException {
+    return retry(() -> client.putBucketCORS(input));
+  }
+
+  @Override
+  public GetBucketCORSOutput getBucketCORS(GetBucketCORSInput input)
+      throws TosException {
+    return retry(() -> client.getBucketCORS(input));
+  }
+
+  @Override
+  public DeleteBucketCORSOutput deleteBucketCORS(DeleteBucketCORSInput input)
+      throws TosException {
+    return retry(() -> client.deleteBucketCORS(input));
+  }
+
+  @Override
+  public PutBucketStorageClassOutput putBucketStorageClass(PutBucketStorageClassInput input)
+      throws TosException {
+    return retry(() -> client.putBucketStorageClass(input));
+  }
+
+  @Override
+  public GetBucketLocationOutput getBucketLocation(GetBucketLocationInput input)
+      throws TosException {
+    return retry(() -> client.getBucketLocation(input));
+  }
+
+  @Override
+  public PutBucketLifecycleOutput putBucketLifecycle(PutBucketLifecycleInput input)
+      throws TosException {
+    return retry(() -> client.putBucketLifecycle(input));
+  }
+
+  @Override
+  public GetBucketLifecycleOutput getBucketLifecycle(GetBucketLifecycleInput input)
+      throws TosException {
+    return retry(() -> client.getBucketLifecycle(input));
+  }
+
+  @Override
+  public DeleteBucketLifecycleOutput deleteBucketLifecycle(DeleteBucketLifecycleInput input)
+      throws TosException {
+    return retry(() -> client.deleteBucketLifecycle(input));
+  }
+
+  @Override
+  public PutBucketMirrorBackOutput putBucketMirrorBack(PutBucketMirrorBackInput input)
+      throws TosException {
+    return retry(() -> client.putBucketMirrorBack(input));
+  }
+
+  @Override
+  public GetBucketMirrorBackOutput getBucketMirrorBack(GetBucketMirrorBackInput input)
+      throws TosException {
+    return retry(() -> client.getBucketMirrorBack(input));
+  }
+
+  @Override
+  public DeleteBucketMirrorBackOutput deleteBucketMirrorBack(DeleteBucketMirrorBackInput input)
+      throws TosException {
+    return retry(() -> client.deleteBucketMirrorBack(input));
+  }
+
+  @Override
+  public PutBucketReplicationOutput putBucketReplication(PutBucketReplicationInput input)
+      throws TosException {
+    return retry(() -> client.putBucketReplication(input));
+  }
+
+  @Override
+  public GetBucketReplicationOutput getBucketReplication(GetBucketReplicationInput input)
+      throws TosException {
+    return retry(() -> client.getBucketReplication(input));
+  }
+
+  @Override
+  public DeleteBucketReplicationOutput deleteBucketReplication(DeleteBucketReplicationInput input)
+      throws TosException {
+    return retry(() -> client.deleteBucketReplication(input));
+  }
+
+  @Override
+  public PutBucketVersioningOutput putBucketVersioning(PutBucketVersioningInput input)
+      throws TosException {
+    return retry(() -> client.putBucketVersioning(input));
+  }
+
+  @Override
+  public GetBucketVersioningOutput getBucketVersioning(GetBucketVersioningInput input)
+      throws TosException {
+    return retry(() -> client.getBucketVersioning(input));
+  }
+
+  @Override
+  public PutBucketWebsiteOutput putBucketWebsite(PutBucketWebsiteInput input)
+      throws TosException {
+    return retry(() -> client.putBucketWebsite(input));
+  }
+
+  @Override
+  public GetBucketWebsiteOutput getBucketWebsite(GetBucketWebsiteInput input)
+      throws TosException {
+    return retry(() -> client.getBucketWebsite(input));
+  }
+
+  @Override
+  public DeleteBucketWebsiteOutput deleteBucketWebsite(DeleteBucketWebsiteInput input)
+      throws TosException {
+    return retry(() -> client.deleteBucketWebsite(input));
+  }
+
+  @Override
+  public PutBucketNotificationOutput putBucketNotification(PutBucketNotificationInput input)
+      throws TosException {
+    return retry(() -> client.putBucketNotification(input));
+  }
+
+  @Override
+  public GetBucketNotificationOutput getBucketNotification(GetBucketNotificationInput input)
+      throws TosException {
+    return retry(() -> client.getBucketNotification(input));
+  }
+
+  @Override
+  public PutBucketNotificationType2Output putBucketNotificationType2(
+      PutBucketNotificationType2Input input) throws TosException {
+    return retry(() -> client.putBucketNotificationType2(input));
+  }
+
+  @Override
+  public GetBucketNotificationType2Output getBucketNotificationType2(
+      GetBucketNotificationType2Input input) throws TosException {
+    return retry(() -> client.getBucketNotificationType2(input));
+  }
+
+  @Override
+  public PutBucketCustomDomainOutput putBucketCustomDomain(PutBucketCustomDomainInput input)
+      throws TosException {
+    return retry(() -> client.putBucketCustomDomain(input));
+  }
+
+  @Override
+  public ListBucketCustomDomainOutput listBucketCustomDomain(ListBucketCustomDomainInput input)
+      throws TosException {
+    return retry(() -> client.listBucketCustomDomain(input));
+  }
+
+  @Override
+  public DeleteBucketCustomDomainOutput deleteBucketCustomDomain(
+      DeleteBucketCustomDomainInput input) throws TosException {
+    return retry(() -> client.deleteBucketCustomDomain(input));
+  }
+
+  @Override
+  public PutBucketRealTimeLogOutput putBucketRealTimeLog(PutBucketRealTimeLogInput input)
+      throws TosException {
+    return retry(() -> client.putBucketRealTimeLog(input));
+  }
+
+  @Override
+  public GetBucketRealTimeLogOutput getBucketRealTimeLog(GetBucketRealTimeLogInput input)
+      throws TosException {
+    return retry(() -> client.getBucketRealTimeLog(input));
+  }
+
+  @Override
+  public DeleteBucketRealTimeLogOutput deleteBucketRealTimeLog(DeleteBucketRealTimeLogInput input)
+      throws TosException {
+    return retry(() -> deleteBucketRealTimeLog(input));
+  }
+
+  @Override
+  public PutBucketACLOutput putBucketACL(PutBucketACLInput input) throws TosException {
+    return retry(() -> client.putBucketACL(input));
+  }
+
+  @Override
+  public GetBucketACLOutput getBucketACL(GetBucketACLInput input) throws TosException {
+    return retry(() -> client.getBucketACL(input));
+  }
+
+  @Override
+  public PutBucketRenameOutput putBucketRename(PutBucketRenameInput input) throws TosException {
+    return retry(() -> client.putBucketRename(input));
+  }
+
+  @Override
+  public GetBucketRenameOutput getBucketRename(GetBucketRenameInput input) throws TosException {
+    return retry(() -> client.getBucketRename(input));
+  }
+
+  @Override
+  public DeleteBucketRenameOutput deleteBucketRename(DeleteBucketRenameInput input)
+      throws TosException {
+    return retry(() -> client.deleteBucketRename(input));
+  }
+
+  @Override
+  public PutBucketEncryptionOutput putBucketEncryption(PutBucketEncryptionInput input)
+      throws TosException {
+    return retry(() -> client.putBucketEncryption(input));
+  }
+
+  @Override
+  public GetBucketEncryptionOutput getBucketEncryption(GetBucketEncryptionInput input)
+      throws TosException {
+    return retry(() -> client.getBucketEncryption(input));
+  }
+
+  @Override
+  public DeleteBucketEncryptionOutput deleteBucketEncryption(DeleteBucketEncryptionInput input)
+      throws TosException {
+    return retry(() -> client.deleteBucketEncryption(input));
+  }
+
+  @Override
+  public PutBucketTaggingOutput putBucketTagging(PutBucketTaggingInput input) throws TosException {
+    return retry(() -> client.putBucketTagging(input));
+  }
+
+  @Override
+  public GetBucketTaggingOutput getBucketTagging(GetBucketTaggingInput input) throws TosException {
+    return retry(() -> client.getBucketTagging(input));
+  }
+
+  @Override
+  public DeleteBucketTaggingOutput deleteBucketTagging(DeleteBucketTaggingInput input)
+      throws TosException {
+    return retry(() -> client.deleteBucketTagging(input));
+  }
+
+  @Override
+  public PutBucketInventoryOutput putBucketInventory(PutBucketInventoryInput input)
+      throws TosException {
+    return retry(() -> client.putBucketInventory(input));
+  }
+
+  @Override
+  public GetBucketInventoryOutput getBucketInventory(GetBucketInventoryInput input)
+      throws TosException {
+    return retry(() -> client.getBucketInventory(input));
+  }
+
+  @Override
+  public ListBucketInventoryOutput listBucketInventory(ListBucketInventoryInput input)
+      throws TosException {
+    return retry(() -> client.listBucketInventory(input));
+  }
+
+  @Override
+  public DeleteBucketInventoryOutput deleteBucketInventory(DeleteBucketInventoryInput input)
+      throws TosException {
+    return retry(() -> client.deleteBucketInventory(input));
+  }
+
+  @Override
+  public GetObjectV2Output getObject(GetObjectV2Input input) throws TosException {
+    return retry(() -> client.getObject(input));
+  }
+
+  @Override
+  public GetObjectToFileOutput getObjectToFile(GetObjectToFileInput input) throws TosException {
+    return retry(() -> client.getObjectToFile(input));
+  }
+
+  @Override
+  public GetFileStatusOutput getFileStatus(GetFileStatusInput input) throws TosException {
+    return retry(() -> client.getFileStatus(input));
+  }
+
+  @Override
+  public UploadFileV2Output uploadFile(UploadFileV2Input input) throws TosException {
+    return retry(() -> client.uploadFile(input));
+  }
+
+  @Override
+  public DownloadFileOutput downloadFile(DownloadFileInput input) throws TosException {
+    return retry(() -> client.downloadFile(input));
+  }
+
+  @Override
+  public ResumableCopyObjectOutput resumableCopyObject(ResumableCopyObjectInput input)
+      throws TosException {
+    return retry(() -> client.resumableCopyObject(input));
+  }
+
+  @Override
+  public HeadObjectV2Output headObject(HeadObjectV2Input input) throws TosException {
+    return retry(() -> client.headObject(input));
+  }
+
+  @Override
+  public DeleteObjectOutput deleteObject(DeleteObjectInput input) throws TosException {
+    return retry(() -> client.deleteObject(input));
+  }
+
+  @Override
+  public DeleteMultiObjectsV2Output deleteMultiObjects(DeleteMultiObjectsV2Input input)
+      throws TosException {
+    return retry(() -> client.deleteMultiObjects(input));
+  }
+
+  public PutObjectOutput put(
+      String bucket, String key, InputStreamProvider streamProvider,
+      long contentLength, ACLType aclType) {
+    return retry(() -> client.putObject(
+        newPutObjectRequest(bucket, key, streamProvider, contentLength, aclType)));
+  }
+
+  private PutObjectInput newPutObjectRequest(
+      String bucket,
+      String key,
+      InputStreamProvider streamProvider,
+      long contentLength,
+      ACLType aclType) {
+
+    return PutObjectInput.builder()
+        .bucket(bucket)
+        .key(key)
+        .content(streamProvider.newStream())
+        .contentLength(contentLength)
+        .options(new ObjectMetaRequestOptions()
+            .setAclType(aclType))
+        .build();
+  }
+
+  public AppendObjectOutput appendObject(String bucket, String key,
+      InputStreamProvider streamProvider, long offset, long contentLength, String originalCrc64,
+      ACLType aclType) {
+    // originalCrc64 is needed when appending data to object. It should be the object's crc64
+    // checksum if the object exists, and null if the object doesn't exist.
+    return retry(() -> client.appendObject(
+        newAppendObjectRequest(bucket, key, streamProvider, offset, contentLength, originalCrc64,
+            aclType)));
+  }
+
+  private AppendObjectInput newAppendObjectRequest(
+      String bucket,
+      String key,
+      InputStreamProvider streamProvider,
+      long offset,
+      long contentLength,
+      String preCrc64ecma,
+      ACLType aclType) {
+    return AppendObjectInput.builder()
+        .bucket(bucket)
+        .key(key)
+        .content(streamProvider.newStream())
+        .offset(offset)
+        .contentLength(contentLength)
+        .preHashCrc64ecma(preCrc64ecma)
+        .options(new ObjectMetaRequestOptions()
+            .setAclType(aclType))
+        .build();
+  }
+
+  @Override
+  public PutObjectOutput putObject(PutObjectInput input) throws TosException {
+    throw new UnsupportedOperationException("Not supported");
+  }
+
+  @Override
+  public PutObjectFromFileOutput putObjectFromFile(PutObjectFromFileInput input)
+      throws TosException {
+    return retry(() -> client.putObjectFromFile(input));
+  }
+
+  @Override
+  public AppendObjectOutput appendObject(AppendObjectInput input)
+      throws TosException {
+    throw new UnsupportedOperationException("Not supported");
+  }
+
+  @Override
+  public SetObjectMetaOutput setObjectMeta(SetObjectMetaInput input)
+      throws TosException {
+    return retry(() -> client.setObjectMeta(input));
+  }
+
+  @Override
+  public SetObjectTimeOutput setObjectTime(SetObjectTimeInput input) throws TosException {
+    return retry(() -> client.setObjectTime(input));
+  }
+
+  @Override
+  public ListObjectsV2Output listObjects(ListObjectsV2Input input)
+      throws TosException {
+    return retry(() -> client.listObjects(input));
+  }
+
+  @Override
+  public ListObjectsType2Output listObjectsType2(ListObjectsType2Input input)
+      throws TosException {
+    return retry(() -> client.listObjectsType2(input));
+  }
+
+  @Override
+  public ListObjectVersionsV2Output listObjectVersions(ListObjectVersionsV2Input input)
+      throws TosException {
+    return retry(() -> client.listObjectVersions(input));
+  }
+
+  @Override
+  public CopyObjectV2Output copyObject(CopyObjectV2Input input)
+      throws TosException {
+    return retry(() -> client.copyObject(input));
+  }
+
+  @Override
+  public UploadPartCopyV2Output uploadPartCopy(UploadPartCopyV2Input input)
+      throws TosException {
+    return retry(() -> client.uploadPartCopy(input));
+  }
+
+  @Override
+  public PutObjectACLOutput putObjectAcl(PutObjectACLInput input)
+      throws TosException {
+    return retry(() -> client.putObjectAcl(input));
+  }
+
+  @Override
+  public GetObjectACLV2Output getObjectAcl(GetObjectACLV2Input input)
+      throws TosException {
+    return retry(() -> client.getObjectAcl(input));
+  }
+
+  @Override
+  public PutObjectTaggingOutput putObjectTagging(PutObjectTaggingInput input)
+      throws TosException {
+    return retry(() -> client.putObjectTagging(input));
+  }
+
+  @Override
+  public GetObjectTaggingOutput getObjectTagging(GetObjectTaggingInput input)
+      throws TosException {
+    return retry(() -> client.getObjectTagging(input));
+  }
+
+  @Override
+  public DeleteObjectTaggingOutput deleteObjectTagging(DeleteObjectTaggingInput input)
+      throws TosException {
+    return retry(() -> client.deleteObjectTagging(input));
+  }
+
+  @Override
+  public FetchObjectOutput fetchObject(FetchObjectInput input) throws TosException {
+    return retry(() -> client.fetchObject(input));
+  }
+
+  @Override
+  public PutFetchTaskOutput putFetchTask(PutFetchTaskInput input) throws TosException {
+    return retry(() -> client.putFetchTask(input));
+  }
+
+  @Override
+  public GetFetchTaskOutput getFetchTask(GetFetchTaskInput input) throws TosException {
+    return retry(() -> client.getFetchTask(input));
+  }
+
+  @Override
+  public CreateMultipartUploadOutput createMultipartUpload(CreateMultipartUploadInput input)
+      throws TosException {
+    return retry(() -> client.createMultipartUpload(input));
+  }
+
+  public Part uploadPart(
+      String bucket,
+      String key,
+      String uploadId,
+      int partNum,
+      InputStreamProvider streamProvider,
+      long contentLength,
+      ACLType aclType) {
+    return retry(() -> {
+      InputStream in = streamProvider.newStream();
+      CountingInputStream countedIn = new CountingInputStream(in);
+      UploadPartV2Input request = UploadPartV2Input.builder()
+          .bucket(bucket)
+          .key(key)
+          .partNumber(partNum)
+          .uploadID(uploadId)
+          .content(countedIn)
+          .contentLength(contentLength)
+          .options(new ObjectMetaRequestOptions()
+              .setAclType(aclType))
+          .build();
+      UploadPartV2Output output = client.uploadPart(request);
+      return new Part(output.getPartNumber(), countedIn.getCount(), output.getEtag());
+    });
+  }
+
+  @Override
+  public UploadPartV2Output uploadPart(UploadPartV2Input input) throws TosException {
+    throw new UnsupportedOperationException("Not supported");
+  }
+
+  @Override
+  public UploadPartFromFileOutput uploadPartFromFile(UploadPartFromFileInput input)
+      throws TosException {
+    return retry(() -> client.uploadPartFromFile(input));
+  }
+
+  @Override
+  public CompleteMultipartUploadV2Output completeMultipartUpload(
+      CompleteMultipartUploadV2Input input) throws TosException {
+    return retry(() -> client.completeMultipartUpload(input));
+  }
+
+  @Override
+  public AbortMultipartUploadOutput abortMultipartUpload(AbortMultipartUploadInput input)
+      throws TosException {
+    return retry(() -> client.abortMultipartUpload(input));
+  }
+
+  @Override
+  public ListPartsOutput listParts(ListPartsInput input) throws TosException {
+    return retry(() -> client.listParts(input));
+  }
+
+  @Override
+  public ListMultipartUploadsV2Output listMultipartUploads(ListMultipartUploadsV2Input input)
+      throws TosException {
+    return retry(() -> client.listMultipartUploads(input));
+  }
+
+  @Override
+  public RenameObjectOutput renameObject(RenameObjectInput input) throws TosException {
+    return retry(() -> client.renameObject(input));
+  }
+
+  @Override
+  public RestoreObjectOutput restoreObject(RestoreObjectInput input) throws TosException {
+    return retry(() -> client.restoreObject(input));
+  }
+
+  @Override
+  public PutSymlinkOutput putSymlink(PutSymlinkInput input) throws TosException {
+    return retry(() -> client.putSymlink(input));
+  }
+
+  @Override
+  public GetSymlinkOutput getSymlink(GetSymlinkInput input) throws TosException {
+    return retry(() -> client.getSymlink(input));
+  }
+
+  @Override
+  public PreSignedURLOutput preSignedURL(PreSignedURLInput input) throws TosException {
+    return retry(() -> client.preSignedURL(input));
+  }
+
+  @Override
+  public PreSignedPostSignatureOutput preSignedPostSignature(PreSignedPostSignatureInput input)
+      throws TosException {
+    return retry(() -> client.preSignedPostSignature(input));
+  }
+
+  @Override
+  public PreSingedPolicyURLOutput preSingedPolicyURL(PreSingedPolicyURLInput input)
+      throws TosException {
+    return retry(() -> client.preSingedPolicyURL(input));
+  }
+
+  @Override
+  public PreSignedPolicyURLOutput preSignedPolicyURL(PreSignedPolicyURLInput input)
+      throws TosException {
+    return retry(() -> client.preSignedPolicyURL(input));
+  }
+
+  @Override
+  public void changeCredentials(Credentials credentials) {
+    retry(() -> {
+      client.changeCredentials(credentials);
+      return null;
+    });
+  }
+
+  @Override
+  public void changeRegionAndEndpoint(String region, String endpoint) {
+    retry(() -> {
+      client.changeRegionAndEndpoint(region, endpoint);
+      return null;
+    });
+  }
+
+  @Override
+  public void changeTransportConfig(TransportConfig conf) {
+    retry(() -> {
+      client.changeTransportConfig(conf);
+      return null;
+    });
+  }
+
+  @Override
+  public boolean refreshEndpointRegion(String s, String s1) {
+    return retry(() -> refreshEndpointRegion(s, s1));
+  }
+
+  @Override
+  public boolean refreshCredentials(String s, String s1, String s2) {
+    return retry(() -> refreshCredentials(s, s1, s2));
+  }
+
+  @Override
+  public void close() throws IOException {
+    client.close();
+  }
+
+  private void refresh() throws TosException {
+    Credential credential = provider.credential();
+    if (credentialIsChanged(credential)) {
+      synchronized (this) {
+        if (credentialIsChanged(credential)) {
+          client.changeCredentials(provider);
+          usedCredential = credential;
+        }
+      }
+    }
+  }
+
+  private boolean credentialIsChanged(Credential credential) {
+    return !Objects.equals(credential.getAccessKeyId(), usedCredential.getAccessKeyId())
+        || !Objects.equals(credential.getAccessKeySecret(), usedCredential.getAccessKeySecret())
+        || !Objects.equals(credential.getSecurityToken(), usedCredential.getSecurityToken());
+  }
+
+  private  T retry(Callable callable) {
+    int attempt = 0;
+    while (true) {
+      attempt++;
+      try {
+        refresh();
+        return callable.call();
+      } catch (TosException e) {
+        if (attempt >= maxRetryTimes) {
+          LOG.error("Retry exhausted after {} times.", maxRetryTimes);
+          throw e;
+        }
+        if (isRetryableException(e, nonRetryable409ErrorCodes)) {
+          LOG.warn("Retry TOS request in the {} times, error: {}", attempt,
+              Throwables.getRootCause(e).getMessage());
+          try {
+            // last time does not need to sleep
+            Thread.sleep(RetryableUtils.backoff(attempt));
+          } catch (InterruptedException ex) {
+            throw new TosClientException("tos: request interrupted.", ex);
+          }
+        } else {
+          throw e;
+        }
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  static boolean isRetryableException(TosException e, List nonRetryable409ErrorCodes) {
+    return e.getStatusCode() >= HttpStatus.INTERNAL_SERVER_ERROR
+        || e.getStatusCode() == HttpStatus.TOO_MANY_REQUESTS
+        || e.getCause() instanceof SocketException
+        || e.getCause() instanceof UnknownHostException
+        || e.getCause() instanceof SSLException
+        || e.getCause() instanceof SocketTimeoutException
+        || e.getCause() instanceof InterruptedException
+        || isRetryableTosClientException(e)
+        || isRetryableTosServerException(e, nonRetryable409ErrorCodes);
+  }
+
+  private static boolean isRetryableTosClientException(TosException e) {
+    return e instanceof TosClientException
+        && e.getCause() instanceof IOException
+        && !(e.getCause() instanceof EOFException);
+  }
+
+  private static boolean isRetryableTosServerException(TosException e,
+      List nonRetryable409ErrorCodes) {
+    return e instanceof TosServerException
+        && e.getStatusCode() == HttpStatus.CONFLICT
+        && isRetryableTosConflictException((TosServerException) e, nonRetryable409ErrorCodes);
+  }
+
+  private static boolean isRetryableTosConflictException(TosServerException e,
+      List nonRetryableCodes) {
+    String errorCode = e.getEc();
+    return StringUtils.isEmpty(errorCode) || !nonRetryableCodes.contains(errorCode);
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClientBuilder.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClientBuilder.java
new file mode 100644
index 0000000000000..cc1640c2e8af6
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClientBuilder.java
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object.tos;
+
+import com.volcengine.tos.TOSClientConfiguration;
+import com.volcengine.tos.TosException;
+import com.volcengine.tos.transport.TransportConfig;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.tosfs.conf.ConfKeys;
+import org.apache.hadoop.fs.tosfs.conf.TosKeys;
+import org.apache.hadoop.fs.tosfs.object.Constants;
+import org.apache.hadoop.fs.tosfs.object.tos.auth.CredentialsProvider;
+import org.apache.hadoop.fs.tosfs.util.ParseUtils;
+import org.apache.hadoop.fs.tosfs.util.TOSClientContextUtils;
+import org.apache.hadoop.util.Preconditions;
+import org.apache.hadoop.util.VersionInfo;
+
+import java.lang.reflect.InvocationTargetException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.apache.hadoop.fs.tosfs.object.tos.TOS.TOS_SCHEME;
+
+public class DelegationClientBuilder {
+
+  public static final int DISABLE_TOS_RETRY_VALUE = -1;
+  private static final String TOS_ENDPOINT_KEY =
+      ConfKeys.FS_OBJECT_STORAGE_ENDPOINT.key(TOS_SCHEME);
+  private static final String TOS_REGION_KEY = ConfKeys.FS_OBJECT_STORAGE_REGION.key(TOS_SCHEME);
+
+  @VisibleForTesting
+  static final Map CACHE = new ConcurrentHashMap<>();
+
+  private String bucket;
+  private Configuration conf;
+
+  public DelegationClientBuilder bucket(String bucketInput) {
+    this.bucket = bucketInput;
+    return this;
+  }
+
+  public DelegationClientBuilder conf(Configuration confInput) {
+    this.conf = confInput;
+    return this;
+  }
+
+  public DelegationClient build() throws TosException {
+    Preconditions.checkNotNull(bucket, "Bucket cannot be null");
+    Preconditions.checkNotNull(conf, "Conf cannot be null");
+    String endpoint = getAndCheckEndpoint(conf);
+    String region = getAndCheckRegion(conf, endpoint);
+
+    if (conf.getBoolean(TosKeys.FS_TOS_DISABLE_CLIENT_CACHE,
+        TosKeys.FS_TOS_DISABLE_CLIENT_CACHE_DEFAULT)) {
+      return createNewClient(conf, endpoint, region, bucket, false);
+    }
+    return CACHE.computeIfAbsent(bucket,
+        client -> createNewClient(conf, endpoint, region, bucket, true));
+  }
+
+  private DelegationClient createNewClient(Configuration config, String endpoint, String region,
+      String bucketName, boolean cached) {
+    CredentialsProvider provider = createProvider(config, bucketName);
+    TOSClientConfiguration clientConfiguration = TOSClientConfiguration.builder()
+        .region(region)
+        .endpoint(endpoint)
+        .credentials(provider)
+        .enableCrc(config.getBoolean(
+            TosKeys.FS_TOS_CRC_CHECK_ENABLED, TosKeys.FS_TOS_CRC_CHECK_ENABLED_DEFAULT))
+        .transportConfig(createTransportConfig(config))
+        .userAgentProductName(config.get(
+            TosKeys.FS_TOS_USER_AGENT_PREFIX, TosKeys.FS_TOS_USER_AGENT_PREFIX_DEFAULT))
+        .userAgentSoftName(Constants.TOS_FS)
+        .userAgentSoftVersion(VersionInfo.getVersion())
+        .build();
+
+    int maxRetryTimes = config.getInt(TosKeys.FS_TOS_REQUEST_MAX_RETRY_TIMES,
+        TosKeys.FS_TOS_REQUEST_MAX_RETRY_TIMES_DEFAULT);
+    List nonRetryable409ErrorCodes = Arrays.asList(
+        config.getTrimmedStrings(TosKeys.FS_TOS_FAST_FAILURE_409_ERROR_CODES,
+            TosKeys.FS_TOS_FAST_FAILURE_409_ERROR_CODES_DEFAULT));
+
+    if (cached) {
+      return new CachedClient(clientConfiguration, maxRetryTimes, nonRetryable409ErrorCodes);
+    } else {
+      return new DelegationClient(clientConfiguration, maxRetryTimes, nonRetryable409ErrorCodes);
+    }
+  }
+
+  private CredentialsProvider createProvider(Configuration config, String bucketName) {
+    try {
+      CredentialsProvider provider = (CredentialsProvider) Class.forName(
+              config.get(TosKeys.FS_TOS_CREDENTIALS_PROVIDER,
+                  TosKeys.FS_TOS_CREDENTIALS_PROVIDER_DEFAULT))
+              .getDeclaredConstructor()
+              .newInstance();
+      provider.initialize(config, bucketName);
+      return provider;
+    } catch (ClassNotFoundException |
+             InstantiationException |
+             IllegalAccessException |
+             InvocationTargetException |
+             NoSuchMethodException e) {
+      throw new TosException(e);
+    }
+  }
+
+  private String getAndCheckEndpoint(Configuration config) {
+    String endpoint = config.get(TOS_ENDPOINT_KEY);
+    if (StringUtils.isBlank(endpoint)) {
+      endpoint = ParseUtils.envAsString(TOS.ENV_TOS_ENDPOINT);
+    }
+    Preconditions.checkNotNull(endpoint, "%s cannot be null", TOS_ENDPOINT_KEY);
+    return endpoint.trim();
+  }
+
+  private String getAndCheckRegion(Configuration config, String endpoint) {
+    String region = config.get(TOS_REGION_KEY);
+    if (StringUtils.isNotBlank(region)) {
+      return region.trim();
+    }
+    region = TOSClientContextUtils.parseRegion(endpoint);
+    Preconditions.checkNotNull(region, "%s cannot be null", TOS_REGION_KEY);
+    return region.trim();
+  }
+
+  private TransportConfig createTransportConfig(Configuration config) {
+    TransportConfig.TransportConfigBuilder builder = TransportConfig.builder();
+    // Disable tos sdk retry with negative number since we have set retry strategy above TOS SDK,
+    // which cannot support retry all input streams via mark & reset API.
+    // It's hard to use it as there are some restrictions.
+    // the TOS SDK will reset the max retry count with 3 if the configured count equal to 0.
+    builder.maxRetryCount(DISABLE_TOS_RETRY_VALUE);
+
+    builder.maxConnections(config.getInt(TosKeys.FS_TOS_HTTP_MAX_CONNECTIONS,
+        TosKeys.FS_TOS_HTTP_MAX_CONNECTIONS_DEFAULT));
+    builder.idleConnectionTimeMills(config.getInt(TosKeys.FS_TOS_HTTP_IDLE_CONNECTION_TIME_MILLS,
+        TosKeys.FS_TOS_HTTP_IDLE_CONNECTION_TIME_MILLS_DEFAULT));
+    builder.connectTimeoutMills(config.getInt(TosKeys.FS_TOS_HTTP_CONNECT_TIMEOUT_MILLS,
+        TosKeys.FS_TOS_HTTP_CONNECT_TIMEOUT_MILLS_DEFAULT));
+    builder.readTimeoutMills(config.getInt(TosKeys.FS_TOS_HTTP_READ_TIMEOUT_MILLS,
+        TosKeys.FS_TOS_HTTP_READ_TIMEOUT_MILLS_DEFAULT));
+    builder.writeTimeoutMills(config.getInt(TosKeys.FS_TOS_HTTP_WRITE_TIMEOUT_MILLS,
+        TosKeys.FS_TOS_HTTP_WRITE_TIMEOUT_MILLS_DEFAULT));
+    builder.enableVerifySSL(config.getBoolean(TosKeys.FS_TOS_HTTP_ENABLE_VERIFY_SSL,
+        TosKeys.FS_TOS_HTTP_ENABLE_VERIFY_SSL_DEFAULT));
+    builder.dnsCacheTimeMinutes(config.getInt(TosKeys.FS_TOS_HTTP_DNS_CACHE_TIME_MINUTES,
+        TosKeys.FS_TOS_HTTP_DNS_CACHE_TIME_MINUTES_DEFAULT));
+
+    return builder.build();
+  }
+
+  static class CachedClient extends DelegationClient {
+
+    protected CachedClient(TOSClientConfiguration configuration, int maxRetryTimes,
+        List nonRetryable409ErrorCodes) {
+      super(configuration, maxRetryTimes, nonRetryable409ErrorCodes);
+    }
+
+    @Override
+    public void close() {
+      // do nothing as this client may be shared by multiple upper-layer instances
+    }
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/GetObjectOutput.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/GetObjectOutput.java
new file mode 100644
index 0000000000000..718cc7e39ef24
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/GetObjectOutput.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object.tos;
+
+import com.volcengine.tos.model.object.GetObjectV2Output;
+import org.apache.hadoop.fs.tosfs.object.exceptions.ChecksumMismatchException;
+import org.apache.hadoop.fs.tosfs.util.CommonUtils;
+import org.apache.hadoop.util.Preconditions;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+
+public class GetObjectOutput {
+  private final GetObjectV2Output output;
+  private final byte[] checksum;
+
+  public GetObjectOutput(GetObjectV2Output output, byte[] checksum) {
+    Preconditions.checkNotNull(checksum, "Checksum should not be null.");
+    this.output = output;
+    this.checksum = checksum;
+  }
+
+  public GetObjectV2Output output() {
+    return output;
+  }
+
+  public byte[] checksum() {
+    return checksum;
+  }
+
+  public InputStream verifiedContent(byte[] expectedChecksum) throws IOException {
+    if (!Arrays.equals(expectedChecksum, checksum)) {
+      CommonUtils.runQuietly(this::forceClose);
+      throw new ChecksumMismatchException(expectedChecksum, checksum);
+    }
+
+    return output.getContent();
+  }
+
+  public void forceClose() throws IOException {
+    output.forceClose();
+  }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOS.java b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOS.java
new file mode 100644
index 0000000000000..aac7b933478bb
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOS.java
@@ -0,0 +1,1038 @@
+/*
+ * ByteDance Volcengine EMR, Copyright 2022.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.tosfs.object.tos;
+
+import com.volcengine.tos.TOSV2;
+import com.volcengine.tos.TosException;
+import com.volcengine.tos.TosServerException;
+import com.volcengine.tos.comm.common.ACLType;
+import com.volcengine.tos.comm.common.BucketType;
+import com.volcengine.tos.internal.util.TypeConverter;
+import com.volcengine.tos.model.bucket.HeadBucketV2Input;
+import com.volcengine.tos.model.bucket.HeadBucketV2Output;
+import com.volcengine.tos.model.bucket.Tag;
+import com.volcengine.tos.model.object.AbortMultipartUploadInput;
+import com.volcengine.tos.model.object.AppendObjectOutput;
+import com.volcengine.tos.model.object.CompleteMultipartUploadV2Input;
+import com.volcengine.tos.model.object.CopyObjectV2Input;
+import com.volcengine.tos.model.object.CreateMultipartUploadInput;
+import com.volcengine.tos.model.object.CreateMultipartUploadOutput;
+import com.volcengine.tos.model.object.DeleteError;
+import com.volcengine.tos.model.object.DeleteMultiObjectsV2Input;
+import com.volcengine.tos.model.object.DeleteMultiObjectsV2Output;
+import com.volcengine.tos.model.object.DeleteObjectInput;
+import com.volcengine.tos.model.object.DeleteObjectTaggingInput;
+import com.volcengine.tos.model.object.GetFileStatusInput;
+import com.volcengine.tos.model.object.GetFileStatusOutput;
+import com.volcengine.tos.model.object.GetObjectBasicOutput;
+import com.volcengine.tos.model.object.GetObjectTaggingInput;
+import com.volcengine.tos.model.object.GetObjectTaggingOutput;
+import com.volcengine.tos.model.object.GetObjectV2Input;
+import com.volcengine.tos.model.object.GetObjectV2Output;
+import com.volcengine.tos.model.object.HeadObjectV2Input;
+import com.volcengine.tos.model.object.HeadObjectV2Output;
+import com.volcengine.tos.model.object.ListMultipartUploadsV2Input;
+import com.volcengine.tos.model.object.ListMultipartUploadsV2Output;
+import com.volcengine.tos.model.object.ListObjectsType2Input;
+import com.volcengine.tos.model.object.ListObjectsType2Output;
+import com.volcengine.tos.model.object.ListedCommonPrefix;
+import com.volcengine.tos.model.object.ListedObjectV2;
+import com.volcengine.tos.model.object.ListedUpload;
+import com.volcengine.tos.model.object.ObjectMetaRequestOptions;
+import com.volcengine.tos.model.object.ObjectTobeDeleted;
+import com.volcengine.tos.model.object.PutObjectOutput;
+import com.volcengine.tos.model.object.PutObjectTaggingInput;
+import com.volcengine.tos.model.object.RenameObjectInput;
+import com.volcengine.tos.model.object.TagSet;
+import com.volcengine.tos.model.object.UploadPartCopyV2Input;
+import com.volcengine.tos.model.object.UploadPartCopyV2Output;
+import com.volcengine.tos.model.object.UploadedPartV2;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.tosfs.conf.ConfKeys;
+import org.apache.hadoop.fs.tosfs.conf.TosKeys;
+import org.apache.hadoop.fs.tosfs.object.BucketInfo;
+import org.apache.hadoop.fs.tosfs.object.ChecksumInfo;
+import org.apache.hadoop.fs.tosfs.object.ChecksumType;
+import org.apache.hadoop.fs.tosfs.object.Constants;
+import org.apache.hadoop.fs.tosfs.object.DirectoryStorage;
+import org.apache.hadoop.fs.tosfs.object.InputStreamProvider;
+import org.apache.hadoop.fs.tosfs.object.MultipartUpload;
+import org.apache.hadoop.fs.tosfs.object.ObjectConstants;
+import org.apache.hadoop.fs.tosfs.object.ObjectContent;
+import org.apache.hadoop.fs.tosfs.object.ObjectInfo;
+import org.apache.hadoop.fs.tosfs.object.ObjectStorage;
+import org.apache.hadoop.fs.tosfs.object.ObjectUtils;
+import org.apache.hadoop.fs.tosfs.object.Part;
+import org.apache.hadoop.fs.tosfs.object.exceptions.InvalidObjectKeyException;
+import org.apache.hadoop.fs.tosfs.object.exceptions.NotAppendableException;
+import org.apache.hadoop.fs.tosfs.object.request.ListObjectsRequest;
+import org.apache.hadoop.fs.tosfs.object.response.ListObjectsResponse;
+import org.apache.hadoop.fs.tosfs.util.LazyReload;
+import org.apache.hadoop.thirdparty.com.google.common.base.Joiner;
+import org.apache.hadoop.thirdparty.com.google.common.base.Strings;
+import org.apache.hadoop.util.Lists;
+import org.apache.hadoop.util.Preconditions;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.fs.tosfs.object.tos.TOSErrorCodes.APPEND_NOT_APPENDABLE;
+import static org.apache.hadoop.fs.tosfs.object.tos.TOSUtils.CHECKSUM_HEADER;
+import static org.apache.hadoop.fs.tosfs.object.tos.TOSUtils.appendable;
+import static org.apache.hadoop.fs.tosfs.object.tos.TOSUtils.crc64ecma;
+import static org.apache.hadoop.fs.tosfs.object.tos.TOSUtils.parseChecksum;
+
+/**
+ * {@link TOS} will be initialized by the {@link ObjectStorage#initialize(Configuration, String)}.
+ */
+public class TOS implements DirectoryStorage {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TOS.class);
+  public static final String TOS_SCHEME = "tos";
+
+  public static final String ENV_TOS_ACCESS_KEY_ID = "TOS_ACCESS_KEY_ID";
+  public static final String ENV_TOS_SECRET_ACCESS_KEY = "TOS_SECRET_ACCESS_KEY";
+  public static final String ENV_TOS_SESSION_TOKEN = "TOS_SESSION_TOKEN";
+  public static final String ENV_TOS_ENDPOINT = "TOS_ENDPOINT";
+
+  private static final int NOT_FOUND_CODE = 404;
+  private static final int PATH_CONFLICT_CODE = 409;
+  private static final int INVALID_RANGE_CODE = 416;
+
+  private static final int MIN_PART_SIZE = 5 * 1024 * 1024;
+  private static final int MAX_PART_COUNT = 10000;
+
+  private static final InputStream EMPTY_STREAM = new ByteArrayInputStream(new byte[0]);
+
+  private Configuration conf;
+  private String bucket;
+  private DelegationClient client;
+  private long maxDrainBytes;
+  private int batchDeleteMaxRetries;
+  private List batchDeleteRetryCodes;
+  private long batchDeleteRetryInterval;
+  private int maxDeleteObjectsCount;
+  private int listObjectsCount;
+  // the max retry times during reading object content
+  private int maxInputStreamRetries;
+  private ACLType defaultAcl;
+  private ChecksumInfo checksumInfo;
+  private BucketInfo bucketInfo;
+
+  static {
+    org.apache.log4j.Logger logger = LogManager.getLogger("com.volcengine.tos");
+    String logLevel = System.getProperty("tos.log.level", "WARN");
+
+    LOG.debug("Reset the log level of com.volcengine.tos with {} ", logLevel);
+    logger.setLevel(Level.toLevel(logLevel.toUpperCase(), Level.WARN));
+  }
+
+  @Override
+  public void initialize(Configuration config, String bucketName) {
+    this.conf = config;
+    this.bucket = bucketName;
+    client = new DelegationClientBuilder().conf(config).bucket(bucketName).build();
+    maxDrainBytes =
+        config.getLong(TosKeys.FS_TOS_MAX_DRAIN_BYTES, TosKeys.FS_TOS_MAX_DRAIN_BYTES_DEFAULT);
+    batchDeleteMaxRetries = config.getInt(TosKeys.FS_TOS_BATCH_DELETE_MAX_RETRIES,
+        TosKeys.FS_TOS_BATCH_DELETE_MAX_RETRIES_DEFAULT);
+    batchDeleteRetryCodes = Arrays.asList(
+        config.getTrimmedStrings(TosKeys.FS_TOS_BATCH_DELETE_RETRY_CODES,
+            TosKeys.FS_TOS_BATCH_DELETE_RETRY_CODES_DEFAULT));
+    batchDeleteRetryInterval = config.getLong(TosKeys.FS_TOS_BATCH_DELETE_RETRY_INTERVAL,
+        TosKeys.FS_TOS_BATCH_DELETE_RETRY_INTERVAL_DEFAULT);
+    maxDeleteObjectsCount = config.getInt(TosKeys.FS_TOS_DELETE_OBJECTS_COUNT,
+        TosKeys.FS_TOS_DELETE_OBJECTS_COUNT_DEFAULT);
+    listObjectsCount =
+        config.getInt(TosKeys.FS_TOS_LIST_OBJECTS_COUNT, TosKeys.FS_TOS_LIST_OBJECTS_COUNT_DEFAULT);
+    maxInputStreamRetries = config.getInt(TosKeys.FS_TOS_MAX_READ_OBJECT_RETRIES,
+        TosKeys.FS_TOS_MAX_READ_OBJECT_RETRIES_DEFAULT);
+    defaultAcl = TypeConverter.convertACLType(config.get(TosKeys.FS_TOS_ACL_DEFAULT));
+
+    String algorithm =
+        config.get(TosKeys.FS_TOS_CHECKSUM_ALGORITHM, TosKeys.FS_TOS_CHECKSUM_ALGORITHM_DEFAULT);
+    ChecksumType checksumType = ChecksumType.valueOf(
+        config.get(TosKeys.FS_TOS_CHECKSUM_TYPE, TosKeys.FS_TOS_CHECKSUM_TYPE_DEFAULT)
+            .toUpperCase());
+    Preconditions.checkArgument(CHECKSUM_HEADER.containsKey(checksumType),
+        "Checksum type %s is not supported by TOS.", checksumType.name());
+    checksumInfo = new ChecksumInfo(algorithm, checksumType);
+
+    bucketInfo = getBucketInfo(bucketName);
+  }
+
+  @Override
+  public String scheme() {
+    return TOS_SCHEME;
+  }
+
+  @Override
+  public Configuration conf() {
+    return conf;
+  }
+
+  @Override
+  public BucketInfo bucket() {
+    return bucketInfo;
+  }
+
+  private BucketInfo getBucketInfo(String bucketName) {
+    try {
+      HeadBucketV2Output res =
+          client.headBucket(HeadBucketV2Input.builder().bucket(bucketName).build());
+
+      // BUCKET_TYPE_FNS is the general purpose bucket, BUCKET_TYPE_HNS is directory bucket.
+      boolean directoryBucket = BucketType.BUCKET_TYPE_HNS.equals(res.getBucketType());
+
+      return new BucketInfo(bucketName, directoryBucket);
+    } catch (TosException e) {
+      if (e.getStatusCode() == NOT_FOUND_CODE) {
+        return null;
+      }
+      throw new RuntimeException(e);
+    }
+  }
+
+  @VisibleForTesting
+  void setClient(DelegationClient client) {
+    this.client = client;
+  }
+
+  private void checkAvailableClient() {
+    Preconditions.checkState(client != null,
+        "Encountered uninitialized ObjectStorage, call initialize(..) please.");
+  }
+
+  @Override
+  public ObjectContent get(String key, long offset, long limit) {
+    checkAvailableClient();
+    Preconditions.checkArgument(offset >= 0, "offset is a negative number: %s", offset);
+
+    if (limit == 0) {
+      // Can not return empty stream when limit = 0, because the requested object might not exist.
+      if (head(key) != null) {
+        return new ObjectContent(Constants.MAGIC_CHECKSUM, EMPTY_STREAM);
+      } else {
+        throw new RuntimeException(String.format("Object %s doesn't exit", key));
+      }
+    }
+
+    long end = limit < 0 ? -1 : offset + limit - 1;
+    GetObjectFactory factory = (k, startOff, endOff) -> getObject(key, startOff, endOff);
+    ChainTOSInputStream chainStream =
+        new ChainTOSInputStream(factory, key, offset, end, maxDrainBytes, maxInputStreamRetries);
+    return new ObjectContent(chainStream.checksum(), chainStream);
+  }
+
+  @Override
+  public Iterable