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
deleted file mode 100644
index 98e654f99b5e0..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/InputStreamProvider.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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
deleted file mode 100644
index e7c89f1f8279b..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/MultipartUpload.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * 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
deleted file mode 100644
index fb42b0091bc63..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectConstants.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 6961fd83bb7bb..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectContent.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 08f38ecbdd961..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectInfo.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 4a9357e46bf75..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectMultiRangeInputStream.java
+++ /dev/null
@@ -1,233 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 8e7b9ebe9f020..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectOutputStream.java
+++ /dev/null
@@ -1,343 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 8e78dbb03e485..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectRangeInputStream.java
+++ /dev/null
@@ -1,199 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 07187bcd237fc..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorage.java
+++ /dev/null
@@ -1,372 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 09c8a72ab743f..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectStorageFactory.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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
deleted file mode 100644
index a0b00f366d963..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/ObjectUtils.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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
deleted file mode 100644
index ace7acaf28a5d..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/Part.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 18185bd1efd14..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/PrefixStorage.java
+++ /dev/null
@@ -1,254 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 309eebe0fcad6..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/ChecksumMismatchException.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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
deleted file mode 100644
index f9eaf021e28e0..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/InvalidObjectKeyException.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 2a7def1176390..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/NotAppendableException.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * 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
deleted file mode 100644
index c87e1a7eaa031..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/exceptions/package-info.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 7819ecd6c8af0..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/package-info.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 170426acb07c2..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/ListObjectsRequest.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 6e7e52f403950..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/request/package-info.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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
deleted file mode 100644
index eb3adf8522ae8..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/ListObjectsResponse.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 6678617057243..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/response/package-info.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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
deleted file mode 100644
index fb39e949febf6..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/FileStagingPart.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/*
- * 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
deleted file mode 100644
index b4fa812397285..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/StagingPart.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 418baa6d9b13a..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/State.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * 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
deleted file mode 100644
index c9c2a9033c40e..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/staging/package-info.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 7790e7c54a33d..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/ChainTOSInputStream.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 358cb99df104b..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClient.java
+++ /dev/null
@@ -1,1248 +0,0 @@
-/*
- * 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.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 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 config) {
- retry(() -> {
- client.changeTransportConfig(config);
- 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
deleted file mode 100644
index a3d779e7ad3ec..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/DelegationClientBuilder.java
+++ /dev/null
@@ -1,185 +0,0 @@
-/*
- * 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 conf, String endpoint, String region,
- String bucket, boolean cached) {
- CredentialsProvider provider = createProvider(conf, bucket);
- TOSClientConfiguration clientConfiguration = TOSClientConfiguration.builder()
- .region(region)
- .endpoint(endpoint)
- .credentials(provider)
- .enableCrc(conf.getBoolean(
- TosKeys.FS_TOS_CRC_CHECK_ENABLED, TosKeys.FS_TOS_CRC_CHECK_ENABLED_DEFAULT))
- .transportConfig(createTransportConfig(conf))
- .userAgentProductName(conf.get(
- TosKeys.FS_TOS_USER_AGENT_PREFIX, TosKeys.FS_TOS_USER_AGENT_PREFIX_DEFAULT))
- .userAgentSoftName(Constants.TOS_FS)
- .userAgentSoftVersion(VersionInfo.getVersion())
- .build();
-
- int maxRetryTimes = conf.getInt(TosKeys.FS_TOS_REQUEST_MAX_RETRY_TIMES,
- TosKeys.FS_TOS_REQUEST_MAX_RETRY_TIMES_DEFAULT);
- List nonRetryable409ErrorCodes = Arrays.asList(
- conf.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 conf, String bucket) {
- try {
- CredentialsProvider provider = (CredentialsProvider) Class.forName(
- conf.get(TosKeys.FS_TOS_CREDENTIALS_PROVIDER,
- TosKeys.FS_TOS_CREDENTIALS_PROVIDER_DEFAULT))
- .getDeclaredConstructor()
- .newInstance();
- provider.initialize(conf, bucket);
- return provider;
- } catch (ClassNotFoundException |
- InstantiationException |
- IllegalAccessException |
- InvocationTargetException |
- NoSuchMethodException e) {
- throw new TosException(e);
- }
- }
-
- private String getAndCheckEndpoint(Configuration conf) {
- String endpoint = conf.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 conf, String endpoint) {
- String region = conf.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 conf) {
- 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(conf.getInt(TosKeys.FS_TOS_HTTP_MAX_CONNECTIONS,
- TosKeys.FS_TOS_HTTP_MAX_CONNECTIONS_DEFAULT));
- builder.idleConnectionTimeMills(conf.getInt(TosKeys.FS_TOS_HTTP_IDLE_CONNECTION_TIME_MILLS,
- TosKeys.FS_TOS_HTTP_IDLE_CONNECTION_TIME_MILLS_DEFAULT));
- builder.connectTimeoutMills(conf.getInt(TosKeys.FS_TOS_HTTP_CONNECT_TIMEOUT_MILLS,
- TosKeys.FS_TOS_HTTP_CONNECT_TIMEOUT_MILLS_DEFAULT));
- builder.readTimeoutMills(conf.getInt(TosKeys.FS_TOS_HTTP_READ_TIMEOUT_MILLS,
- TosKeys.FS_TOS_HTTP_READ_TIMEOUT_MILLS_DEFAULT));
- builder.writeTimeoutMills(conf.getInt(TosKeys.FS_TOS_HTTP_WRITE_TIMEOUT_MILLS,
- TosKeys.FS_TOS_HTTP_WRITE_TIMEOUT_MILLS_DEFAULT));
- builder.enableVerifySSL(conf.getBoolean(TosKeys.FS_TOS_HTTP_ENABLE_VERIFY_SSL,
- TosKeys.FS_TOS_HTTP_ENABLE_VERIFY_SSL_DEFAULT));
- builder.dnsCacheTimeMinutes(conf.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
deleted file mode 100644
index 718cc7e39ef24..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/GetObjectOutput.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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
deleted file mode 100644
index 5f7c674352303..0000000000000
--- a/hadoop-cloud-storage-project/hadoop-tos/src/main/java/org/apache/hadoop/fs/tosfs/object/tos/TOS.java
+++ /dev/null
@@ -1,1037 +0,0 @@
-/*
- * 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