flags,
+ int bufferSize, short replication, long blockSize,
+ Progressable progress) throws IOException {
+ return createNonRecursive(f, permission,
+ flags.contains(CreateFlag.OVERWRITE),
+ bufferSize, replication, blockSize, progress);
+ }
+
+ @Override
+ @Deprecated
+ public boolean delete(Path path) throws IOException {
+ return delete(path, true);
+ }
+
+ @Override
+ public boolean delete(Path f, boolean recursive)
+ throws IOException {
+ Path absolutePath = makeAbsolute(f);
+ String key = pathToKey(absolutePath);
+
+ if (store.isHierarchy() && recursive) {
+ try {
+ getFileStatus(f);
+ } catch (FileNotFoundException e) {
+ return false;
+ }
+ store.deleteDirs(key, recursive);
+ } else {
+ FileStatus status;
+ try {
+ status = getFileStatus(f);
+ } catch (FileNotFoundException e) {
+ return false;
+ }
+
+ if (status.isDirectory()) {
+ store.deleteDirs(key, recursive);
+ } else {
+ store.delete(key);
+ }
+ }
+
+ if (!store.isHierarchy()) {
+ createParent(f);
+ }
+ return true;
+ }
+
+ @Override
+ public FileStatus getFileStatus(Path f) throws IOException {
+ Path absolutePath = makeAbsolute(f);
+ String key = pathToKey(absolutePath);
+ if (key.isEmpty()) {
+ return newDirectory(null, absolutePath);
+ }
+
+ try {
+ FileMetadata meta = store.retrieveMetadata(key);
+ if (meta != null) {
+ if (meta.isFolder()) {
+ return meta.getLastModified() == 0
+ ? newDirectory(null, absolutePath)
+ : newDirectory(meta, absolutePath);
+ } else {
+ return newFile(meta, absolutePath);
+ }
+ }
+ } catch (FileNotFoundException e) {
+ throw new FileNotFoundException(
+ absolutePath + ": No such file or directory.");
+ } catch (IOException e) {
+ LOG.error("bos-fs getFileStatus error: ", e);
+ throw e;
+ }
+ throw new FileNotFoundException(
+ absolutePath + ": No such file or directory.");
+ }
+
+ @Override
+ public FileChecksum getFileChecksum(Path f, long length)
+ throws IOException {
+ LOG.debug("call the checksum for the path: {}",
+ f.getName());
+ Path absolutePath = makeAbsolute(f);
+ String key = pathToKey(absolutePath);
+ return this.store.getFileChecksum(key);
+ }
+
+ @Override
+ public boolean isDirectory(Path f) throws IOException {
+ String key = pathToKey(makeAbsolute(f));
+ try {
+ return store.isDirectory(key);
+ } catch (FileNotFoundException e) {
+ return false;
+ }
+ }
+
+ @Override
+ public URI getUri() {
+ return uri;
+ }
+
+ /**
+ * List the statuses of the files/directories in the given path.
+ *
+ * If {@code f} is a file, this method will make a single call
+ * to BOS. If {@code f} is a directory, this method will make a
+ * maximum of (n / 1000) + 2 calls to BOS, where n
+ * is the total number of files and directories contained directly
+ * in {@code f}.
+ *
+ * @param f the path
+ * @return file status array
+ * @throws IOException on IO failure
+ */
+ @Override
+ public FileStatus[] listStatus(Path f) throws IOException {
+ Path absolutePath = makeAbsolute(f);
+ String key = pathToKey(absolutePath);
+
+ URI pathUri = absolutePath.toUri();
+ Set status = new TreeSet<>();
+ String priorLastKey = null;
+
+ do {
+ PartialListing listing = store.list(key,
+ BaiduBosConstants.BOS_MAX_LISTING_LENGTH,
+ priorLastKey, PATH_DELIMITER);
+ if (listing != null) {
+ for (FileMetadata fileMetadata : listing.getFiles()) {
+ if (fileMetadata.getKey().endsWith("/")) {
+ continue;
+ }
+ Path subpath = keyToPath(fileMetadata.getKey());
+ String relativePath =
+ pathUri.relativize(subpath.toUri()).getPath();
+ status.add(getBosFileStatus(fileMetadata,
+ new Path(absolutePath, relativePath)));
+ }
+ for (FileMetadata dirMetadata
+ : listing.getDirectories()) {
+ Path subpath = keyToPath(dirMetadata.getKey());
+ String relativePath =
+ pathUri.relativize(subpath.toUri()).getPath();
+ Path dirPath = new Path(absolutePath, relativePath);
+ status.add(getBosDirStatus(dirMetadata, dirPath));
+ }
+ priorLastKey = listing.getPriorLastKey();
+ }
+ } while (priorLastKey != null && !priorLastKey.isEmpty());
+
+ if (status.isEmpty()) {
+ FileStatus stat = getFileStatus(f);
+ if (stat.isFile()) {
+ status.add(stat);
+ }
+ }
+ return status.toArray(new FileStatus[0]);
+ }
+
+ /**
+ * Get BOS file status. Only used in listStatus.
+ */
+ private FileStatus getBosFileStatus(FileMetadata fileMetadata,
+ Path path) {
+ boolean needExtHead = !this.store.isHierarchy();
+ try {
+ if (needExtHead
+ && forceGetNonHierarchyMetadataInListStatus) {
+ FileMetadata fMeta =
+ store.retrieveMetadata(fileMetadata.getKey());
+ if (fMeta != null) {
+ return newFile(fMeta, path);
+ }
+ }
+ } catch (IOException e) {
+ // throw nothing
+ }
+ return newFile(fileMetadata, path);
+ }
+
+ private FileStatus getBosDirStatus(FileMetadata dirMetadata,
+ Path path) {
+ boolean needExtHead = !this.store.isHierarchy();
+ try {
+ if (needExtHead) {
+ FileMetadata dirMeta =
+ store.retrieveMetadata(
+ prefixToDir(dirMetadata.getKey()));
+ if (dirMeta != null) {
+ return newDirectory(dirMeta, path);
+ }
+ }
+ } catch (IOException e) {
+ // throw nothing
+ }
+ return newDirectory(dirMetadata, path);
+ }
+
+ /**
+ * Add a trailing slash to a prefix if not already present.
+ *
+ * @param prefix the prefix to convert
+ * @return the prefix ending with "/"
+ */
+ public static String prefixToDir(String prefix) {
+ if (prefix != null && !prefix.isEmpty()
+ && !prefix.endsWith(PATH_DELIMITER)) {
+ prefix += PATH_DELIMITER;
+ }
+ return prefix;
+ }
+
+ /**
+ * Non-recursive getContentSummary.
+ *
+ * @param f path to use
+ * @return the content summary
+ * @throws IOException on IO failure
+ */
+ @Override
+ public ContentSummary getContentSummary(Path f)
+ throws IOException {
+ Path absolutePath = makeAbsolute(f);
+ String key = pathToKey(absolutePath);
+ FileStatus status = this.getFileStatus(f);
+ if (status.isFile()) {
+ return new ContentSummary.Builder()
+ .length(status.getLen()).fileCount(1L)
+ .directoryCount(0L).build();
+ }
+ if (store.isHierarchy()) {
+ return multiThreadGetContentSummary(f);
+ }
+ long[] summary = new long[]{0L, 0L, 0L};
+ String priorLastKey = null;
+ do {
+ PartialListing listing = store.list(
+ store.prefixToDir(key),
+ BaiduBosConstants.BOS_MAX_LISTING_LENGTH,
+ priorLastKey, null);
+ if (listing != null) {
+ for (FileMetadata fileMetadata : listing.getFiles()) {
+ if (fileMetadata.getKey() != null
+ && !fileMetadata.getKey().isEmpty()) {
+ summary[0] += fileMetadata.getLength();
+ if (!fileMetadata.getKey().endsWith("/")) {
+ summary[1] += 1;
+ } else {
+ summary[2] += 1;
+ }
+ }
+ }
+ priorLastKey = listing.getPriorLastKey();
+ }
+ } while (priorLastKey != null && !priorLastKey.isEmpty());
+ return new ContentSummary.Builder()
+ .length(summary[0]).fileCount(summary[1])
+ .directoryCount(summary[2]).build();
+ }
+
+ /**
+ * Multi-threaded content summary for hierarchy buckets.
+ *
+ * @param root the root path
+ * @return the content summary
+ * @throws IOException on IO failure
+ */
+ public ContentSummary multiThreadGetContentSummary(Path root)
+ throws IOException {
+ ExecutorService es = store.getBoundedThreadPool();
+ Queue> futures =
+ new LinkedBlockingQueue<>();
+ AtomicBoolean exceptionThrow = new AtomicBoolean(false);
+ long[] summary = new long[]{0L, 0L, 1L};
+ FileStatus[] statuses = listStatus(root);
+ for (FileStatus status : statuses) {
+ if (status.isFile()) {
+ summary[0] += status.getLen();
+ summary[1]++;
+ } else {
+ summary[2]++;
+ processDirectory(status.getPath(), futures, es,
+ exceptionThrow);
+ }
+ }
+
+ while (!exceptionThrow.get() && !futures.isEmpty()) {
+ Future future = futures.poll();
+ try {
+ ContentSummary subSummary = future.get();
+ summary[0] += subSummary.getLength();
+ summary[1] += subSummary.getFileCount();
+ summary[2] += subSummary.getDirectoryCount();
+ } catch (InterruptedException | ExecutionException e) {
+ LOG.error(e.getMessage(), e);
+ throw new RuntimeException(e);
+ }
+ }
+ if (exceptionThrow.get()) {
+ throw new IOException(
+ "Exception while get content summary");
+ }
+ return new ContentSummary.Builder()
+ .length(summary[0]).fileCount(summary[1])
+ .directoryCount(summary[2]).build();
+ }
+
+ private void processDirectory(Path p,
+ Queue> futures,
+ ExecutorService es, AtomicBoolean exceptionThrow) {
+ futures.add(es.submit(() -> {
+ long[] summary = new long[]{0L, 0L, 0L};
+ try {
+ if (!exceptionThrow.get()) {
+ FileStatus[] statuses = listStatus(p);
+ for (FileStatus status : statuses) {
+ if (status.isFile()) {
+ summary[0] += status.getLen();
+ summary[1]++;
+ } else {
+ summary[2]++;
+ processDirectory(status.getPath(), futures,
+ es, exceptionThrow);
+ }
+ }
+ }
+ } catch (Throwable e) {
+ LOG.error(e.getMessage(), e);
+ exceptionThrow.set(true);
+ }
+ return new ContentSummary.Builder()
+ .length(summary[0]).fileCount(summary[1])
+ .directoryCount(summary[2]).build();
+ }));
+ }
+
+ // SpotBugs fix: removed dead-store variables owner/group.
+ // SpotBugs fix: meta is never null here (length is set),
+ // so the null-check was misleading.
+ private FileStatus newFile(FileMetadata meta, Path path) {
+ long blockSize = getConf().getLong(
+ BaiduBosConstants.BOS_BLOCK_SIZE,
+ BaiduBosConstants.DEFAULT_BOS_BLOCK_SIZE);
+ String owner = meta.getUserName() != null
+ ? meta.getUserName() : store.getEnvUserName();
+ String group = meta.getGroupName() != null
+ ? meta.getGroupName() : store.getEnvGroupName();
+ return new FileStatus(
+ meta.getLength(),
+ false,
+ 1,
+ blockSize,
+ meta.getLastModified(),
+ 0L,
+ null,
+ owner,
+ group,
+ null,
+ path.makeQualified(this),
+ FileStatus.NONE);
+ }
+
+ private FileStatus newDirectory(FileMetadata meta,
+ Path path) {
+ long blockSize = getConf().getLong(
+ BaiduBosConstants.BOS_BLOCK_SIZE,
+ BaiduBosConstants.DEFAULT_BOS_BLOCK_SIZE);
+ String owner = (meta != null && meta.getUserName() != null)
+ ? meta.getUserName() : store.getEnvUserName();
+ String group = (meta != null && meta.getGroupName() != null)
+ ? meta.getGroupName() : store.getEnvGroupName();
+ return new FileStatus(
+ 0,
+ true,
+ 1,
+ blockSize,
+ (meta == null ? 0 : meta.getLastModified()),
+ 0L,
+ null,
+ owner,
+ group,
+ null,
+ path.makeQualified(this),
+ FileStatus.NONE);
+ }
+
+ @Override
+ public boolean mkdirs(Path f, FsPermission permission)
+ throws IOException {
+ Path absolutePath = makeAbsolute(f);
+
+ List paths = new ArrayList<>();
+ do {
+ try {
+ FileStatus fileStatus = getFileStatus(absolutePath);
+ if (fileStatus.isFile()) {
+ throw new FileAlreadyExistsException(
+ absolutePath + " is a file");
+ } else {
+ break;
+ }
+ } catch (FileNotFoundException e) {
+ paths.add(0, absolutePath);
+ absolutePath = absolutePath.getParent();
+ }
+ } while (!pathToKey(absolutePath).isEmpty());
+
+ boolean result = true;
+ for (Path path : paths) {
+ result &= mkdir(path);
+ }
+ return result;
+ }
+
+ private boolean mkdir(Path f) throws IOException {
+ String key = pathToKey(f);
+ if (!key.endsWith(FOLDER_SUFFIX)) {
+ key = pathToKey(f) + FOLDER_SUFFIX;
+ }
+
+ store.storeEmptyFile(key, this.store.getEnvUserName(),
+ this.store.getEnvGroupName());
+ return true;
+ }
+
+ @Override
+ public FSDataInputStream open(Path f, int bufferSize)
+ throws IOException {
+ Path absolutePath = makeAbsolute(f);
+ String key = pathToKey(absolutePath);
+
+ FileMetadata fileMetaData = null;
+ try {
+ fileMetaData = store.retrieveMetadata(key);
+ } catch (FileNotFoundException ignore) {
+ throw new FileNotFoundException(f.toString());
+ }
+
+ if (fileMetaData.isFolder()) {
+ throw new FileNotFoundException("Can not open a folder");
+ }
+
+ BosInputStream bosFsInputStream = new BosInputStream(
+ key, fileMetaData, this.store, this.statistics);
+
+ bosFsInputStream.setReadahead(this.readAhead);
+ return new FSDataInputStream(
+ new BufferedFSInputStream(
+ bosFsInputStream, this.readBufferSize));
+ }
+
+ private void createParent(Path path) throws IOException {
+ Path parent = path.getParent();
+ if (parent != null) {
+ String key = pathToKey(makeAbsolute(parent));
+ if (key.length() > 0) {
+ if (!store.isDirectory(key)) {
+ LOG.warn("create parent when rename or delete, "
+ + "common sense the code won't reach here");
+ store.storeEmptyFile(store.prefixToDir(key));
+ }
+ }
+ }
+ }
+
+ @Override
+ public boolean rename(Path srcPath, Path dstPath)
+ throws IOException {
+ srcPath = makeAbsolute(srcPath);
+ dstPath = makeAbsolute(dstPath);
+
+ if (srcPath.equals(dstPath)) {
+ return true;
+ }
+
+ FileStatus srcStatus;
+ try {
+ srcStatus = getFileStatus(srcPath);
+ } catch (FileNotFoundException e) {
+ return false;
+ }
+
+ if (store.isHierarchy()) {
+ store.rename(pathToKey(srcPath),
+ pathToKey(dstPath), true);
+ return true;
+ }
+
+ Path parent = dstPath.getParent();
+ while (parent != null && !srcPath.equals(parent)) {
+ parent = parent.getParent();
+ }
+ if (parent != null) {
+ return false;
+ }
+
+ FileStatus dstStatus;
+ try {
+ dstStatus = getFileStatus(dstPath);
+ } catch (FileNotFoundException fnde) {
+ dstStatus = null;
+ }
+
+ if (dstStatus == null) {
+ Path dstParent = dstPath.getParent();
+ if (dstParent != null) {
+ Path currentPath = dstParent;
+ while (currentPath != null
+ && !currentPath.isRoot()) {
+ FileStatus ancestorStatus;
+ try {
+ ancestorStatus = getFileStatus(currentPath);
+ if (!ancestorStatus.isDirectory()) {
+ throw new IOException(String.format(
+ "Failed to rename %s to %s, %s is a file",
+ srcPath, dstPath, currentPath));
+ }
+ break;
+ } catch (FileNotFoundException fnde) {
+ currentPath = currentPath.getParent();
+ }
+ }
+ }
+ LOG.debug("Parent directory {} does not exist "
+ + "or will be implicitly created during rename",
+ dstPath.getParent());
+ } else {
+ if (dstStatus.isDirectory()) {
+ dstPath = new Path(dstPath, srcPath.getName());
+ FileStatus status;
+ try {
+ status = getFileStatus(dstPath);
+ } catch (FileNotFoundException fnde) {
+ status = null;
+ }
+ if (status != null) {
+ if (status.isFile()) {
+ LOG.debug("Deleting existing destination "
+ + "file {} before rename", dstPath);
+ delete(dstPath, false);
+ } else {
+ throw new FileAlreadyExistsException(
+ String.format(
+ "Failed to rename %s to %s, "
+ + "file already exists or not empty!",
+ srcPath, dstPath));
+ }
+ }
+ } else {
+ LOG.debug("Deleting existing destination "
+ + "file {} before rename", dstPath);
+ delete(dstPath, false);
+ }
+ }
+
+ store.rename(pathToKey(srcPath), pathToKey(dstPath),
+ srcStatus.isFile());
+ return true;
+ }
+
+ @Override
+ public Path getWorkingDirectory() {
+ return workingDir;
+ }
+
+ @Override
+ public boolean truncate(Path f, long newLength)
+ throws IOException {
+ throw new UnsupportedOperationException(
+ "Truncate is not supported by BaiduBosFileSystem");
+ }
+
+ /**
+ * Set the working directory to the given directory.
+ *
+ * @param newDir the new working directory
+ */
+ @Override
+ public void setWorkingDirectory(Path newDir) {
+ workingDir = newDir;
+ }
+
+ @Override
+ public String getCanonicalServiceName() {
+ return null;
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BaiduBosFileSystemAdapter.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BaiduBosFileSystemAdapter.java
new file mode 100644
index 0000000000000..c4df7cfa31d2f
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BaiduBosFileSystemAdapter.java
@@ -0,0 +1,76 @@
+/*
+ * 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.bos;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegateToFileSystem;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+/**
+ * Adapter that bridges {@link BaiduBosFileSystem} to the
+ * Hadoop {@link DelegateToFileSystem} abstraction for the
+ * "bos" URI scheme.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class BaiduBosFileSystemAdapter
+ extends DelegateToFileSystem {
+
+ /**
+ * Constructs a BaiduBosFileSystemAdapter.
+ *
+ * @param theUri the URI for this filesystem
+ * @param conf the Hadoop configuration
+ * @throws IOException if an I/O error occurs
+ * @throws URISyntaxException if the URI is malformed
+ */
+ public BaiduBosFileSystemAdapter(
+ URI theUri, Configuration conf)
+ throws IOException, URISyntaxException {
+ super(theUri, new BaiduBosFileSystem(),
+ conf, "bos", false);
+ }
+
+ /**
+ * Returns the default port for the URI scheme.
+ *
+ * @return the default port
+ */
+ @Override
+ public int getUriDefaultPort() {
+ return super.getUriDefaultPort();
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public String toString() {
+ final StringBuilder sb =
+ new StringBuilder("bos:{");
+ sb.append("URI =").append(fsImpl.getUri());
+ sb.append("; fsImpl=").append(fsImpl);
+ sb.append('}');
+ return sb.toString();
+ }
+
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosBlockBuffer.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosBlockBuffer.java
new file mode 100644
index 0000000000000..adaafeccea571
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosBlockBuffer.java
@@ -0,0 +1,129 @@
+/*
+ * 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.bos;
+
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A buffer that holds data for a single block during
+ * multipart upload. Data is written to an output buffer and
+ * then moved to an input buffer for uploading.
+ */
+public class BosBlockBuffer implements Closeable {
+
+ /** The output buffer for writing data. */
+ private DataOutputBuffer outBuffer;
+
+ /** The input buffer for reading data during upload. */
+ private DataInputBuffer inBuffer = new DataInputBuffer();
+
+ private String key;
+ private int blkId;
+
+ /**
+ * Constructs a BosBlockBuffer.
+ *
+ * @param key the object key
+ * @param blkId the block identifier
+ * @param size the initial capacity of the output buffer
+ */
+ public BosBlockBuffer(
+ String key, int blkId, int size) {
+ this.key = key;
+ this.blkId = blkId;
+ outBuffer = new DataOutputBuffer(size);
+ }
+
+ /**
+ * Returns the object key.
+ *
+ * @return the key
+ */
+ public String getKey() {
+ return key;
+ }
+
+ /**
+ * Returns the block identifier.
+ *
+ * @return the block ID
+ */
+ public int getBlkId() {
+ return blkId;
+ }
+
+ /**
+ * Sets the block identifier.
+ *
+ * @param blkId the new block ID
+ */
+ public void setBlkId(int blkId) {
+ this.blkId = blkId;
+ }
+
+ /**
+ * Returns the output buffer for writing data.
+ *
+ * @return the output buffer
+ */
+ DataOutputBuffer getOutBuffer() {
+ return outBuffer;
+ }
+
+ /**
+ * Returns the input buffer for reading data.
+ *
+ * @return the input buffer
+ */
+ DataInputBuffer getInBuffer() {
+ return inBuffer;
+ }
+
+ /**
+ * Moves data from the output buffer to the input buffer
+ * and resets the output buffer for reuse.
+ */
+ void moveData() {
+ inBuffer.reset(
+ outBuffer.getData(), outBuffer.getLength());
+ outBuffer.reset();
+ }
+
+ /**
+ * Clears the input buffer.
+ */
+ void clear() {
+ inBuffer.reset(null, 0);
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public void close() throws IOException {
+ if (outBuffer != null) {
+ outBuffer.close();
+ }
+ if (inBuffer != null) {
+ inBuffer.close();
+ }
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosClientProxy.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosClientProxy.java
new file mode 100644
index 0000000000000..f4f064e5344d9
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosClientProxy.java
@@ -0,0 +1,262 @@
+/*
+ * 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.bos;
+
+import com.baidubce.services.bos.model.BosObject;
+import com.baidubce.services.bos.model.CompleteMultipartUploadRequest;
+import com.baidubce.services.bos.model.CompleteMultipartUploadResponse;
+import com.baidubce.services.bos.model.CopyObjectRequest;
+import com.baidubce.services.bos.model.CopyObjectResponse;
+import com.baidubce.services.bos.model.DeleteDirectoryResponse;
+import com.baidubce.services.bos.model.DeleteMultipleObjectsRequest;
+import com.baidubce.services.bos.model.GetObjectRequest;
+import com.baidubce.services.bos.model.InitiateMultipartUploadRequest;
+import com.baidubce.services.bos.model.InitiateMultipartUploadResponse;
+import com.baidubce.services.bos.model.ListObjectsRequest;
+import com.baidubce.services.bos.model.ListObjectsResponse;
+import com.baidubce.services.bos.model.ObjectMetadata;
+import com.baidubce.services.bos.model.PutObjectResponse;
+import com.baidubce.services.bos.model.RenameObjectResponse;
+import com.baidubce.services.bos.model.UploadPartCopyRequest;
+import com.baidubce.services.bos.model.UploadPartCopyResponse;
+import com.baidubce.services.bos.model.UploadPartRequest;
+import com.baidubce.services.bos.model.UploadPartResponse;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+
+/**
+ * Interface defining the client operations for interacting
+ * with the Baidu Object Storage (BOS) service.
+ */
+public interface BosClientProxy {
+
+ /**
+ * Initializes the BOS client with the given URI and
+ * configuration.
+ *
+ * @param uri the URI of the BOS filesystem
+ * @param conf the Hadoop configuration
+ */
+ void init(URI uri, Configuration conf);
+
+ /**
+ * Uploads an object to BOS.
+ *
+ * @param bucketName the name of the bucket
+ * @param key the object key
+ * @param input the input stream containing object data
+ * @param metadata the object metadata
+ * @return the response from the put operation
+ * @throws IOException if an I/O error occurs
+ */
+ PutObjectResponse putObject(
+ String bucketName, String key,
+ InputStream input, ObjectMetadata metadata)
+ throws IOException;
+
+ /**
+ * Uploads a zero-length object to BOS.
+ *
+ * @param bucketName the name of the bucket
+ * @param key the object key
+ * @param metadata the object metadata
+ * @throws IOException if an I/O error occurs
+ */
+ void putEmptyObject(
+ String bucketName, String key,
+ ObjectMetadata metadata) throws IOException;
+
+ /**
+ * Retrieves the metadata for an object in BOS.
+ *
+ * @param bucketName the name of the bucket
+ * @param key the object key
+ * @return the object metadata
+ * @throws IOException if an I/O error occurs
+ */
+ ObjectMetadata getObjectMetadata(
+ String bucketName, String key) throws IOException;
+
+ /**
+ * Retrieves an object from BOS.
+ *
+ * @param request the get object request
+ * @return the BOS object
+ * @throws IOException if an I/O error occurs
+ */
+ BosObject getObject(GetObjectRequest request)
+ throws IOException;
+
+ /**
+ * Lists objects in a BOS bucket.
+ *
+ * @param request the list objects request
+ * @return the list objects response
+ * @throws IOException if an I/O error occurs
+ */
+ ListObjectsResponse listObjects(ListObjectsRequest request)
+ throws IOException;
+
+ /**
+ * Uploads a part in a multipart upload.
+ *
+ * @param request the upload part request
+ * @return the upload part response
+ * @throws IOException if an I/O error occurs
+ */
+ UploadPartResponse uploadPart(UploadPartRequest request)
+ throws IOException;
+
+ /**
+ * Deletes an object from BOS.
+ *
+ * @param bucketName the name of the bucket
+ * @param key the object key
+ * @throws IOException if an I/O error occurs
+ */
+ void deleteObject(String bucketName, String key)
+ throws IOException;
+
+ /**
+ * Deletes a directory from a hierarchy-enabled BOS bucket.
+ *
+ * @param bucketName the name of the bucket
+ * @param key the directory key
+ * @param isDeleteRecursive whether to delete recursively
+ * @param marker the marker for pagination
+ * @return the delete directory response
+ * @throws IOException if an I/O error occurs
+ */
+ DeleteDirectoryResponse deleteDirectory(
+ String bucketName, String key,
+ boolean isDeleteRecursive, String marker)
+ throws IOException;
+
+ /**
+ * Deletes multiple objects from BOS in a single request.
+ *
+ * @param request the delete multiple objects request
+ * @throws IOException if an I/O error occurs
+ */
+ void deleteMultipleObjects(
+ DeleteMultipleObjectsRequest request)
+ throws IOException;
+
+ /**
+ * Copies an object within BOS using bucket names and keys.
+ *
+ * @param sourceBucketName the source bucket name
+ * @param sourceKey the source object key
+ * @param destinationBucketName the destination bucket name
+ * @param destinationKey the destination object key
+ * @return the copy object response
+ * @throws IOException if an I/O error occurs
+ */
+ CopyObjectResponse copyObject(
+ String sourceBucketName, String sourceKey,
+ String destinationBucketName,
+ String destinationKey) throws IOException;
+
+ /**
+ * Copies an object within BOS using a copy request.
+ *
+ * @param request the copy object request
+ * @return the copy object response
+ * @throws IOException if an I/O error occurs
+ */
+ CopyObjectResponse copyObject(CopyObjectRequest request)
+ throws IOException;
+
+ /**
+ * Renames an object in a BOS bucket.
+ *
+ * @param bucketName the name of the bucket
+ * @param sourceKey the source object key
+ * @param destinationKey the destination object key
+ * @return the rename object response
+ * @throws IOException if an I/O error occurs
+ */
+ RenameObjectResponse renameObject(
+ String bucketName, String sourceKey,
+ String destinationKey) throws IOException;
+
+ /**
+ * Completes a multipart upload.
+ *
+ * @param request the complete multipart upload request
+ * @return the complete multipart upload response
+ * @throws IOException if an I/O error occurs
+ */
+ CompleteMultipartUploadResponse completeMultipartUpload(
+ CompleteMultipartUploadRequest request)
+ throws IOException;
+
+ /**
+ * Initiates a multipart upload.
+ *
+ * @param request the initiate multipart upload request
+ * @return the initiate multipart upload response
+ * @throws IOException if an I/O error occurs
+ */
+ InitiateMultipartUploadResponse initiateMultipartUpload(
+ InitiateMultipartUploadRequest request)
+ throws IOException;
+
+ /**
+ * Aborts a multipart upload.
+ *
+ * @param bucketName the name of the bucket
+ * @param key the object key
+ * @param uploadId the upload ID to abort
+ * @throws IOException if an I/O error occurs
+ */
+ void abortMultipartUpload(
+ String bucketName, String key, String uploadId)
+ throws IOException;
+
+ /**
+ * Checks whether the specified bucket is a hierarchy
+ * (namespace) bucket.
+ *
+ * @param bucketName the name of the bucket
+ * @return true if the bucket is a hierarchy bucket
+ * @throws IOException if an I/O error occurs
+ */
+ boolean isHierarchyBucket(String bucketName)
+ throws IOException;
+
+ /**
+ * Closes the BOS client and releases resources.
+ */
+ void close();
+
+ /**
+ * Copies a part of an object using a multipart copy request.
+ *
+ * @param uploadPartCopyRequest the upload part copy request
+ * @return the upload part copy response
+ * @throws IOException if an I/O error occurs
+ */
+ UploadPartCopyResponse uploadPartCopy(
+ UploadPartCopyRequest uploadPartCopyRequest)
+ throws IOException;
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosClientProxyImpl.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosClientProxyImpl.java
new file mode 100644
index 0000000000000..bd9b90e4e04cc
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosClientProxyImpl.java
@@ -0,0 +1,601 @@
+/*
+ * 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.bos;
+
+import com.baidubce.BceServiceException;
+import com.baidubce.auth.DefaultBceCredentials;
+import com.baidubce.auth.DefaultBceSessionCredentials;
+import com.baidubce.services.bos.BosClient;
+import com.baidubce.services.bos.BosClientConfiguration;
+import com.baidubce.services.bos.model.AbortMultipartUploadRequest;
+import com.baidubce.services.bos.model.BosObject;
+import com.baidubce.services.bos.model.CompleteMultipartUploadRequest;
+import com.baidubce.services.bos.model.CompleteMultipartUploadResponse;
+import com.baidubce.services.bos.model.CopyObjectRequest;
+import com.baidubce.services.bos.model.CopyObjectResponse;
+import com.baidubce.services.bos.model.DeleteDirectoryRequest;
+import com.baidubce.services.bos.model.DeleteDirectoryResponse;
+import com.baidubce.services.bos.model.DeleteMultipleObjectsRequest;
+import com.baidubce.services.bos.model.DeleteObjectRequest;
+import com.baidubce.services.bos.model.GetObjectMetadataRequest;
+import com.baidubce.services.bos.model.GetObjectRequest;
+import com.baidubce.services.bos.model.HeadBucketRequest;
+import com.baidubce.services.bos.model.HeadBucketResponse;
+import com.baidubce.services.bos.model.InitiateMultipartUploadRequest;
+import com.baidubce.services.bos.model.InitiateMultipartUploadResponse;
+import com.baidubce.services.bos.model.ListObjectsRequest;
+import com.baidubce.services.bos.model.ListObjectsResponse;
+import com.baidubce.services.bos.model.ObjectMetadata;
+import com.baidubce.services.bos.model.PutObjectRequest;
+import com.baidubce.services.bos.model.PutObjectResponse;
+import com.baidubce.services.bos.model.RenameObjectRequest;
+import com.baidubce.services.bos.model.RenameObjectResponse;
+import com.baidubce.services.bos.model.UploadPartCopyRequest;
+import com.baidubce.services.bos.model.UploadPartCopyResponse;
+import com.baidubce.services.bos.model.UploadPartRequest;
+import com.baidubce.services.bos.model.UploadPartResponse;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.bos.credentials.BceCredentialsProvider;
+import org.apache.hadoop.fs.bos.credentials.HadoopCredentialsProvider;
+import org.apache.hadoop.fs.bos.exceptions.BandwidthLimitException;
+import org.apache.hadoop.fs.bos.exceptions.BosException;
+import org.apache.hadoop.fs.bos.exceptions.BosHotObjectException;
+import org.apache.hadoop.fs.bos.exceptions.BosServerException;
+import org.apache.hadoop.fs.bos.exceptions.SessionTokenExpireException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+
+/**
+ * Default implementation of {@link BosClientProxy} that
+ * delegates operations to the BOS SDK {@link BosClient}.
+ */
+public class BosClientProxyImpl implements BosClientProxy {
+
+ /**
+ * Constructs a BosClientProxyImpl.
+ * Call {@link #init(URI, Configuration)} to initialize.
+ */
+ public BosClientProxyImpl() {
+ }
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(BosClientProxyImpl.class);
+
+ private volatile BosClient bosClient = null;
+ private volatile BosClient preBosClient = null;
+ private Configuration conf = null;
+ private URI uri = null;
+ private static final int BOS_UPLOAD_CONFLICT_CODE = 412;
+ private static final int BOS_REQUEST_LIMIT_CODE = 429;
+ private static final int BOS_NO_SUCH_KEY_CODE = 404;
+
+ private String user = null;
+ private static final int STREAM_BUFFER_DEFAULT_SIZE =
+ 64 * 1024;
+ private BceCredentialsProvider provider;
+
+ /** {@inheritDoc} */
+ @Override
+ public void init(URI uri, Configuration conf) {
+ this.conf = conf;
+ this.uri = uri;
+
+ BosClientConfiguration config =
+ new BosClientConfiguration();
+ updateUserForInit();
+ provider =
+ BceCredentialsProvider
+ .getBceCredentialsProviderImpl(conf);
+ if (provider instanceof HadoopCredentialsProvider) {
+ provider.setConf(this.conf);
+ }
+ DefaultBceSessionCredentials token =
+ provider.getCredentials(uri, user);
+
+ if (token == null
+ || StringUtils.isEmpty(token.getAccessKeyId())
+ || StringUtils.isEmpty(token.getSecretKey())) {
+ throw new IllegalArgumentException(
+ "user accessKey and secretAccessKey"
+ + " should not be null");
+ }
+ if (token.getSessionToken() == null
+ || token.getSessionToken().trim().isEmpty()) {
+ config.setCredentials(
+ new DefaultBceCredentials(
+ token.getAccessKeyId(),
+ token.getSecretKey()
+ ));
+ } else {
+ config.setCredentials(
+ new DefaultBceSessionCredentials(
+ token.getAccessKeyId(),
+ token.getSecretKey(),
+ token.getSessionToken()
+ ));
+ }
+ // support ConsistencyView
+ config.setConsistencyView(
+ BosClientConfiguration.STRONG_CONSISTENCY_VIEW);
+
+ String endPoint = conf.get(
+ "fs.bos.bucket." + uri.getHost() + ".endpoint",
+ null);
+ if (endPoint == null) {
+ endPoint = conf.get(BaiduBosConstants.BOS_ENDPOINT);
+ }
+ config.setEndpoint(endPoint);
+ int maxConnections = conf.getInt(
+ BaiduBosConstants.BOS_MAX_CONNECTIONS, 1000);
+ config.setMaxConnections(maxConnections);
+ String userAgent =
+ BaiduBosConstants.BOS_FILESYSTEM_USER_AGENT
+ + "/"
+ + BosClientConfiguration.DEFAULT_USER_AGENT;
+ config.setUserAgent(userAgent);
+ config.withStreamBufferSize(conf.getInt(
+ BaiduBosConstants.BOS_STREAM_BUFFER_SIZE,
+ STREAM_BUFFER_DEFAULT_SIZE));
+ bosClient = new BosClient(config);
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public PutObjectResponse putObject(
+ String bucketName, String key,
+ InputStream input, ObjectMetadata metadata)
+ throws IOException {
+ PutObjectResponse response = null;
+ try {
+ PutObjectRequest request =
+ new PutObjectRequest(
+ bucketName, key, input, metadata);
+ response = bosClient.putObject(request);
+ } catch (BceServiceException e) {
+ handleBosServiceException(e);
+ }
+ return response;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public void putEmptyObject(
+ String bucketName, String key,
+ ObjectMetadata objectMeta) throws IOException {
+ InputStream in = null;
+ try {
+ in = new ByteArrayInputStream(new byte[0]);
+ objectMeta.setContentType("binary/octet-stream");
+ objectMeta.setContentLength(0);
+ PutObjectRequest request =
+ new PutObjectRequest(
+ bucketName, key, in, objectMeta);
+ bosClient.putObject(request);
+ } catch (BceServiceException e) {
+ if (BOS_UPLOAD_CONFLICT_CODE == e.getStatusCode()
+ || BOS_REQUEST_LIMIT_CODE
+ == e.getStatusCode()) {
+ // status code 412 or 429 means another thread
+ // is uploading the same object
+ LOG.debug(
+ "another thread uploading same bos object");
+ try {
+ bosClient.getObjectMetadata(bucketName, key);
+ } catch (BceServiceException ex) {
+ if (BOS_NO_SUCH_KEY_CODE
+ == ex.getStatusCode()) {
+ LOG.error("concurrentUpload bos failed");
+ handleBosServiceException(key, ex);
+ }
+ }
+ } else {
+ handleBosServiceException(e);
+ }
+ } finally {
+ try {
+ if (in != null) {
+ in.close();
+ }
+ } catch (IOException ex) {
+ // ignore close exception
+ }
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public ObjectMetadata getObjectMetadata(
+ String bucketName, String key) throws IOException {
+ ObjectMetadata response = null;
+ try {
+ GetObjectMetadataRequest request =
+ new GetObjectMetadataRequest(bucketName, key);
+ response = bosClient.getObjectMetadata(request);
+ } catch (BceServiceException e) {
+ handleBosServiceException(key, e);
+ }
+ return response;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public BosObject getObject(GetObjectRequest request)
+ throws IOException {
+ BosObject response = null;
+ try {
+ response = bosClient.getObject(request);
+ } catch (BceServiceException e) {
+ handleBosServiceException(request.getKey(), e);
+ }
+ return response;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public ListObjectsResponse listObjects(
+ ListObjectsRequest request) throws IOException {
+ ListObjectsResponse response = null;
+ request.setNeedExtMeta(true);
+ try {
+ response = bosClient.listObjects(request);
+ } catch (BceServiceException e) {
+ handleBosServiceException(e);
+ }
+ return response;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public UploadPartResponse uploadPart(
+ UploadPartRequest request) throws IOException {
+ UploadPartResponse response = null;
+ try {
+ long now = System.currentTimeMillis();
+ response = bosClient.uploadPart(request);
+ LOG.debug("upload use time : {} ms",
+ System.currentTimeMillis() - now);
+ } catch (BceServiceException e) {
+ handleBosServiceException(e);
+ }
+ return response;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public void deleteObject(String bucketName, String key)
+ throws IOException {
+ try {
+ DeleteObjectRequest request =
+ new DeleteObjectRequest(bucketName, key);
+ bosClient.deleteObject(request);
+ } catch (BceServiceException e) {
+ if (BOS_NO_SUCH_KEY_CODE == e.getStatusCode()) {
+ // if key does not exist, treat as success
+ LOG.warn(
+ "Deleting key: {} from bucket: {}"
+ + " but key is not exist",
+ key, bucketName);
+ } else {
+ handleBosServiceException(e);
+ }
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public void deleteMultipleObjects(
+ DeleteMultipleObjectsRequest request)
+ throws IOException {
+ try {
+ bosClient.deleteMultipleObjects(request);
+ } catch (BceServiceException e) {
+ // BOS_NO_SUCH_KEY_CODE in response.getErrors,
+ // don't handle
+ handleBosServiceException(e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public DeleteDirectoryResponse deleteDirectory(
+ String bucketName, String key,
+ boolean isDeleteRecursive, String marker)
+ throws IOException {
+ DeleteDirectoryResponse response = null;
+ try {
+ DeleteDirectoryRequest request =
+ new DeleteDirectoryRequest(
+ bucketName, key,
+ isDeleteRecursive, marker);
+ response = bosClient.deleteDirectory(request);
+ } catch (BceServiceException e) {
+ if (BOS_NO_SUCH_KEY_CODE == e.getStatusCode()) {
+ // if key does not exist, treat as success
+ LOG.warn(
+ "Deleting Directory: {} from bucket: {}"
+ + " but Directory is not exist",
+ key, bucketName);
+ } else {
+ handleBosServiceException(e);
+ }
+ }
+ return response;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public CopyObjectResponse copyObject(
+ String sourceBucketName, String sourceKey,
+ String destinationBucketName,
+ String destinationKey) throws IOException {
+ CopyObjectRequest request = new CopyObjectRequest(
+ sourceBucketName, sourceKey,
+ destinationBucketName, destinationKey);
+ CopyObjectResponse response = null;
+ try {
+ response = bosClient.copyObject(request);
+ } catch (BceServiceException e) {
+ handleBosServiceException(sourceKey, e);
+ }
+ return response;
+ }
+
+ /** {@inheritDoc} */
+ public CopyObjectResponse copyObject(
+ CopyObjectRequest request) throws IOException {
+ CopyObjectResponse response = null;
+ try {
+ response = bosClient.copyObject(request);
+ } catch (BceServiceException e) {
+ handleBosServiceException(
+ request.getSourceKey(), e);
+ }
+ return response;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public RenameObjectResponse renameObject(
+ String bucketName, String sourceKey,
+ String destinationKey) throws IOException {
+ RenameObjectResponse response = null;
+ try {
+ RenameObjectRequest request =
+ new RenameObjectRequest(
+ bucketName, sourceKey, destinationKey);
+ response = bosClient.renameObject(request);
+ } catch (BceServiceException e) {
+ if (BOS_NO_SUCH_KEY_CODE == e.getStatusCode()) {
+ try {
+ bosClient.getObjectMetadata(
+ bucketName, destinationKey);
+ } catch (BceServiceException e2) {
+ handleBosServiceException(sourceKey, e);
+ }
+ } else {
+ handleBosServiceException(sourceKey, e);
+ }
+ }
+ return response;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public CompleteMultipartUploadResponse
+ completeMultipartUpload(
+ CompleteMultipartUploadRequest request)
+ throws IOException {
+ CompleteMultipartUploadResponse response = null;
+ try {
+ response =
+ bosClient.completeMultipartUpload(request);
+ } catch (BceServiceException e) {
+ if (BOS_NO_SUCH_KEY_CODE == e.getStatusCode()
+ && e.getErrorCode() != null
+ && e.getErrorCode().trim()
+ .equals("NoSuchUpload")) {
+ LOG.warn("The upload ID might be invalid, or"
+ + " the multipart upload might have been"
+ + " aborted or completed.");
+ throw new IOException(
+ "NoSuchUpload: upload ID is invalid or"
+ + " the upload has been aborted", e);
+ } else {
+ handleBosServiceException(e);
+ }
+
+ }
+ return response;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public InitiateMultipartUploadResponse
+ initiateMultipartUpload(
+ InitiateMultipartUploadRequest request)
+ throws IOException {
+ InitiateMultipartUploadResponse response = null;
+ try {
+ response =
+ bosClient.initiateMultipartUpload(request);
+ } catch (BceServiceException e) {
+ handleBosServiceException(e);
+ }
+ return response;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public void abortMultipartUpload(
+ String bucketName, String key, String uploadId)
+ throws IOException {
+ try {
+ AbortMultipartUploadRequest request =
+ new AbortMultipartUploadRequest(
+ bucketName, key, uploadId);
+ bosClient.abortMultipartUpload(request);
+ } catch (BceServiceException e) {
+ handleBosServiceException(e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public void close() {
+ if (bosClient != null) {
+ bosClient.shutdown();
+ bosClient = null;
+ }
+ }
+
+ /**
+ * Handles a BOS service exception with key context. If the
+ * error is a 404, throws {@link FileNotFoundException}.
+ *
+ * @param key the object key involved in the operation
+ * @param e the BOS service exception
+ * @throws IOException wrapping the service exception
+ */
+ private void handleBosServiceException(
+ String key, BceServiceException e)
+ throws IOException {
+ if (BOS_NO_SUCH_KEY_CODE == e.getStatusCode()) {
+ throw new FileNotFoundException(
+ "Key '" + key + "' does not exist in BOS");
+ } else {
+ handleBosServiceException(e);
+ }
+ }
+
+ /**
+ * Handles a BOS service exception, translating it into the
+ * appropriate IOException subclass based on status code and
+ * error code.
+ *
+ * @param e the BOS service exception
+ * @throws IOException wrapping the service exception
+ */
+ private void handleBosServiceException(
+ BceServiceException e) throws IOException {
+ // process all 400 status, because as token expired
+ // bos server return "Error Code=null" when request
+ // object use a http head method
+ if (400 == e.getStatusCode()
+ && (e.getErrorCode() == null
+ || "null".equals(e.getErrorCode())
+ || "InvalidSessionToken".equals(
+ e.getErrorCode().trim()))) {
+ LOG.error(
+ "sts session expired or invalid,"
+ + " need to update client");
+ throw new SessionTokenExpireException(e);
+ } else if (400 == e.getStatusCode()) {
+ LOG.error("request unknown error : {}",
+ e.getCause());
+ throw new SessionTokenExpireException(e);
+ } else if (BOS_REQUEST_LIMIT_CODE
+ == e.getStatusCode()
+ && (e.getErrorCode() == null
+ || e.getErrorCode().trim()
+ .equals("null"))) {
+ throw new BandwidthLimitException(
+ new IOException(
+ "trigger bos rate limit"
+ + " for too many requests !!!"));
+ } else if (BOS_REQUEST_LIMIT_CODE
+ == e.getStatusCode()
+ && e.getErrorCode().trim()
+ .equals("RequestRateLimitExceeded")) {
+ throw new BosHotObjectException(
+ new IOException(
+ "trigger bos object rate limit !!!"));
+ } else if (BOS_REQUEST_LIMIT_CODE
+ == e.getStatusCode()) {
+ throw new BosHotObjectException(
+ new IOException(
+ "status code 429 !!!" + e.getCause()));
+ } else if (e.getCause() instanceof IOException) {
+ throw (IOException) e.getCause();
+ } else {
+ LOG.debug(
+ "BOS Error code: {}; BOS Error message: {}",
+ e.getErrorCode(), e.getErrorMessage());
+ if (5 == (e.getStatusCode() / 100)) {
+ throw new BosServerException(e);
+ }
+ throw new BosException(e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ public boolean isHierarchyBucket(String bucketName)
+ throws IOException {
+ HeadBucketResponse response = null;
+ try {
+ HeadBucketRequest request =
+ new HeadBucketRequest(bucketName);
+ response = this.bosClient.headBucket(request);
+ } catch (BceServiceException e) {
+ handleBosServiceException(e);
+ }
+
+ if (response != null
+ && response.getMetadata() != null) {
+ return "namespace".equals(
+ response.getMetadata().getBucketType());
+ }
+
+ return false;
+ }
+
+ /**
+ * Initializes the current user name during client
+ * initialization.
+ */
+ private void updateUserForInit() {
+ try {
+ UserGroupInformation ugi =
+ UserGroupInformation.getCurrentUser();
+ user = ugi.getShortUserName();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ public UploadPartCopyResponse uploadPartCopy(
+ UploadPartCopyRequest request) throws IOException {
+ UploadPartCopyResponse response = null;
+ try {
+ response = bosClient.uploadPartCopy(request);
+ } catch (BceServiceException e) {
+ handleBosServiceException(e);
+ }
+ return response;
+ }
+
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosInputStream.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosInputStream.java
new file mode 100644
index 0000000000000..e22c9c7257dcc
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosInputStream.java
@@ -0,0 +1,587 @@
+/*
+ * 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.bos;
+
+import org.apache.hadoop.fs.CanSetReadahead;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Input stream implementation for reading objects from BOS.
+ * Supports lazy seeking and readahead buffering.
+ */
+public class BosInputStream extends FSInputStream
+ implements CanSetReadahead {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(BosInputStream.class);
+
+ private FileSystem.Statistics statistics;
+ private BosNativeFileSystemStore store;
+ private static final long DEFAULT_READAHEAD_LEN =
+ 1024 * 1024;
+ private volatile boolean closed;
+ private final String key;
+ private final long contentLength;
+ private InputStream in = null;
+ private long pos = 0;
+ private FileMetadata fileMetaData;
+ private long readahead = DEFAULT_READAHEAD_LEN;
+
+ /**
+ * This is the actual position within the object, used by
+ * lazy seek to decide whether to seek on the next read or
+ * not.
+ */
+ private long nextReadPos = 0;
+
+ /**
+ * The start of the content range of the last request.
+ */
+ private long contentRangeStart = 0;
+
+ /**
+ * The end of the content range of the last request.
+ * This is an absolute value of the range, not a length
+ * field.
+ */
+ private long contentRangeFinish = 0;
+
+ /**
+ * Constructs a BosInputStream for the given object key.
+ *
+ * @param key the object key in BOS
+ * @param fileMetaData the file metadata
+ * @param store the native file system store
+ * @param statistics the file system statistics
+ */
+ public BosInputStream(
+ String key, FileMetadata fileMetaData,
+ BosNativeFileSystemStore store,
+ FileSystem.Statistics statistics) {
+ this.key = key;
+ this.fileMetaData = fileMetaData;
+ this.contentLength = fileMetaData.getLength();
+ this.store = store;
+ this.statistics = statistics;
+ }
+
+ /**
+ * Reads a single byte from the stream.
+ *
+ * @return the byte read, or -1 if end of stream
+ * @throws IOException if an I/O error occurs
+ */
+ public synchronized int read() throws IOException {
+ checkNotClosed();
+
+ if (this.contentLength == 0
+ || (nextReadPos >= contentLength)) {
+ return -1;
+ }
+
+ int byteRead;
+ try {
+ lazySeek(nextReadPos, 1);
+ byteRead = in.read();
+ } catch (EOFException e) {
+ return -1;
+ } catch (IOException e) {
+ onReadFailure(e, 1);
+ byteRead = in.read();
+ }
+
+ if (byteRead >= 0) {
+ pos++;
+ nextReadPos++;
+ }
+
+ if (statistics != null && byteRead >= 0) {
+ statistics.incrementBytesRead(1);
+ }
+ return byteRead;
+ }
+
+ /**
+ * Reads bytes into a buffer from the stream.
+ *
+ * @param buf the buffer to read into
+ * @param off the start offset in the buffer
+ * @param len the maximum number of bytes to read
+ * @return the number of bytes read, or -1 if end of stream
+ * @throws IOException if an I/O error occurs
+ */
+ public synchronized int read(
+ byte[] buf, int off, int len) throws IOException {
+ checkNotClosed();
+
+ if (len == 0) {
+ return 0;
+ }
+
+ if (this.contentLength == 0
+ || (nextReadPos >= contentLength)) {
+ return -1;
+ }
+
+ // Validate and adjust parameters
+ if (off < 0 || len < 0
+ || len > buf.length - off) {
+ throw new IndexOutOfBoundsException(
+ String.format(
+ "Invalid read parameters:"
+ + " buf.length=%d, off=%d, len=%d",
+ buf.length, off, len));
+ }
+
+ try {
+ lazySeek(nextReadPos, len);
+ } catch (EOFException e) {
+ // the end of the file has moved
+ return -1;
+ }
+
+ int bytesRead;
+ try {
+ bytesRead = in.read(buf, off, len);
+ } catch (EOFException e) {
+ onReadFailure(e, len);
+ // the base implementation swallows EOFs.
+ return -1;
+ } catch (IOException e) {
+ onReadFailure(e, len);
+ bytesRead = in.read(buf, off, len);
+ } catch (Exception e) {
+ int intervalSeconds = 3;
+ int retry = 3;
+ while (true) {
+ try {
+ LOG.info(
+ "Read exception occur:"
+ + " retry at most {} times.",
+ retry);
+ // Reopen the stream at the current position
+ // to recover from the exception
+ onReadFailure(e, len);
+ // Read from the reopened stream
+ bytesRead = in.read(buf, off, len);
+ break;
+ } catch (EOFException eof) {
+ return -1;
+ } catch (IOException ioe) {
+ // For IOException, try to reopen once more
+ // and retry
+ LOG.info(
+ "IOException during retry,"
+ + " attempting to reopen stream");
+ onReadFailure(ioe, len);
+ bytesRead = in.read(buf, off, len);
+ break;
+ } catch (Exception ex) {
+ if (retry <= 0) {
+ String errorMsg =
+ ex.getMessage() != null
+ ? ex.getMessage()
+ : ex.getClass().getSimpleName();
+ throw new IOException(
+ "Retry " + retry
+ + " times to read still"
+ + " exception: " + errorMsg,
+ ex);
+ }
+
+ try {
+ TimeUnit.SECONDS.sleep(
+ intervalSeconds);
+ } catch (InterruptedException ite) {
+ Thread.currentThread().interrupt();
+ throw new IOException(
+ "Thread interrupted during retry", ite);
+ }
+ intervalSeconds *= 2;
+ retry--;
+ // Update e to ex for the next iteration
+ e = ex;
+ }
+ }
+ }
+
+ if (bytesRead > 0) {
+ pos += bytesRead;
+ nextReadPos += bytesRead;
+ }
+
+ if (statistics != null && bytesRead >= 0) {
+ statistics.incrementBytesRead(bytesRead);
+ }
+
+ return bytesRead;
+ }
+
+ /**
+ * Closes this input stream and releases resources.
+ *
+ * @throws IOException if an I/O error occurs
+ */
+ public synchronized void close() throws IOException {
+ if (!closed) {
+ closed = true;
+
+ // close or abort the stream
+ closeStream("close() operation");
+ // this is actually a no-op
+ super.close();
+ }
+ }
+
+ /**
+ * Seeks to the specified position in the stream. This
+ * performs a lazy seek; the actual stream repositioning
+ * happens on the next read.
+ *
+ * @param targetPos the target position to seek to
+ * @throws IOException if an I/O error occurs
+ */
+ public synchronized void seek(long targetPos)
+ throws IOException {
+ checkNotClosed();
+
+ // Do not allow negative seek
+ if (targetPos < 0 || targetPos > contentLength) {
+ throw new EOFException(
+ FSExceptionMessages.NEGATIVE_SEEK
+ + " " + targetPos);
+ }
+
+ if (this.contentLength <= 0) {
+ return;
+ }
+
+ // Lazy seek
+ nextReadPos = targetPos;
+ }
+
+ /**
+ * Returns the current position in the stream.
+ *
+ * @return the current position
+ * @throws IOException if an I/O error occurs
+ */
+ public synchronized long getPos() throws IOException {
+ return nextReadPos;
+ }
+
+ /**
+ * Seeks to a new source. Not supported; always returns
+ * false.
+ *
+ * @param targetPos the target position
+ * @return false always
+ * @throws IOException if an I/O error occurs
+ */
+ public boolean seekToNewSource(long targetPos)
+ throws IOException {
+ return false;
+ }
+
+ /**
+ * Sets the readahead value for this stream.
+ *
+ * @param readahead the readahead length, or null to reset
+ * to default
+ */
+ public synchronized void setReadahead(Long readahead) {
+ if (readahead == null) {
+ this.readahead = DEFAULT_READAHEAD_LEN;
+ } else {
+ this.readahead = Math.max(
+ readahead, DEFAULT_READAHEAD_LEN);
+ }
+ }
+
+ /**
+ * Verify that the input stream is open. Non blocking; this
+ * gives the last state of the volatile {@link #closed}
+ * field.
+ *
+ * @throws IOException if the connection is closed.
+ */
+ private void checkNotClosed() throws IOException {
+ if (closed) {
+ throw new IOException(
+ key + ": "
+ + FSExceptionMessages.STREAM_IS_CLOSED);
+ }
+ }
+
+ /**
+ * Perform lazy seek and adjust stream to correct position
+ * for reading.
+ *
+ * @param targetPos position from where data should be read
+ * @param len length of the content that needs to be read
+ * @throws IOException if an I/O error occurs
+ */
+ private void lazySeek(long targetPos, long len)
+ throws IOException {
+ // For lazy seek
+ try {
+ seekInStream(targetPos, len);
+ } catch (Exception e) {
+ closeStream("seekInStream failed");
+ }
+ // re-open at specific location if needed
+ if (in == null) {
+ LOG.debug(
+ "re-open at specific locaition: {}",
+ targetPos);
+ reopen("read from new offset", targetPos, len);
+ }
+ }
+
+ /**
+ * Adjust the stream to a specific position.
+ *
+ * @param targetPos target seek position
+ * @param length length of content that needs to be read
+ * from targetPos
+ * @throws IOException if an I/O error occurs
+ */
+ private void seekInStream(
+ long targetPos, long length) throws IOException {
+ long diff = targetPos - pos;
+ checkNotClosed();
+
+ if (in == null) {
+ return;
+ }
+ // compute how much more to skip
+ if (diff > 0) {
+ // if seek then reopen the stream
+
+ // forward seek - this is where data can be skipped
+ int available = in.available();
+ // always seek at least as far as what is available
+ long forwardSeekRange =
+ Math.max(readahead, available);
+ // work out how much is actually left in the stream
+ // then choose whichever comes first: the range or
+ // the EOF
+ long remainingInCurrentRequest =
+ remainingInCurrentRequest();
+
+ long forwardSeekLimit = Math.min(
+ remainingInCurrentRequest, forwardSeekRange);
+
+ boolean skipForward =
+ remainingInCurrentRequest > 0
+ && diff <= forwardSeekLimit;
+
+ if (skipForward) {
+ // the forward seek range is within the limits
+ LOG.debug(
+ "Forward seek on {} of {}bytes",
+ key, diff);
+
+ long skipped = in.skip(diff);
+ if (skipped > 0) {
+ pos += skipped;
+ // as these bytes have been read, they are
+ // included in the counter
+ statistics.incrementBytesRead(skipped);
+ }
+
+ if (pos == targetPos) {
+ // all is well
+ return;
+ } else {
+ // log a warning; continue to attempt to
+ // re-open
+ LOG.debug(
+ "Failed to seek on {} to {}."
+ + " Current position {}",
+ key, targetPos, pos);
+ }
+ }
+ } else if (diff < 0) {
+ // backwards seek
+ LOG.debug(
+ "seek on {} backwards {}."
+ + " Current position {}",
+ key, diff, pos);
+ } else {
+ // targetPos == pos
+ if (remainingInCurrentRequest() > 0) {
+ // if there is data left in the stream, keep
+ // going
+ return;
+ }
+ }
+ // if the code reaches here, the stream needs to be
+ // reopened. close the stream; if read the object will
+ // be opened at the new pos
+ closeStream("seekInStream()");
+ pos = targetPos;
+ }
+
+ /**
+ * Bytes left in the current request.
+ * Only valid if there is an active request.
+ *
+ * @return how many bytes are left to read in the current
+ * GET.
+ */
+ public synchronized long remainingInCurrentRequest() {
+ // contentRangeFinish is inclusive (last byte index),
+ // so add 1
+ return this.contentRangeFinish - this.pos + 1;
+ }
+
+ /**
+ * Close a stream.
+ *
+ * This does not set the {@link #closed} flag.
+ *
+ * @param reason reason for stream being closed; used in
+ * messages
+ */
+ private void closeStream(String reason) {
+ if (in != null) {
+ try {
+ in.close();
+ } catch (IOException e) {
+ LOG.debug(
+ "When closing {} stream for {}",
+ key, reason, e);
+ }
+ }
+ long remaining = remainingInCurrentRequest();
+ LOG.debug(
+ "Stream {} : {}; remaining={} streamPos={},"
+ + " nextReadPos={}, request range {}-{}",
+ key, reason, remaining, pos,
+ nextReadPos, contentRangeStart,
+ contentRangeFinish);
+
+ in = null;
+ }
+
+ /**
+ * Handle an IOE on a read by attempting to re-open the
+ * stream. The filesystem's readException count will be
+ * incremented.
+ *
+ * @param ioe exception caught.
+ * @param length length of data being attempted to read
+ * @throws IOException any exception thrown on the re-open
+ * attempt.
+ */
+ private void onReadFailure(
+ Exception ioe, int length) throws IOException {
+ LOG.info(
+ "Got exception while trying to read from"
+ + " stream {} trying to recover from"
+ + " position: {}",
+ key, pos);
+ LOG.info(
+ "While trying to read from stream {}"
+ + " exception: {}",
+ key, ioe);
+ reopen("failure recovery", pos, length);
+ }
+
+ /**
+ * Opens up the stream at specified target position and for
+ * given length.
+ *
+ * @param reason reason for reopen
+ * @param targetPos target position
+ * @param length length requested
+ * @throws IOException on any failure to open the object
+ */
+ private synchronized void reopen(
+ String reason, long targetPos, long length)
+ throws IOException {
+ if (in != null) {
+ closeStream("reopen(" + reason + ")");
+ }
+
+ contentRangeFinish = calculateRequestLimit(
+ targetPos, length, contentLength, readahead);
+
+ LOG.info(
+ "reopen({}) for {} range[{}-{}], length={},"
+ + " contentLength={}, streamPosition={},"
+ + " nextReadPosition={}",
+ key, reason, targetPos, contentRangeFinish,
+ length, contentLength, pos, nextReadPos);
+ try {
+ in = store.retrieve(
+ key, targetPos, contentRangeFinish,
+ fileMetaData);
+ contentRangeStart = targetPos;
+ } catch (Exception e) {
+ LOG.info(
+ "native store retrieve failed reason : {}",
+ e.getMessage());
+ closeStream("native store retrieve failed");
+ }
+ if (in == null) {
+ throw new IOException(
+ "Null IO stream from reopen of ("
+ + reason + ") " + key);
+ }
+ this.pos = targetPos;
+ }
+
+ /**
+ * Calculate the limit for a get request.
+ *
+ * @param targetPos position of the read
+ * @param length length of bytes requested; if less than
+ * zero "unknown"
+ * @param fileContentLength total length of file
+ * @param readaheadLen current readahead value
+ * @return the absolute value of the limit of the request
+ * (inclusive, last byte index).
+ */
+ private long calculateRequestLimit(
+ long targetPos,
+ long length,
+ long fileContentLength,
+ long readaheadLen) {
+ if (length < 0) {
+ // For unknown length, read to end of file
+ // (last byte index)
+ return fileContentLength - 1;
+ }
+
+ return Math.min(
+ fileContentLength,
+ targetPos + Math.max(readaheadLen, length)) - 1;
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosNativeFileSystemStore.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosNativeFileSystemStore.java
new file mode 100644
index 0000000000000..67c152d9c90bc
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosNativeFileSystemStore.java
@@ -0,0 +1,735 @@
+/*
+ * 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.bos;
+
+import com.baidubce.BceClientException;
+import com.baidubce.services.bos.model.BosObject;
+import com.baidubce.services.bos.model.BosObjectSummary;
+import com.baidubce.services.bos.model.BosPrefixInfo;
+import com.baidubce.services.bos.model.GetObjectRequest;
+import com.baidubce.services.bos.model.ListObjectsRequest;
+import com.baidubce.services.bos.model.ListObjectsResponse;
+import com.baidubce.services.bos.model.ObjectMetadata;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.bos.exceptions.BandwidthLimitException;
+import org.apache.hadoop.fs.bos.exceptions.BosHotObjectException;
+import org.apache.hadoop.fs.bos.exceptions.BosServerException;
+import org.apache.hadoop.fs.bos.exceptions.SessionTokenExpireException;
+import org.apache.hadoop.fs.bos.utils.BosCRC32CCheckSum;
+import org.apache.hadoop.io.retry.DefaultFailoverProxyProvider;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryProxy;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Abstract base class for BOS native file system store
+ * implementations. Provides common operations such as object
+ * retrieval, listing, deletion, copy, and rename. Concrete
+ * subclasses handle hierarchy vs. non-hierarchy bucket
+ * differences.
+ */
+public abstract class BosNativeFileSystemStore {
+
+ /** Logger instance using the runtime class name. */
+ public final Logger log =
+ LoggerFactory.getLogger(this.getClass().getName());
+
+ /** The BOS client proxy for service operations. */
+ protected final BosClientProxy bosClientProxy;
+
+ /** The name of the BOS bucket. */
+ protected final String bucketName;
+
+ /** The Hadoop configuration. */
+ protected final Configuration conf;
+
+ /** Thread pool for bounded parallel operations. */
+ protected ExecutorService boundedThreadPool;
+
+ private String envUserName;
+ private String envGroupName;
+
+ /**
+ * Constructs a BosNativeFileSystemStore.
+ *
+ * @param bosClientProxy the BOS client proxy
+ * @param bucketName the name of the bucket
+ * @param conf the Hadoop configuration
+ */
+ public BosNativeFileSystemStore(
+ BosClientProxy bosClientProxy,
+ String bucketName, Configuration conf) {
+ this.bosClientProxy = bosClientProxy;
+ this.bucketName = bucketName;
+ this.conf = conf;
+ }
+
+ /**
+ * Creates a default BOS client proxy with retry policies.
+ *
+ * @param conf the Hadoop configuration
+ * @return a retry-wrapped BOS client proxy
+ */
+ private static BosClientProxy
+ createDefaultBosClientProxy(Configuration conf) {
+ BosClientProxy bosClientProxy =
+ new BosClientProxyImpl();
+ RetryPolicy basePolicy =
+ RetryPolicies
+ .retryUpToMaximumCountWithProportionalSleep(
+ conf.getInt(
+ BaiduBosConstants.BOS_MAX_RETRIES,
+ 3),
+ conf.getLong(
+ BaiduBosConstants
+ .BOS_SLEEP_MILL_SECONDS,
+ 200),
+ TimeUnit.MILLISECONDS);
+
+ BosRandomRetryPolicy bosHotObjectRandomRetryPolicy =
+ new BosRandomRetryPolicy(
+ conf.getInt(
+ BaiduBosConstants.BOS_MAX_RETRIES, 3),
+ 2000,
+ 8000,
+ TimeUnit.MILLISECONDS);
+
+ Map, RetryPolicy>
+ exceptionToPolicyMap = new HashMap<>();
+
+ exceptionToPolicyMap.put(
+ IOException.class, basePolicy);
+ exceptionToPolicyMap.put(
+ SessionTokenExpireException.class, basePolicy);
+ exceptionToPolicyMap.put(
+ BceClientException.class, basePolicy);
+ exceptionToPolicyMap.put(
+ BosServerException.class, basePolicy);
+ exceptionToPolicyMap.put(
+ BandwidthLimitException.class,
+ bosHotObjectRandomRetryPolicy);
+ exceptionToPolicyMap.put(
+ BosHotObjectException.class,
+ bosHotObjectRandomRetryPolicy);
+ RetryPolicy methodPolicy =
+ RetryPolicies.retryByException(
+ RetryPolicies.TRY_ONCE_THEN_FAIL,
+ exceptionToPolicyMap);
+
+ Map methodNameToPolicyMap =
+ new HashMap<>();
+
+ methodNameToPolicyMap.put(
+ "putObject", methodPolicy);
+ methodNameToPolicyMap.put(
+ "putEmptyObject", methodPolicy);
+ methodNameToPolicyMap.put(
+ "getObjectMetadata", methodPolicy);
+ methodNameToPolicyMap.put(
+ "getObject", methodPolicy);
+ methodNameToPolicyMap.put(
+ "listObjects", methodPolicy);
+ methodNameToPolicyMap.put(
+ "uploadPart", methodPolicy);
+ methodNameToPolicyMap.put(
+ "deleteObject", methodPolicy);
+ methodNameToPolicyMap.put(
+ "deleteDirectory", methodPolicy);
+ methodNameToPolicyMap.put(
+ "deleteMultipleObjects", methodPolicy);
+ methodNameToPolicyMap.put(
+ "copyObject", methodPolicy);
+ methodNameToPolicyMap.put(
+ "renameObject", methodPolicy);
+ methodNameToPolicyMap.put(
+ "completeMultipartUpload", methodPolicy);
+ methodNameToPolicyMap.put(
+ "initiateMultipartUpload", methodPolicy);
+ methodNameToPolicyMap.put(
+ "isHierarchyBucket", methodPolicy);
+
+ return (BosClientProxy) RetryProxy.create(
+ BosClientProxy.class,
+ new DefaultFailoverProxyProvider<>(
+ BosClientProxy.class, bosClientProxy),
+ methodNameToPolicyMap,
+ new BosTryOnceThenFail());
+ }
+
+ /**
+ * Initializes this store, including setting up thread
+ * pools.
+ *
+ * @throws IOException if initialization fails
+ */
+ public void initialize() throws IOException {
+ initThreadPools();
+ }
+
+ /**
+ * Initializes the bounded thread pool used for parallel
+ * operations.
+ */
+ private void initThreadPools() {
+ int activeTasks = conf.getInt(
+ BaiduBosConstants.BOS_THREADS_MAX_NUM,
+ BaiduBosConstants.BOS_THREADS_MAX_NUM_DEFAULT);
+ int waitingTasks = conf.getInt(
+ BaiduBosConstants.BOS_THREADS_MAX_NUM,
+ BaiduBosConstants.BOS_THREADS_MAX_NUM_DEFAULT)
+ * 2;
+ boundedThreadPool =
+ BlockingThreadPoolExecutorService.newInstance(
+ activeTasks,
+ waitingTasks,
+ 60, TimeUnit.SECONDS,
+ this.bucketName + "-pool");
+ }
+
+ /**
+ * Creates a BosNativeFileSystemStore for the given URI.
+ * Determines whether the bucket is hierarchy-enabled and
+ * returns the appropriate implementation.
+ *
+ * @param uri the URI of the BOS filesystem
+ * @param conf the Hadoop configuration
+ * @return a new store instance
+ * @throws IOException if creation fails
+ */
+ public static BosNativeFileSystemStore createStore(
+ URI uri, Configuration conf) throws IOException {
+ BosClientProxy bosClientProxy =
+ createDefaultBosClientProxy(conf);
+ bosClientProxy.init(uri, conf);
+ String bucketName = uri.getHost();
+ boolean isHierarchy;
+ String hierarchyConf = conf.get(
+ BaiduBosConstants.BOS_BUCKET_HIERARCHY, "null");
+ if (hierarchyConf.equals("true")) {
+ isHierarchy = true;
+ } else if (hierarchyConf.equals("false")) {
+ isHierarchy = false;
+ } else {
+ isHierarchy =
+ bosClientProxy.isHierarchyBucket(bucketName);
+ }
+ BosNativeFileSystemStore store = null;
+ if (isHierarchy) {
+ store = new HierarchyBosNativeFileSystemStore(
+ bosClientProxy, bucketName, conf);
+ } else {
+ store = new NonHierarchyBosNativeFileSystemStore(
+ bosClientProxy, bucketName, conf);
+ }
+ store.initialize();
+ return store;
+ }
+
+ /**
+ * Closes the BOS client and releases resources.
+ */
+ public synchronized void close() {
+ try {
+ if (boundedThreadPool != null) {
+ boundedThreadPool.shutdownNow();
+ }
+ } catch (Exception e) {
+ log.error("thread pool shutdown fail:", e);
+ }
+ try {
+ if (bosClientProxy != null) {
+ bosClientProxy.close();
+ }
+ } catch (Exception e) {
+ log.error("bos client close fail:", e);
+ }
+ }
+
+ /**
+ * Stores an empty file (zero-length object) in BOS using
+ * the environment user and group names.
+ *
+ * @param key the object key
+ * @throws IOException if the operation fails
+ */
+ public void storeEmptyFile(String key)
+ throws IOException {
+ storeEmptyFile(
+ key, this.getEnvUserName(),
+ this.getEnvGroupName());
+ }
+
+ /**
+ * Stores an empty file (zero-length object) in BOS with
+ * specified user and group names.
+ *
+ * @param key the object key
+ * @param userName the user name metadata
+ * @param groupName the group name metadata
+ * @throws IOException if the operation fails
+ */
+ public void storeEmptyFile(
+ String key, String userName, String groupName)
+ throws IOException {
+ ObjectMetadata objectMeta = new ObjectMetadata();
+ objectMeta.addUserMetadata(
+ BaiduBosConstants.BOS_FILE_PATH_USER_KEY,
+ userName);
+ objectMeta.addUserMetadata(
+ BaiduBosConstants.BOS_FILE_PATH_GROUP_KEY,
+ groupName);
+ bosClientProxy.putEmptyObject(
+ bucketName, key, objectMeta);
+ }
+
+ /**
+ * Creates an output stream for writing a file to BOS.
+ *
+ * @param key the object key
+ * @param hadoopConf the Hadoop configuration
+ * @return an output stream for writing the file
+ */
+ public OutputStream createFile(
+ String key, Configuration hadoopConf) {
+ return new BosOutputStream(
+ this, this.bucketName, key, hadoopConf);
+ }
+
+ /**
+ * Gets the file checksum for the specified key.
+ *
+ * @param key the object key
+ * @return the file checksum, or null if unavailable
+ * @throws IOException if the key does not exist or an I/O
+ * error occurs
+ */
+ public FileChecksum getFileChecksum(String key)
+ throws IOException {
+ try {
+ ObjectMetadata metadata =
+ bosClientProxy.getObjectMetadata(
+ bucketName, key);
+ if (metadata.getxBceCrc32c() == null) {
+ return null;
+ }
+ return new BosCRC32CCheckSum(
+ metadata.getxBceCrc32c());
+ } catch (FileNotFoundException e) {
+ throw new FileNotFoundException(
+ key + ": No such file or directory.");
+ }
+ }
+
+ /**
+ * Checks whether the given key represents a directory.
+ *
+ * @param key the object key
+ * @return true if the key is a directory, false if it is a
+ * file or does not exist
+ * @throws IOException if an I/O error occurs
+ */
+ public abstract boolean isDirectory(String key)
+ throws IOException;
+
+ /**
+ * Retrieves the metadata for the specified key.
+ *
+ * @param key the object key
+ * @return the file metadata
+ * @throws IOException if the key is not found or an I/O
+ * error occurs
+ */
+ public abstract FileMetadata retrieveMetadata(
+ String key) throws IOException;
+
+ /**
+ * Retrieves a range of bytes from the object at the given
+ * key.
+ *
+ * @param key the object key
+ * @param byteRangeStart the start of the byte range
+ * @param byteRangeEnd the end of the byte range
+ * @param meta the file metadata for verification
+ * @return an input stream for reading the range, or null
+ * if the key is not found
+ * @throws IOException if an I/O error occurs
+ */
+ public InputStream retrieve(
+ String key, long byteRangeStart,
+ long byteRangeEnd, FileMetadata meta)
+ throws IOException {
+ log.debug(
+ "Getting key: {} from bucket: {} with"
+ + " byteRangeStart: {} and"
+ + " byteRangeEnd: {}.",
+ key, bucketName, byteRangeStart, byteRangeEnd);
+ if (meta == null || !meta.getKey().equals(key)) {
+ throw new IOException(
+ "FileMetadata not match key:" + key);
+ }
+ GetObjectRequest request =
+ new GetObjectRequest(bucketName, key);
+ // Due to the InvalidRange exception of bos, we
+ // shouldn't set range for empty file
+ if (meta.getLength() != 0) {
+ request.setRange(byteRangeStart, byteRangeEnd);
+ }
+ BosObject object =
+ bosClientProxy.getObject(request);
+ return object.getObjectContent();
+ }
+
+ /**
+ * Converts a prefix to a directory-style key by appending
+ * a trailing delimiter if not already present.
+ *
+ * @param prefix the prefix to convert
+ * @return the directory-style key
+ */
+ public String prefixToDir(String prefix) {
+ // !prefix.isEmpty() must not be removed since ""
+ // is root dir.
+ if (prefix != null && !prefix.isEmpty()
+ && !prefix.endsWith(
+ BaiduBosFileSystem.PATH_DELIMITER)) {
+ prefix += BaiduBosFileSystem.PATH_DELIMITER;
+ }
+
+ return prefix;
+ }
+
+ /**
+ * Lists objects in BOS with the default path delimiter.
+ *
+ * @param prefix the prefix to filter by
+ * @param maxListingLength the maximum number of results
+ * @param priorLastKey the marker for pagination
+ * @return the partial listing result
+ * @throws IOException if an I/O error occurs
+ */
+ public PartialListing list(
+ String prefix, int maxListingLength,
+ String priorLastKey) throws IOException {
+ return list(
+ prefix, BaiduBosFileSystem.PATH_DELIMITER,
+ maxListingLength, priorLastKey);
+ }
+
+ /**
+ * Lists objects in BOS with a specified delimiter.
+ *
+ * @param prefix the prefix to filter by
+ * @param maxListingLength the maximum number of results
+ * @param priorLastKey the marker for pagination
+ * @param delimiter the delimiter for grouping
+ * @return the partial listing result
+ * @throws IOException if an I/O error occurs
+ */
+ public PartialListing list(
+ String prefix, int maxListingLength,
+ String priorLastKey, String delimiter)
+ throws IOException {
+ if (this.isHierarchy()) {
+ delimiter = BaiduBosFileSystem.PATH_DELIMITER;
+ }
+ return list(
+ prefix, delimiter,
+ maxListingLength, priorLastKey);
+ }
+
+ /**
+ * Returns whether this store is backed by a hierarchy
+ * (namespace) bucket.
+ *
+ * @return true if the bucket is a hierarchy bucket
+ */
+ protected abstract boolean isHierarchy();
+
+ /**
+ * Lists objects in BOS.
+ *
+ * @param prefix the prefix to filter by
+ * @param delimiter the delimiter for grouping
+ * @param maxListingLength the maximum number of results
+ * @param priorLastKey the marker for pagination
+ * @return the partial listing result, or null if the list
+ * could not be populated
+ * @throws IOException if an I/O error occurs
+ */
+ private PartialListing list(
+ String prefix, String delimiter,
+ int maxListingLength, String priorLastKey)
+ throws IOException {
+ ListObjectsRequest request =
+ new ListObjectsRequest(bucketName);
+ request.setPrefix(prefixToDir(prefix));
+ request.setMarker(priorLastKey);
+ if (delimiter != null) {
+ request.setDelimiter(delimiter);
+ }
+ request.setMaxKeys(maxListingLength);
+
+ ListObjectsResponse objects =
+ bosClientProxy.listObjects(request);
+
+ List fileMetadata =
+ new LinkedList();
+ for (int i = 0;
+ i < objects.getContents().size(); i++) {
+ BosObjectSummary object =
+ objects.getContents().get(i);
+ if (object.getKey() != null
+ && !object.getKey().isEmpty()) {
+ long lastMod =
+ object.getLastModified() != null
+ ? object.getLastModified().getTime()
+ : 0L;
+ String userMeta =
+ object.getUserMeta() == null
+ ? null
+ : object.getUserMeta().get(
+ BaiduBosConstants
+ .BOS_FILE_PATH_USER_KEY);
+ String groupMeta =
+ object.getUserMeta() == null
+ ? null
+ : object.getUserMeta().get(
+ BaiduBosConstants
+ .BOS_FILE_PATH_GROUP_KEY);
+ fileMetadata.add(new FileMetadata(
+ object.getKey(),
+ object.getSize(),
+ lastMod,
+ false,
+ userMeta,
+ groupMeta,
+ null,
+ false
+ ));
+ }
+ }
+
+ List commonPrefix =
+ new LinkedList();
+ if (objects.getCommonPrefixesWithExtMeta() != null) {
+ for (int i = 0;
+ i < objects.getCommonPrefixesWithExtMeta()
+ .size();
+ i++) {
+ BosPrefixInfo bosPrefixInfo =
+ objects.getCommonPrefixesWithExtMeta()
+ .get(i);
+ long lastMod =
+ bosPrefixInfo.getLastModified() == null
+ ? 0L
+ : bosPrefixInfo.getLastModified()
+ .getTime();
+ String userMeta =
+ bosPrefixInfo.getUserMeta() == null
+ ? null
+ : bosPrefixInfo.getUserMeta().get(
+ BaiduBosConstants
+ .BOS_FILE_PATH_USER_KEY);
+ String groupMeta =
+ bosPrefixInfo.getUserMeta() == null
+ ? null
+ : bosPrefixInfo.getUserMeta().get(
+ BaiduBosConstants
+ .BOS_FILE_PATH_GROUP_KEY);
+ commonPrefix.add(new FileMetadata(
+ bosPrefixInfo.getPrefix(),
+ 0L,
+ lastMod,
+ true,
+ userMeta,
+ groupMeta,
+ null,
+ false
+ ));
+ }
+ }
+
+ return new PartialListing(
+ objects.getNextMarker(),
+ fileMetadata.toArray(
+ new FileMetadata[fileMetadata.size()]),
+ commonPrefix.toArray(
+ new FileMetadata[commonPrefix.size()]));
+ }
+
+ /**
+ * Deletes the directory at the given key.
+ *
+ * @param key the directory key
+ * @param recursive whether to delete recursively
+ * @throws IOException if the deletion fails
+ */
+ public abstract void deleteDirs(
+ String key, boolean recursive) throws IOException;
+
+ /**
+ * Deletes the object at the given key.
+ *
+ * @param key the object key
+ * @throws IOException if the deletion fails
+ */
+ public void delete(String key) throws IOException {
+ log.debug(
+ "Deleting key: {} from bucket {}",
+ key, bucketName);
+ bosClientProxy.deleteObject(bucketName, key);
+ }
+
+ /**
+ * Copies an object from source key to destination key
+ * within the same bucket.
+ *
+ * @param srcKey the source object key
+ * @param dstKey the destination object key
+ * @throws IOException if the copy fails
+ */
+ public void copy(String srcKey, String dstKey)
+ throws IOException {
+ log.debug(
+ "Copying srcKey: {} to dstKey: {}"
+ + " in bucket: {}",
+ srcKey, dstKey, bucketName);
+ bosClientProxy.copyObject(
+ bucketName, srcKey, bucketName, dstKey);
+ }
+
+ /**
+ * Renames an object from source key to destination key.
+ *
+ * @param srcKey the source object key
+ * @param dstKey the destination object key
+ * @param isFile whether the source key is a file
+ * @throws IOException if the rename fails
+ */
+ public abstract void rename(
+ String srcKey, String dstKey,
+ boolean isFile) throws IOException;
+
+ /**
+ * Returns the BOS client proxy.
+ *
+ * @return the BOS client proxy
+ */
+ protected BosClientProxy getBosClientProxy() {
+ return bosClientProxy;
+ }
+
+ /**
+ * Returns the bucket name.
+ *
+ * @return the bucket name
+ */
+ protected String getBucketName() {
+ return bucketName;
+ }
+
+ /**
+ * Returns the Hadoop configuration.
+ *
+ * @return the configuration
+ */
+ protected Configuration getConf() {
+ return conf;
+ }
+
+ /**
+ * Returns the bounded thread pool.
+ *
+ * @return the executor service
+ */
+ protected ExecutorService getBoundedThreadPool() {
+ return boundedThreadPool;
+ }
+
+ /**
+ * Returns the current user name from the environment.
+ * Falls back to the system property if UGI lookup fails.
+ *
+ * @return the user name
+ */
+ public String getEnvUserName() {
+ if (this.envUserName != null) {
+ return this.envUserName;
+ }
+ try {
+ this.envUserName =
+ UserGroupInformation.getCurrentUser()
+ .getShortUserName();
+ } catch (IOException ex) {
+ log.error(
+ "get user fail,fallback to system"
+ + " property user.name");
+ this.envUserName =
+ System.getProperty("user.name");
+ }
+ return this.envUserName;
+ }
+
+ /**
+ * Returns the current group name from the environment.
+ * Falls back to the user name if group lookup fails.
+ *
+ * @return the group name
+ */
+ public String getEnvGroupName() {
+ if (this.envGroupName != null) {
+ return this.envGroupName;
+ }
+ try {
+ String[] groupNames =
+ UserGroupInformation.getCurrentUser()
+ .getGroupNames();
+ if (groupNames.length != 0) {
+ this.envGroupName = groupNames[0];
+ } else {
+ this.envGroupName = getEnvUserName();
+ }
+ } catch (IOException e) {
+ log.warn(
+ "get current user failed!"
+ + " use user.name prop {}",
+ e.getMessage());
+ this.envGroupName =
+ System.getProperty("user.name");
+ }
+ return this.envGroupName;
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosOutputStream.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosOutputStream.java
new file mode 100644
index 0000000000000..2f27a8391ae2e
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosOutputStream.java
@@ -0,0 +1,496 @@
+/*
+ * 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.bos;
+
+import com.baidubce.services.bos.model.CompleteMultipartUploadRequest;
+import com.baidubce.services.bos.model.InitiateMultipartUploadRequest;
+import com.baidubce.services.bos.model.InitiateMultipartUploadResponse;
+import com.baidubce.services.bos.model.ObjectMetadata;
+import com.baidubce.services.bos.model.PartETag;
+import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Future;
+
+/**
+ * Output stream implementation for writing data to BOS using
+ * multipart uploads. Data is buffered locally and uploaded in
+ * parts concurrently.
+ */
+public class BosOutputStream extends OutputStream {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(BosOutputStream.class);
+
+ private final BosNativeFileSystemStore store;
+ private final Map blocksMap =
+ new HashMap<>();
+ private final String bucketName;
+ private final BosClientProxy bosClientProxy;
+ private final String key;
+ private final Configuration conf;
+ private final int blockSize;
+ private final int uploadThreadSize;
+ private final List eTags =
+ Collections.synchronizedList(new ArrayList<>());
+ private final List futureList = new LinkedList<>();
+ private final Map exceptionMap =
+ new ConcurrentHashMap<>();
+ private final ArrayBlockingQueue blocks;
+
+ private BosBlockBuffer currBlock;
+ private int blkIndex = 1;
+ private boolean closed;
+ private long filePos = 0;
+ private int bytesWrittenToBlock = 0;
+ private String uploadId = null;
+ private int createBlockSize;
+
+ /**
+ * Constructs a BosOutputStream for writing to BOS.
+ *
+ * @param store the native file system store
+ * @param bucketName the name of the bucket
+ * @param key the object key
+ * @param conf the Hadoop configuration
+ */
+ public BosOutputStream(
+ BosNativeFileSystemStore store,
+ String bucketName, String key,
+ Configuration conf) {
+ this.store = store;
+ this.bosClientProxy = store.getBosClientProxy();
+ this.bucketName = bucketName;
+ this.key = key;
+ this.conf = conf;
+ this.blockSize = this.conf.getInt(
+ BaiduBosConstants.BOS_MULTI_UPLOAD_BLOCK_SIZE,
+ BaiduBosConstants
+ .BOS_MULTI_UPLOAD_BLOCK_SIZE_DEFAULT);
+ this.uploadThreadSize = this.conf.getInt(
+ BaiduBosConstants.BOS_MULTI_UPLOAD_CONCURRENT_SIZE,
+ BaiduBosConstants
+ .DEFAULT_MULTI_UPLOAD_CONCURRENT_SIZE);
+ this.blocks =
+ new ArrayBlockingQueue<>(uploadThreadSize);
+ createBlockSize = 0;
+ }
+
+ /**
+ * Creates a new block buffer if the current one is null.
+ *
+ * @throws IOException if allocation fails
+ */
+ private void createBlockBufferIfNull()
+ throws IOException {
+ if (this.currBlock == null) {
+ LOG.debug(
+ "Ready to get Block Buffer !"
+ + " key is {}. blkIndex is {}."
+ + " blockSize is {}",
+ this.key, this.blkIndex, this.blockSize);
+ try {
+ if (createBlockSize < this.uploadThreadSize) {
+ this.currBlock = new BosBlockBuffer(
+ this.key, this.blkIndex, blockSize);
+ ++createBlockSize;
+ LOG.debug(
+ "create {}th BosBlockBuffer",
+ createBlockSize);
+ } else {
+ this.currBlock = this.blocks.take();
+ this.currBlock.setBlkId(this.blkIndex);
+ LOG.debug(
+ "get free block: {}", this.currBlock);
+ }
+ LOG.debug(
+ "Block Buffer get !"
+ + " key is {}. blkIndex is {}."
+ + " blockSize is {}",
+ this.key, this.blkIndex,
+ this.blockSize);
+ this.bytesWrittenToBlock = 0;
+ this.blkIndex++;
+ } catch (Throwable throwable) {
+ LOG.error(
+ "catch exception when allocating"
+ + " BosBlockBuffer: ",
+ throwable);
+ throw new IOException(
+ "catch exception when allocating"
+ + " BosBlockBuffer: ",
+ throwable);
+ }
+ }
+ }
+
+ /**
+ * Writes a single byte to this output stream.
+ *
+ * @param b the byte to write
+ * @throws IOException if the stream is closed or an I/O
+ * error occurs
+ */
+ @Override
+ public synchronized void write(int b)
+ throws IOException {
+ if (this.closed) {
+ throw new IOException("Stream closed");
+ }
+
+ flush();
+ createBlockBufferIfNull();
+
+ this.currBlock.getOutBuffer().write(b);
+ this.bytesWrittenToBlock++;
+ this.filePos++;
+ }
+
+ /**
+ * Writes bytes from the specified buffer to this output
+ * stream.
+ *
+ * @param b the data buffer
+ * @param off the start offset in the data
+ * @param len the number of bytes to write
+ * @throws IOException if the stream is closed or an I/O
+ * error occurs
+ */
+ @Override
+ public synchronized void write(
+ byte[] b, int off, int len) throws IOException {
+ if (this.closed) {
+ throw new IOException("Stream closed");
+ }
+
+ while (len > 0) {
+ flush();
+ createBlockBufferIfNull();
+
+ int remaining =
+ this.blockSize - this.bytesWrittenToBlock;
+ int toWrite = Math.min(remaining, len);
+ this.currBlock.getOutBuffer().write(b, off, toWrite);
+ this.bytesWrittenToBlock += toWrite;
+ this.filePos += toWrite;
+ off += toWrite;
+ len -= toWrite;
+ }
+ }
+
+ /**
+ * Flushes this output stream. If the current block is full,
+ * it will be uploaded as a part. Note: flush does not
+ * guarantee data durability; it cannot be used for HBase
+ * WAL recovery.
+ *
+ * @throws IOException if the stream is closed or an I/O
+ * error occurs
+ */
+ @Override
+ public synchronized void flush() throws IOException {
+ if (this.closed) {
+ throw new IOException("Stream closed");
+ }
+
+ if (this.bytesWrittenToBlock >= this.blockSize) {
+ endBlock();
+ }
+ }
+
+ /**
+ * Ends the current block by uploading it as a multipart
+ * upload part.
+ *
+ * @throws IOException if an upload error occurs
+ */
+ private synchronized void endBlock() throws IOException {
+ if (this.currBlock == null) {
+ return;
+ }
+
+ LOG.debug(
+ "Enter endBlock() ! key is {}."
+ + " blkIndex is {}. blockSize is {}."
+ + " Size of eTags is {}."
+ + " concurrentUpload is {}",
+ this.key, this.blkIndex, this.blockSize,
+ this.eTags.size(), this.uploadThreadSize);
+
+ if (this.currBlock.getBlkId() == 1) {
+ this.uploadId = initMultipartUpload(this.key);
+ LOG.debug(
+ "init multipart upload!"
+ + " key is {} upload id is {}",
+ this.key, this.uploadId);
+ }
+
+ //
+ // Move outBuffer to inBuffer
+ //
+ this.currBlock.getOutBuffer().close();
+ this.currBlock.moveData();
+
+ //
+ // Block this when too many active UploadPartThread
+ //
+ while ((this.blkIndex - this.eTags.size())
+ > this.uploadThreadSize) {
+ synchronized (this.blocksMap) {
+ try {
+ this.blocksMap.wait(10);
+ } catch (InterruptedException e) {
+ // ignore
+ }
+ }
+ if (this.exceptionMap.size() > 0) {
+ //
+ // Exception happens during upload
+ //
+ throw new IOException(
+ "Exception happens during upload:"
+ + exceptionMap);
+ }
+ }
+
+ if (this.exceptionMap.size() > 0) {
+ //
+ // Exception happens during upload
+ //
+ throw new IOException(
+ "Exception happens during upload:"
+ + exceptionMap);
+ }
+
+ synchronized (this.blocksMap) {
+ this.blocksMap.put(
+ this.currBlock.getBlkId(), this.currBlock);
+ }
+
+ UploadPartThread upThread = new UploadPartThread(
+ bosClientProxy,
+ bucketName,
+ this.key,
+ this.currBlock,
+ this.uploadId,
+ this.eTags,
+ this.blocksMap,
+ this.exceptionMap,
+ this.conf,
+ this.blocks
+ );
+ this.futureList.add(
+ store.getBoundedThreadPool().submit(upThread));
+
+ // Clear current BosBlockBuffer
+ this.currBlock = null;
+ }
+
+ /**
+ * Puts the current block as a single object (used when
+ * only one block has been written).
+ *
+ * @throws IOException if the put operation fails
+ */
+ private synchronized void putObject()
+ throws IOException {
+ if (this.currBlock == null) {
+ return;
+ }
+
+ // Move outBuffer to inBuffer
+ //
+ this.currBlock.getOutBuffer().close();
+ this.currBlock.moveData();
+
+ ObjectMetadata objectMeta = new ObjectMetadata();
+ objectMeta.setContentLength(
+ this.currBlock.getInBuffer().getLength());
+ objectMeta.addUserMetadata(
+ BaiduBosConstants.BOS_FILE_PATH_USER_KEY,
+ store.getEnvUserName());
+ objectMeta.addUserMetadata(
+ BaiduBosConstants.BOS_FILE_PATH_GROUP_KEY,
+ store.getEnvGroupName());
+ bosClientProxy.putObject(
+ bucketName, this.key,
+ this.currBlock.getInBuffer(), objectMeta);
+ this.currBlock = null;
+ }
+
+ /**
+ * Closes this output stream, completing the multipart
+ * upload if necessary.
+ *
+ * @throws IOException if an I/O error occurs during close
+ */
+ @Override
+ public synchronized void close() throws IOException {
+ if (this.closed) {
+ return;
+ }
+
+ if (this.filePos == 0) {
+ ObjectMetadata objectMeta = new ObjectMetadata();
+ objectMeta.addUserMetadata(
+ BaiduBosConstants.BOS_FILE_PATH_USER_KEY,
+ store.getEnvUserName());
+ objectMeta.addUserMetadata(
+ BaiduBosConstants.BOS_FILE_PATH_GROUP_KEY,
+ store.getEnvGroupName());
+ bosClientProxy.putEmptyObject(
+ bucketName, key, objectMeta);
+ } else if (null != this.currBlock
+ && this.currBlock.getBlkId() == 1) {
+ putObject();
+ } else {
+ if (this.bytesWrittenToBlock != 0) {
+ endBlock();
+ }
+
+ LOG.debug(
+ "start to wait upload part threads done."
+ + " futureList size: {}",
+ futureList.size());
+ // wait UploadPartThread threads done
+ Exception uploadException = null;
+ try {
+ int index = 0;
+ for (Future future : this.futureList) {
+ future.get();
+ index += 1;
+ LOG.debug(
+ "future.get() index: {} is done",
+ index);
+ }
+ } catch (Exception e) {
+ uploadException = e;
+ LOG.warn(
+ "catch exception when waiting"
+ + " UploadPartThread done: ",
+ e);
+ }
+
+ if (uploadException != null) {
+ abortMultipartUpload();
+ throw new IOException(
+ "Multipart upload failed", uploadException);
+ }
+
+ LOG.debug(
+ "success to wait upload part threads done");
+
+ LOG.debug(
+ "Size of eTags is {}. blkIndex is {}",
+ this.eTags.size(), this.blkIndex);
+
+ if (this.eTags.size() != this.blkIndex - 1) {
+ abortMultipartUpload();
+ throw new IOException(
+ "Multipart upload incomplete: expected "
+ + (this.blkIndex - 1) + " parts but got "
+ + this.eTags.size());
+ }
+
+ completeMultipartUpload(
+ bucketName, this.key,
+ this.uploadId, this.eTags);
+ }
+
+ super.close();
+ this.closed = true;
+ }
+
+ /**
+ * Aborts the in-progress multipart upload to prevent
+ * storage leakage on failure.
+ */
+ private void abortMultipartUpload() {
+ if (this.uploadId != null) {
+ try {
+ bosClientProxy.abortMultipartUpload(
+ bucketName, this.key, this.uploadId);
+ } catch (IOException e) {
+ LOG.warn("Failed to abort multipart upload"
+ + " for key={}, uploadId={}",
+ this.key, this.uploadId, e);
+ }
+ }
+ }
+
+ /**
+ * Initiates a multipart upload for the given key.
+ *
+ * @param objectKey the object key
+ * @return the upload ID
+ * @throws IOException if the initiation fails
+ */
+ private String initMultipartUpload(String objectKey)
+ throws IOException {
+ InitiateMultipartUploadRequest request =
+ new InitiateMultipartUploadRequest(
+ bucketName, objectKey);
+
+ InitiateMultipartUploadResponse result =
+ bosClientProxy.initiateMultipartUpload(request);
+ return result.getUploadId();
+ }
+
+ /**
+ * Completes a multipart upload by assembling previously
+ * uploaded parts.
+ *
+ * @param bucket the bucket name
+ * @param objectKey the object key
+ * @param mpUploadId the upload ID
+ * @param partETags the list of part ETags
+ * @throws IOException if the completion fails
+ */
+ private void completeMultipartUpload(
+ String bucket, String objectKey,
+ String mpUploadId, List partETags)
+ throws IOException {
+ Collections.sort(partETags,
+ Comparator.comparingInt(PartETag::getPartNumber));
+ ObjectMetadata objectMeta = new ObjectMetadata();
+ objectMeta.addUserMetadata(
+ BaiduBosConstants.BOS_FILE_PATH_USER_KEY,
+ store.getEnvUserName());
+ objectMeta.addUserMetadata(
+ BaiduBosConstants.BOS_FILE_PATH_GROUP_KEY,
+ store.getEnvGroupName());
+ CompleteMultipartUploadRequest request =
+ new CompleteMultipartUploadRequest(
+ bucket, objectKey, mpUploadId,
+ partETags, objectMeta);
+ bosClientProxy.completeMultipartUpload(request);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosRandomRetryPolicy.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosRandomRetryPolicy.java
new file mode 100644
index 0000000000000..2f43cc0db6bcb
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosRandomRetryPolicy.java
@@ -0,0 +1,107 @@
+/*
+ * 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.bos;
+
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A retry policy that uses random sleep times between a
+ * configured minimum and maximum, with sleep duration
+ * increasing proportionally to the number of retries.
+ */
+public class BosRandomRetryPolicy implements RetryPolicy {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(
+ BosRandomRetryPolicy.class);
+
+ private int maxRetries;
+ private TimeUnit timeUnit;
+ private int minSleep;
+ private int maxSleep;
+
+ /**
+ * Constructs a BosRandomRetryPolicy.
+ *
+ * @param maxRetries the maximum number of retries
+ * @param minSleep the minimum sleep time
+ * @param maxSleep the maximum sleep time
+ * @param timeUnit the time unit for sleep values
+ * @throws IllegalArgumentException if minSleep is negative
+ * or maxSleep is not greater than minSleep
+ */
+ public BosRandomRetryPolicy(
+ int maxRetries, int minSleep,
+ int maxSleep, TimeUnit timeUnit) {
+ if (minSleep < 0 || maxSleep <= minSleep) {
+ throw new IllegalArgumentException(
+ "min sleep time must be positive number and"
+ + " max sleep time must be greater than"
+ + " min sleep time !");
+ }
+ this.maxRetries = maxRetries;
+ this.minSleep = minSleep;
+ this.maxSleep = maxSleep;
+ this.timeUnit = timeUnit;
+ }
+
+ /**
+ * Determines whether the operation should be retried.
+ *
+ * @param e the exception that triggered the retry
+ * @param retries the number of retries so far
+ * @param failovers the number of failovers so far
+ * @param isIdempotentOrAtMostOnce whether the operation is
+ * idempotent or at-most-once
+ * @return the retry action indicating whether to retry or
+ * fail
+ */
+ @Override
+ public RetryAction shouldRetry(
+ Exception e, int retries, int failovers,
+ boolean isIdempotentOrAtMostOnce) {
+ if (retries >= this.maxRetries) {
+ return RetryAction.FAIL;
+ }
+ long sleepMillis = this.timeUnit.toMillis(
+ this.calculateSleepTime(retries));
+ return new RetryAction(
+ RetryAction.RetryDecision.RETRY,
+ sleepMillis);
+ }
+
+ /**
+ * Calculates the sleep time for the given retry attempt.
+ *
+ * @param retries the current retry count
+ * @return the sleep time in the configured time unit
+ */
+ public long calculateSleepTime(int retries) {
+ int sleepTime = minSleep
+ + ThreadLocalRandom.current()
+ .nextInt(maxSleep - minSleep);
+ LOG.warn("sleep random time : {}", sleepTime);
+ return sleepTime * (retries + 1);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosTryOnceThenFail.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosTryOnceThenFail.java
new file mode 100644
index 0000000000000..620ff4440f0e5
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/BosTryOnceThenFail.java
@@ -0,0 +1,54 @@
+/*
+ * 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.bos;
+
+import org.apache.hadoop.io.retry.RetryPolicy;
+
+/**
+ * A retry policy that always fails on the first attempt.
+ * Used as the default policy for methods not explicitly
+ * configured with a retry policy.
+ */
+public class BosTryOnceThenFail implements RetryPolicy {
+
+ /**
+ * Constructs a BosTryOnceThenFail retry policy.
+ */
+ public BosTryOnceThenFail() {
+ }
+
+ /**
+ * Always returns a FAIL action, indicating no retry.
+ *
+ * @param e the exception that caused the failure
+ * @param retries the number of retries attempted
+ * @param failovers the number of failovers attempted
+ * @param isIdempotentOrAtMostOnce whether the operation
+ * is idempotent or at-most-once
+ * @return a retry action indicating failure
+ * @throws Exception if an error occurs
+ */
+ public RetryAction shouldRetry(
+ Exception e, int retries, int failovers,
+ boolean isIdempotentOrAtMostOnce)
+ throws Exception {
+ return new RetryAction(
+ RetryAction.RetryDecision.FAIL);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/FileMetadata.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/FileMetadata.java
new file mode 100644
index 0000000000000..6e64c94ac4502
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/FileMetadata.java
@@ -0,0 +1,217 @@
+/*
+ * 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.bos;
+
+import com.baidubce.services.bos.model.ObjectMetadata;
+import org.apache.hadoop.fs.FileStatus;
+
+import java.util.Set;
+
+/**
+ * Holds basic metadata for a file stored in a
+ * {@link BosNativeFileSystemStore}.
+ */
+public class FileMetadata {
+
+ private final String key;
+ private final long length;
+ private final long lastModified;
+ private final boolean isFolder;
+ private final String userName;
+ private final String groupName;
+ private final String permission;
+ private final Set attr;
+ private ObjectMetadata objectMetadata = null;
+
+ /**
+ * Constructs a FileMetadata with minimal fields.
+ *
+ * @param key the object key
+ * @param length the content length in bytes
+ * @param lastModified the last modified timestamp
+ * @param isFolder whether this entry is a folder
+ */
+ public FileMetadata(
+ String key, long length,
+ long lastModified, boolean isFolder) {
+ this(key, length, lastModified, isFolder,
+ null, null, null, false);
+ }
+
+ /**
+ * Constructs a FileMetadata with user, group, and
+ * permission fields.
+ *
+ * @param key the object key
+ * @param length the content length in bytes
+ * @param lastModified the last modified timestamp
+ * @param isFolder whether this entry is a folder
+ * @param userName the owner user name
+ * @param groupName the owner group name
+ * @param permission the permission string
+ */
+ public FileMetadata(
+ String key, long length,
+ long lastModified, boolean isFolder,
+ String userName, String groupName,
+ String permission) {
+ this(key, length, lastModified, isFolder,
+ userName, groupName, permission, false);
+ }
+
+ /**
+ * Constructs a FileMetadata from BOS object metadata.
+ *
+ * @param key the object key
+ * @param length the content length in bytes
+ * @param lastModified the last modified timestamp
+ * @param isFolder whether this entry is a folder
+ * @param objectMetadata the BOS object metadata
+ */
+ public FileMetadata(
+ String key, long length,
+ long lastModified, boolean isFolder,
+ ObjectMetadata objectMetadata) {
+ this(key, length, lastModified, isFolder,
+ objectMetadata.getUserMetaDataOf(
+ BaiduBosConstants.BOS_FILE_PATH_USER_KEY),
+ objectMetadata.getUserMetaDataOf(
+ BaiduBosConstants.BOS_FILE_PATH_GROUP_KEY),
+ null,
+ false);
+ this.objectMetadata = objectMetadata;
+ }
+
+ /**
+ * Constructs a FileMetadata with all fields.
+ *
+ * @param key the object key
+ * @param length the content length in bytes
+ * @param lastModified the last modified timestamp
+ * @param isFolder whether this entry is a folder
+ * @param userName the owner user name
+ * @param groupName the owner group name
+ * @param permission the permission string
+ * @param hasAcl whether the object has an ACL
+ */
+ public FileMetadata(
+ String key, long length,
+ long lastModified, boolean isFolder,
+ String userName, String groupName,
+ String permission, boolean hasAcl) {
+ this.key = key;
+ this.length = length;
+ this.lastModified = lastModified;
+ this.isFolder = isFolder;
+ this.userName = userName;
+ this.groupName = groupName;
+ this.permission = permission;
+ this.attr = FileStatus.attributes(
+ hasAcl, false, false, false);
+ }
+
+ /**
+ * Returns the object key.
+ *
+ * @return the key
+ */
+ public String getKey() {
+ return key;
+ }
+
+ /**
+ * Returns the content length in bytes.
+ *
+ * @return the length
+ */
+ public long getLength() {
+ return length;
+ }
+
+ /**
+ * Returns the last modified timestamp.
+ *
+ * @return the last modified time in milliseconds
+ */
+ public long getLastModified() {
+ return lastModified;
+ }
+
+ /**
+ * Returns whether this entry is a folder.
+ *
+ * @return true if this is a folder
+ */
+ public boolean isFolder() {
+ return isFolder;
+ }
+
+ /**
+ * Returns the owner user name.
+ *
+ * @return the user name
+ */
+ public String getUserName() {
+ return this.userName;
+ }
+
+ /**
+ * Returns the owner group name.
+ *
+ * @return the group name
+ */
+ public String getGroupName() {
+ return this.groupName;
+ }
+
+ /**
+ * Returns the permission string.
+ *
+ * @return the permission
+ */
+ public String getPermission() {
+ return this.permission;
+ }
+
+ /**
+ * Returns the attribute flags.
+ *
+ * @return the set of attribute flags
+ */
+ public Set getAttr() {
+ return attr;
+ }
+
+ /**
+ * Returns the BOS object metadata, if available.
+ *
+ * @return the object metadata, or null
+ */
+ public ObjectMetadata getObjectMetadata() {
+ return objectMetadata;
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public String toString() {
+ return "FileMetadata[" + key + ", " + length
+ + ", " + lastModified + ", " + isFolder + "]";
+ }
+
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/HierarchyBosNativeFileSystemStore.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/HierarchyBosNativeFileSystemStore.java
new file mode 100644
index 0000000000000..0cfd23551ad8a
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/HierarchyBosNativeFileSystemStore.java
@@ -0,0 +1,160 @@
+/*
+ * 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.bos;
+
+import com.baidubce.services.bos.model.DeleteDirectoryResponse;
+import com.baidubce.services.bos.model.ObjectMetadata;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+/**
+ * Store implementation for hierarchy (namespace-enabled)
+ * BOS buckets. Supports native directory operations such as
+ * recursive delete and atomic rename.
+ */
+public class HierarchyBosNativeFileSystemStore
+ extends BosNativeFileSystemStore {
+
+ /**
+ * Constructs a store for a hierarchy bucket.
+ *
+ * @param bosClientProxy the BOS client proxy
+ * @param bucketName the name of the bucket
+ * @param conf the Hadoop configuration
+ */
+ public HierarchyBosNativeFileSystemStore(
+ BosClientProxy bosClientProxy,
+ String bucketName, Configuration conf) {
+ super(bosClientProxy, bucketName, conf);
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * Checks whether the key is a directory by examining the
+ * object type in the metadata.
+ */
+ @Override
+ public boolean isDirectory(String key)
+ throws IOException {
+ try {
+ ObjectMetadata meta =
+ bosClientProxy.getObjectMetadata(
+ bucketName, key);
+ return "Directory".equals(meta.getObjectType());
+ } catch (FileNotFoundException e) {
+ return false;
+ }
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * Retrieves file metadata. For hierarchy buckets, the
+ * object type metadata determines whether the key is a
+ * directory.
+ */
+ @Override
+ public FileMetadata retrieveMetadata(String key)
+ throws IOException {
+ boolean isFolder = false;
+
+ ObjectMetadata meta =
+ bosClientProxy.getObjectMetadata(
+ bucketName, key);
+ if (BaiduBosConstants.BOS_FILE_TYPE_DIRECTORY
+ .equals(meta.getObjectType())) {
+ isFolder = true;
+ }
+ long lastMod =
+ meta.getLastModified() != null
+ ? meta.getLastModified().getTime() : 0L;
+ return new FileMetadata(
+ key, meta.getContentLength(),
+ lastMod, isFolder, meta);
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ protected boolean isHierarchy() {
+ return true;
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * Deletes a directory. For non-recursive deletes, verifies
+ * the directory is empty first.
+ */
+ @Override
+ public void deleteDirs(
+ String key, boolean recursive) throws IOException {
+ if (!recursive) {
+ PartialListing listing =
+ list(key, 1, null, "/");
+ if (listing.getFiles().length
+ + listing.getDirectories().length > 0) {
+ throw new IOException(
+ "Directory " + key + " is not empty.");
+ }
+ delete(key);
+ } else {
+ String marker = null;
+ DeleteDirectoryResponse response = null;
+ do {
+ response = bosClientProxy.deleteDirectory(
+ bucketName, key, true, marker);
+ if (response != null
+ && response.isTruncated()) {
+ marker = response.getNextDeleteMarker();
+ } else {
+ break;
+ }
+ } while (response.isTruncated());
+ }
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * Renames an object using the BOS rename API.
+ */
+ @Override
+ public void rename(
+ String srcKey, String dstKey,
+ final boolean isFile) throws IOException {
+ log.debug(
+ "Renaming srcKey: {} to dstKey: {}"
+ + " in bucket: {}",
+ srcKey, dstKey, bucketName);
+ try {
+ bosClientProxy.renameObject(
+ bucketName, srcKey, dstKey);
+ } catch (FileNotFoundException e) {
+ log.debug(
+ "Renaming srcKey: {} to dstKey: {}"
+ + " in bucket: {} failed,"
+ + " FileNotFoundException",
+ srcKey, dstKey, bucketName);
+ throw e;
+ }
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/NonHierarchyBosNativeFileSystemStore.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/NonHierarchyBosNativeFileSystemStore.java
new file mode 100644
index 0000000000000..d94e470af814d
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/NonHierarchyBosNativeFileSystemStore.java
@@ -0,0 +1,457 @@
+/*
+ * 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.bos;
+
+import com.baidubce.services.bos.model.CompleteMultipartUploadRequest;
+import com.baidubce.services.bos.model.DeleteMultipleObjectsRequest;
+import com.baidubce.services.bos.model.InitiateMultipartUploadRequest;
+import com.baidubce.services.bos.model.InitiateMultipartUploadResponse;
+import com.baidubce.services.bos.model.ObjectMetadata;
+import com.baidubce.services.bos.model.PartETag;
+import com.baidubce.services.bos.model.UploadPartCopyRequest;
+import com.baidubce.services.bos.model.UploadPartCopyResponse;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+/**
+ * Store implementation for non-hierarchy (flat) BOS buckets.
+ * Directories are simulated using key prefixes and delimiter
+ * conventions.
+ */
+public class NonHierarchyBosNativeFileSystemStore
+ extends BosNativeFileSystemStore {
+
+ private final int multipartBlockSize;
+ private final long copyLargeFileThreshold;
+
+ /**
+ * Constructs a store for a non-hierarchy bucket.
+ *
+ * @param bosClientProxy the BOS client proxy
+ * @param bucketName the name of the bucket
+ * @param conf the Hadoop configuration
+ */
+ public NonHierarchyBosNativeFileSystemStore(
+ BosClientProxy bosClientProxy,
+ String bucketName, Configuration conf) {
+ super(bosClientProxy, bucketName, conf);
+ this.multipartBlockSize = conf.getInt(
+ BaiduBosConstants.BOS_MULTI_UPLOAD_BLOCK_SIZE,
+ BaiduBosConstants
+ .BOS_MULTI_UPLOAD_BLOCK_SIZE_DEFAULT);
+ this.copyLargeFileThreshold = conf.getLong(
+ BaiduBosConstants
+ .BOS_COPY_LARGE_FILE_THRESHOLD,
+ BaiduBosConstants
+ .BOS_COPY_LARGE_FILE_THRESHOLD_DEFAULT);
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * Checks whether the key is a directory by looking for
+ * the directory marker or listing sub-objects.
+ */
+ @Override
+ public boolean isDirectory(String key)
+ throws IOException {
+ try {
+ key = prefixToDir(key);
+ bosClientProxy.getObjectMetadata(bucketName, key);
+ return true;
+ } catch (FileNotFoundException ex) {
+ PartialListing listing =
+ list(key, 1, null, null);
+ if (listing != null) {
+ if (listing.getFiles().length > 0) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * Retrieves metadata for the given key. If the key is not
+ * found as-is, tries appending a trailing delimiter. If
+ * sub-objects exist, rebuilds the directory marker.
+ */
+ @Override
+ public FileMetadata retrieveMetadata(String key)
+ throws IOException {
+ boolean isFolder = false;
+ try {
+ ObjectMetadata meta =
+ bosClientProxy.getObjectMetadata(
+ bucketName, key);
+ if (key.endsWith(
+ BaiduBosFileSystem.PATH_DELIMITER)) {
+ isFolder = true;
+ }
+ long lastMod =
+ meta.getLastModified() != null
+ ? meta.getLastModified().getTime() : 0L;
+ return new FileMetadata(
+ key, meta.getContentLength(),
+ lastMod, isFolder, meta);
+ } catch (FileNotFoundException e) {
+ try {
+ key = prefixToDir(key);
+ ObjectMetadata meta =
+ bosClientProxy.getObjectMetadata(
+ bucketName, key);
+ isFolder = true;
+ long lastMod =
+ meta.getLastModified() != null
+ ? meta.getLastModified().getTime()
+ : 0L;
+ return new FileMetadata(
+ key, meta.getContentLength(),
+ lastMod, isFolder, meta);
+ } catch (FileNotFoundException ex) {
+ PartialListing listing =
+ list(key, 1, null, null);
+ if (listing != null) {
+ if (listing.getFiles().length > 0) {
+ // rebuild dir
+ this.storeEmptyFile(key);
+ return new FileMetadata(
+ key, 0, 0, true);
+ }
+ }
+ throw new FileNotFoundException(
+ key + ": No such file or directory.");
+ }
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ protected boolean isHierarchy() {
+ return false;
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * Deletes a directory and optionally its contents by
+ * listing and batch-deleting objects with the given prefix.
+ */
+ @Override
+ public void deleteDirs(
+ String key, boolean recursive) throws IOException {
+ log.debug(
+ "Deleting dirs key:{}, bucket: {},"
+ + " recursive:{}",
+ key, bucketName, recursive);
+ String listKey = key;
+ ArrayList keys = new ArrayList();
+ String priorLastKey = null;
+
+ do {
+ keys.clear();
+ PartialListing listing =
+ list(listKey, 1000, priorLastKey, null);
+ if (listing != null) {
+ DeleteMultipleObjectsRequest deleteRequest =
+ (new DeleteMultipleObjectsRequest())
+ .withBucketName(bucketName);
+ for (FileMetadata fileMetadata
+ : listing.getFiles()) {
+ keys.add(fileMetadata.getKey());
+ }
+ if (!recursive
+ && (keys.size() >= 2
+ || (keys.size() == 1
+ && !keys.get(0).equals(
+ prefixToDir(key))))) {
+ throw new IOException(
+ key + " is a directory.");
+ }
+ if (!keys.isEmpty()) {
+ deleteRequest.setObjectKeys(keys);
+ bosClientProxy.deleteMultipleObjects(
+ deleteRequest);
+ }
+ priorLastKey = listing.getPriorLastKey();
+ }
+ } while (priorLastKey != null
+ && !priorLastKey.isEmpty());
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * Renames an object. For files, delegates to
+ * {@code renameObject}. For directories, collects rename
+ * tasks and executes them in parallel.
+ */
+ @Override
+ public void rename(
+ String srcKey, String dstKey,
+ final boolean isFile) throws IOException {
+ log.debug(
+ "Renaming srcKey: {} to dstKey: {}"
+ + " in bucket: {}",
+ srcKey, dstKey, bucketName);
+ try {
+ if (isFile) {
+ bosClientProxy.renameObject(
+ bucketName, srcKey, dstKey);
+ } else {
+ this.storeEmptyFile(
+ this.prefixToDir(srcKey));
+ List tasks =
+ collectRenameTasks(srcKey, dstKey);
+ runRenameTask(tasks);
+ }
+ } catch (FileNotFoundException e) {
+ log.debug(
+ "Renaming srcKey: {} to dstKey: {}"
+ + " in bucket: {} failed,"
+ + " FileNotFoundException",
+ srcKey, dstKey, bucketName);
+ throw e;
+ }
+ }
+
+ /**
+ * Collects rename tasks for all objects under the source
+ * directory prefix.
+ *
+ * @param src the source directory key
+ * @param des the destination directory key
+ * @return the list of rename tasks
+ * @throws IOException if listing fails
+ */
+ private List collectRenameTasks(
+ String src, String des) throws IOException {
+ List tasks = new ArrayList<>();
+ String priorLastKey = null;
+ String srcKey = this.prefixToDir(src);
+ String dstKey = this.prefixToDir(des);
+ do {
+ PartialListing listing = this.list(
+ prefixToDir(srcKey),
+ 1000, priorLastKey, null);
+ for (FileMetadata file : listing.getFiles()) {
+ String newKey = dstKey
+ + file.getKey().substring(
+ srcKey.length());
+ tasks.add(new RenameTask(
+ this, file.getKey(), newKey));
+ }
+ priorLastKey = listing.getPriorLastKey();
+ } while (priorLastKey != null);
+ return tasks;
+ }
+
+ /**
+ * Runs rename tasks in parallel using the bounded thread
+ * pool.
+ *
+ * @param tasks the list of rename tasks
+ * @throws IOException if any rename operation fails
+ */
+ private void runRenameTask(List tasks)
+ throws IOException {
+ List> futures = new ArrayList<>();
+ // Submit rename tasks for parallel execution
+ Future curFuture = null;
+ try {
+ for (RenameTask task : tasks) {
+ futures.add(
+ this.boundedThreadPool.submit(task));
+ }
+
+ for (Future future : futures) {
+ curFuture = future;
+ curFuture.get();
+ }
+ } catch (InterruptedException
+ | ExecutionException e) {
+ for (Future f : futures) {
+ f.cancel(true);
+ }
+ throw new IOException(e);
+ }
+ }
+
+ /**
+ * Copies a large file using multipart copy.
+ *
+ * @param srcKey the source object key
+ * @param dstKey the destination object key
+ * @param fileSize the size of the file in bytes
+ * @throws IOException if the copy fails
+ */
+ private void copyLargeFile(
+ String srcKey, String dstKey, long fileSize)
+ throws IOException {
+ String copyUploadId =
+ initMultipartUpload(dstKey);
+ long leftSize = fileSize;
+ long skipBytes = 0;
+ int partNumber = 1;
+ List partETags =
+ new ArrayList();
+ long partSize = multipartBlockSize;
+
+ while (leftSize > 0) {
+ if (leftSize < partSize) {
+ partSize = leftSize;
+ }
+ UploadPartCopyRequest copyRequest =
+ new UploadPartCopyRequest();
+ copyRequest.setBucketName(bucketName);
+ copyRequest.setKey(dstKey);
+ copyRequest.setSourceBucketName(bucketName);
+ copyRequest.setSourceKey(srcKey);
+ copyRequest.setUploadId(copyUploadId);
+ copyRequest.setPartSize(partSize);
+ copyRequest.setOffSet(skipBytes);
+ copyRequest.setPartNumber(partNumber);
+ UploadPartCopyResponse copyResponse =
+ bosClientProxy.uploadPartCopy(copyRequest);
+ // Save the returned PartETag to the list
+ PartETag partETag = new PartETag(
+ partNumber, copyResponse.getETag());
+ partETags.add(partETag);
+ leftSize -= partSize;
+ skipBytes += partSize;
+ partNumber += 1;
+ }
+
+ completeMultipartUpload(
+ bucketName, dstKey, copyUploadId, partETags);
+ }
+
+ /**
+ * Initiates a multipart upload for the given key.
+ *
+ * @param key the object key
+ * @return the upload ID
+ * @throws IOException if the initiation fails
+ */
+ private String initMultipartUpload(String key)
+ throws IOException {
+ InitiateMultipartUploadRequest request =
+ new InitiateMultipartUploadRequest(
+ bucketName, key);
+ InitiateMultipartUploadResponse result =
+ bosClientProxy.initiateMultipartUpload(
+ request);
+ return result.getUploadId();
+ }
+
+ /**
+ * Completes a multipart upload by assembling the parts.
+ *
+ * @param bucket the bucket name
+ * @param objectKey the object key
+ * @param mpUploadId the upload ID
+ * @param partETags the list of part ETags
+ * @throws IOException if the completion fails
+ */
+ private void completeMultipartUpload(
+ String bucket, String objectKey,
+ String mpUploadId, List partETags)
+ throws IOException {
+ Collections.sort(partETags,
+ new Comparator() {
+ public int compare(
+ PartETag arg1, PartETag arg2) {
+ PartETag part1 = arg1;
+ PartETag part2 = arg2;
+ return part1.getPartNumber()
+ - part2.getPartNumber();
+ }
+ });
+ CompleteMultipartUploadRequest request =
+ new CompleteMultipartUploadRequest(
+ bucket, objectKey, mpUploadId, partETags);
+ bosClientProxy.completeMultipartUpload(request);
+ }
+
+ /**
+ * A callable task that renames a single object in BOS.
+ */
+ private static class RenameTask
+ implements Callable {
+
+ private final String srcKey;
+ private final String dstKey;
+ private final BosNativeFileSystemStore store;
+
+ /**
+ * Constructs a RenameTask.
+ *
+ * @param store the file system store
+ * @param srcKey the source object key
+ * @param dstKey the destination object key
+ */
+ RenameTask(
+ BosNativeFileSystemStore store,
+ String srcKey, String dstKey) {
+ this.store = store;
+ this.srcKey = srcKey;
+ this.dstKey = dstKey;
+ }
+
+ /**
+ * Executes the rename operation.
+ *
+ * @return null
+ * @throws IOException if the rename fails
+ */
+ public Void call() throws IOException {
+ store.rename(srcKey, dstKey, true);
+ return null;
+ }
+
+ /**
+ * Returns the source key.
+ *
+ * @return the source key
+ */
+ String getSrcKey() {
+ return srcKey;
+ }
+
+ /**
+ * Returns the destination key.
+ *
+ * @return the destination key
+ */
+ String getDstKey() {
+ return dstKey;
+ }
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/PartialListing.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/PartialListing.java
new file mode 100644
index 0000000000000..6c97911c250cb
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/PartialListing.java
@@ -0,0 +1,84 @@
+/*
+ * 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.bos;
+
+/**
+ * Holds information on a directory listing for a
+ * {@link BosNativeFileSystemStore}.
+ * This includes the {@link FileMetadata files} and
+ * directories (their names) contained in a directory.
+ *
+ * This listing may be returned in chunks, so a
+ * {@code priorLastKey} is provided so that the next chunk
+ * may be requested.
+ *
+ * @see BosNativeFileSystemStore#list(String, int, String)
+ */
+class PartialListing {
+
+ private final String priorLastKey;
+ private final FileMetadata[] files;
+ private final FileMetadata[] directories;
+
+ /**
+ * Constructs a PartialListing.
+ *
+ * @param priorLastKey the marker for requesting the next
+ * chunk
+ * @param files the file entries in this listing
+ * @param commonPrefixesWithExtMeta the directory entries
+ * in this listing
+ */
+ PartialListing(
+ String priorLastKey, FileMetadata[] files,
+ FileMetadata[] commonPrefixesWithExtMeta) {
+ this.priorLastKey = priorLastKey;
+ this.files = files;
+ this.directories = commonPrefixesWithExtMeta;
+ }
+
+ /**
+ * Returns the file entries in this listing.
+ *
+ * @return the array of file metadata
+ */
+ public FileMetadata[] getFiles() {
+ return files;
+ }
+
+ /**
+ * Returns the marker for requesting the next listing
+ * chunk.
+ *
+ * @return the prior last key, or null if this is the
+ * last chunk
+ */
+ public String getPriorLastKey() {
+ return priorLastKey;
+ }
+
+ /**
+ * Returns the directory entries in this listing.
+ *
+ * @return the array of directory metadata
+ */
+ public FileMetadata[] getDirectories() {
+ return directories;
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/UploadPartThread.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/UploadPartThread.java
new file mode 100644
index 0000000000000..68684209579d1
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/UploadPartThread.java
@@ -0,0 +1,180 @@
+/*
+ * 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.bos;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+
+import com.baidubce.BceServiceException;
+import com.baidubce.services.bos.model.PartETag;
+import com.baidubce.services.bos.model.UploadPartRequest;
+import com.baidubce.services.bos.model.UploadPartResponse;
+import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Runnable that uploads a single part of a multipart upload
+ * with configurable retries.
+ */
+class UploadPartThread implements Runnable {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(UploadPartThread.class);
+
+ private final Configuration conf;
+ private final String bucket;
+ private final String object;
+ private final long size;
+ private final List eTags;
+ private final int partId;
+ private final BosClientProxy client;
+ private final String uploadId;
+ private final InputStream in;
+ private final Map dataMap;
+ private final Map exceptionMap;
+ private final ArrayBlockingQueue blocks;
+
+ /**
+ * Construct an upload part thread.
+ *
+ * @param client the BOS client proxy
+ * @param bucket the bucket name
+ * @param object the object key
+ * @param block the block buffer to upload
+ * @param uploadId the multipart upload id
+ * @param eTags the list to collect part ETags
+ * @param dataMap the block buffer map
+ * @param exceptionMap the map to collect exceptions
+ * @param conf the configuration
+ * @param blocks the queue for recycling block buffers
+ */
+ UploadPartThread(BosClientProxy client, String bucket,
+ String object, BosBlockBuffer block,
+ String uploadId, List eTags,
+ Map dataMap,
+ Map exceptionMap,
+ Configuration conf,
+ ArrayBlockingQueue blocks) {
+ this.bucket = bucket;
+ this.object = object;
+ this.size = block.getInBuffer().getLength();
+ this.eTags = eTags;
+ this.partId = block.getBlkId();
+ this.client = client;
+ this.uploadId = uploadId;
+ this.in = block.getInBuffer();
+ this.dataMap = dataMap;
+ this.exceptionMap = exceptionMap;
+ this.conf = conf;
+ this.blocks = blocks;
+ }
+
+ private int getMultiUploadAttempts() {
+ return this.conf.getInt(
+ BaiduBosConstants.BOS_MAX_RETRIES, 3);
+ }
+
+ private int getMultiUploadRetryInterval() {
+ return this.conf.getInt(
+ BaiduBosConstants.BOS_SLEEP_MILL_SECONDS, 1000);
+ }
+
+ @Override
+ public void run() {
+ String partInfo = object + " with part id " + partId;
+ LOG.debug("start uploading {}", partInfo);
+ int attempts = this.getMultiUploadAttempts();
+ int retryInterval = this.getMultiUploadRetryInterval();
+ boolean isSuccess = false;
+ UploadPartResponse uploadPartResult = null;
+ for (int i = 0; i < attempts; i++) {
+ try {
+ in.reset();
+ } catch (IOException e) {
+ LOG.warn("failed to reset input stream for {}",
+ partInfo);
+ break;
+ }
+
+ UploadPartRequest uploadPartRequest =
+ new UploadPartRequest();
+ uploadPartRequest.setBucketName(bucket);
+ uploadPartRequest.setKey(object);
+ uploadPartRequest.setUploadId(uploadId);
+ uploadPartRequest.setInputStream(in);
+ uploadPartRequest.setPartSize(size);
+ uploadPartRequest.setPartNumber(partId);
+ try {
+ LOG.debug("[upload attempt {}] start uploadPart {}",
+ i, partInfo);
+ uploadPartResult =
+ client.uploadPart(uploadPartRequest);
+ LOG.debug("[upload attempt {}] end uploadPart {}",
+ i, partInfo);
+ isSuccess = true;
+ break;
+ } catch (IOException e) {
+ LOG.error("[upload attempt {}] failed to upload {}",
+ i, partInfo, e);
+ try {
+ Thread.sleep(retryInterval);
+ } catch (InterruptedException ex) {
+ LOG.info("interrupted while sleep");
+ Thread.currentThread().interrupt();
+ }
+ }
+ }
+
+ if (!isSuccess) {
+ BceServiceException e = new BceServiceException(
+ "failed to upload part " + partInfo);
+ exceptionMap.put(partId, e);
+ LOG.error("upload {} failed after {} attempts",
+ partInfo, attempts);
+ } else {
+ eTags.add(uploadPartResult.getPartETag());
+ if (dataMap != null) {
+ synchronized (dataMap) {
+ dataMap.get(partId).clear();
+ dataMap.notifyAll();
+ }
+ }
+ LOG.debug("upload key {} success", partInfo);
+ }
+
+ try {
+ if (dataMap != null) {
+ this.blocks.put(dataMap.get(partId));
+ }
+ if (in != null) {
+ in.close();
+ }
+ } catch (IOException e) {
+ LOG.debug("Error during cleanup for {}", partInfo, e);
+ } catch (InterruptedException e) {
+ LOG.debug("Interrupted during block recycle for {}",
+ partInfo);
+ Thread.currentThread().interrupt();
+ }
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/credentials/BceCredentialsProvider.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/credentials/BceCredentialsProvider.java
new file mode 100644
index 0000000000000..0a920f29b8cd6
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/credentials/BceCredentialsProvider.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.credentials;
+
+import com.baidubce.auth.DefaultBceSessionCredentials;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.bos.BaiduBosConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+
+/**
+ * Interface for providing BCE (Baidu Cloud Engine)
+ * credentials. Implementations supply access keys and
+ * optional session tokens for authenticating with BOS.
+ */
+public interface BceCredentialsProvider {
+
+ /** Logger for credentials provider operations. */
+ Logger LOG = LoggerFactory.getLogger(
+ BceCredentialsProvider.class);
+
+ /**
+ * Creates a credentials provider instance based on the
+ * class name specified in the Hadoop configuration.
+ *
+ * @param configuration the Hadoop configuration
+ * @return the credentials provider instance
+ */
+ static BceCredentialsProvider
+ getBceCredentialsProviderImpl(
+ Configuration configuration) {
+ try {
+ String className = configuration.get(
+ BaiduBosConstants.BOS_CREDENTIALS_PROVIDER,
+ BaiduBosConstants
+ .BOS_CONFIG_CREDENTIALS_PROVIDER);
+ LOG.debug(
+ "getBceCredentialsProviderImpl use"
+ + " configuration value: {}",
+ className);
+ Class> providerClass =
+ Class.forName(className);
+ BceCredentialsProvider provider =
+ (BceCredentialsProvider) providerClass
+ .getDeclaredConstructor().newInstance();
+ provider.setConf(configuration);
+ return provider;
+ } catch (ReflectiveOperationException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ /**
+ * Gets credentials for the specified user.
+ *
+ * @param user the user name
+ * @return the session credentials
+ */
+ default DefaultBceSessionCredentials getCredentials(
+ String user) {
+ throw new UnsupportedOperationException(
+ "getCredentials(String) not implemented");
+ }
+
+ /**
+ * Gets credentials for the specified URI and user.
+ *
+ * @param uri the filesystem URI, may be null
+ * @param user the user name
+ * @return the session credentials
+ */
+ default DefaultBceSessionCredentials getCredentials(
+ URI uri, String user) {
+ return getCredentials(user);
+ }
+
+ /**
+ * Gets credentials by user ID.
+ *
+ * @param userId the user ID
+ * @return the session credentials
+ * @throws UnsupportedOperationException by default
+ */
+ default DefaultBceSessionCredentials getCredentialsById(
+ String userId) {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * Gets credentials by URI and user ID.
+ *
+ * @param uri the filesystem URI, may be null
+ * @param userId the user ID
+ * @return the session credentials
+ */
+ default DefaultBceSessionCredentials getCredentialsById(
+ URI uri, String userId) {
+ return getCredentialsById(userId);
+ }
+
+ /**
+ * Sets the Hadoop configuration on this provider.
+ *
+ * @param configs the Hadoop configuration
+ */
+ default void setConf(Configuration configs) {
+ // do nothing default
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/credentials/ConfigurationCredentialsProvider.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/credentials/ConfigurationCredentialsProvider.java
new file mode 100644
index 0000000000000..6ef025998d536
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/credentials/ConfigurationCredentialsProvider.java
@@ -0,0 +1,135 @@
+/*
+ * 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.bos.credentials;
+
+import com.baidubce.auth.DefaultBceSessionCredentials;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.bos.BaiduBosConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+
+/**
+ * Credentials provider that reads access key, secret key,
+ * and optional session token from the Hadoop configuration.
+ */
+public class ConfigurationCredentialsProvider
+ implements HadoopCredentialsProvider {
+
+ /**
+ * Constructs a ConfigurationCredentialsProvider.
+ */
+ public ConfigurationCredentialsProvider() {
+ }
+
+ private final Logger log =
+ LoggerFactory.getLogger(
+ ConfigurationCredentialsProvider.class);
+ private Configuration conf;
+
+ /**
+ * Sets the Hadoop configuration.
+ *
+ * @param conf the Hadoop configuration
+ */
+ @Override
+ public void setConf(Configuration conf) {
+ this.conf = conf;
+ }
+
+ /**
+ * Gets credentials from the Hadoop configuration. If a
+ * bucket-specific configuration exists for the given URI,
+ * it takes precedence.
+ *
+ * @param uri the filesystem URI, may be null
+ * @param user the user name
+ * @return the session credentials
+ * @throws IllegalArgumentException if access key or
+ * secret key is null
+ */
+ @Override
+ public DefaultBceSessionCredentials getCredentials(
+ URI uri, String user) {
+ String accessKey;
+ String secretAccessKey;
+ String sessionToken;
+ if (uri != null) {
+ String host = uri.getHost();
+ accessKey = conf.get(
+ "fs.bos.bucket." + host + ".access.key",
+ null);
+ secretAccessKey = conf.get(
+ "fs.bos.bucket." + host
+ + ".secret.access.key",
+ null);
+ sessionToken = conf.get(
+ "fs.bos.bucket." + host
+ + ".session.token.key",
+ null);
+ if (accessKey != null
+ && secretAccessKey != null) {
+ if (sessionToken == null) {
+ return new DefaultBceSessionCredentials(
+ accessKey, secretAccessKey, " ");
+ }
+ return new DefaultBceSessionCredentials(
+ accessKey, secretAccessKey, sessionToken);
+ }
+ }
+
+ accessKey = conf.get(
+ BaiduBosConstants.BOS_ACCESS_KEY, null);
+ secretAccessKey = conf.get(
+ BaiduBosConstants.BOS_SECRET_KEY, null);
+ sessionToken = conf.get(
+ BaiduBosConstants.BOS_SESSION_TOKEN, null);
+
+ if (accessKey == null
+ || secretAccessKey == null) {
+ throw new IllegalArgumentException(
+ "accessKey and secretAccessKey"
+ + " should not be null");
+ }
+
+ if (sessionToken == null
+ || sessionToken.trim().isEmpty()) {
+ return new DefaultBceSessionCredentials(
+ accessKey, secretAccessKey, " ");
+ } else {
+ return new DefaultBceSessionCredentials(
+ accessKey, secretAccessKey, sessionToken);
+ }
+ }
+
+ /**
+ * Gets credentials by user ID, delegating to
+ * {@link #getCredentials(URI, String)}.
+ *
+ * @param uri the filesystem URI
+ * @param userId the user ID
+ * @return the session credentials
+ */
+ @Override
+ public DefaultBceSessionCredentials getCredentialsById(
+ URI uri, String userId) {
+ return getCredentials(uri, userId);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/credentials/EnvironmentVariableCredentialsProvider.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/credentials/EnvironmentVariableCredentialsProvider.java
new file mode 100644
index 0000000000000..607ac076fbbce
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/credentials/EnvironmentVariableCredentialsProvider.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.credentials;
+
+import com.baidubce.auth.DefaultBceSessionCredentials;
+import org.apache.hadoop.fs.bos.BaiduBosConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+
+/**
+ * Credentials provider that reads access key, secret key,
+ * and optional session token from environment variables.
+ */
+public class EnvironmentVariableCredentialsProvider
+ implements BceCredentialsProvider {
+
+ /**
+ * Constructs an EnvironmentVariableCredentialsProvider.
+ */
+ public EnvironmentVariableCredentialsProvider() {
+ }
+
+ /** Logger for this provider. */
+ public static final Logger LOG =
+ LoggerFactory.getLogger(
+ EnvironmentVariableCredentialsProvider.class);
+
+ /**
+ * Gets credentials from environment variables.
+ *
+ * @param uri the filesystem URI, may be null
+ * @param user the user name
+ * @return the session credentials, or null if access key
+ * or secret key is missing
+ */
+ public DefaultBceSessionCredentials getCredentials(
+ URI uri, String user) {
+ String accessKey =
+ System.getenv(BaiduBosConstants.BOS_AK_ENV);
+ String secretAccessKey =
+ System.getenv(BaiduBosConstants.BOS_SK_ENV);
+ String sessionToken = System.getenv(
+ BaiduBosConstants.BOS_STS_TOKEN_ENV);
+
+ if (accessKey == null
+ || secretAccessKey == null) {
+ LOG.error(
+ "accessKey and secretAccessKey"
+ + " should not be null");
+ return null;
+ }
+
+ if (sessionToken == null) {
+ return new DefaultBceSessionCredentials(
+ accessKey, secretAccessKey, " ");
+ }
+ return new DefaultBceSessionCredentials(
+ accessKey, secretAccessKey, sessionToken);
+ }
+
+ /**
+ * Gets credentials by user ID, delegating to
+ * {@link #getCredentials(URI, String)}.
+ *
+ * @param uri the filesystem URI
+ * @param userId the user ID
+ * @return the session credentials
+ */
+ @Override
+ public DefaultBceSessionCredentials getCredentialsById(
+ URI uri, String userId) {
+ return getCredentials(uri, userId);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/credentials/HadoopCredentialsProvider.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/credentials/HadoopCredentialsProvider.java
new file mode 100644
index 0000000000000..9a405e41fc203
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/credentials/HadoopCredentialsProvider.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.credentials;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A credentials provider that requires Hadoop configuration
+ * to be set via {@link #setConf(Configuration)} before
+ * credentials can be retrieved.
+ */
+public interface HadoopCredentialsProvider
+ extends BceCredentialsProvider {
+
+ /**
+ * Sets the Hadoop configuration on this provider.
+ *
+ * @param conf the Hadoop configuration
+ */
+ void setConf(Configuration conf);
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/credentials/package-info.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/credentials/package-info.java
new file mode 100644
index 0000000000000..1d019cc39eb9c
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/credentials/package-info.java
@@ -0,0 +1,18 @@
+/**
+ * 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.bos.credentials;
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/BandwidthLimitException.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/BandwidthLimitException.java
new file mode 100644
index 0000000000000..f304d2eacca56
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/BandwidthLimitException.java
@@ -0,0 +1,45 @@
+/*
+ * 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.bos.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.io.IOException;
+
+/**
+ * Exception thrown when BOS enforces a bandwidth or request
+ * rate limit (HTTP 429 with no specific error code).
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class BandwidthLimitException extends IOException {
+
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * Constructs a BandwidthLimitException with the
+ * specified cause.
+ *
+ * @param t the cause of this exception
+ */
+ public BandwidthLimitException(Throwable t) {
+ super(t);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/BosException.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/BosException.java
new file mode 100644
index 0000000000000..0cac2430132ab
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/BosException.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.io.IOException;
+
+/**
+ * Thrown if there is a problem communicating with Baidu BOS.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class BosException extends IOException {
+
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * Constructs a BosException with the specified cause.
+ *
+ * @param t the cause of this exception
+ */
+ public BosException(Throwable t) {
+ super(t);
+ }
+
+ /**
+ * Constructs a BosException with the specified message.
+ *
+ * @param message the detail message
+ */
+ public BosException(String message) {
+ super(message);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/BosHotObjectException.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/BosHotObjectException.java
new file mode 100644
index 0000000000000..7763a0d7fcac5
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/BosHotObjectException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.exceptions;
+
+import java.io.IOException;
+
+/**
+ * Exception thrown when BOS rejects a request due to
+ * per-object rate limiting (HTTP 429 with
+ * RequestRateLimitExceeded error code).
+ */
+public class BosHotObjectException extends IOException {
+
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * Constructs a BosHotObjectException with the specified
+ * cause.
+ *
+ * @param t the cause of this exception
+ */
+ public BosHotObjectException(Throwable t) {
+ super(t);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/BosServerException.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/BosServerException.java
new file mode 100644
index 0000000000000..f4bb44300588d
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/BosServerException.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.exceptions;
+
+import java.io.IOException;
+
+/**
+ * Exception thrown when the BOS server returns a 5xx error,
+ * indicating a server-side failure.
+ */
+public class BosServerException extends IOException {
+
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * Constructs a BosServerException with the specified
+ * cause.
+ *
+ * @param t the cause of this exception
+ */
+ public BosServerException(Throwable t) {
+ super(t);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/SessionTokenExpireException.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/SessionTokenExpireException.java
new file mode 100644
index 0000000000000..f1c5277cadc24
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/SessionTokenExpireException.java
@@ -0,0 +1,44 @@
+/*
+ * 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.bos.exceptions;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.io.IOException;
+
+/**
+ * Exception thrown when the BOS session token has expired
+ * or is invalid, requiring credential refresh.
+ */
+@InterfaceStability.Stable
+public class SessionTokenExpireException
+ extends IOException {
+
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * Constructs a SessionTokenExpireException with the
+ * specified cause.
+ *
+ * @param t the cause of this exception
+ */
+ public SessionTokenExpireException(Throwable t) {
+ super(t);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/package-info.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/package-info.java
new file mode 100644
index 0000000000000..046e05399e452
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/exceptions/package-info.java
@@ -0,0 +1,18 @@
+/**
+ * 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.bos.exceptions;
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/package-info.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/package-info.java
new file mode 100644
index 0000000000000..182bdc962cb60
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/package-info.java
@@ -0,0 +1,18 @@
+/**
+ * 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.bos;
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/utils/BosCRC32CCheckSum.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/utils/BosCRC32CCheckSum.java
new file mode 100644
index 0000000000000..954907a46f418
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/utils/BosCRC32CCheckSum.java
@@ -0,0 +1,168 @@
+/*
+ * 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.bos.utils;
+
+import org.apache.hadoop.fs.FileChecksum;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.math.BigInteger;
+
+/**
+ * A {@link FileChecksum} implementation that wraps a CRC32C
+ * value returned by BOS.
+ */
+public class BosCRC32CCheckSum extends FileChecksum {
+
+ private static final String ALGORITHM_NAME =
+ "COMPOSITE-CRC";
+
+ private int crc32c;
+
+ /**
+ * Constructs a BosCRC32CCheckSum from a string
+ * representation of an unsigned 32-bit CRC32C value.
+ *
+ * @param crc32cUInt32 the CRC32C value as a string
+ */
+ public BosCRC32CCheckSum(String crc32cUInt32) {
+ try {
+ BigInteger bigInteger =
+ new BigInteger(crc32cUInt32);
+ this.crc32c = bigInteger.intValue();
+ } catch (NumberFormatException e) {
+ this.crc32c = 0;
+ }
+ }
+
+ /**
+ * Writes big-endian representation of {@code value}
+ * into {@code buf} starting at {@code offset}.
+ * buf.length must be greater than or equal to
+ * offset + 4.
+ *
+ * @param buf the byte buffer to write into
+ * @param offset the starting offset in the buffer
+ * @param value the integer value to write
+ * @throws IOException if the buffer is too small
+ */
+ public static void writeInt(
+ byte[] buf, int offset, int value)
+ throws IOException {
+ if (offset + 4 > buf.length) {
+ throw new IOException(String.format(
+ "writeInt out of bounds:"
+ + " buf.length=%d, offset=%d",
+ buf.length, offset));
+ }
+ buf[offset + 0] =
+ (byte) ((value >>> 24) & 0xff);
+ buf[offset + 1] =
+ (byte) ((value >>> 16) & 0xff);
+ buf[offset + 2] =
+ (byte) ((value >>> 8) & 0xff);
+ buf[offset + 3] =
+ (byte) (value & 0xff);
+ }
+
+ /**
+ * Converts an integer to a 4-byte big-endian byte array.
+ *
+ * @param value the integer value
+ * @return 4-byte array holding the big-endian
+ * representation of {@code value}
+ */
+ public static byte[] intToBytes(int value) {
+ byte[] buf = new byte[4];
+ try {
+ writeInt(buf, 0, value);
+ } catch (IOException ioe) {
+ // Since this should only be able to occur from
+ // code bugs within this class rather than user
+ // input, we throw as a RuntimeException rather
+ // than requiring this method to declare throwing
+ // IOException for something the caller can't
+ // control.
+ throw new RuntimeException(ioe);
+ }
+ return buf;
+ }
+
+ /**
+ * Returns the algorithm name for this checksum.
+ *
+ * @return the algorithm name
+ */
+ @Override
+ public String getAlgorithmName() {
+ return BosCRC32CCheckSum.ALGORITHM_NAME;
+ }
+
+ /**
+ * Returns the length of this checksum in bytes.
+ *
+ * @return the checksum length
+ */
+ @Override
+ public int getLength() {
+ return Integer.SIZE / Byte.SIZE;
+ }
+
+ /**
+ * Returns the checksum value as a byte array.
+ *
+ * @return the checksum bytes
+ */
+ @Override
+ public byte[] getBytes() {
+ return intToBytes(crc32c);
+ }
+
+ /**
+ * Serializes this checksum to a data output stream.
+ *
+ * @param dataOutput the data output to write to
+ * @throws IOException if an I/O error occurs
+ */
+ @Override
+ public void write(DataOutput dataOutput)
+ throws IOException {
+ dataOutput.writeInt(this.crc32c);
+ }
+
+ /**
+ * Deserializes this checksum from a data input stream.
+ *
+ * @param dataInput the data input to read from
+ * @throws IOException if an I/O error occurs
+ */
+ @Override
+ public void readFields(DataInput dataInput)
+ throws IOException {
+ this.crc32c = dataInput.readInt();
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ public String toString() {
+ return getAlgorithmName() + ":"
+ + String.format("0x%08x", crc32c);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/utils/package-info.java b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/utils/package-info.java
new file mode 100644
index 0000000000000..015a41f79ed78
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/main/java/org/apache/hadoop/fs/bos/utils/package-info.java
@@ -0,0 +1,18 @@
+/**
+ * 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.bos.utils;
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/site/markdown/cloud-storage/index.md b/hadoop-cloud-storage-project/hadoop-bos/src/site/markdown/cloud-storage/index.md
new file mode 100644
index 0000000000000..1d6501b2988f7
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/site/markdown/cloud-storage/index.md
@@ -0,0 +1,392 @@
+
+
+# Integration of Baidu BOS in Hadoop
+
+## Overview
+
+BOS (Baidu Object Storage) is the object storage service provided by Baidu Cloud. Hadoop-BOS is a connector between computing systems and underlying storage. For systems like Hadoop MR, Hive, Spark and Alluxio, hadoop-bos helps them use BOS as the underlying storage system instead of HDFS.
+
+## Quick Start
+
+In quick start, we will use hadoop shell command to access a BOS bucket.
+
+### Requirements
+
+1. A Baidu Cloud account. Use the account to create a BOS bucket.
+2. A dev environment that can access BOS. E.g. a local server or a Baidu Cloud cloud server.
+3. Install hadoop to the dev environment. Hadoop is installed at `$HADOOP_HOME`.
+
+### Usage
+
+* Compile hadoop-bos bundle tar. The hadoop-bos bundle is not packaged in hadoop final tar file.
+So we have to compile it manually. Download the hadoop project, and build it with command below.
+```bash
+mvn package -DskipTests -pl org.apache.hadoop:hadoop-bos
+```
+
+* The bundle jar file is placed at
+`$HADOOP_HOME/hadoop-cloud-storage-project/hadoop-bos/target/hadoop-bos-{VERSION}.jar`.
+
+* Copy the bundle jar to hdfs lib path. The hdfs lib path is
+`$HADOOP_HOME/share/hadoop/hdfs/lib`. Remember copying to all hadoop nodes.
+```bash
+cp hadoop-bos-{VERSION}.jar $HADOOP_HOME/share/hadoop/hdfs/lib/
+```
+
+* Configure properties below.
+
+```xml
+
+
+ fs.defaultFS
+ bos://{your_bucket_name}/
+
+ The name of the default file system. Make it your BOS bucket.
+
+
+
+
+ fs.bos.endpoint
+
+
+ Object storage endpoint to connect to, e.g. 'fs.bos.endpoint'='bj.bcebos.com'.
+
+
+
+
+ fs.bos.impl
+ org.apache.hadoop.fs.bos.BaiduBosFileSystem
+
+ The implementation class of the BOS FileSystem.
+
+
+
+
+ fs.AbstractFileSystem.bos.impl
+ org.apache.hadoop.fs.bos.BOS
+
+ The implementation class of the BOS AbstractFileSystem.
+
+
+
+
+ fs.bos.access.key
+
+
+ The access key of Baidu Cloud's user or role.
+
+
+
+
+ fs.bos.secret.access.key
+
+
+ The secret key of the access key specified by 'fs.bos.access.key'.
+
+
+
+```
+
+* Use hadoop shell command to access BOS.
+
+```bash
+# 1. List root dir.
+hadoop fs -ls /
+
+# 2. Make directory.
+hadoop fs -mkdir /hadoop-bos
+
+# 3. Write and read.
+echo "hello BOS." > hello.txt
+hadoop fs -put hello.txt /hadoop-bos/
+hadoop fs -cat /hadoop-bos/hello.txt
+
+# 4. Delete file and directory.
+hadoop fs -rm -r /hadoop-bos/
+```
+
+## Introduction
+
+This is a brief introduction of hadoop-bos design and basic functions. The following contents are
+based on flat mode by default. The differences between hierarchy mode will be explained at the end
+of each section.
+
+### BOS
+
+BOS is the object storage service of Baidu Cloud. It is similar to AWS S3, Azure Blob Storage
+and Aliyun OSS. BOS has 2 modes: the flat mode and the hierarchy mode. In flat mode, there are no directories,
+all objects are files indexed by the object names. User can use 'slash' in the object name to
+logically divide objects into different "directories", though the "directories divided by the slash"
+are not real. Cleanup a logic directory is to clean all the files with "directory path" as prefix.
+
+In hierarchy mode, there are directories and files. A directory object is the object whose name
+ends with slash. All objects start with the directory object name are the directory object's
+consecutive objects, and together they form a directory tree. A directory object can't contain
+any data. Delete or rename a directory object will clean or rename all objects under the
+directory tree atomically.
+
+### Directory and file
+
+This section illustrates how hadoop-bos transforms BOS to a hadoop FileSystem. BOS requires object's
+name must not start with slash, must not contain consecutive slash and must not be empty. Here is
+the transformation rules.
+
+* Object name is divided by slash to form hierarchy.
+* An object whose name ends with slash is a directory.
+* An object whose name doesn't end with slash is a file.
+* A file's parents are directories, no matter whether the parent exists or not.
+
+For example, supposing we have 2 objects "user/table/" and "user/table/part-0". The first object
+is mapped to "/user/table" in hadoop and is a directory. The second object is mapped to
+"/user/table/part-0" as a file. The non-existent object "user/" is mapped to "/user" as a directory
+because it's the parent of file "/user/table/part-0".
+
+| Object name | Object existence | FileSystem path | FileSystem Type |
+|-------------------|------------------|--------------------|-----------------|
+| user/table/ | yes | /user/table | Directory |
+| user/table/part-0 | yes | /user/table/part-0 | File |
+| user/ | no | /user | Directory |
+
+The FileSystem requirements above are not enforced rules in flat mode, users can construct
+cases violating the requirements above. For example, creating a file with its parent is a file. In
+hierarchy mode, the requirements are enforced rules controlled by BOS service, so there won't be
+semantic violations.
+
+### List, Rename and Delete
+
+List, rename and delete are costly operations in flat mode. Since the namespace is flat, to list
+a directory, the client needs to scan all objects with directory as the prefix and filter with
+delimiter. For rename and delete directory, the client needs to first list the directory to get all
+objects and then rename or delete objects one by one. So they are not atomic operations and costs a
+lot comparing to hdfs.
+
+The idiosyncrasies of hierarchy mode is supporting directory. So it can list very fast and
+support atomic rename and delete directory. Rename or delete failure in flat mode may leave
+the bucket in an inconsistent state, the hierarchy mode won't have this problem.
+
+### Read and write file
+
+The read behaviour in hadoop-bos is very like reading an HDFS file. The challenge is how to keep the
+input stream consistent with object. If the object is changed after we open the file, the input
+stream should fail. This is implemented by saving the file checksum when open file. If the
+file is changed while reading, the input stream will compare the checksum and trigger an exception.
+
+The write behaviour in hadoop-bos is slightly different from hdfs. Firstly, the append interface
+is not supported. Secondly, the file is not visible until it is successfully closed. Finally,
+when 2 clients try to write one file, the last client to close the file will override the previous
+one.
+
+Both read and write has many performance optimizations. E.g. range read, connection reuse, local
+write buffer, put for small files, multipart-upload for big files etc.
+
+### Permissions
+
+BOS permission model is different from hadoop filesystem permission model. BOS supports permissions
+based on IAM, Bucket Policy, Bucket and Object ACL, while hadoop filesystem permission model uses
+mode and acl. There is no way to map BOS permission to hadoop filesystem permission, so we have
+to use fake permissions in BaiduBosFileSystem and BOS. Users can read and change the filesystem
+permissions, they can only be seen but not effective. Permission control eventually depends on BOS
+permission model.
+
+### Times
+
+Hadoop-bos supports last modified time and doesn't support access time. For files, the last modified
+time is the object's modified time. For directories, if the directory object doesn't exist, the last
+modified time is the current system time. If the directory object exists, the last modified time is
+the object's modify time when `getFileStatus` and current system time when `listStatus`.
+
+### File checksum
+
+BOS supports CRC32 checksum. We can retrieve it by calling `FileSystem#getFileChecksum`.
+To be compatible with HDFS, BOS provides optional CRC32C checksum. When we distcp
+between HDFS and BOS, we can rely on distcp checksum mechanisms to keep data consistent.
+
+### Credential
+
+BOS client uses access key id and secret access key to authenticate with BOS service. There are 2
+ways to configure them. First is adding to hadoop configuration, such as adding to core-site.xml or
+configuring through `-D` parameter. The second is setting environment variable, hadoop-bos will
+search for environment variables automatically.
+
+To configure ak, sk in hadoop configuration, using the key below.
+
+```xml
+
+
+
+ fs.bos.access.key
+
+
+ The accessKey key to access the BOS object storage.
+
+
+
+ fs.bos.secret.access.key
+
+
+ The secret access key to access the object storage.
+
+
+
+ fs.bos.session.token.key
+
+
+ The session token to access the object storage.
+
+
+
+
+
+ fs.bos.bucket.{bucket_name}.access.key
+
+
+ The access key to access the object storage for the configured bucket.
+
+
+
+ fs.bos.bucket.{bucket_name}.secret.access.key
+
+
+ The secret access key to access the object storage for the configured bucket.
+
+
+
+ fs.bos.bucket.{bucket_name}.session.token.key
+
+
+ The session token to access the object storage for the configured bucket.
+
+
+
+```
+
+The ak, sk in environment variables have the top priority and automatically fall back to hadoop
+configuration if not found.
+
+## Properties Summary
+
+| properties | description | default value | required |
+|-------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------|----------|
+| fs.bos.access.key | The accessKey key to access the BOS object storage | NONE | YES |
+| fs.bos.secret.access.key | The secret access key to access the object storage | NONE | YES |
+| fs.bos.session.token.key | The session token to access the object storage | NONE | NO |
+| fs.bos.endpoint | Object storage endpoint to connect to, e.g. bj.bcebos.com, gz.bcebos.com | NONE | NO |
+| fs.bos.bucket.%s.access.key | The access key to access the object storage for the configured bucket, where %s is the bucket name. | NONE | NO |
+| fs.bos.bucket.%s.secret.access.key | The secret access key to access the object storage for the configured bucket, where %s is the bucket name | NONE | NO |
+| fs.bos.bucket.%s.session.token.key | The session token to access the object storage for the configured bucket, where %s is the bucketname | NONE | NO |
+| fs.bos.credentials.provider | Credentials provider class for BOS authentication. Available providers: ConfigurationCredentialsProvider, EnvironmentVariableCredentialsProvider, HadoopCredentialsProvider | org.apache.hadoop.fs.bos.credentials.ConfigurationCredentialsProvider | NO |
+| fs.bos.multipart.uploads.block.size | The size of each part for multipart uploads (in bytes). | 16777216 (16MB) | NO |
+| fs.bos.multipart.uploads.concurrent.size | The number of concurrent threads for multipart uploads. | 10 | NO |
+| fs.bos.copy.large.file.threshold | The threshold (in bytes) for using multipart upload during copy operations. | 5368709120 (5GB) | NO |
+| fs.bos.max.connections | The maximum number of connections to the BOS service that a client can create. | 50 | NO |
+| fs.bos.readahead.size | The size of readahead buffer (in bytes). | 4194304 (4MB) | NO |
+| fs.bos.read.buffer.size | The size of read buffer (in bytes). | 131072 (128KB) | NO |
+| fs.bos.threads.max.num | The maximum number of threads for BOS operations. | 64 | NO |
+| fs.bos.bucket.hierarchy | Whether the bucket uses hierarchical namespace mode. | NONE | NO |
+
+## Running unit tests in hadoop-bos module
+
+Unit tests need to connect to BOS service. Setting the environment variables below to run unit
+tests.
+
+```bash
+export FS_BOS_ACCESS_KEY=YOUR_ACCESS_KEY
+export FS_BOS_SECRET_ACCESS_KEY=YOUR_SECRET_ACCESS_KEY
+export FS_BOS_ENDPOINT=bj.bcebos.com
+```
+
+Then cd to hadoop project root directory, and run the test command below.
+
+```bash
+mvn -Dtest=org.apache.hadoop.fs.bos.** test -pl org.apache.hadoop:hadoop-bos
+```
+
+## Testing the hadoop-bos Module
+
+To test BOS filesystem, the following configuration file is needed:
+
+### `contract-test-options.xml`
+
+All configurations related to support contract tests need to be specified in `contract-test-options.xml`. Here is an example of `contract-test-options.xml`.
+
+```xml
+
+
+
+
+ fs.contract.test.fs.bos
+ bos://your-bucket-name/test
+
+
+ fs.bos.endpoint
+ bj.bcebos.com
+
+
+ fs.bos.credentials.provider
+ org.apache.hadoop.fs.bos.credentials.EnvironmentVariableCredentialsProvider
+
+
+```
+
+If the option `fs.contract.test.fs.bos` is not defined in the file, all contract tests will be skipped.
+
+## Features and Limitations
+
+### Supported Features
+
+- File create, read, delete
+- Directory operations (mkdir, delete)
+- Rename (via copy + delete)
+- Random access (seek)
+- Content summary
+- Checksum verification
+- Hierarchical namespace buckets (atomic rename/delete)
+
+### Unsupported Features
+
+- Append to files
+- File concatenation
+- File truncation
+- Symbolic links
+- Extended attributes (xattr)
+- Full Unix permissions (limited support in hierarchy mode)
+- hflush/hsync operations
+
+## Performance Notes
+
+Object Storage is not a file system and it has some limitations:
+
+1. Object storage is a key-value storage and it does not support hierarchical directory naturally. Usually, using the directory separatory in object key to simulate the hierarchical directory, such as "/hadoop/data/words.dat".
+
+2. BOS Object storage can not support the object's append operation currently. It means that you can not append content to the end of an existing object(file).
+
+3. Both `delete` and `rename` operations are non-atomic in flat mode, which means that the operations are interrupted, the operation result may be inconsistent state. In hierarchy mode, these operations are atomic.
+
+4. Object storages have different authorization models:
+ - Directory permissions are reported as 777.
+ - File permissions are reported as 666.
+ - File owner is reported as the local current user.
+ - File group is also reported as the local current user.
+
+5. Supports multipart uploads for a large file (up to 5TB), but the number of parts is limited.
+
+6. The number of files listed each time is limited to 1000.
+
+## Getting Help
+
+- Check Hadoop FileSystem specification: https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-common/filesystem/
+- Review Baidu BOS documentation: https://cloud.baidu.com/doc/BOS/index.html
+- Check project README.md
+- File issues on project issue tracker
\ No newline at end of file
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/BosContract.java b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/BosContract.java
new file mode 100644
index 0000000000000..cda215b9c3ac2
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/BosContract.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
+
+/**
+ * Contract for Baidu BOS FileSystem.
+ * Only enabled if the test filesystem URI is provided via
+ * {@code fs.contract.test.fs.bos} in contract-test-options.xml.
+ */
+public class BosContract extends AbstractBondedFSContract {
+
+ public static final String CONTRACT_XML = "contract/bos.xml";
+
+ /**
+ * Create a new BOS contract.
+ *
+ * @param conf configuration to use
+ */
+ protected BosContract(Configuration conf) {
+ super(conf);
+ addConfResource(CONTRACT_XML);
+ }
+
+ /**
+ * Get the filesystem scheme.
+ *
+ * @return "bos"
+ */
+ @Override
+ public String getScheme() {
+ return "bos";
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractAppend.java b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractAppend.java
new file mode 100644
index 0000000000000..fec36c1d5376a
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractAppend.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractAppendTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for append operations on BOS FileSystem.
+ * Tests the append() operation for adding data to existing files.
+ */
+public class TestBosContractAppend extends AbstractContractAppendTest {
+
+ /**
+ * Create the contract for this test.
+ * @param conf configuration
+ * @return BOS contract
+ */
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new BosContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractConcat.java b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractConcat.java
new file mode 100644
index 0000000000000..0e7bf81003190
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractConcat.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractConcatTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for concat operations on BOS FileSystem.
+ * Tests the concat() operation for concatenating multiple files into one.
+ */
+public class TestBosContractConcat extends AbstractContractConcatTest {
+
+ /**
+ * Create the contract for this test.
+ * @param conf configuration
+ * @return BOS contract
+ */
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new BosContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractContentSummary.java b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractContentSummary.java
new file mode 100644
index 0000000000000..7d92682268af1
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractContentSummary.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractContentSummaryTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for content summary operations on BOS FileSystem.
+ * Tests the getContentSummary() operation.
+ */
+public class TestBosContractContentSummary extends AbstractContractContentSummaryTest {
+
+ /**
+ * Create the contract for this test.
+ * @param conf configuration
+ * @return BOS contract
+ */
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new BosContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractCreate.java b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractCreate.java
new file mode 100644
index 0000000000000..1ee91e692ee5e
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractCreate.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for file creation operations on BOS FileSystem.
+ * Tests the create(), createNonRecursive(), and related operations.
+ */
+public class TestBosContractCreate extends AbstractContractCreateTest {
+
+ /**
+ * Create the contract for this test.
+ * @param conf configuration
+ * @return BOS contract
+ */
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new BosContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractDelete.java b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractDelete.java
new file mode 100644
index 0000000000000..c8792a4adee50
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractDelete.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for delete operations on BOS FileSystem.
+ * Tests the delete() operation for files and directories.
+ */
+public class TestBosContractDelete extends AbstractContractDeleteTest {
+
+ /**
+ * Create the contract for this test.
+ * @param conf configuration
+ * @return BOS contract
+ */
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new BosContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractGetFileStatus.java b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractGetFileStatus.java
new file mode 100644
index 0000000000000..9468c65ff219c
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractGetFileStatus.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for getFileStatus operations on BOS FileSystem.
+ * Tests the getFileStatus() and related metadata operations.
+ */
+public class TestBosContractGetFileStatus extends AbstractContractGetFileStatusTest {
+
+ /**
+ * Create the contract for this test.
+ * @param conf configuration
+ * @return BOS contract
+ */
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new BosContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractMkdir.java b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractMkdir.java
new file mode 100644
index 0000000000000..7e22192b6a551
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractMkdir.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for mkdir operations on BOS FileSystem.
+ * Tests the mkdirs() operation and directory creation.
+ */
+public class TestBosContractMkdir extends AbstractContractMkdirTest {
+
+ /**
+ * Create the contract for this test.
+ * @param conf configuration
+ * @return BOS contract
+ */
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new BosContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractOpen.java b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractOpen.java
new file mode 100644
index 0000000000000..9c3398f7adb14
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractOpen.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for file open operations on BOS FileSystem.
+ * Tests the open() operation and input stream behavior.
+ */
+public class TestBosContractOpen extends AbstractContractOpenTest {
+
+ /**
+ * Create the contract for this test.
+ * @param conf configuration
+ * @return BOS contract
+ */
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new BosContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractRename.java b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractRename.java
new file mode 100644
index 0000000000000..7353df5299332
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractRename.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for rename operations on BOS FileSystem.
+ * Tests the rename() operation for files and directories.
+ */
+public class TestBosContractRename extends AbstractContractRenameTest {
+
+ /**
+ * Create the contract for this test.
+ * @param conf configuration
+ * @return BOS contract
+ */
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new BosContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractRootDir.java b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractRootDir.java
new file mode 100644
index 0000000000000..a29ce89697b02
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractRootDir.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for root directory operations on BOS FileSystem.
+ * Tests operations directly on the root directory (/).
+ *
+ * Note: These tests are disabled by default (via contract configuration)
+ * to avoid potentially destructive operations on the bucket root.
+ */
+public class TestBosContractRootDir extends AbstractContractRootDirectoryTest {
+
+ /**
+ * Create the contract for this test.
+ * @param conf configuration
+ * @return BOS contract
+ */
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new BosContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractSeek.java b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractSeek.java
new file mode 100644
index 0000000000000..656b8e141a59b
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractSeek.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for seek operations on BOS FileSystem.
+ * Tests the seek() operation and random access capabilities.
+ */
+public class TestBosContractSeek extends AbstractContractSeekTest {
+
+ /**
+ * Create the contract for this test.
+ * @param conf configuration
+ * @return BOS contract
+ */
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new BosContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractSetTimes.java b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractSetTimes.java
new file mode 100644
index 0000000000000..bffb5e8e95ebd
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractSetTimes.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSetTimesTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for setTimes operations on BOS FileSystem.
+ * Tests the setTimes() operation for modifying file timestamps.
+ */
+public class TestBosContractSetTimes extends AbstractContractSetTimesTest {
+
+ /**
+ * Create the contract for this test.
+ * @param conf configuration
+ * @return BOS contract
+ */
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new BosContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractUnbuffer.java b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractUnbuffer.java
new file mode 100644
index 0000000000000..47342e22f7594
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/TestBosContractUnbuffer.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.bos.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractUnbufferTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test for unbuffer operations on BOS FileSystem.
+ * Tests the unbuffer() operation that releases buffered data from input streams.
+ */
+public class TestBosContractUnbuffer extends AbstractContractUnbufferTest {
+
+ /**
+ * Create the contract for this test.
+ * @param conf configuration
+ * @return BOS contract
+ */
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new BosContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/package-info.java b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/package-info.java
new file mode 100644
index 0000000000000..60c8f0b44fbd8
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/contract/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Contract tests for the Baidu BOS FileSystem.
+ */
+package org.apache.hadoop.fs.bos.contract;
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/credentials/TestConfigurationCredentialsProvider.java b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/credentials/TestConfigurationCredentialsProvider.java
new file mode 100644
index 0000000000000..ec20d51463848
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/credentials/TestConfigurationCredentialsProvider.java
@@ -0,0 +1,64 @@
+/*
+ * 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.bos.credentials;
+
+import com.baidubce.auth.DefaultBceSessionCredentials;
+import org.apache.hadoop.conf.Configuration;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * Tests for {@link ConfigurationCredentialsProvider}.
+ */
+public class TestConfigurationCredentialsProvider {
+ private static Configuration conf = new Configuration();
+
+ /**
+ * Set up test configuration with BOS credentials.
+ */
+ @BeforeAll
+ public static void beforeTest() {
+ conf.set("fs.bos.access.key", "ak");
+ conf.set("fs.bos.secret.access.key", "sk");
+ conf.set("fs.bos.session.token.key", "token");
+ conf.set("fs.bos.credentials.provider",
+ "org.apache.hadoop.fs.bos.credentials"
+ + ".ConfigurationCredentialsProvider");
+ }
+
+ /**
+ * Test that the provider returns correct credentials.
+ */
+ @Test
+ public void testProvider() {
+ BceCredentialsProvider provider =
+ BceCredentialsProvider
+ .getBceCredentialsProviderImpl(conf);
+ if (provider instanceof HadoopCredentialsProvider) {
+ ((HadoopCredentialsProvider) provider).setConf(conf);
+ }
+ DefaultBceSessionCredentials token =
+ provider.getCredentials(null, "hdfs");
+ assertEquals("ak", token.getAccessKeyId());
+ assertEquals("sk", token.getSecretKey());
+ assertEquals("token", token.getSessionToken());
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/credentials/package-info.java b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/credentials/package-info.java
new file mode 100644
index 0000000000000..a1a0a78fa0ee0
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/java/org/apache/hadoop/fs/bos/credentials/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Tests for Baidu BOS credentials provider implementations.
+ */
+package org.apache.hadoop.fs.bos.credentials;
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/README-TEST.md b/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/README-TEST.md
new file mode 100644
index 0000000000000..2d659ae23ae65
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/README-TEST.md
@@ -0,0 +1,324 @@
+
+
+# BOS FileSystem Testing Guide
+
+This document provides guidance on running and understanding the BOS FileSystem test suite.
+
+## Table of Contents
+
+1. [Prerequisites](#prerequisites)
+2. [Test Configuration](#test-configuration)
+3. [Running Tests](#running-tests)
+4. [Test Structure](#test-structure)
+5. [Troubleshooting](#troubleshooting)
+
+## Prerequisites
+
+### Required
+
+- Java 8 or higher
+- Maven 3.3 or higher
+- Access to a Baidu BOS bucket for testing
+- BOS credentials (Access Key ID and Secret Access Key)
+
+### BOS Bucket Setup
+
+1. Create a BOS bucket for testing (or use an existing one)
+2. Ensure the bucket is accessible with your credentials
+3. Note: Tests will create and delete files under `/test` directory in the bucket
+
+## Test Configuration
+
+### Step 1: Create Test Configuration File
+
+Copy the template configuration file:
+
+```bash
+cd src/test/resources
+cp contract-test-options.xml.template contract-test-options.xml
+```
+
+### Step 2: Configure BOS Credentials
+
+Edit `contract-test-options.xml` and set your BOS configuration:
+
+```xml
+
+ fs.contract.test.fs.bos
+ bos://your-bucket-name/test
+
+
+
+ fs.bos.endpoint
+ http://bd.bcebos.com
+
+```
+
+### Step 3: Set Authentication
+
+**Recommended: Use Environment Variables**
+
+```bash
+export BOS_ACCESS_KEY_ID=your_access_key_id
+export BOS_SECRET_ACCESS_KEY=your_secret_access_key
+```
+
+Ensure the credentials provider is set to `EnvironmentVariableCredentialsProvider`:
+
+```xml
+
+ fs.bos.credentials.provider
+ org.apache.hadoop.fs.bos.credentials.EnvironmentVariableCredentialsProvider
+
+```
+
+**Alternative: Configuration-based (Not Recommended)**
+
+You can also configure credentials directly in `contract-test-options.xml`, but this is **NOT recommended** for security reasons:
+
+```xml
+
+ fs.bos.access.key
+ YOUR_ACCESS_KEY
+
+
+
+ fs.bos.secret.key
+ YOUR_SECRET_KEY
+
+```
+
+**⚠️ SECURITY WARNING**: Never commit `contract-test-options.xml` with credentials to version control!
+
+## Running Tests
+
+### Run All Tests
+
+```bash
+mvn test
+```
+
+### Run Contract Tests Only
+
+```bash
+mvn test -Dtest="**/contract/*Test.java"
+```
+
+### Run Specific Contract Test Suite
+
+```bash
+# Test file creation
+mvn test -Dtest=TestBosContractCreate
+
+# Test rename operations
+mvn test -Dtest=TestBosContractRename
+
+# Test delete operations
+mvn test -Dtest=TestBosContractDelete
+```
+
+### Run Integration Tests
+
+```bash
+mvn test -Dtest="**/integration/*Test.java"
+```
+
+### Run with Specific Bucket Type
+
+For namespace-enabled bucket:
+```bash
+mvn test -Dfs.contract.test.fs.bos=bos://namespace-bucket/test
+```
+
+For flat (non-namespace) bucket:
+```bash
+mvn test -Dfs.contract.test.fs.bos=bos://flat-bucket/test
+```
+
+### Skip Tests
+
+```bash
+mvn package -DskipTests
+```
+
+## Test Structure
+
+### Contract Tests (`org.apache.hadoop.fs.bos.contract`)
+
+These tests verify that BOS FileSystem conforms to the Hadoop FileSystem contract:
+
+- **TestBosContractCreate**: File creation operations
+- **TestBosContractOpen**: File open and read operations
+- **TestBosContractDelete**: File and directory deletion
+- **TestBosContractMkdir**: Directory creation
+- **TestBosContractRename**: File and directory rename
+- **TestBosContractSeek**: Random access (seek) operations
+- **TestBosContractGetFileStatus**: Metadata operations
+- **TestBosContractRootDir**: Root directory operations (disabled by default)
+- **TestBosContractContentSummary**: Content summary operations
+
+### Integration Tests (`org.apache.hadoop.fs.bos.integration`)
+
+These tests verify BOS-specific functionality:
+
+- **TestBosIntegrationIO**: Input/Output stream operations
+- **TestBosIntegrationChecksum**: Checksum calculation and verification
+- **TestBosIntegrationMultipart**: Multi-part upload/download (if applicable)
+
+### Credential Tests (`org.apache.hadoop.fs.bos.credentials`)
+
+- **TestConfigurationCredentialsProvider**: Configuration-based authentication
+
+## Test Features and Limitations
+
+### Supported Features
+
+- ✅ File create, read, delete
+- ✅ Directory operations (mkdir, delete)
+- ✅ Rename (via copy + delete)
+- ✅ Random access (seek)
+- ✅ Content summary
+- ✅ Checksum verification
+
+### Unsupported Features (Tests Will Skip)
+
+- ❌ Append to files
+- ❌ File concatenation
+- ❌ File truncation
+- ❌ Symbolic links
+- ❌ Extended attributes (xattr)
+- ❌ Full Unix permissions (limited support in namespace mode)
+
+## Troubleshooting
+
+### Authentication Failures
+
+**Problem**: Tests fail with authentication errors
+
+**Solutions**:
+1. Verify credentials are correct
+2. Check environment variables are set: `echo $BOS_ACCESS_KEY_ID`
+3. Ensure credentials provider class is correctly configured
+4. Verify network connectivity to BOS endpoint
+
+### Bucket Not Found
+
+**Problem**: `FileNotFoundException` or bucket access errors
+
+**Solutions**:
+1. Verify bucket name is correct in `contract-test-options.xml`
+2. Ensure bucket exists in BOS
+3. Check bucket region matches endpoint
+4. Verify credentials have access to the bucket
+
+### Test Timeouts
+
+**Problem**: Tests hang or timeout
+
+**Solutions**:
+1. Check network connectivity to BOS
+2. Verify firewall settings
+3. Try different BOS endpoint
+4. Increase timeout values in test configuration
+
+### Inconsistent Test Results
+
+**Problem**: Tests pass sometimes but fail other times
+
+**Causes**:
+- BOS is eventually consistent; some operations may have delays
+- Network issues causing intermittent failures
+- Concurrent test runs interfering with each other
+
+**Solutions**:
+1. Run tests sequentially: `mvn test -DforkCount=1`
+2. Add retry logic for flaky tests
+3. Ensure proper test cleanup
+
+### Contract Test Failures
+
+**Problem**: Contract tests fail or are skipped
+
+**Expected Behavior**:
+- Some tests may be skipped if features are not supported (e.g., append, concat)
+- Check `contract/bos.xml` for feature flags
+
+**Debugging**:
+1. Enable debug logging: `mvn test -X`
+2. Check test output for specific failure reasons
+3. Verify BOS behavior matches Hadoop expectations
+
+## Performance Testing
+
+### Enable Scale Tests
+
+Scale tests are disabled by default. To enable:
+
+```xml
+
+ scale.test.enabled
+ true
+
+```
+
+**Warning**: Scale tests may take a long time and incur costs!
+
+### Configure Scale Test Parameters
+
+```xml
+
+ scale.test.operation.count
+ 1000
+
+```
+
+## Best Practices
+
+1. **Use Dedicated Test Bucket**: Don't use production buckets for testing
+2. **Clean Up**: Tests should clean up after themselves, but verify manually
+3. **Network**: Run tests in the same region as your BOS bucket for better performance
+4. **Credentials**: Always use environment variables for credentials
+5. **Version Control**: Never commit `contract-test-options.xml` to git
+
+## Getting Help
+
+- Check Hadoop FileSystem specification: https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-common/filesystem/
+- Review Baidu BOS documentation
+- Check project README.md
+- File issues on project issue tracker
+
+## Contributing Tests
+
+When adding new tests:
+
+1. Follow Hadoop contract test patterns
+2. Add appropriate documentation
+3. Ensure tests clean up resources
+4. Handle both namespace and non-namespace modes
+5. Add tests to appropriate package (contract vs integration)
+6. Update this README with new test information
+
+## CI/CD Integration
+
+For automated testing in CI/CD pipelines:
+
+```bash
+# Example GitHub Actions / GitLab CI
+export BOS_ACCESS_KEY_ID=${{ secrets.BOS_ACCESS_KEY_ID }}
+export BOS_SECRET_ACCESS_KEY=${{ secrets.BOS_SECRET_ACCESS_KEY }}
+mvn test
+```
+
+Ensure secrets are properly configured in your CI/CD system.
\ No newline at end of file
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/contract-test-options.xml b/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/contract-test-options.xml
new file mode 100644
index 0000000000000..bd9ff3053ee24
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/contract-test-options.xml
@@ -0,0 +1,49 @@
+
+
+
+
+
+
+
+
+ fs.contract.test.fs.bos
+
+
+ The BOS bucket URI for contract tests.
+ Set this in auth-keys.xml, e.g. bos://your-bucket-name/test
+
+
+
+
+ fs.contract.test.root-tests-enabled
+ false
+
+ Root directory tests are disabled by default.
+
+
+
+
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/contract-test-options.xml.template b/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/contract-test-options.xml.template
new file mode 100644
index 0000000000000..14b2f1872cb0d
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/contract-test-options.xml.template
@@ -0,0 +1,72 @@
+
+
+
+
+
+
+
+
+ fs.contract.test.fs.bos
+ bos://your-bucket-name/test
+
+ The BOS bucket URI to use for contract tests.
+ Format: bos://bucket-name/path
+
+
+
+
+ fs.bos.endpoint
+ http://bj.bcebos.com
+
+ BOS service endpoint.
+
+ Common endpoints:
+ - Beijing: http://bj.bcebos.com
+ - Guangzhou: http://gz.bcebos.com
+ - Shanghai: http://su.bcebos.com
+ - Hong Kong: http://hkg.bcebos.com
+ - Baoding: http://bd.bcebos.com
+
+
+
+
+ fs.bos.access.key
+ YOUR_ACCESS_KEY_HERE
+ BOS Access Key ID.
+
+
+
+ fs.bos.secret.access.key
+ YOUR_SECRET_KEY_HERE
+ BOS Secret Access Key.
+
+
+
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/contract/bos.xml b/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/contract/bos.xml
new file mode 100644
index 0000000000000..46696c0769670
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/contract/bos.xml
@@ -0,0 +1,324 @@
+
+
+
+
+
+
+
+
+
+
+
+
+ fs.contract.supports-append
+ false
+
+ BOS does not support appending to existing files.
+ Object stores generally do not support append operations.
+
+
+
+
+ fs.contract.supports-concat
+ false
+
+ BOS does not support concatenating multiple files into one.
+
+
+
+
+ fs.contract.supports-truncate
+ false
+
+ BOS does not support truncating files.
+
+
+
+
+ fs.contract.is-blobstore
+ true
+
+ BOS is an object/blob store, not a real filesystem.
+ This means files are only visible after close(), not after create() or flush().
+
+
+
+
+ fs.contract.create-visibility-delayed
+ true
+
+ BOS files are not visible until after close() is called.
+ This is standard behavior for object stores.
+
+
+
+
+ fs.contract.supports-hflush
+ false
+
+ BOS does not support hflush(). Data is only persisted on close().
+
+
+
+
+ fs.contract.supports-hsync
+ false
+
+ BOS does not support hsync(). Data is only persisted on close().
+
+
+
+
+ fs.contract.supports-seek
+ true
+
+ BOS supports random access (seek) operations on files.
+
+
+
+
+ fs.contract.supports-seek-on-closed-file
+ true
+
+ BOS allows seek operations on closed files (will fail on subsequent read).
+
+
+
+
+ fs.contract.rejects-seek-past-eof
+ true
+
+ BOS will throw an exception when seeking past end of file.
+
+
+
+
+
+
+
+
+ fs.contract.is-case-sensitive
+ true
+
+ BOS paths are case-sensitive.
+
+
+
+
+ fs.contract.is-banded-seek
+ false
+
+ BOS does not use banded seeking.
+
+
+
+
+
+
+
+
+ fs.contract.rename-overwrites-dest
+ true
+
+ BOS rename will overwrite the destination if it exists.
+ Note: This is implemented via copy + delete.
+
+
+
+
+ fs.contract.rename-returns-false-if-source-missing
+ true
+
+ BOS rename returns false if source does not exist.
+
+
+
+
+ fs.contract.rename-returns-false-if-dest-parent-missing
+ false
+
+ BOS rename will create parent directories of destination if needed.
+ This is set to false because rename-creates-dest-dirs is true.
+
+
+
+
+ fs.contract.rename-remove-dest-if-empty-dir
+ false
+
+ BOS rename will remove destination if it is an empty directory.
+
+
+
+
+ fs.contract.rename-creates-dest-dirs
+ true
+
+ BOS rename will create parent directories of destination if needed.
+
+
+
+
+
+
+
+
+ fs.contract.supports-atomic-delete
+ false
+
+ BOS delete operations are not atomic (implemented via multiple API calls).
+
+
+
+
+ fs.contract.supports-atomic-rename
+ false
+
+ BOS rename is not atomic (implemented via copy + delete).
+
+
+
+
+
+
+
+
+ fs.contract.create-overwrites-directory
+ false
+
+ BOS create() will not overwrite a directory.
+
+
+
+
+ fs.contract.supports-getfilestatus
+ true
+
+ BOS supports getFileStatus operation.
+
+
+
+
+ fs.contract.supports-block-locality
+ true
+
+ BOS provides block location information (though it may be synthetic).
+
+
+
+
+ fs.contract.is-emulated-directory
+ true
+
+ BOS emulates directories in object storage mode.
+ Directories are not immediately visible until explicitly created.
+
+
+
+
+
+
+
+
+ fs.contract.supports-unix-permissions
+ false
+
+ BOS does not fully support Unix-style permissions.
+ Note: Hierarchical namespace buckets have limited permission support.
+
+
+
+
+ fs.contract.supports-strict-exceptions
+ true
+
+ BOS throws specific exception types (FileNotFoundException, etc.)
+ rather than generic IOException.
+
+
+
+
+
+
+
+
+ fs.contract.supports-symlinks
+ false
+
+ BOS does not support symbolic links.
+
+
+
+
+ fs.contract.supports-xattr
+ false
+
+ BOS does not support extended attributes (xattrs).
+
+
+
+
+ fs.contract.supports-availablebytes
+ true
+
+ BOS InputStream supports available() method.
+
+
+
+
+
+
+
+
+ fs.contract.supports-content-summary
+ true
+
+ BOS supports getContentSummary() operation.
+
+
+
+
+
+
+
+
+ fs.contract.test-root-tests-enabled
+ false
+
+ Root directory tests are disabled for BOS to avoid
+ destructive operations on the bucket root.
+
+
+
+
+
+
+
+
+ fs.contract.supports-unbuffer
+ false
+
+ BOS does not support the unbuffer() operation on streams.
+
+
+
+
\ No newline at end of file
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/core-site.xml b/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/core-site.xml
new file mode 100644
index 0000000000000..429259757c895
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/core-site.xml
@@ -0,0 +1,76 @@
+
+
+
+
+
+
+
+
+
+ hadoop.tmp.dir
+ target/build/test
+ A base for other temporary directories.
+ true
+
+
+
+
+ hadoop.security.authentication
+ simple
+
+
+
+ fs.bos.impl
+ org.apache.hadoop.fs.bos.BaiduBosFileSystem
+ The implementation class of the BOS FileSystem.
+
+
+
+
+
+
+
+
diff --git a/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/log4j.properties b/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/log4j.properties
new file mode 100644
index 0000000000000..a3398d499bee8
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-bos/src/test/resources/log4j.properties
@@ -0,0 +1,346 @@
+# 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.
+
+# Define some default values that can be overridden by system properties
+hadoop.root.logger=INFO,console
+hadoop.log.dir=.
+hadoop.log.file=hadoop.log
+
+# Define the root logger to the system property "hadoop.root.logger".
+log4j.rootLogger=${hadoop.root.logger}, EventCounter
+
+# Logging Threshold
+log4j.threshold=ALL
+
+# Null Appender
+log4j.appender.NullAppender=org.apache.log4j.varia.NullAppender
+
+#
+# Rolling File Appender - cap space usage at 5gb.
+#
+hadoop.log.maxfilesize=256MB
+hadoop.log.maxbackupindex=20
+log4j.appender.RFA=org.apache.log4j.RollingFileAppender
+log4j.appender.RFA.File=${hadoop.log.dir}/${hadoop.log.file}
+
+log4j.appender.RFA.MaxFileSize=${hadoop.log.maxfilesize}
+log4j.appender.RFA.MaxBackupIndex=${hadoop.log.maxbackupindex}
+
+log4j.appender.RFA.layout=org.apache.log4j.PatternLayout
+
+# Pattern format: Date LogLevel LoggerName LogMessage
+log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+# Debugging Pattern format
+#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+
+#
+# Daily Rolling File Appender
+#
+
+log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.DRFA.File=${hadoop.log.dir}/${hadoop.log.file}
+
+# Rollover at midnight
+log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
+
+log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
+
+# Pattern format: Date LogLevel LoggerName LogMessage
+log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+# Debugging Pattern format
+#log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+
+#
+# console
+# Add "console" to rootlogger above if you want to use this
+#
+
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
+
+#
+# TaskLog Appender
+#
+log4j.appender.TLA=org.apache.hadoop.mapred.TaskLogAppender
+
+log4j.appender.TLA.layout=org.apache.log4j.PatternLayout
+log4j.appender.TLA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+
+#
+# HDFS block state change log from block manager
+#
+# Uncomment the following to log normal block state change
+# messages from BlockManager in NameNode.
+#log4j.logger.BlockStateChange=DEBUG
+
+#
+#Security appender
+#
+hadoop.security.logger=INFO,NullAppender
+hadoop.security.log.maxfilesize=256MB
+hadoop.security.log.maxbackupindex=20
+log4j.category.SecurityLogger=${hadoop.security.logger}
+hadoop.security.log.file=SecurityAuth-${user.name}.audit
+log4j.appender.RFAS=org.apache.log4j.RollingFileAppender
+log4j.appender.RFAS.File=${hadoop.log.dir}/${hadoop.security.log.file}
+log4j.appender.RFAS.layout=org.apache.log4j.PatternLayout
+log4j.appender.RFAS.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+log4j.appender.RFAS.MaxFileSize=${hadoop.security.log.maxfilesize}
+log4j.appender.RFAS.MaxBackupIndex=${hadoop.security.log.maxbackupindex}
+
+#
+# Daily Rolling Security appender
+#
+log4j.appender.DRFAS=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.DRFAS.File=${hadoop.log.dir}/${hadoop.security.log.file}
+log4j.appender.DRFAS.layout=org.apache.log4j.PatternLayout
+log4j.appender.DRFAS.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+log4j.appender.DRFAS.DatePattern=.yyyy-MM-dd
+
+#
+# hadoop configuration logging
+#
+
+# Uncomment the following line to turn off configuration deprecation warnings.
+# log4j.logger.org.apache.hadoop.conf.Configuration.deprecation=WARN
+
+#
+# hdfs audit logging
+#
+hdfs.audit.logger=INFO,NullAppender
+hdfs.audit.log.maxfilesize=256MB
+hdfs.audit.log.maxbackupindex=20
+log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=${hdfs.audit.logger}
+log4j.additivity.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=false
+log4j.appender.RFAAUDIT=org.apache.log4j.RollingFileAppender
+log4j.appender.RFAAUDIT.File=${hadoop.log.dir}/hdfs-audit.log
+log4j.appender.RFAAUDIT.layout=org.apache.log4j.PatternLayout
+log4j.appender.RFAAUDIT.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
+log4j.appender.RFAAUDIT.MaxFileSize=${hdfs.audit.log.maxfilesize}
+log4j.appender.RFAAUDIT.MaxBackupIndex=${hdfs.audit.log.maxbackupindex}
+
+#
+# NameNode metrics logging.
+# The default is to retain two namenode-metrics.log files up to 64MB each.
+#
+namenode.metrics.logger=INFO,NullAppender
+log4j.logger.NameNodeMetricsLog=${namenode.metrics.logger}
+log4j.additivity.NameNodeMetricsLog=false
+log4j.appender.NNMETRICSRFA=org.apache.log4j.RollingFileAppender
+log4j.appender.NNMETRICSRFA.File=${hadoop.log.dir}/namenode-metrics.log
+log4j.appender.NNMETRICSRFA.layout=org.apache.log4j.PatternLayout
+log4j.appender.NNMETRICSRFA.layout.ConversionPattern=%d{ISO8601} %m%n
+log4j.appender.NNMETRICSRFA.MaxBackupIndex=1
+log4j.appender.NNMETRICSRFA.MaxFileSize=64MB
+
+#
+# DataNode metrics logging.
+# The default is to retain two datanode-metrics.log files up to 64MB each.
+#
+datanode.metrics.logger=INFO,NullAppender
+log4j.logger.DataNodeMetricsLog=${datanode.metrics.logger}
+log4j.additivity.DataNodeMetricsLog=false
+log4j.appender.DNMETRICSRFA=org.apache.log4j.RollingFileAppender
+log4j.appender.DNMETRICSRFA.File=${hadoop.log.dir}/datanode-metrics.log
+log4j.appender.DNMETRICSRFA.layout=org.apache.log4j.PatternLayout
+log4j.appender.DNMETRICSRFA.layout.ConversionPattern=%d{ISO8601} %m%n
+log4j.appender.DNMETRICSRFA.MaxBackupIndex=1
+log4j.appender.DNMETRICSRFA.MaxFileSize=64MB
+
+# Custom Logging levels
+
+#log4j.logger.org.apache.hadoop.mapred.JobTracker=DEBUG
+#log4j.logger.org.apache.hadoop.mapred.TaskTracker=DEBUG
+#log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=DEBUG
+
+
+# AWS SDK & S3A FileSystem
+#log4j.logger.com.amazonaws=ERROR
+log4j.logger.com.amazonaws.http.AmazonHttpClient=ERROR
+#log4j.logger.org.apache.hadoop.fs.s3a.S3AFileSystem=WARN
+
+#
+# Job Summary Appender
+#
+# Use following logger to send summary to separate file defined by
+# hadoop.mapreduce.jobsummary.log.file :
+# hadoop.mapreduce.jobsummary.logger=INFO,JSA
+#
+hadoop.mapreduce.jobsummary.logger=${hadoop.root.logger}
+hadoop.mapreduce.jobsummary.log.file=hadoop-mapreduce.jobsummary.log
+hadoop.mapreduce.jobsummary.log.maxfilesize=256MB
+hadoop.mapreduce.jobsummary.log.maxbackupindex=20
+log4j.appender.JSA=org.apache.log4j.RollingFileAppender
+log4j.appender.JSA.File=${hadoop.log.dir}/${hadoop.mapreduce.jobsummary.log.file}
+log4j.appender.JSA.MaxFileSize=${hadoop.mapreduce.jobsummary.log.maxfilesize}
+log4j.appender.JSA.MaxBackupIndex=${hadoop.mapreduce.jobsummary.log.maxbackupindex}
+log4j.appender.JSA.layout=org.apache.log4j.PatternLayout
+log4j.appender.JSA.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
+log4j.logger.org.apache.hadoop.mapred.JobInProgress$JobSummary=${hadoop.mapreduce.jobsummary.logger}
+log4j.additivity.org.apache.hadoop.mapred.JobInProgress$JobSummary=false
+
+#
+# shuffle connection log from shuffleHandler
+# Uncomment the following line to enable logging of shuffle connections
+# log4j.logger.org.apache.hadoop.mapred.ShuffleHandler.audit=DEBUG
+
+#
+# Yarn ResourceManager Application Summary Log
+#
+# Set the ResourceManager summary log filename
+yarn.server.resourcemanager.appsummary.log.file=rm-appsummary.log
+# Set the ResourceManager summary log level and appender
+yarn.server.resourcemanager.appsummary.logger=${hadoop.root.logger}
+#yarn.server.resourcemanager.appsummary.logger=INFO,RMSUMMARY
+
+# To enable AppSummaryLogging for the RM,
+# set yarn.server.resourcemanager.appsummary.logger to
+# ,RMSUMMARY in hadoop-env.sh
+
+# Appender for ResourceManager Application Summary Log
+# Requires the following properties to be set
+# - hadoop.log.dir (Hadoop Log directory)
+# - yarn.server.resourcemanager.appsummary.log.file (resource manager app summary log filename)
+# - yarn.server.resourcemanager.appsummary.logger (resource manager app summary log level and appender)
+
+log4j.logger.org.apache.hadoop.yarn.server.resourcemanager.RMAppManager$ApplicationSummary=${yarn.server.resourcemanager.appsummary.logger}
+log4j.additivity.org.apache.hadoop.yarn.server.resourcemanager.RMAppManager$ApplicationSummary=false
+log4j.appender.RMSUMMARY=org.apache.log4j.RollingFileAppender
+log4j.appender.RMSUMMARY.File=${hadoop.log.dir}/${yarn.server.resourcemanager.appsummary.log.file}
+log4j.appender.RMSUMMARY.MaxFileSize=256MB
+log4j.appender.RMSUMMARY.MaxBackupIndex=20
+log4j.appender.RMSUMMARY.layout=org.apache.log4j.PatternLayout
+log4j.appender.RMSUMMARY.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
+
+#
+# YARN ResourceManager audit logging
+#
+rm.audit.logger=INFO,NullAppender
+rm.audit.log.maxfilesize=256MB
+rm.audit.log.maxbackupindex=20
+log4j.logger.org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger=${rm.audit.logger}
+log4j.additivity.org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger=false
+log4j.appender.RMAUDIT=org.apache.log4j.RollingFileAppender
+log4j.appender.RMAUDIT.File=${hadoop.log.dir}/rm-audit.log
+log4j.appender.RMAUDIT.layout=org.apache.log4j.PatternLayout
+log4j.appender.RMAUDIT.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
+log4j.appender.RMAUDIT.MaxFileSize=${rm.audit.log.maxfilesize}
+log4j.appender.RMAUDIT.MaxBackupIndex=${rm.audit.log.maxbackupindex}
+
+#
+# YARN NodeManager audit logging
+#
+nm.audit.logger=INFO,NullAppender
+nm.audit.log.maxfilesize=256MB
+nm.audit.log.maxbackupindex=20
+log4j.logger.org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger=${nm.audit.logger}
+log4j.additivity.org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger=false
+log4j.appender.NMAUDIT=org.apache.log4j.RollingFileAppender
+log4j.appender.NMAUDIT.File=${hadoop.log.dir}/nm-audit.log
+log4j.appender.NMAUDIT.layout=org.apache.log4j.PatternLayout
+log4j.appender.NMAUDIT.layout.ConversionPattern=%d{ISO8601}%p %c{2}: %m%n
+log4j.appender.NMAUDIT.MaxFileSize=${nm.audit.log.maxfilesize}
+log4j.appender.NMAUDIT.MaxBackupIndex=${nm.audit.log.maxbackupindex}
+
+# HS audit log configs
+#mapreduce.hs.audit.logger=INFO,HSAUDIT
+#log4j.logger.org.apache.hadoop.mapreduce.v2.hs.HSAuditLogger=${mapreduce.hs.audit.logger}
+#log4j.additivity.org.apache.hadoop.mapreduce.v2.hs.HSAuditLogger=false
+#log4j.appender.HSAUDIT=org.apache.log4j.DailyRollingFileAppender
+#log4j.appender.HSAUDIT.File=${hadoop.log.dir}/hs-audit.log
+#log4j.appender.HSAUDIT.layout=org.apache.log4j.PatternLayout
+#log4j.appender.HSAUDIT.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
+#log4j.appender.HSAUDIT.DatePattern=.yyyy-MM-dd
+
+# Http Server Request Logs
+#log4j.logger.http.requests.namenode=INFO,namenoderequestlog
+#log4j.appender.namenoderequestlog=org.apache.hadoop.http.HttpRequestLogAppender
+#log4j.appender.namenoderequestlog.Filename=${hadoop.log.dir}/jetty-namenode-yyyy_mm_dd.log
+#log4j.appender.namenoderequestlog.RetainDays=3
+
+#log4j.logger.http.requests.datanode=INFO,datanoderequestlog
+#log4j.appender.datanoderequestlog=org.apache.hadoop.http.HttpRequestLogAppender
+#log4j.appender.datanoderequestlog.Filename=${hadoop.log.dir}/jetty-datanode-yyyy_mm_dd.log
+#log4j.appender.datanoderequestlog.RetainDays=3
+
+#log4j.logger.http.requests.resourcemanager=INFO,resourcemanagerrequestlog
+#log4j.appender.resourcemanagerrequestlog=org.apache.hadoop.http.HttpRequestLogAppender
+#log4j.appender.resourcemanagerrequestlog.Filename=${hadoop.log.dir}/jetty-resourcemanager-yyyy_mm_dd.log
+#log4j.appender.resourcemanagerrequestlog.RetainDays=3
+
+#log4j.logger.http.requests.jobhistory=INFO,jobhistoryrequestlog
+#log4j.appender.jobhistoryrequestlog=org.apache.hadoop.http.HttpRequestLogAppender
+#log4j.appender.jobhistoryrequestlog.Filename=${hadoop.log.dir}/jetty-jobhistory-yyyy_mm_dd.log
+#log4j.appender.jobhistoryrequestlog.RetainDays=3
+
+#log4j.logger.http.requests.nodemanager=INFO,nodemanagerrequestlog
+#log4j.appender.nodemanagerrequestlog=org.apache.hadoop.http.HttpRequestLogAppender
+#log4j.appender.nodemanagerrequestlog.Filename=${hadoop.log.dir}/jetty-nodemanager-yyyy_mm_dd.log
+#log4j.appender.nodemanagerrequestlog.RetainDays=3
+
+
+# WebHdfs request log on datanodes
+# Specify -Ddatanode.webhdfs.logger=INFO,HTTPDRFA on datanode startup to
+# direct the log to a separate file.
+#datanode.webhdfs.logger=INFO,console
+#log4j.logger.datanode.webhdfs=${datanode.webhdfs.logger}
+#log4j.appender.HTTPDRFA=org.apache.log4j.DailyRollingFileAppender
+#log4j.appender.HTTPDRFA.File=${hadoop.log.dir}/hadoop-datanode-webhdfs.log
+#log4j.appender.HTTPDRFA.layout=org.apache.log4j.PatternLayout
+#log4j.appender.HTTPDRFA.layout.ConversionPattern=%d{ISO8601} %m%n
+#log4j.appender.HTTPDRFA.DatePattern=.yyyy-MM-dd
+
+
+# Appender for viewing information for errors and warnings
+yarn.ewma.cleanupInterval=300
+yarn.ewma.messageAgeLimitSeconds=86400
+yarn.ewma.maxUniqueMessages=250
+log4j.appender.EWMA=org.apache.hadoop.yarn.util.Log4jWarningErrorMetricsAppender
+log4j.appender.EWMA.cleanupInterval=${yarn.ewma.cleanupInterval}
+log4j.appender.EWMA.messageAgeLimitSeconds=${yarn.ewma.messageAgeLimitSeconds}
+log4j.appender.EWMA.maxUniqueMessages=${yarn.ewma.maxUniqueMessages}
+
+#
+# Fair scheduler state dump
+#
+# Use following logger to dump the state to a separate file
+
+#log4j.logger.org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler.statedump=DEBUG,FSSTATEDUMP
+#log4j.additivity.org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler.statedump=false
+#log4j.appender.FSSTATEDUMP=org.apache.log4j.RollingFileAppender
+#log4j.appender.FSSTATEDUMP.File=${hadoop.log.dir}/fairscheduler-statedump.log
+#log4j.appender.FSSTATEDUMP.layout=org.apache.log4j.PatternLayout
+#log4j.appender.FSSTATEDUMP.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+#log4j.appender.FSSTATEDUMP.MaxFileSize=${hadoop.log.maxfilesize}
+#log4j.appender.FSSTATEDUMP.MaxBackupIndex=${hadoop.log.maxbackupindex}
+
+# Log levels of third-party libraries
+log4j.logger.org.apache.commons.beanutils=WARN
+
+# log4j.logger.org.apache.hadoop.fs.bos=DEBUG
+#log4j.logger.com.baidubce.request=DEBUG
+
+
+#log4j.logger.com.baidubce.http=DEBUG, console
+
+## All HTTP requests
+log4j.logger.bfs.com.baidubce.request=WARN
+
+log4j.logger.org.apache.hadoop.fs.bos=INFO
\ No newline at end of file
diff --git a/hadoop-cloud-storage-project/hadoop-cloud-storage-dist/pom.xml b/hadoop-cloud-storage-project/hadoop-cloud-storage-dist/pom.xml
index 18049a760d04d..122ef2443f06c 100644
--- a/hadoop-cloud-storage-project/hadoop-cloud-storage-dist/pom.xml
+++ b/hadoop-cloud-storage-project/hadoop-cloud-storage-dist/pom.xml
@@ -56,6 +56,7 @@
hadoop-aliyun-package
hadoop-aws-package
hadoop-azure-datalake-package
+ hadoop-bos-package
hadoop-cos-package
hadoop-gcp-package
hadoop-huaweicloud-package
@@ -67,6 +68,7 @@ mvn package -Pdist -DskipTests -Dtar -Dmaven.javadoc.skip=true \
-Dhadoop-aliyun-package \
-Dhadoop-aws-package \
-Dhadoop-azure-datalake-package \
+ -Dhadoop-bos-package \
-Dhadoop-cos-package \
-Dhadoop-gcp-package \
-Dhadoop-huaweicloud-package \
@@ -130,6 +132,17 @@ mvn package -Pdist -DskipTests -Dtar -Dmaven.javadoc.skip=true \
+
+ org.apache.hadoop
+ hadoop-bos
+ compile
+
+
+ *
+ *
+
+
+
org.apache.hadoop
hadoop-huaweicloud
@@ -284,6 +297,21 @@ mvn package -Pdist -DskipTests -Dtar -Dmaven.javadoc.skip=true \
+
+
+ hadoop-bos-package
+
+ hadoop-bos-package
+
+
+
+ org.apache.hadoop
+ hadoop-bos
+ compile
+
+
+
+
hadoop-huaweicloud-package
diff --git a/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml b/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml
index 0370a9186a538..c4916f839ddfd 100644
--- a/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml
+++ b/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml
@@ -120,6 +120,21 @@
hadoop-cos
compile
+
+ org.apache.hadoop
+ hadoop-bos
+ compile
+
+
+
+ *
+ *
+
+
+
org.apache.hadoop
hadoop-huaweicloud
diff --git a/hadoop-cloud-storage-project/pom.xml b/hadoop-cloud-storage-project/pom.xml
index 43d4bc7386aaf..370cdae415607 100644
--- a/hadoop-cloud-storage-project/pom.xml
+++ b/hadoop-cloud-storage-project/pom.xml
@@ -31,6 +31,7 @@
hadoop-cloud-storage
+ hadoop-bos
hadoop-cos
hadoop-gcp
hadoop-huaweicloud
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 15c3df947c33d..38bb043d3424f 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -772,6 +772,12 @@
${hadoop.version}
+
+ org.apache.hadoop
+ hadoop-bos
+ ${hadoop.version}
+
+
org.apache.hadoop
hadoop-huaweicloud