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