tryGetConstructor(final Class mainClss,
+ final Class[] args) {
+ try {
+ Constructor constructor = mainClss.getDeclaredConstructor(args);
+ return Optional.ofNullable(constructor);
+ } catch (NoSuchMethodException e) {
+ // ignore
+ return Optional.empty();
+ }
+ }
+
+ @Override
+ public ObsClient createObsClient(final URI name) throws IOException {
+ Configuration conf = getConf();
+ ExtObsConfiguration obsConf = new ExtObsConfiguration();
+ initConnectionSettings(conf, obsConf);
+ initProxySupport(conf, obsConf);
+
+ return createHuaweiObsClient(conf, obsConf, name);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/FileConflictException.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/FileConflictException.java
new file mode 100644
index 00000000000..7384251b708
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/FileConflictException.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.obs;
+
+import java.io.IOException;
+
+/**
+ * OBS file conflict exception.
+ */
+class FileConflictException extends IOException {
+ private static final long serialVersionUID = -897856973823710492L;
+
+ /**
+ * Constructs a FileConflictException
with the specified detail
+ * message. The string s
can be retrieved later by the
+ * {@link Throwable#getMessage}
+ * method of class java.lang.Throwable
.
+ *
+ * @param s the detail message.
+ */
+ FileConflictException(final String s) {
+ super(s);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBS.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBS.java
new file mode 100644
index 00000000000..3f05f007ee5
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBS.java
@@ -0,0 +1,53 @@
+/*
+ * 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.obs;
+
+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;
+
+/**
+ * OBS implementation of AbstractFileSystem, which delegates to the {@link
+ * OBSFileSystem}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class OBS extends DelegateToFileSystem {
+
+ /**
+ * @param theUri URI of the file system
+ * @param conf Configuration for the file system
+ * @throws IOException on any failure to initialize this instance
+ * @throws URISyntaxException theUri
has syntax error
+ */
+ public OBS(final URI theUri, final Configuration conf)
+ throws IOException, URISyntaxException {
+ super(theUri, new OBSFileSystem(), conf, "obs", false);
+ }
+
+ @Override
+ public int getUriDefaultPort() {
+ return OBSConstants.OBS_DEFAULT_PORT;
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSBlockOutputStream.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSBlockOutputStream.java
new file mode 100644
index 00000000000..d1967402776
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSBlockOutputStream.java
@@ -0,0 +1,814 @@
+/*
+ * 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.obs;
+
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures;
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableFuture;
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService;
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecutors;
+import com.obs.services.exception.ObsException;
+import com.obs.services.model.CompleteMultipartUploadResult;
+import com.obs.services.model.PartEtag;
+import com.obs.services.model.PutObjectRequest;
+import com.obs.services.model.UploadPartRequest;
+import com.obs.services.model.UploadPartResult;
+import com.obs.services.model.fs.WriteFileRequest;
+import com.sun.istack.NotNull;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Syncable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * OBS output stream based on block buffering.
+ *
+ * Upload files/parts directly via different buffering mechanisms: including
+ * memory and disk.
+ *
+ *
If the stream is closed and no update has started, then the upload is
+ * instead done as a single PUT operation.
+ *
+ *
Unstable: statistics and error handling might evolve.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+class OBSBlockOutputStream extends OutputStream implements Syncable {
+
+ /**
+ * Class logger.
+ */
+ private static final Logger LOG = LoggerFactory.getLogger(
+ OBSBlockOutputStream.class);
+
+ /**
+ * Owner FileSystem.
+ */
+ private final OBSFileSystem fs;
+
+ /**
+ * Key of the object being uploaded.
+ */
+ private final String key;
+
+ /**
+ * Length of object.
+ */
+ private long objectLen;
+
+ /**
+ * Size of all blocks.
+ */
+ private final int blockSize;
+
+ /**
+ * Callback for progress.
+ */
+ private final ListeningExecutorService executorService;
+
+ /**
+ * Factory for creating blocks.
+ */
+ private final OBSDataBlocks.BlockFactory blockFactory;
+
+ /**
+ * Preallocated byte buffer for writing single characters.
+ */
+ private final byte[] singleCharWrite = new byte[1];
+
+ /**
+ * Closed flag.
+ */
+ private final AtomicBoolean closed = new AtomicBoolean(false);
+
+ /**
+ * Has exception flag.
+ */
+ private final AtomicBoolean hasException = new AtomicBoolean(false);
+
+ /**
+ * Has flushed flag.
+ */
+ private final AtomicBoolean appendAble;
+
+ /**
+ * Multipart upload details; null means none started.
+ */
+ private MultiPartUpload multiPartUpload;
+
+ /**
+ * Current data block. Null means none currently active.
+ */
+ private OBSDataBlocks.DataBlock activeBlock;
+
+ /**
+ * Count of blocks uploaded.
+ */
+ private long blockCount = 0;
+
+ /**
+ * Write operation helper; encapsulation of the filesystem operations.
+ */
+ private OBSWriteOperationHelper writeOperationHelper;
+
+ /**
+ * Flag for mocking upload part error.
+ */
+ private boolean mockUploadPartError = false;
+
+ /**
+ * An OBS output stream which uploads partitions in a separate pool of
+ * threads; different {@link OBSDataBlocks.BlockFactory} instances can control
+ * where data is buffered.
+ *
+ * @param owner OBSFilesystem
+ * @param obsObjectKey OBS object to work on
+ * @param objLen object length
+ * @param execService the executor service to use to schedule work
+ * @param isAppendable if append is supported
+ * @throws IOException on any problem
+ */
+ OBSBlockOutputStream(
+ final OBSFileSystem owner,
+ final String obsObjectKey,
+ final long objLen,
+ final ExecutorService execService,
+ final boolean isAppendable)
+ throws IOException {
+ this.appendAble = new AtomicBoolean(isAppendable);
+ this.fs = owner;
+ this.key = obsObjectKey;
+ this.objectLen = objLen;
+ this.blockFactory = owner.getBlockFactory();
+ this.blockSize = (int) owner.getPartSize();
+ this.writeOperationHelper = owner.getWriteHelper();
+ Preconditions.checkArgument(
+ owner.getPartSize() >= OBSConstants.MULTIPART_MIN_SIZE,
+ "Block size is too small: %d", owner.getPartSize());
+ this.executorService = MoreExecutors.listeningDecorator(
+ execService);
+ this.multiPartUpload = null;
+ // create that first block. This guarantees that an open + close
+ // sequence writes a 0-byte entry.
+ createBlockIfNeeded();
+ LOG.debug(
+ "Initialized OBSBlockOutputStream for {}" + " output to {}",
+ owner.getWriteHelper(),
+ activeBlock);
+ }
+
+ /**
+ * Demand create a destination block.
+ *
+ * @return the active block; null if there isn't one.
+ * @throws IOException on any failure to create
+ */
+ private synchronized OBSDataBlocks.DataBlock createBlockIfNeeded()
+ throws IOException {
+ if (activeBlock == null) {
+ blockCount++;
+ if (blockCount >= OBSConstants.MAX_MULTIPART_COUNT) {
+ LOG.warn(
+ "Number of partitions in stream exceeds limit for OBS: "
+ + OBSConstants.MAX_MULTIPART_COUNT
+ + " write may fail.");
+ }
+ activeBlock = blockFactory.create(blockCount, this.blockSize);
+ }
+ return activeBlock;
+ }
+
+ /**
+ * Synchronized accessor to the active block.
+ *
+ * @return the active block; null if there isn't one.
+ */
+ synchronized OBSDataBlocks.DataBlock getActiveBlock() {
+ return activeBlock;
+ }
+
+ /**
+ * Set mock error.
+ *
+ * @param isException mock error
+ */
+ @VisibleForTesting
+ public void mockPutPartError(final boolean isException) {
+ this.mockUploadPartError = isException;
+ }
+
+ /**
+ * Predicate to query whether or not there is an active block.
+ *
+ * @return true if there is an active block.
+ */
+ private synchronized boolean hasActiveBlock() {
+ return activeBlock != null;
+ }
+
+ /**
+ * Clear the active block.
+ */
+ private synchronized void clearActiveBlock() {
+ if (activeBlock != null) {
+ LOG.debug("Clearing active block");
+ }
+ activeBlock = null;
+ }
+
+ /**
+ * Check for the filesystem being open.
+ *
+ * @throws IOException if the filesystem is closed.
+ */
+ private void checkOpen() throws IOException {
+ if (closed.get()) {
+ throw new IOException(
+ "Filesystem " + writeOperationHelper.toString(key) + " closed");
+ }
+ }
+
+ /**
+ * The flush operation does not trigger an upload; that awaits the next block
+ * being full. What it does do is call {@code flush() } on the current block,
+ * leaving it to choose how to react.
+ *
+ * @throws IOException Any IO problem.
+ */
+ @Override
+ public synchronized void flush() throws IOException {
+ checkOpen();
+ OBSDataBlocks.DataBlock dataBlock = getActiveBlock();
+ if (dataBlock != null) {
+ dataBlock.flush();
+ }
+ }
+
+ /**
+ * Writes a byte to the destination. If this causes the buffer to reach its
+ * limit, the actual upload is submitted to the threadpool.
+ *
+ * @param b the int of which the lowest byte is written
+ * @throws IOException on any problem
+ */
+ @Override
+ public synchronized void write(final int b) throws IOException {
+ singleCharWrite[0] = (byte) b;
+ write(singleCharWrite, 0, 1);
+ }
+
+ /**
+ * Writes a range of bytes from to the memory buffer. If this causes the
+ * buffer to reach its limit, the actual upload is submitted to the threadpool
+ * and the remainder of the array is written to memory (recursively).
+ *
+ * @param source byte array containing
+ * @param offset offset in array where to start
+ * @param len number of bytes to be written
+ * @throws IOException on any problem
+ */
+ @Override
+ public synchronized void write(@NotNull final byte[] source,
+ final int offset, final int len)
+ throws IOException {
+ if (hasException.get()) {
+ String closeWarning = String.format(
+ "write has error. bs : pre upload obs[%s] has error.", key);
+ LOG.warn(closeWarning);
+ throw new IOException(closeWarning);
+ }
+ OBSDataBlocks.validateWriteArgs(source, offset, len);
+ checkOpen();
+ if (len == 0) {
+ return;
+ }
+
+ OBSDataBlocks.DataBlock block = createBlockIfNeeded();
+ int written = block.write(source, offset, len);
+ int remainingCapacity = block.remainingCapacity();
+ try {
+ innerWrite(source, offset, len, written, remainingCapacity);
+ } catch (IOException e) {
+ LOG.error(
+ "Write data for key {} of bucket {} error, error message {}",
+ key, fs.getBucket(),
+ e.getMessage());
+ throw e;
+ }
+ }
+
+ private synchronized void innerWrite(final byte[] source, final int offset,
+ final int len,
+ final int written, final int remainingCapacity)
+ throws IOException {
+
+ if (written < len) {
+ // not everything was written the block has run out
+ // of capacity
+ // Trigger an upload then process the remainder.
+ LOG.debug(
+ "writing more data than block has capacity -triggering upload");
+ if (appendAble.get()) {
+ // to write a buffer then append to obs
+ LOG.debug("[Append] open stream and single write size {} "
+ + "greater than buffer size {}, append buffer to obs.",
+ len, blockSize);
+ flushCurrentBlock();
+ } else {
+ // block output stream logic, multi-part upload
+ uploadCurrentBlock();
+ }
+ // tail recursion is mildly expensive, but given buffer sizes
+ // must be MB. it's unlikely to recurse very deeply.
+ this.write(source, offset + written, len - written);
+ } else {
+ if (remainingCapacity == 0) {
+ // the whole buffer is done, trigger an upload
+ if (appendAble.get()) {
+ // to write a buffer then append to obs
+ LOG.debug("[Append] open stream and already write size "
+ + "equal to buffer size {}, append buffer to obs.",
+ blockSize);
+ flushCurrentBlock();
+ } else {
+ // block output stream logic, multi-part upload
+ uploadCurrentBlock();
+ }
+ }
+ }
+ }
+
+ /**
+ * Start an asynchronous upload of the current block.
+ *
+ * @throws IOException Problems opening the destination for upload or
+ * initializing the upload.
+ */
+ private synchronized void uploadCurrentBlock() throws IOException {
+ Preconditions.checkState(hasActiveBlock(), "No active block");
+ LOG.debug("Writing block # {}", blockCount);
+
+ try {
+ if (multiPartUpload == null) {
+ LOG.debug("Initiating Multipart upload");
+ multiPartUpload = new MultiPartUpload();
+ }
+ multiPartUpload.uploadBlockAsync(getActiveBlock());
+ } catch (IOException e) {
+ hasException.set(true);
+ LOG.error("Upload current block on ({}/{}) failed.", fs.getBucket(),
+ key, e);
+ throw e;
+ } finally {
+ // set the block to null, so the next write will create a new block.
+ clearActiveBlock();
+ }
+ }
+
+ /**
+ * Close the stream.
+ *
+ *
This will not return until the upload is complete or the attempt to
+ * perform the upload has failed. Exceptions raised in this method are
+ * indicative that the write has failed and data is at risk of being lost.
+ *
+ * @throws IOException on any failure.
+ */
+ @Override
+ public synchronized void close() throws IOException {
+ if (closed.getAndSet(true)) {
+ // already closed
+ LOG.debug("Ignoring close() as stream is already closed");
+ return;
+ }
+ if (hasException.get()) {
+ String closeWarning = String.format(
+ "closed has error. bs : pre write obs[%s] has error.", key);
+ LOG.warn(closeWarning);
+ throw new IOException(closeWarning);
+ }
+ // do upload
+ completeCurrentBlock();
+
+ // clear
+ clearHFlushOrSync();
+
+ // All end of write operations, including deleting fake parent
+ // directories
+ writeOperationHelper.writeSuccessful(key);
+ }
+
+ /**
+ * If flush has take place, need to append file, else to put object.
+ *
+ * @throws IOException any problem in append or put object
+ */
+ private synchronized void putObjectIfNeedAppend() throws IOException {
+ if (appendAble.get() && fs.exists(
+ OBSCommonUtils.keyToQualifiedPath(fs, key))) {
+ appendFsFile();
+ } else {
+ putObject();
+ }
+ }
+
+ /**
+ * Append posix file.
+ *
+ * @throws IOException any problem
+ */
+ private synchronized void appendFsFile() throws IOException {
+ LOG.debug("bucket is posix, to append file. key is {}", key);
+ final OBSDataBlocks.DataBlock block = getActiveBlock();
+ WriteFileRequest writeFileReq;
+ if (block instanceof OBSDataBlocks.DiskBlock) {
+ writeFileReq = OBSCommonUtils.newAppendFileRequest(fs, key,
+ objectLen, (File) block.startUpload());
+ } else {
+ writeFileReq = OBSCommonUtils.newAppendFileRequest(fs, key,
+ objectLen, (InputStream) block.startUpload());
+ }
+ OBSCommonUtils.appendFile(fs, writeFileReq);
+ objectLen += block.dataSize();
+ }
+
+ /**
+ * Upload the current block as a single PUT request; if the buffer is empty a
+ * 0-byte PUT will be invoked, as it is needed to create an entry at the far
+ * end.
+ *
+ * @throws IOException any problem.
+ */
+ private synchronized void putObject() throws IOException {
+ LOG.debug("Executing regular upload for {}",
+ writeOperationHelper.toString(key));
+
+ final OBSDataBlocks.DataBlock block = getActiveBlock();
+ clearActiveBlock();
+ final int size = block.dataSize();
+ final PutObjectRequest putObjectRequest;
+ if (block instanceof OBSDataBlocks.DiskBlock) {
+ putObjectRequest = writeOperationHelper.newPutRequest(key,
+ (File) block.startUpload());
+
+ } else {
+ putObjectRequest =
+ writeOperationHelper.newPutRequest(key,
+ (InputStream) block.startUpload(), size);
+
+ }
+ putObjectRequest.setAcl(fs.getCannedACL());
+ fs.getSchemeStatistics().incrementWriteOps(1);
+ try {
+ // the putObject call automatically closes the input
+ // stream afterwards.
+ writeOperationHelper.putObject(putObjectRequest);
+ } finally {
+ OBSCommonUtils.closeAll(block);
+ }
+ }
+
+ @Override
+ public synchronized String toString() {
+ final StringBuilder sb = new StringBuilder("OBSBlockOutputStream{");
+ sb.append(writeOperationHelper.toString());
+ sb.append(", blockSize=").append(blockSize);
+ OBSDataBlocks.DataBlock block = activeBlock;
+ if (block != null) {
+ sb.append(", activeBlock=").append(block);
+ }
+ sb.append('}');
+ return sb.toString();
+ }
+
+ public synchronized void sync() {
+ // need to do
+ }
+
+ @Override
+ public synchronized void hflush() throws IOException {
+ // hflush hsyn same
+ flushOrSync();
+ }
+
+ /**
+ * Flush local file or multipart to obs. focus: not posix bucket is not
+ * support
+ *
+ * @throws IOException io exception
+ */
+ private synchronized void flushOrSync() throws IOException {
+
+ checkOpen();
+ if (hasException.get()) {
+ String flushWarning = String.format(
+ "flushOrSync has error. bs : pre write obs[%s] has error.",
+ key);
+ LOG.warn(flushWarning);
+ throw new IOException(flushWarning);
+ }
+ if (fs.isFsBucket()) {
+ // upload
+ flushCurrentBlock();
+
+ // clear
+ clearHFlushOrSync();
+ } else {
+ LOG.warn("not posix bucket, not support hflush or hsync.");
+ flush();
+ }
+ }
+
+ /**
+ * Clear for hflush or hsync.
+ */
+ private synchronized void clearHFlushOrSync() {
+ appendAble.set(true);
+ multiPartUpload = null;
+ }
+
+ /**
+ * Upload block to obs.
+ *
+ * @param block block
+ * @param hasBlock jungle if has block
+ * @throws IOException io exception
+ */
+ private synchronized void uploadWriteBlocks(
+ final OBSDataBlocks.DataBlock block,
+ final boolean hasBlock)
+ throws IOException {
+ if (multiPartUpload == null) {
+ if (hasBlock) {
+ // no uploads of data have taken place, put the single block
+ // up. This must happen even if there is no data, so that 0 byte
+ // files are created.
+ putObjectIfNeedAppend();
+ }
+ } else {
+ // there has already been at least one block scheduled for upload;
+ // put up the current then wait
+ if (hasBlock && block.hasData()) {
+ // send last part
+ uploadCurrentBlock();
+ }
+ // wait for the partial uploads to finish
+ final List partETags
+ = multiPartUpload.waitForAllPartUploads();
+ // then complete the operation
+ multiPartUpload.complete(partETags);
+ }
+ LOG.debug("Upload complete for {}", writeOperationHelper.toString(key));
+ }
+
+ private synchronized void completeCurrentBlock() throws IOException {
+ OBSDataBlocks.DataBlock block = getActiveBlock();
+ boolean hasBlock = hasActiveBlock();
+ LOG.debug("{}: complete block #{}: current block= {}", this, blockCount,
+ hasBlock ? block : "(none)");
+ try {
+ uploadWriteBlocks(block, hasBlock);
+ } catch (IOException ioe) {
+ LOG.error("Upload data to obs error. io exception : {}",
+ ioe.getMessage());
+ throw ioe;
+ } catch (Exception e) {
+ LOG.error("Upload data to obs error. other exception : {}",
+ e.getMessage());
+ throw e;
+ } finally {
+ OBSCommonUtils.closeAll(block);
+ clearActiveBlock();
+ }
+ }
+
+ private synchronized void flushCurrentBlock() throws IOException {
+ OBSDataBlocks.DataBlock block = getActiveBlock();
+ boolean hasBlock = hasActiveBlock();
+ LOG.debug(
+ "{}: complete block #{}: current block= {}", this, blockCount,
+ hasBlock ? block : "(none)");
+ try {
+ uploadWriteBlocks(block, hasBlock);
+ } catch (IOException ioe) {
+ LOG.error("hflush data to obs error. io exception : {}",
+ ioe.getMessage());
+ hasException.set(true);
+ throw ioe;
+ } catch (Exception e) {
+ LOG.error("hflush data to obs error. other exception : {}",
+ e.getMessage());
+ hasException.set(true);
+ throw e;
+ } finally {
+ OBSCommonUtils.closeAll(block);
+ clearActiveBlock();
+ }
+ }
+
+ @Override
+ public synchronized void hsync() throws IOException {
+ flushOrSync();
+ }
+
+ /**
+ * Multiple partition upload.
+ */
+ private class MultiPartUpload {
+ /**
+ * Upload id for multipart upload.
+ */
+ private final String uploadId;
+
+ /**
+ * List for async part upload future.
+ */
+ private final List> partETagsFutures;
+
+ MultiPartUpload() throws IOException {
+ this.uploadId = writeOperationHelper.initiateMultiPartUpload(key);
+ this.partETagsFutures = new ArrayList<>(2);
+ LOG.debug(
+ "Initiated multi-part upload for {} with , the key is {}"
+ + "id '{}'",
+ writeOperationHelper,
+ uploadId,
+ key);
+ }
+
+ /**
+ * Upload a block of data asynchronously.
+ *
+ * @param block block to upload
+ * @throws IOException upload failure
+ */
+ private void uploadBlockAsync(final OBSDataBlocks.DataBlock block)
+ throws IOException {
+ LOG.debug("Queueing upload of {}", block);
+
+ final int size = block.dataSize();
+ final int currentPartNumber = partETagsFutures.size() + 1;
+ final UploadPartRequest request;
+ if (block instanceof OBSDataBlocks.DiskBlock) {
+ request = writeOperationHelper.newUploadPartRequest(
+ key,
+ uploadId,
+ currentPartNumber,
+ size,
+ (File) block.startUpload());
+ } else {
+ request = writeOperationHelper.newUploadPartRequest(
+ key,
+ uploadId,
+ currentPartNumber,
+ size,
+ (InputStream) block.startUpload());
+
+ }
+ ListenableFuture partETagFuture = executorService.submit(
+ () -> {
+ // this is the queued upload operation
+ LOG.debug("Uploading part {} for id '{}'",
+ currentPartNumber, uploadId);
+ // do the upload
+ PartEtag partETag = null;
+ try {
+ if (mockUploadPartError) {
+ throw new ObsException("mock upload part error");
+ }
+ UploadPartResult uploadPartResult
+ = OBSCommonUtils.uploadPart(fs, request);
+ partETag =
+ new PartEtag(uploadPartResult.getEtag(),
+ uploadPartResult.getPartNumber());
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Completed upload of {} to part {}",
+ block, partETag);
+ }
+ } catch (ObsException e) {
+ // catch all exception
+ hasException.set(true);
+ LOG.error("UploadPart failed (ObsException). {}",
+ OBSCommonUtils.translateException("UploadPart", key,
+ e).getMessage());
+ } finally {
+ // close the stream and block
+ OBSCommonUtils.closeAll(block);
+ }
+ return partETag;
+ });
+ partETagsFutures.add(partETagFuture);
+ }
+
+ /**
+ * Block awaiting all outstanding uploads to complete.
+ *
+ * @return list of results
+ * @throws IOException IO Problems
+ */
+ private List waitForAllPartUploads() throws IOException {
+ LOG.debug("Waiting for {} uploads to complete",
+ partETagsFutures.size());
+ try {
+ return Futures.allAsList(partETagsFutures).get();
+ } catch (InterruptedException ie) {
+ LOG.warn("Interrupted partUpload", ie);
+ LOG.debug("Cancelling futures");
+ for (ListenableFuture future : partETagsFutures) {
+ future.cancel(true);
+ }
+ // abort multipartupload
+ this.abort();
+ throw new IOException(
+ "Interrupted multi-part upload with id '" + uploadId
+ + "' to " + key);
+ } catch (ExecutionException ee) {
+ // there is no way of recovering so abort
+ // cancel all partUploads
+ LOG.debug("While waiting for upload completion", ee);
+ LOG.debug("Cancelling futures");
+ for (ListenableFuture future : partETagsFutures) {
+ future.cancel(true);
+ }
+ // abort multipartupload
+ this.abort();
+ throw OBSCommonUtils.extractException(
+ "Multi-part upload with id '" + uploadId + "' to " + key,
+ key, ee);
+ }
+ }
+
+ /**
+ * This completes a multipart upload. Sometimes it fails; here retries are
+ * handled to avoid losing all data on a transient failure.
+ *
+ * @param partETags list of partial uploads
+ * @return result for completing multipart upload
+ * @throws IOException on any problem
+ */
+ private CompleteMultipartUploadResult complete(
+ final List partETags) throws IOException {
+ String operation = String.format(
+ "Completing multi-part upload for key '%s',"
+ + " id '%s' with %s partitions ",
+ key, uploadId, partETags.size());
+ try {
+ LOG.debug(operation);
+ return writeOperationHelper.completeMultipartUpload(key,
+ uploadId, partETags);
+ } catch (ObsException e) {
+ throw OBSCommonUtils.translateException(operation, key, e);
+ }
+ }
+
+ /**
+ * Abort a multi-part upload. Retries are attempted on failures.
+ * IOExceptions are caught; this is expected to be run as a cleanup
+ * process.
+ */
+ void abort() {
+ String operation =
+ String.format(
+ "Aborting multi-part upload for '%s', id '%s",
+ writeOperationHelper, uploadId);
+ try {
+ LOG.debug(operation);
+ writeOperationHelper.abortMultipartUpload(key, uploadId);
+ } catch (ObsException e) {
+ LOG.warn(
+ "Unable to abort multipart upload, you may need to purge "
+ + "uploaded parts",
+ e);
+ }
+ }
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSClientFactory.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSClientFactory.java
new file mode 100644
index 00000000000..fbd54feae80
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSClientFactory.java
@@ -0,0 +1,46 @@
+/*
+ * 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.obs;
+
+import com.obs.services.ObsClient;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.io.IOException;
+import java.net.URI;
+
+/**
+ * Factory for creating OBS client instance to be used by {@link
+ * OBSFileSystem}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+interface OBSClientFactory {
+ /**
+ * Creates a new {@link ObsClient} client. This method accepts the OBS file
+ * system URI both in raw input form and validated form as separate arguments,
+ * because both values may be useful in logging.
+ *
+ * @param name raw input OBS file system URI
+ * @return OBS client
+ * @throws IOException IO problem
+ */
+ ObsClient createObsClient(URI name) throws IOException;
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSCommonUtils.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSCommonUtils.java
new file mode 100644
index 00000000000..ba7550bc0d6
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSCommonUtils.java
@@ -0,0 +1,1546 @@
+/*
+ * 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.obs;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
+import com.obs.services.ObsClient;
+import com.obs.services.exception.ObsException;
+import com.obs.services.model.AbortMultipartUploadRequest;
+import com.obs.services.model.DeleteObjectsRequest;
+import com.obs.services.model.DeleteObjectsResult;
+import com.obs.services.model.KeyAndVersion;
+import com.obs.services.model.ListMultipartUploadsRequest;
+import com.obs.services.model.ListObjectsRequest;
+import com.obs.services.model.MultipartUpload;
+import com.obs.services.model.MultipartUploadListing;
+import com.obs.services.model.ObjectListing;
+import com.obs.services.model.ObjectMetadata;
+import com.obs.services.model.ObsObject;
+import com.obs.services.model.PutObjectRequest;
+import com.obs.services.model.PutObjectResult;
+import com.obs.services.model.UploadPartRequest;
+import com.obs.services.model.UploadPartResult;
+import com.obs.services.model.fs.FSStatusEnum;
+import com.obs.services.model.fs.GetAttributeRequest;
+import com.obs.services.model.fs.GetBucketFSStatusRequest;
+import com.obs.services.model.fs.GetBucketFSStatusResult;
+import com.obs.services.model.fs.ObsFSAttribute;
+import com.obs.services.model.fs.WriteFileRequest;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.InvalidRequestException;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.security.ProviderUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.nio.file.AccessDeniedException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+
+/**
+ * Common utils for {@link OBSFileSystem}.
+ */
+final class OBSCommonUtils {
+ /**
+ * Class logger.
+ */
+ private static final Logger LOG = LoggerFactory.getLogger(
+ OBSCommonUtils.class);
+
+ /**
+ * Moved permanently response code.
+ */
+ static final int MOVED_PERMANENTLY_CODE = 301;
+
+ /**
+ * Unauthorized response code.
+ */
+ static final int UNAUTHORIZED_CODE = 401;
+
+ /**
+ * Forbidden response code.
+ */
+ static final int FORBIDDEN_CODE = 403;
+
+ /**
+ * Not found response code.
+ */
+ static final int NOT_FOUND_CODE = 404;
+
+ /**
+ * File conflict.
+ */
+ static final int CONFLICT_CODE = 409;
+
+ /**
+ * Gone response code.
+ */
+ static final int GONE_CODE = 410;
+
+ /**
+ * EOF response code.
+ */
+ static final int EOF_CODE = 416;
+
+ /**
+ * Core property for provider path. Duplicated here for consistent code across
+ * Hadoop version: {@value}.
+ */
+ static final String CREDENTIAL_PROVIDER_PATH
+ = "hadoop.security.credential.provider.path";
+
+ /**
+ * Max number of retry times.
+ */
+ static final int MAX_RETRY_TIME = 3;
+
+ /**
+ * Delay time between two retries.
+ */
+ static final int DELAY_TIME = 10;
+
+ /**
+ * Max number of listing keys for checking folder empty.
+ */
+ static final int MAX_KEYS_FOR_CHECK_FOLDER_EMPTY = 3;
+
+ /**
+ * Max number of listing keys for checking folder empty.
+ */
+ static final int BYTE_TO_INT_MASK = 0xFF;
+
+ private OBSCommonUtils() {
+ }
+
+ /**
+ * Get the fs status of the bucket.
+ *
+ * @param obs OBS client instance
+ * @param bucketName bucket name
+ * @return boolean value indicating if this bucket is a posix bucket
+ * @throws FileNotFoundException the bucket is absent
+ * @throws IOException any other problem talking to OBS
+ */
+ static boolean getBucketFsStatus(final ObsClient obs,
+ final String bucketName)
+ throws FileNotFoundException, IOException {
+ try {
+ GetBucketFSStatusRequest getBucketFsStatusRequest
+ = new GetBucketFSStatusRequest();
+ getBucketFsStatusRequest.setBucketName(bucketName);
+ GetBucketFSStatusResult getBucketFsStatusResult =
+ obs.getBucketFSStatus(getBucketFsStatusRequest);
+ FSStatusEnum fsStatus = getBucketFsStatusResult.getStatus();
+ return fsStatus == FSStatusEnum.ENABLED;
+ } catch (ObsException e) {
+ LOG.error(e.toString());
+ throw translateException("getBucketFsStatus", bucketName, e);
+ }
+ }
+
+ /**
+ * Turns a path (relative or otherwise) into an OBS key.
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param path input path, may be relative to the working dir
+ * @return a key excluding the leading "/", or, if it is the root path, ""
+ */
+ static String pathToKey(final OBSFileSystem owner, final Path path) {
+ Path absolutePath = path;
+ if (!path.isAbsolute()) {
+ absolutePath = new Path(owner.getWorkingDirectory(), path);
+ }
+
+ if (absolutePath.toUri().getScheme() != null && absolutePath.toUri()
+ .getPath()
+ .isEmpty()) {
+ return "";
+ }
+
+ return absolutePath.toUri().getPath().substring(1);
+ }
+
+ /**
+ * Turns a path (relative or otherwise) into an OBS key, adding a trailing "/"
+ * if the path is not the root and does not already have a "/" at the
+ * end.
+ *
+ * @param key obs key or ""
+ * @return the with a trailing "/", or, if it is the root key, "",
+ */
+ static String maybeAddTrailingSlash(final String key) {
+ if (!StringUtils.isEmpty(key) && !key.endsWith("/")) {
+ return key + '/';
+ } else {
+ return key;
+ }
+ }
+
+ /**
+ * Convert a path back to a key.
+ *
+ * @param key input key
+ * @return the path from this key
+ */
+ static Path keyToPath(final String key) {
+ return new Path("/" + key);
+ }
+
+ /**
+ * Convert a key to a fully qualified path.
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param key input key
+ * @return the fully qualified path including URI scheme and bucket name.
+ */
+ static Path keyToQualifiedPath(final OBSFileSystem owner,
+ final String key) {
+ return qualify(owner, keyToPath(key));
+ }
+
+ /**
+ * Qualify a path.
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param path path to qualify
+ * @return a qualified path.
+ */
+ static Path qualify(final OBSFileSystem owner, final Path path) {
+ return path.makeQualified(owner.getUri(), owner.getWorkingDirectory());
+ }
+
+ /**
+ * Delete obs key started '/'.
+ *
+ * @param key object key
+ * @return new key
+ */
+ static String maybeDeleteBeginningSlash(final String key) {
+ return !StringUtils.isEmpty(key) && key.startsWith("/") ? key.substring(
+ 1) : key;
+ }
+
+ /**
+ * Add obs key started '/'.
+ *
+ * @param key object key
+ * @return new key
+ */
+ static String maybeAddBeginningSlash(final String key) {
+ return !StringUtils.isEmpty(key) && !key.startsWith("/")
+ ? "/" + key
+ : key;
+ }
+
+ /**
+ * Translate an exception raised in an operation into an IOException. HTTP
+ * error codes are examined and can be used to build a more specific
+ * response.
+ *
+ * @param operation operation
+ * @param path path operated on (may be null)
+ * @param exception obs exception raised
+ * @return an IOE which wraps the caught exception.
+ */
+ static IOException translateException(
+ final String operation, final String path,
+ final ObsException exception) {
+ String message = String.format("%s%s: status [%d] - request id [%s] "
+ + "- error code [%s] - error message [%s] - trace :%s ",
+ operation, path != null ? " on " + path : "",
+ exception.getResponseCode(), exception.getErrorRequestId(),
+ exception.getErrorCode(),
+ exception.getErrorMessage(), exception);
+
+ IOException ioe;
+
+ int status = exception.getResponseCode();
+ switch (status) {
+ case MOVED_PERMANENTLY_CODE:
+ message =
+ String.format("Received permanent redirect response, "
+ + "status [%d] - request id [%s] - "
+ + "error code [%s] - message [%s]",
+ exception.getResponseCode(),
+ exception.getErrorRequestId(), exception.getErrorCode(),
+ exception.getErrorMessage());
+ ioe = new OBSIOException(message, exception);
+ break;
+ // permissions
+ case UNAUTHORIZED_CODE:
+ case FORBIDDEN_CODE:
+ ioe = new AccessDeniedException(path, null, message);
+ ioe.initCause(exception);
+ break;
+
+ // the object isn't there
+ case NOT_FOUND_CODE:
+ case GONE_CODE:
+ ioe = new FileNotFoundException(message);
+ ioe.initCause(exception);
+ break;
+
+ // out of range. This may happen if an object is overwritten with
+ // a shorter one while it is being read.
+ case EOF_CODE:
+ ioe = new EOFException(message);
+ break;
+
+ default:
+ // no specific exit code. Choose an IOE subclass based on the
+ // class
+ // of the caught exception
+ ioe = new OBSIOException(message, exception);
+ break;
+ }
+ return ioe;
+ }
+
+ /**
+ * Reject any request to delete an object where the key is root.
+ *
+ * @param bucket bucket name
+ * @param key key to validate
+ * @throws InvalidRequestException if the request was rejected due to a
+ * mistaken attempt to delete the root
+ * directory.
+ */
+ static void blockRootDelete(final String bucket, final String key)
+ throws InvalidRequestException {
+ if (key.isEmpty() || "/".equals(key)) {
+ throw new InvalidRequestException(
+ "Bucket " + bucket + " cannot be deleted");
+ }
+ }
+
+ /**
+ * Delete an object. Increments the {@code OBJECT_DELETE_REQUESTS} and write
+ * operation statistics.
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param key key to blob to delete.
+ * @throws IOException on any failure to delete object
+ */
+ static void deleteObject(final OBSFileSystem owner, final String key)
+ throws IOException {
+ blockRootDelete(owner.getBucket(), key);
+ ObsException lastException = null;
+ for (int retryTime = 1; retryTime <= MAX_RETRY_TIME; retryTime++) {
+ try {
+ owner.getObsClient().deleteObject(owner.getBucket(), key);
+ owner.getSchemeStatistics().incrementWriteOps(1);
+ return;
+ } catch (ObsException e) {
+ lastException = e;
+ LOG.warn("Delete path failed with [{}], "
+ + "retry time [{}] - request id [{}] - "
+ + "error code [{}] - error message [{}]",
+ e.getResponseCode(), retryTime, e.getErrorRequestId(),
+ e.getErrorCode(), e.getErrorMessage());
+ if (retryTime < MAX_RETRY_TIME) {
+ try {
+ Thread.sleep(DELAY_TIME);
+ } catch (InterruptedException ie) {
+ throw translateException("delete", key, e);
+ }
+ }
+ }
+ }
+ throw translateException(
+ String.format("retry max times [%s] delete failed", MAX_RETRY_TIME),
+ key, lastException);
+ }
+
+ /**
+ * Perform a bulk object delete operation. Increments the {@code
+ * OBJECT_DELETE_REQUESTS} and write operation statistics.
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param deleteRequest keys to delete on the obs-backend
+ * @throws IOException on any failure to delete objects
+ */
+ static void deleteObjects(final OBSFileSystem owner,
+ final DeleteObjectsRequest deleteRequest) throws IOException {
+ DeleteObjectsResult result;
+ deleteRequest.setQuiet(true);
+ try {
+ result = owner.getObsClient().deleteObjects(deleteRequest);
+ owner.getSchemeStatistics().incrementWriteOps(1);
+ } catch (ObsException e) {
+ LOG.warn("delete objects failed, request [{}], request id [{}] - "
+ + "error code [{}] - error message [{}]",
+ deleteRequest, e.getErrorRequestId(), e.getErrorCode(),
+ e.getErrorMessage());
+ for (KeyAndVersion keyAndVersion
+ : deleteRequest.getKeyAndVersionsList()) {
+ deleteObject(owner, keyAndVersion.getKey());
+ }
+ return;
+ }
+
+ // delete one by one if there is errors
+ if (result != null) {
+ List errorResults
+ = result.getErrorResults();
+ if (!errorResults.isEmpty()) {
+ LOG.warn("bulk delete {} objects, {} failed, begin to delete "
+ + "one by one.",
+ deleteRequest.getKeyAndVersionsList().size(),
+ errorResults.size());
+ for (DeleteObjectsResult.ErrorResult errorResult
+ : errorResults) {
+ deleteObject(owner, errorResult.getObjectKey());
+ }
+ }
+ }
+ }
+
+ /**
+ * Create a putObject request. Adds the ACL and metadata
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param key key of object
+ * @param metadata metadata header
+ * @param srcfile source file
+ * @return the request
+ */
+ static PutObjectRequest newPutObjectRequest(final OBSFileSystem owner,
+ final String key, final ObjectMetadata metadata, final File srcfile) {
+ Preconditions.checkNotNull(srcfile);
+ PutObjectRequest putObjectRequest = new PutObjectRequest(
+ owner.getBucket(), key, srcfile);
+ putObjectRequest.setAcl(owner.getCannedACL());
+ putObjectRequest.setMetadata(metadata);
+ if (owner.getSse().isSseCEnable()) {
+ putObjectRequest.setSseCHeader(owner.getSse().getSseCHeader());
+ } else if (owner.getSse().isSseKmsEnable()) {
+ putObjectRequest.setSseKmsHeader(owner.getSse().getSseKmsHeader());
+ }
+ return putObjectRequest;
+ }
+
+ /**
+ * Create a {@link PutObjectRequest} request. The metadata is assumed to have
+ * been configured with the size of the operation.
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param key key of object
+ * @param metadata metadata header
+ * @param inputStream source data.
+ * @return the request
+ */
+ static PutObjectRequest newPutObjectRequest(final OBSFileSystem owner,
+ final String key, final ObjectMetadata metadata,
+ final InputStream inputStream) {
+ Preconditions.checkNotNull(inputStream);
+ PutObjectRequest putObjectRequest = new PutObjectRequest(
+ owner.getBucket(), key, inputStream);
+ putObjectRequest.setAcl(owner.getCannedACL());
+ putObjectRequest.setMetadata(metadata);
+ if (owner.getSse().isSseCEnable()) {
+ putObjectRequest.setSseCHeader(owner.getSse().getSseCHeader());
+ } else if (owner.getSse().isSseKmsEnable()) {
+ putObjectRequest.setSseKmsHeader(owner.getSse().getSseKmsHeader());
+ }
+ return putObjectRequest;
+ }
+
+ /**
+ * PUT an object directly (i.e. not via the transfer manager). Byte length is
+ * calculated from the file length, or, if there is no file, from the content
+ * length of the header. Important: this call will close any input stream
+ * in the request.
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param putObjectRequest the request
+ * @return the upload initiated
+ * @throws ObsException on problems
+ */
+ static PutObjectResult putObjectDirect(final OBSFileSystem owner,
+ final PutObjectRequest putObjectRequest) throws ObsException {
+ long len;
+ if (putObjectRequest.getFile() != null) {
+ len = putObjectRequest.getFile().length();
+ } else {
+ len = putObjectRequest.getMetadata().getContentLength();
+ }
+
+ PutObjectResult result = owner.getObsClient()
+ .putObject(putObjectRequest);
+ owner.getSchemeStatistics().incrementWriteOps(1);
+ owner.getSchemeStatistics().incrementBytesWritten(len);
+ return result;
+ }
+
+ /**
+ * Upload part of a multi-partition file. Increments the write and put
+ * counters. Important: this call does not close any input stream in the
+ * request.
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param request request
+ * @return the result of the operation.
+ * @throws ObsException on problems
+ */
+ static UploadPartResult uploadPart(final OBSFileSystem owner,
+ final UploadPartRequest request) throws ObsException {
+ long len = request.getPartSize();
+ UploadPartResult uploadPartResult = owner.getObsClient()
+ .uploadPart(request);
+ owner.getSchemeStatistics().incrementWriteOps(1);
+ owner.getSchemeStatistics().incrementBytesWritten(len);
+ return uploadPartResult;
+ }
+
+ static void removeKeys(final OBSFileSystem owner,
+ final List keysToDelete, final boolean clearKeys,
+ final boolean checkRootDelete) throws IOException {
+ if (keysToDelete.isEmpty()) {
+ // exit fast if there are no keys to delete
+ return;
+ }
+
+ if (checkRootDelete) {
+ for (KeyAndVersion keyVersion : keysToDelete) {
+ blockRootDelete(owner.getBucket(), keyVersion.getKey());
+ }
+ }
+
+ if (!owner.isEnableMultiObjectDelete()
+ || keysToDelete.size() < owner.getMultiDeleteThreshold()) {
+ // delete one by one.
+ for (KeyAndVersion keyVersion : keysToDelete) {
+ deleteObject(owner, keyVersion.getKey());
+ }
+ } else if (keysToDelete.size() <= owner.getMaxEntriesToDelete()) {
+ // Only one batch.
+ DeleteObjectsRequest deleteObjectsRequest
+ = new DeleteObjectsRequest(owner.getBucket());
+ deleteObjectsRequest.setKeyAndVersions(
+ keysToDelete.toArray(new KeyAndVersion[0]));
+ deleteObjects(owner, deleteObjectsRequest);
+ } else {
+ // Multi batches.
+ List keys = new ArrayList<>(
+ owner.getMaxEntriesToDelete());
+ for (KeyAndVersion key : keysToDelete) {
+ keys.add(key);
+ if (keys.size() == owner.getMaxEntriesToDelete()) {
+ // Delete one batch.
+ removeKeys(owner, keys, true, false);
+ }
+ }
+ // Delete the last batch
+ removeKeys(owner, keys, true, false);
+ }
+
+ if (clearKeys) {
+ keysToDelete.clear();
+ }
+ }
+
+ /**
+ * Translate an exception raised in an operation into an IOException. The
+ * specific type of IOException depends on the class of {@link ObsException}
+ * passed in, and any status codes included in the operation. That is: HTTP
+ * error codes are examined and can be used to build a more specific
+ * response.
+ *
+ * @param operation operation
+ * @param path path operated on (must not be null)
+ * @param exception obs exception raised
+ * @return an IOE which wraps the caught exception.
+ */
+ static IOException translateException(final String operation,
+ final Path path, final ObsException exception) {
+ return translateException(operation, path.toString(), exception);
+ }
+
+ /**
+ * List the statuses of the files/directories in the given path if the path is
+ * a directory.
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param f given path
+ * @param recursive flag indicating if list is recursive
+ * @return the statuses of the files/directories in the given patch
+ * @throws FileNotFoundException when the path does not exist;
+ * @throws IOException due to an IO problem.
+ * @throws ObsException on failures inside the OBS SDK
+ */
+ static FileStatus[] innerListStatus(final OBSFileSystem owner, final Path f,
+ final boolean recursive)
+ throws FileNotFoundException, IOException, ObsException {
+ Path path = qualify(owner, f);
+ String key = pathToKey(owner, path);
+
+ List result;
+ final FileStatus fileStatus = owner.getFileStatus(path);
+
+ if (fileStatus.isDirectory()) {
+ key = maybeAddTrailingSlash(key);
+ String delimiter = recursive ? null : "/";
+ ListObjectsRequest request = createListObjectsRequest(owner, key,
+ delimiter);
+ LOG.debug(
+ "listStatus: doing listObjects for directory {} - recursive {}",
+ f, recursive);
+
+ OBSListing.FileStatusListingIterator files = owner.getObsListing()
+ .createFileStatusListingIterator(
+ path, request, OBSListing.ACCEPT_ALL,
+ new OBSListing.AcceptAllButSelfAndS3nDirs(path));
+ result = new ArrayList<>(files.getBatchSize());
+ while (files.hasNext()) {
+ result.add(files.next());
+ }
+
+ return result.toArray(new FileStatus[0]);
+ } else {
+ LOG.debug("Adding: rd (not a dir): {}", path);
+ FileStatus[] stats = new FileStatus[1];
+ stats[0] = fileStatus;
+ return stats;
+ }
+ }
+
+ /**
+ * Create a {@code ListObjectsRequest} request against this bucket.
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param key key for request
+ * @param delimiter any delimiter
+ * @return the request
+ */
+ static ListObjectsRequest createListObjectsRequest(
+ final OBSFileSystem owner, final String key, final String delimiter) {
+ return createListObjectsRequest(owner, key, delimiter, -1);
+ }
+
+ static ListObjectsRequest createListObjectsRequest(
+ final OBSFileSystem owner, final String key, final String delimiter,
+ final int maxKeyNum) {
+ ListObjectsRequest request = new ListObjectsRequest();
+ request.setBucketName(owner.getBucket());
+ if (maxKeyNum > 0 && maxKeyNum < owner.getMaxKeys()) {
+ request.setMaxKeys(maxKeyNum);
+ } else {
+ request.setMaxKeys(owner.getMaxKeys());
+ }
+ request.setPrefix(key);
+ if (delimiter != null) {
+ request.setDelimiter(delimiter);
+ }
+ return request;
+ }
+
+ /**
+ * Implements the specific logic to reject root directory deletion. The caller
+ * must return the result of this call, rather than attempt to continue with
+ * the delete operation: deleting root directories is never allowed. This
+ * method simply implements the policy of when to return an exit code versus
+ * raise an exception.
+ *
+ * @param bucket bucket name
+ * @param isEmptyDir flag indicating if the directory is empty
+ * @param recursive recursive flag from command
+ * @return a return code for the operation
+ * @throws PathIOException if the operation was explicitly rejected.
+ */
+ static boolean rejectRootDirectoryDelete(final String bucket,
+ final boolean isEmptyDir,
+ final boolean recursive)
+ throws IOException {
+ LOG.info("obs delete the {} root directory of {}", bucket, recursive);
+ if (isEmptyDir) {
+ return true;
+ }
+ if (recursive) {
+ return false;
+ } else {
+ // reject
+ throw new PathIOException(bucket, "Cannot delete root path");
+ }
+ }
+
+ /**
+ * Make the given path and all non-existent parents into directories.
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param path path to create
+ * @return true if a directory was created
+ * @throws FileAlreadyExistsException there is a file at the path specified
+ * @throws IOException other IO problems
+ * @throws ObsException on failures inside the OBS SDK
+ */
+ static boolean innerMkdirs(final OBSFileSystem owner, final Path path)
+ throws IOException, FileAlreadyExistsException, ObsException {
+ LOG.debug("Making directory: {}", path);
+ FileStatus fileStatus;
+ try {
+ fileStatus = owner.getFileStatus(path);
+
+ if (fileStatus.isDirectory()) {
+ return true;
+ } else {
+ throw new FileAlreadyExistsException("Path is a file: " + path);
+ }
+ } catch (FileNotFoundException e) {
+ Path fPart = path.getParent();
+ do {
+ try {
+ fileStatus = owner.getFileStatus(fPart);
+ if (fileStatus.isDirectory()) {
+ break;
+ }
+ if (fileStatus.isFile()) {
+ throw new FileAlreadyExistsException(
+ String.format("Can't make directory for path '%s'"
+ + " since it is a file.", fPart));
+ }
+ } catch (FileNotFoundException fnfe) {
+ LOG.debug("file {} not fount, but ignore.", path);
+ }
+ fPart = fPart.getParent();
+ } while (fPart != null);
+
+ String key = pathToKey(owner, path);
+ if (owner.isFsBucket()) {
+ OBSPosixBucketUtils.fsCreateFolder(owner, key);
+ } else {
+ OBSObjectBucketUtils.createFakeDirectory(owner, key);
+ }
+ return true;
+ }
+ }
+
+ /**
+ * Initiate a {@code listObjects} operation, incrementing metrics in the
+ * process.
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param request request to initiate
+ * @return the results
+ * @throws IOException on any failure to list objects
+ */
+ static ObjectListing listObjects(final OBSFileSystem owner,
+ final ListObjectsRequest request) throws IOException {
+ if (request.getDelimiter() == null && request.getMarker() == null
+ && owner.isFsBucket() && owner.isObsClientDFSListEnable()) {
+ return OBSFsDFSListing.fsDFSListObjects(owner, request);
+ }
+
+ return commonListObjects(owner, request);
+ }
+
+ static ObjectListing commonListObjects(final OBSFileSystem owner,
+ final ListObjectsRequest request) {
+ for (int retryTime = 1; retryTime < MAX_RETRY_TIME; retryTime++) {
+ try {
+ owner.getSchemeStatistics().incrementReadOps(1);
+ return owner.getObsClient().listObjects(request);
+ } catch (ObsException e) {
+ LOG.warn("Failed to commonListObjects for request[{}], retry "
+ + "time [{}], due to exception[{}]",
+ request, retryTime, e);
+ try {
+ Thread.sleep(DELAY_TIME);
+ } catch (InterruptedException ie) {
+ LOG.error("Failed to commonListObjects for request[{}], "
+ + "retry time [{}], due to exception[{}]",
+ request, retryTime, e);
+ throw e;
+ }
+ }
+ }
+
+ owner.getSchemeStatistics().incrementReadOps(1);
+ return owner.getObsClient().listObjects(request);
+ }
+
+ /**
+ * List the next set of objects.
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param objects paged result
+ * @return the next result object
+ * @throws IOException on any failure to list the next set of objects
+ */
+ static ObjectListing continueListObjects(final OBSFileSystem owner,
+ final ObjectListing objects) throws IOException {
+ if (objects.getDelimiter() == null && owner.isFsBucket()
+ && owner.isObsClientDFSListEnable()) {
+ return OBSFsDFSListing.fsDFSContinueListObjects(owner,
+ (OBSFsDFSListing) objects);
+ }
+
+ return commonContinueListObjects(owner, objects);
+ }
+
+ private static ObjectListing commonContinueListObjects(
+ final OBSFileSystem owner, final ObjectListing objects) {
+ String delimiter = objects.getDelimiter();
+ int maxKeyNum = objects.getMaxKeys();
+ // LOG.debug("delimiters: "+objects.getDelimiter());
+ ListObjectsRequest request = new ListObjectsRequest();
+ request.setMarker(objects.getNextMarker());
+ request.setBucketName(owner.getBucket());
+ request.setPrefix(objects.getPrefix());
+ if (maxKeyNum > 0 && maxKeyNum < owner.getMaxKeys()) {
+ request.setMaxKeys(maxKeyNum);
+ } else {
+ request.setMaxKeys(owner.getMaxKeys());
+ }
+ if (delimiter != null) {
+ request.setDelimiter(delimiter);
+ }
+ return commonContinueListObjects(owner, request);
+ }
+
+ static ObjectListing commonContinueListObjects(final OBSFileSystem owner,
+ final ListObjectsRequest request) {
+ for (int retryTime = 1; retryTime < MAX_RETRY_TIME; retryTime++) {
+ try {
+ owner.getSchemeStatistics().incrementReadOps(1);
+ return owner.getObsClient().listObjects(request);
+ } catch (ObsException e) {
+ LOG.warn("Continue list objects failed for request[{}], retry"
+ + " time[{}], due to exception[{}]",
+ request, retryTime, e);
+ try {
+ Thread.sleep(DELAY_TIME);
+ } catch (InterruptedException ie) {
+ LOG.error("Continue list objects failed for request[{}], "
+ + "retry time[{}], due to exception[{}]",
+ request, retryTime, e);
+ throw e;
+ }
+ }
+ }
+
+ owner.getSchemeStatistics().incrementReadOps(1);
+ return owner.getObsClient().listObjects(request);
+ }
+
+ /**
+ * Predicate: does the object represent a directory?.
+ *
+ * @param name object name
+ * @param size object size
+ * @return true if it meets the criteria for being an object
+ */
+ public static boolean objectRepresentsDirectory(final String name,
+ final long size) {
+ return !name.isEmpty() && name.charAt(name.length() - 1) == '/'
+ && size == 0L;
+ }
+
+ /**
+ * Date to long conversion. Handles null Dates that can be returned by OBS by
+ * returning 0
+ *
+ * @param date date from OBS query
+ * @return timestamp of the object
+ */
+ public static long dateToLong(final Date date) {
+ if (date == null) {
+ return 0L;
+ }
+
+ return date.getTime() / OBSConstants.SEC2MILLISEC_FACTOR
+ * OBSConstants.SEC2MILLISEC_FACTOR;
+ }
+
+ // Used to check if a folder is empty or not.
+ static boolean isFolderEmpty(final OBSFileSystem owner, final String key)
+ throws FileNotFoundException, ObsException {
+ for (int retryTime = 1; retryTime < MAX_RETRY_TIME; retryTime++) {
+ try {
+ return innerIsFolderEmpty(owner, key);
+ } catch (ObsException e) {
+ LOG.warn(
+ "Failed to check empty folder for [{}], retry time [{}], "
+ + "exception [{}]", key, retryTime, e);
+
+ try {
+ Thread.sleep(DELAY_TIME);
+ } catch (InterruptedException ie) {
+ throw e;
+ }
+ }
+ }
+
+ return innerIsFolderEmpty(owner, key);
+ }
+
+ // Used to check if a folder is empty or not by counting the number of
+ // sub objects in list.
+ private static boolean isFolderEmpty(final String key,
+ final ObjectListing objects) {
+ int count = objects.getObjects().size();
+ if (count >= 2) {
+ // There is a sub file at least.
+ return false;
+ } else if (count == 1 && !objects.getObjects()
+ .get(0)
+ .getObjectKey()
+ .equals(key)) {
+ // There is a sub file at least.
+ return false;
+ }
+
+ count = objects.getCommonPrefixes().size();
+ // There is a sub file at least.
+ // There is no sub object.
+ if (count >= 2) {
+ // There is a sub file at least.
+ return false;
+ } else {
+ return count != 1 || objects.getCommonPrefixes().get(0).equals(key);
+ }
+ }
+
+ // Used to check if a folder is empty or not.
+ static boolean innerIsFolderEmpty(final OBSFileSystem owner,
+ final String key)
+ throws FileNotFoundException, ObsException {
+ String obsKey = maybeAddTrailingSlash(key);
+ ListObjectsRequest request = new ListObjectsRequest();
+ request.setBucketName(owner.getBucket());
+ request.setPrefix(obsKey);
+ request.setDelimiter("/");
+ request.setMaxKeys(MAX_KEYS_FOR_CHECK_FOLDER_EMPTY);
+ owner.getSchemeStatistics().incrementReadOps(1);
+ ObjectListing objects = owner.getObsClient().listObjects(request);
+
+ if (!objects.getCommonPrefixes().isEmpty() || !objects.getObjects()
+ .isEmpty()) {
+ if (isFolderEmpty(obsKey, objects)) {
+ LOG.debug("Found empty directory {}", obsKey);
+ return true;
+ }
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Found path as directory (with /): {}/{}",
+ objects.getCommonPrefixes().size(),
+ objects.getObjects().size());
+
+ for (ObsObject summary : objects.getObjects()) {
+ LOG.debug("Summary: {} {}", summary.getObjectKey(),
+ summary.getMetadata().getContentLength());
+ }
+ for (String prefix : objects.getCommonPrefixes()) {
+ LOG.debug("Prefix: {}", prefix);
+ }
+ }
+ LOG.debug("Found non-empty directory {}", obsKey);
+ return false;
+ } else if (obsKey.isEmpty()) {
+ LOG.debug("Found root directory");
+ return true;
+ } else if (owner.isFsBucket()) {
+ LOG.debug("Found empty directory {}", obsKey);
+ return true;
+ }
+
+ LOG.debug("Not Found: {}", obsKey);
+ throw new FileNotFoundException("No such file or directory: " + obsKey);
+ }
+
+ /**
+ * Build a {@link LocatedFileStatus} from a {@link FileStatus} instance.
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param status file status
+ * @return a located status with block locations set up from this FS.
+ * @throws IOException IO Problems.
+ */
+ static LocatedFileStatus toLocatedFileStatus(final OBSFileSystem owner,
+ final FileStatus status) throws IOException {
+ return new LocatedFileStatus(
+ status, status.isFile() ? owner.getFileBlockLocations(status, 0,
+ status.getLen()) : null);
+ }
+
+ /**
+ * Create a appendFile request. Adds the ACL and metadata
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param key key of object
+ * @param tmpFile temp file or input stream
+ * @param recordPosition client record next append position
+ * @return the request
+ * @throws IOException any problem
+ */
+ static WriteFileRequest newAppendFileRequest(final OBSFileSystem owner,
+ final String key, final long recordPosition, final File tmpFile)
+ throws IOException {
+ Preconditions.checkNotNull(key);
+ Preconditions.checkNotNull(tmpFile);
+ ObsFSAttribute obsFsAttribute;
+ try {
+ GetAttributeRequest getAttributeReq = new GetAttributeRequest(
+ owner.getBucket(), key);
+ obsFsAttribute = owner.getObsClient().getAttribute(getAttributeReq);
+ } catch (ObsException e) {
+ throw translateException("GetAttributeRequest", key, e);
+ }
+
+ long appendPosition = Math.max(recordPosition,
+ obsFsAttribute.getContentLength());
+ if (recordPosition != obsFsAttribute.getContentLength()) {
+ LOG.warn("append url[{}] position[{}], file contentLength[{}] not"
+ + " equal to recordPosition[{}].", key, appendPosition,
+ obsFsAttribute.getContentLength(), recordPosition);
+ }
+ WriteFileRequest writeFileReq = new WriteFileRequest(owner.getBucket(),
+ key, tmpFile, appendPosition);
+ writeFileReq.setAcl(owner.getCannedACL());
+ return writeFileReq;
+ }
+
+ /**
+ * Create a appendFile request. Adds the ACL and metadata
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param key key of object
+ * @param inputStream temp file or input stream
+ * @param recordPosition client record next append position
+ * @return the request
+ * @throws IOException any problem
+ */
+ static WriteFileRequest newAppendFileRequest(final OBSFileSystem owner,
+ final String key, final long recordPosition,
+ final InputStream inputStream) throws IOException {
+ Preconditions.checkNotNull(key);
+ Preconditions.checkNotNull(inputStream);
+ ObsFSAttribute obsFsAttribute;
+ try {
+ GetAttributeRequest getAttributeReq = new GetAttributeRequest(
+ owner.getBucket(), key);
+ obsFsAttribute = owner.getObsClient().getAttribute(getAttributeReq);
+ } catch (ObsException e) {
+ throw translateException("GetAttributeRequest", key, e);
+ }
+
+ long appendPosition = Math.max(recordPosition,
+ obsFsAttribute.getContentLength());
+ if (recordPosition != obsFsAttribute.getContentLength()) {
+ LOG.warn("append url[{}] position[{}], file contentLength[{}] not"
+ + " equal to recordPosition[{}].", key, appendPosition,
+ obsFsAttribute.getContentLength(), recordPosition);
+ }
+ WriteFileRequest writeFileReq = new WriteFileRequest(owner.getBucket(),
+ key, inputStream, appendPosition);
+ writeFileReq.setAcl(owner.getCannedACL());
+ return writeFileReq;
+ }
+
+ /**
+ * Append File.
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param appendFileRequest append object request
+ * @throws IOException on any failure to append file
+ */
+ static void appendFile(final OBSFileSystem owner,
+ final WriteFileRequest appendFileRequest) throws IOException {
+ long len = 0;
+ if (appendFileRequest.getFile() != null) {
+ len = appendFileRequest.getFile().length();
+ }
+
+ try {
+ LOG.debug("Append file, key {} position {} size {}",
+ appendFileRequest.getObjectKey(),
+ appendFileRequest.getPosition(),
+ len);
+ owner.getObsClient().writeFile(appendFileRequest);
+ owner.getSchemeStatistics().incrementWriteOps(1);
+ owner.getSchemeStatistics().incrementBytesWritten(len);
+ } catch (ObsException e) {
+ throw translateException("AppendFile",
+ appendFileRequest.getObjectKey(), e);
+ }
+ }
+
+ /**
+ * Close the Closeable objects and ignore any Exception or null
+ * pointers. (This is the SLF4J equivalent of that in {@code IOUtils}).
+ *
+ * @param closeables the objects to close
+ */
+ static void closeAll(final java.io.Closeable... closeables) {
+ for (java.io.Closeable c : closeables) {
+ if (c != null) {
+ try {
+ if (LOG != null) {
+ LOG.debug("Closing {}", c);
+ }
+ c.close();
+ } catch (Exception e) {
+ if (LOG != null && LOG.isDebugEnabled()) {
+ LOG.debug("Exception in closing {}", c, e);
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Extract an exception from a failed future, and convert to an IOE.
+ *
+ * @param operation operation which failed
+ * @param path path operated on (may be null)
+ * @param ee execution exception
+ * @return an IOE which can be thrown
+ */
+ static IOException extractException(final String operation,
+ final String path, final ExecutionException ee) {
+ IOException ioe;
+ Throwable cause = ee.getCause();
+ if (cause instanceof ObsException) {
+ ioe = translateException(operation, path, (ObsException) cause);
+ } else if (cause instanceof IOException) {
+ ioe = (IOException) cause;
+ } else {
+ ioe = new IOException(operation + " failed: " + cause, cause);
+ }
+ return ioe;
+ }
+
+ /**
+ * Create a files status instance from a listing.
+ *
+ * @param keyPath path to entry
+ * @param summary summary from OBS
+ * @param blockSize block size to declare.
+ * @param owner owner of the file
+ * @return a status entry
+ */
+ static OBSFileStatus createFileStatus(
+ final Path keyPath, final ObsObject summary, final long blockSize,
+ final String owner) {
+ if (objectRepresentsDirectory(
+ summary.getObjectKey(), summary.getMetadata().getContentLength())) {
+ return new OBSFileStatus(keyPath, owner);
+ } else {
+ return new OBSFileStatus(
+ summary.getMetadata().getContentLength(),
+ dateToLong(summary.getMetadata().getLastModified()),
+ keyPath,
+ blockSize,
+ owner);
+ }
+ }
+
+ /**
+ * Return the access key and secret for OBS API use. Credentials may exist in
+ * configuration, within credential providers or indicated in the UserInfo of
+ * the name URI param.
+ *
+ * @param name the URI for which we need the access keys.
+ * @param conf the Configuration object to interrogate for keys.
+ * @return OBSAccessKeys
+ * @throws IOException problems retrieving passwords from KMS.
+ */
+ static OBSLoginHelper.Login getOBSAccessKeys(final URI name,
+ final Configuration conf)
+ throws IOException {
+ OBSLoginHelper.Login login
+ = OBSLoginHelper.extractLoginDetailsWithWarnings(name);
+ Configuration c =
+ ProviderUtils.excludeIncompatibleCredentialProviders(conf,
+ OBSFileSystem.class);
+ String accessKey = getPassword(c, OBSConstants.ACCESS_KEY,
+ login.getUser());
+ String secretKey = getPassword(c, OBSConstants.SECRET_KEY,
+ login.getPassword());
+ String sessionToken = getPassword(c, OBSConstants.SESSION_TOKEN,
+ login.getToken());
+ return new OBSLoginHelper.Login(accessKey, secretKey, sessionToken);
+ }
+
+ /**
+ * Get a password from a configuration, or, if a value is passed in, pick that
+ * up instead.
+ *
+ * @param conf configuration
+ * @param key key to look up
+ * @param val current value: if non empty this is used instead of querying
+ * the configuration.
+ * @return a password or "".
+ * @throws IOException on any problem
+ */
+ private static String getPassword(final Configuration conf,
+ final String key, final String val) throws IOException {
+ return StringUtils.isEmpty(val) ? lookupPassword(conf, key) : val;
+ }
+
+ /**
+ * Get a password from a configuration/configured credential providers.
+ *
+ * @param conf configuration
+ * @param key key to look up
+ * @return a password or the value in {@code defVal}
+ * @throws IOException on any problem
+ */
+ private static String lookupPassword(final Configuration conf,
+ final String key) throws IOException {
+ try {
+ final char[] pass = conf.getPassword(key);
+ return pass != null ? new String(pass).trim() : "";
+ } catch (IOException ioe) {
+ throw new IOException("Cannot find password option " + key, ioe);
+ }
+ }
+
+ /**
+ * String information about a summary entry for debug messages.
+ *
+ * @param summary summary object
+ * @return string value
+ */
+ static String stringify(final ObsObject summary) {
+ return summary.getObjectKey() + " size=" + summary.getMetadata()
+ .getContentLength();
+ }
+
+ /**
+ * Get a integer option not smaller than the minimum allowed value.
+ *
+ * @param conf configuration
+ * @param key key to look up
+ * @param defVal default value
+ * @param min minimum value
+ * @return the value
+ * @throws IllegalArgumentException if the value is below the minimum
+ */
+ static int intOption(final Configuration conf, final String key,
+ final int defVal,
+ final int min) {
+ int v = conf.getInt(key, defVal);
+ Preconditions.checkArgument(
+ v >= min,
+ String.format("Value of %s: %d is below the minimum value %d", key,
+ v, min));
+ LOG.debug("Value of {} is {}", key, v);
+ return v;
+ }
+
+ /**
+ * Get a long option not smaller than the minimum allowed value.
+ *
+ * @param conf configuration
+ * @param key key to look up
+ * @param defVal default value
+ * @param min minimum value
+ * @return the value
+ * @throws IllegalArgumentException if the value is below the minimum
+ */
+ static long longOption(final Configuration conf, final String key,
+ final long defVal,
+ final long min) {
+ long v = conf.getLong(key, defVal);
+ Preconditions.checkArgument(
+ v >= min,
+ String.format("Value of %s: %d is below the minimum value %d", key,
+ v, min));
+ LOG.debug("Value of {} is {}", key, v);
+ return v;
+ }
+
+ /**
+ * Get a long option not smaller than the minimum allowed value, supporting
+ * memory prefixes K,M,G,T,P.
+ *
+ * @param conf configuration
+ * @param key key to look up
+ * @param defVal default value
+ * @param min minimum value
+ * @return the value
+ * @throws IllegalArgumentException if the value is below the minimum
+ */
+ static long longBytesOption(final Configuration conf, final String key,
+ final long defVal,
+ final long min) {
+ long v = conf.getLongBytes(key, defVal);
+ Preconditions.checkArgument(
+ v >= min,
+ String.format("Value of %s: %d is below the minimum value %d", key,
+ v, min));
+ LOG.debug("Value of {} is {}", key, v);
+ return v;
+ }
+
+ /**
+ * Get a size property from the configuration: this property must be at least
+ * equal to {@link OBSConstants#MULTIPART_MIN_SIZE}. If it is too small, it is
+ * rounded up to that minimum, and a warning printed.
+ *
+ * @param conf configuration
+ * @param property property name
+ * @param defVal default value
+ * @return the value, guaranteed to be above the minimum size
+ */
+ public static long getMultipartSizeProperty(final Configuration conf,
+ final String property, final long defVal) {
+ long partSize = conf.getLongBytes(property, defVal);
+ if (partSize < OBSConstants.MULTIPART_MIN_SIZE) {
+ LOG.warn("{} must be at least 5 MB; configured value is {}",
+ property, partSize);
+ partSize = OBSConstants.MULTIPART_MIN_SIZE;
+ }
+ return partSize;
+ }
+
+ /**
+ * Ensure that the long value is in the range of an integer.
+ *
+ * @param name property name for error messages
+ * @param size original size
+ * @return the size, guaranteed to be less than or equal to the max value of
+ * an integer.
+ */
+ static int ensureOutputParameterInRange(final String name,
+ final long size) {
+ if (size > Integer.MAX_VALUE) {
+ LOG.warn(
+ "obs: {} capped to ~2.14GB"
+ + " (maximum allowed size with current output mechanism)",
+ name);
+ return Integer.MAX_VALUE;
+ } else {
+ return (int) size;
+ }
+ }
+
+ /**
+ * Propagates bucket-specific settings into generic OBS configuration keys.
+ * This is done by propagating the values of the form {@code
+ * fs.obs.bucket.${bucket}.key} to {@code fs.obs.key}, for all values of "key"
+ * other than a small set of unmodifiable values.
+ *
+ * The source of the updated property is set to the key name of the
+ * bucket property, to aid in diagnostics of where things came from.
+ *
+ *
Returns a new configuration. Why the clone? You can use the same conf
+ * for different filesystems, and the original values are not updated.
+ *
+ *
The {@code fs.obs.impl} property cannot be set, nor can any with the
+ * prefix {@code fs.obs.bucket}.
+ *
+ *
This method does not propagate security provider path information
+ * from the OBS property into the Hadoop common provider: callers must call
+ * {@link #patchSecurityCredentialProviders(Configuration)} explicitly.
+ *
+ * @param source Source Configuration object.
+ * @param bucket bucket name. Must not be empty.
+ * @return a (potentially) patched clone of the original.
+ */
+ static Configuration propagateBucketOptions(final Configuration source,
+ final String bucket) {
+
+ Preconditions.checkArgument(StringUtils.isNotEmpty(bucket), "bucket");
+ final String bucketPrefix = OBSConstants.FS_OBS_BUCKET_PREFIX + bucket
+ + '.';
+ LOG.debug("Propagating entries under {}", bucketPrefix);
+ final Configuration dest = new Configuration(source);
+ for (Map.Entry entry : source) {
+ final String key = entry.getKey();
+ // get the (unexpanded) value.
+ final String value = entry.getValue();
+ if (!key.startsWith(bucketPrefix) || bucketPrefix.equals(key)) {
+ continue;
+ }
+ // there's a bucket prefix, so strip it
+ final String stripped = key.substring(bucketPrefix.length());
+ if (stripped.startsWith("bucket.") || "impl".equals(stripped)) {
+ // tell user off
+ LOG.debug("Ignoring bucket option {}", key);
+ } else {
+ // propagate the value, building a new origin field.
+ // to track overwrites, the generic key is overwritten even if
+ // already matches the new one.
+ final String generic = OBSConstants.FS_OBS_PREFIX + stripped;
+ LOG.debug("Updating {}", generic);
+ dest.set(generic, value, key);
+ }
+ }
+ return dest;
+ }
+
+ /**
+ * Patch the security credential provider information in {@link
+ * #CREDENTIAL_PROVIDER_PATH} with the providers listed in {@link
+ * OBSConstants#OBS_SECURITY_CREDENTIAL_PROVIDER_PATH}.
+ *
+ * This allows different buckets to use different credential files.
+ *
+ * @param conf configuration to patch
+ */
+ static void patchSecurityCredentialProviders(final Configuration conf) {
+ Collection customCredentials =
+ conf.getStringCollection(
+ OBSConstants.OBS_SECURITY_CREDENTIAL_PROVIDER_PATH);
+ Collection hadoopCredentials = conf.getStringCollection(
+ CREDENTIAL_PROVIDER_PATH);
+ if (!customCredentials.isEmpty()) {
+ List all = Lists.newArrayList(customCredentials);
+ all.addAll(hadoopCredentials);
+ String joined = StringUtils.join(all, ',');
+ LOG.debug("Setting {} to {}", CREDENTIAL_PROVIDER_PATH, joined);
+ conf.set(CREDENTIAL_PROVIDER_PATH, joined, "patch of "
+ + OBSConstants.OBS_SECURITY_CREDENTIAL_PROVIDER_PATH);
+ }
+ }
+
+ /**
+ * Verify that the bucket exists. This does not check permissions, not even
+ * read access.
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @throws FileNotFoundException the bucket is absent
+ * @throws IOException any other problem talking to OBS
+ */
+ static void verifyBucketExists(final OBSFileSystem owner)
+ throws FileNotFoundException, IOException {
+ int retryTime = 1;
+ while (true) {
+ try {
+ if (!owner.getObsClient().headBucket(owner.getBucket())) {
+ throw new FileNotFoundException(
+ "Bucket " + owner.getBucket() + " does not exist");
+ }
+ return;
+ } catch (ObsException e) {
+ LOG.warn("Failed to head bucket for [{}], retry time [{}], "
+ + "exception [{}]", owner.getBucket(), retryTime,
+ translateException("doesBucketExist", owner.getBucket(),
+ e));
+
+ if (MAX_RETRY_TIME == retryTime) {
+ throw translateException("doesBucketExist",
+ owner.getBucket(), e);
+ }
+
+ try {
+ Thread.sleep(DELAY_TIME);
+ } catch (InterruptedException ie) {
+ throw e;
+ }
+ }
+ retryTime++;
+ }
+ }
+
+ /**
+ * initialize multi-part upload, purge larger than the value of
+ * PURGE_EXISTING_MULTIPART_AGE.
+ *
+ * @param owner the owner OBSFileSystem instance
+ * @param conf the configuration to use for the FS
+ * @throws IOException on any failure to initialize multipart upload
+ */
+ static void initMultipartUploads(final OBSFileSystem owner,
+ final Configuration conf)
+ throws IOException {
+ boolean purgeExistingMultipart =
+ conf.getBoolean(OBSConstants.PURGE_EXISTING_MULTIPART,
+ OBSConstants.DEFAULT_PURGE_EXISTING_MULTIPART);
+ long purgeExistingMultipartAge =
+ longOption(conf, OBSConstants.PURGE_EXISTING_MULTIPART_AGE,
+ OBSConstants.DEFAULT_PURGE_EXISTING_MULTIPART_AGE, 0);
+
+ if (!purgeExistingMultipart) {
+ return;
+ }
+
+ final Date purgeBefore = new Date(
+ new Date().getTime() - purgeExistingMultipartAge * 1000);
+
+ try {
+ ListMultipartUploadsRequest request
+ = new ListMultipartUploadsRequest(owner.getBucket());
+ while (true) {
+ // List + purge
+ MultipartUploadListing uploadListing = owner.getObsClient()
+ .listMultipartUploads(request);
+ for (MultipartUpload upload
+ : uploadListing.getMultipartTaskList()) {
+ if (upload.getInitiatedDate().compareTo(purgeBefore) < 0) {
+ owner.getObsClient().abortMultipartUpload(
+ new AbortMultipartUploadRequest(
+ owner.getBucket(), upload.getObjectKey(),
+ upload.getUploadId()));
+ }
+ }
+ if (!uploadListing.isTruncated()) {
+ break;
+ }
+ request.setUploadIdMarker(
+ uploadListing.getNextUploadIdMarker());
+ request.setKeyMarker(uploadListing.getNextKeyMarker());
+ }
+ } catch (ObsException e) {
+ if (e.getResponseCode() == FORBIDDEN_CODE) {
+ LOG.debug("Failed to purging multipart uploads against {},"
+ + " FS may be read only", owner.getBucket(),
+ e);
+ } else {
+ throw translateException("purging multipart uploads",
+ owner.getBucket(), e);
+ }
+ }
+ }
+
+ static void shutdownAll(final ExecutorService... executors) {
+ for (ExecutorService exe : executors) {
+ if (exe != null) {
+ try {
+ if (LOG != null) {
+ LOG.debug("Shutdown {}", exe);
+ }
+ exe.shutdown();
+ } catch (Exception e) {
+ if (LOG != null && LOG.isDebugEnabled()) {
+ LOG.debug("Exception in shutdown {}", exe, e);
+ }
+ }
+ }
+ }
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSConstants.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSConstants.java
new file mode 100644
index 00000000000..ac72e0404c4
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSConstants.java
@@ -0,0 +1,726 @@
+/*
+ * 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.obs;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * All constants used by {@link OBSFileSystem}.
+ *
+ * Some of the strings are marked as {@code Unstable}. This means that they
+ * may be unsupported in future; at which point they will be marked as
+ * deprecated and simply ignored.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+final class OBSConstants {
+ /**
+ * Minimum multipart size which OBS supports.
+ */
+ static final int MULTIPART_MIN_SIZE = 5 * 1024 * 1024;
+
+ /**
+ * OBS access key.
+ */
+ static final String ACCESS_KEY = "fs.obs.access.key";
+
+ /**
+ * OBS secret key.
+ */
+ static final String SECRET_KEY = "fs.obs.secret.key";
+
+ /**
+ * OBS credentials provider.
+ */
+ static final String OBS_CREDENTIALS_PROVIDER
+ = "fs.obs.credentials.provider";
+
+ /**
+ * OBS client security provider.
+ */
+ static final String OBS_SECURITY_PROVIDER = "fs.obs.security.provider";
+
+ /**
+ * Extra set of security credentials which will be prepended to that set in
+ * {@code "hadoop.security.credential.provider.path"}. This extra option
+ * allows for per-bucket overrides.
+ */
+ static final String OBS_SECURITY_CREDENTIAL_PROVIDER_PATH =
+ "fs.obs.security.credential.provider.path";
+
+ /**
+ * Session token for when using TemporaryOBSCredentialsProvider.
+ */
+ static final String SESSION_TOKEN = "fs.obs.session.token";
+
+ /**
+ * Maximum number of simultaneous connections to obs.
+ */
+ static final String MAXIMUM_CONNECTIONS = "fs.obs.connection.maximum";
+
+ /**
+ * Default value of {@link #MAXIMUM_CONNECTIONS}.
+ */
+ static final int DEFAULT_MAXIMUM_CONNECTIONS = 1000;
+
+ /**
+ * Connect to obs over ssl.
+ */
+ static final String SECURE_CONNECTIONS = "fs.obs.connection.ssl.enabled";
+
+ /**
+ * Default value of {@link #SECURE_CONNECTIONS}.
+ */
+ static final boolean DEFAULT_SECURE_CONNECTIONS = false;
+
+ /**
+ * Use a custom endpoint.
+ */
+ static final String ENDPOINT = "fs.obs.endpoint";
+
+ /**
+ * Host for connecting to OBS through proxy server.
+ */
+ static final String PROXY_HOST = "fs.obs.proxy.host";
+
+ /**
+ * Port for connecting to OBS through proxy server.
+ */
+ static final String PROXY_PORT = "fs.obs.proxy.port";
+
+ /**
+ * User name for connecting to OBS through proxy server.
+ */
+ static final String PROXY_USERNAME = "fs.obs.proxy.username";
+
+ /**
+ * Password for connecting to OBS through proxy server.
+ */
+ static final String PROXY_PASSWORD = "fs.obs.proxy.password";
+
+ /**
+ * Default port for HTTPS.
+ */
+ static final int DEFAULT_HTTPS_PORT = 443;
+
+ /**
+ * Default port for HTTP.
+ */
+ static final int DEFAULT_HTTP_PORT = 80;
+
+ /**
+ * Number of times we should retry errors.
+ */
+ static final String MAX_ERROR_RETRIES = "fs.obs.attempts.maximum";
+
+ /**
+ * Default value of {@link #MAX_ERROR_RETRIES}.
+ */
+ static final int DEFAULT_MAX_ERROR_RETRIES = 3;
+
+ /**
+ * Seconds until we give up trying to establish a connection to obs.
+ */
+ static final String ESTABLISH_TIMEOUT
+ = "fs.obs.connection.establish.timeout";
+
+ /**
+ * Default value of {@link #ESTABLISH_TIMEOUT}.
+ */
+ static final int DEFAULT_ESTABLISH_TIMEOUT = 120000;
+
+ /**
+ * Seconds until we give up on a connection to obs.
+ */
+ static final String SOCKET_TIMEOUT = "fs.obs.connection.timeout";
+
+ /**
+ * Default value of {@link #SOCKET_TIMEOUT}.
+ */
+ static final int DEFAULT_SOCKET_TIMEOUT = 120000;
+
+ /**
+ * Socket send buffer to be used in OBS SDK.
+ */
+ static final String SOCKET_SEND_BUFFER = "fs.obs.socket.send.buffer";
+
+ /**
+ * Default value of {@link #SOCKET_SEND_BUFFER}.
+ */
+ static final int DEFAULT_SOCKET_SEND_BUFFER = 256 * 1024;
+
+ /**
+ * Socket receive buffer to be used in OBS SDK.
+ */
+ static final String SOCKET_RECV_BUFFER = "fs.obs.socket.recv.buffer";
+
+ /**
+ * Default value of {@link #SOCKET_RECV_BUFFER}.
+ */
+ static final int DEFAULT_SOCKET_RECV_BUFFER = 256 * 1024;
+
+ /**
+ * Number of records to get while paging through a directory listing.
+ */
+ static final String MAX_PAGING_KEYS = "fs.obs.paging.maximum";
+
+ /**
+ * Default value of {@link #MAX_PAGING_KEYS}.
+ */
+ static final int DEFAULT_MAX_PAGING_KEYS = 1000;
+
+ /**
+ * Maximum number of threads to allow in the pool used by TransferManager.
+ */
+ static final String MAX_THREADS = "fs.obs.threads.max";
+
+ /**
+ * Default value of {@link #MAX_THREADS}.
+ */
+ static final int DEFAULT_MAX_THREADS = 20;
+
+ /**
+ * Maximum number of tasks cached if all threads are already uploading.
+ */
+ static final String MAX_TOTAL_TASKS = "fs.obs.max.total.tasks";
+
+ /**
+ * Default value of {@link #MAX_TOTAL_TASKS}.
+ */
+ static final int DEFAULT_MAX_TOTAL_TASKS = 20;
+
+ /**
+ * Max number of copy threads.
+ */
+ static final String MAX_COPY_THREADS = "fs.obs.copy.threads.max";
+
+ /**
+ * Default value of {@link #MAX_COPY_THREADS}.
+ */
+ static final int DEFAULT_MAX_COPY_THREADS = 40;
+
+ /**
+ * Max number of delete threads.
+ */
+ static final String MAX_DELETE_THREADS = "fs.obs.delete.threads.max";
+
+ /**
+ * Default value of {@link #MAX_DELETE_THREADS}.
+ */
+ static final int DEFAULT_MAX_DELETE_THREADS = 20;
+
+ /**
+ * Unused option: maintained for compile-time compatibility. If set, a warning
+ * is logged in OBS during init.
+ */
+ @Deprecated
+ static final String CORE_THREADS = "fs.obs.threads.core";
+
+ /**
+ * The time that an idle thread waits before terminating.
+ */
+ static final String KEEPALIVE_TIME = "fs.obs.threads.keepalivetime";
+
+ /**
+ * Default value of {@link #KEEPALIVE_TIME}.
+ */
+ static final int DEFAULT_KEEPALIVE_TIME = 60;
+
+ /**
+ * Size of each of or multipart pieces in bytes.
+ */
+ static final String MULTIPART_SIZE = "fs.obs.multipart.size";
+
+ /**
+ * Default value of {@link #MULTIPART_SIZE}.
+ */
+ static final long DEFAULT_MULTIPART_SIZE = 104857600; // 100 MB
+
+ /**
+ * Enable multi-object delete calls.
+ */
+ static final String ENABLE_MULTI_DELETE = "fs.obs.multiobjectdelete.enable";
+
+ /**
+ * Max number of objects in one multi-object delete call. This option takes
+ * effect only when the option 'ENABLE_MULTI_DELETE' is set to 'true'.
+ */
+ static final String MULTI_DELETE_MAX_NUMBER
+ = "fs.obs.multiobjectdelete.maximum";
+
+ /**
+ * Default value of {@link #MULTI_DELETE_MAX_NUMBER}.
+ */
+ static final int DEFAULT_MULTI_DELETE_MAX_NUMBER = 1000;
+
+ /**
+ * Delete recursively or not.
+ */
+ static final String MULTI_DELETE_RECURSION
+ = "fs.obs.multiobjectdelete.recursion";
+
+ /**
+ * Minimum number of objects in one multi-object delete call.
+ */
+ static final String MULTI_DELETE_THRESHOLD
+ = "fs.obs.multiobjectdelete.threshold";
+
+ /**
+ * Default value of {@link #MULTI_DELETE_THRESHOLD}.
+ */
+ static final int MULTI_DELETE_DEFAULT_THRESHOLD = 3;
+
+ /**
+ * Comma separated list of directories.
+ */
+ static final String BUFFER_DIR = "fs.obs.buffer.dir";
+
+ /**
+ * Switch to the fast block-by-block upload mechanism.
+ */
+ static final String FAST_UPLOAD = "fs.obs.fast.upload";
+
+ /**
+ * What buffer to use. Default is {@link #FAST_UPLOAD_BUFFER_DISK} Value:
+ * {@value}
+ */
+ @InterfaceStability.Unstable
+ static final String FAST_UPLOAD_BUFFER = "fs.obs.fast.upload.buffer";
+
+ /**
+ * Buffer blocks to disk: {@value}. Capacity is limited to available disk
+ * space.
+ */
+ @InterfaceStability.Unstable
+ static final String FAST_UPLOAD_BUFFER_DISK = "disk";
+
+ /**
+ * Use an in-memory array. Fast but will run of heap rapidly: {@value}.
+ */
+ @InterfaceStability.Unstable
+ static final String FAST_UPLOAD_BUFFER_ARRAY = "array";
+
+ /**
+ * Use a byte buffer. May be more memory efficient than the {@link
+ * #FAST_UPLOAD_BUFFER_ARRAY}: {@value}.
+ */
+ @InterfaceStability.Unstable
+ static final String FAST_UPLOAD_BYTEBUFFER = "bytebuffer";
+
+ /**
+ * Maximum number of blocks a single output stream can have active (uploading,
+ * or queued to the central FileSystem instance's pool of queued operations.
+ * )This stops a single stream overloading the shared thread pool. {@value}
+ *
+ *
Default is {@link #DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS}
+ */
+ @InterfaceStability.Unstable
+ static final String FAST_UPLOAD_ACTIVE_BLOCKS
+ = "fs.obs.fast.upload.active.blocks";
+
+ /**
+ * Limit of queued block upload operations before writes block. Value:
+ * {@value}
+ */
+ @InterfaceStability.Unstable
+ static final int DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS = 4;
+
+ /**
+ * Canned acl options: Private | PublicRead | PublicReadWrite |
+ * AuthenticatedRead | LogDeliveryWrite | BucketOwnerRead |
+ * BucketOwnerFullControl.
+ */
+ static final String CANNED_ACL = "fs.obs.acl.default";
+
+ /**
+ * Default value of {@link #CANNED_ACL}.
+ */
+ static final String DEFAULT_CANNED_ACL = "";
+
+ /**
+ * Should we try to purge old multipart uploads when starting up.
+ */
+ static final String PURGE_EXISTING_MULTIPART = "fs.obs.multipart.purge";
+
+ /**
+ * Default value of {@link #PURGE_EXISTING_MULTIPART}.
+ */
+ static final boolean DEFAULT_PURGE_EXISTING_MULTIPART = false;
+
+ /**
+ * Purge any multipart uploads older than this number of seconds.
+ */
+ static final String PURGE_EXISTING_MULTIPART_AGE
+ = "fs.obs.multipart.purge.age";
+
+ /**
+ * Default value of {@link #PURGE_EXISTING_MULTIPART_AGE}.
+ */
+ static final long DEFAULT_PURGE_EXISTING_MULTIPART_AGE = 86400;
+
+ /**
+ * OBS folder suffix.
+ */
+ static final String OBS_FOLDER_SUFFIX = "_$folder$";
+
+ /**
+ * Block size for
+ * {@link org.apache.hadoop.fs.FileSystem#getDefaultBlockSize()}.
+ */
+ static final String FS_OBS_BLOCK_SIZE = "fs.obs.block.size";
+
+ /**
+ * Default value of {@link #FS_OBS_BLOCK_SIZE}.
+ */
+ static final int DEFAULT_FS_OBS_BLOCK_SIZE = 128 * 1024 * 1024;
+
+ /**
+ * OBS scheme.
+ */
+ static final String OBS_SCHEME = "obs";
+
+ /**
+ * Prefix for all OBS properties: {@value}.
+ */
+ static final String FS_OBS_PREFIX = "fs.obs.";
+
+ /**
+ * Prefix for OBS bucket-specific properties: {@value}.
+ */
+ static final String FS_OBS_BUCKET_PREFIX = "fs.obs.bucket.";
+
+ /**
+ * OBS default port.
+ */
+ static final int OBS_DEFAULT_PORT = -1;
+
+ /**
+ * User agent prefix.
+ */
+ static final String USER_AGENT_PREFIX = "fs.obs.user.agent.prefix";
+
+ /**
+ * Read ahead buffer size to prevent connection re-establishments.
+ */
+ static final String READAHEAD_RANGE = "fs.obs.readahead.range";
+
+ /**
+ * Default value of {@link #READAHEAD_RANGE}.
+ */
+ static final long DEFAULT_READAHEAD_RANGE = 1024 * 1024;
+
+ /**
+ * Flag indicating if {@link OBSInputStream#read(long, byte[], int, int)} will
+ * use the implementation of
+ * {@link org.apache.hadoop.fs.FSInputStream#read(long,
+ * byte[], int, int)}.
+ */
+ static final String READ_TRANSFORM_ENABLE = "fs.obs.read.transform.enable";
+
+ /**
+ * OBS client factory implementation class.
+ */
+ @InterfaceAudience.Private
+ @InterfaceStability.Unstable
+ static final String OBS_CLIENT_FACTORY_IMPL
+ = "fs.obs.client.factory.impl";
+
+ /**
+ * Default value of {@link #OBS_CLIENT_FACTORY_IMPL}.
+ */
+ @InterfaceAudience.Private
+ @InterfaceStability.Unstable
+ static final Class extends OBSClientFactory>
+ DEFAULT_OBS_CLIENT_FACTORY_IMPL =
+ DefaultOBSClientFactory.class;
+
+ /**
+ * Maximum number of partitions in a multipart upload: {@value}.
+ */
+ @InterfaceAudience.Private
+ static final int MAX_MULTIPART_COUNT = 10000;
+
+ // OBS Client configuration
+
+ /**
+ * Idle connection time.
+ */
+ static final String IDLE_CONNECTION_TIME = "fs.obs.idle.connection.time";
+
+ /**
+ * Default value of {@link #IDLE_CONNECTION_TIME}.
+ */
+ static final int DEFAULT_IDLE_CONNECTION_TIME = 30000;
+
+ /**
+ * Maximum number of idle connections.
+ */
+ static final String MAX_IDLE_CONNECTIONS = "fs.obs.max.idle.connections";
+
+ /**
+ * Default value of {@link #MAX_IDLE_CONNECTIONS}.
+ */
+ static final int DEFAULT_MAX_IDLE_CONNECTIONS = 1000;
+
+ /**
+ * Keep alive.
+ */
+ static final String KEEP_ALIVE = "fs.obs.keep.alive";
+
+ /**
+ * Default value of {@link #KEEP_ALIVE}.
+ */
+ static final boolean DEFAULT_KEEP_ALIVE = true;
+
+ /**
+ * Validate certificate.
+ */
+ static final String VALIDATE_CERTIFICATE = "fs.obs.validate.certificate";
+
+ /**
+ * Default value of {@link #VALIDATE_CERTIFICATE}.
+ */
+ static final boolean DEFAULT_VALIDATE_CERTIFICATE = false;
+
+ /**
+ * Verify response content type.
+ */
+ static final String VERIFY_RESPONSE_CONTENT_TYPE
+ = "fs.obs.verify.response.content.type";
+
+ /**
+ * Default value of {@link #VERIFY_RESPONSE_CONTENT_TYPE}.
+ */
+ static final boolean DEFAULT_VERIFY_RESPONSE_CONTENT_TYPE = true;
+
+ /**
+ * UploadStreamRetryBufferSize.
+ */
+ static final String UPLOAD_STREAM_RETRY_SIZE
+ = "fs.obs.upload.stream.retry.buffer.size";
+
+ /**
+ * Default value of {@link #UPLOAD_STREAM_RETRY_SIZE}.
+ */
+ static final int DEFAULT_UPLOAD_STREAM_RETRY_SIZE = 512 * 1024;
+
+ /**
+ * Read buffer size.
+ */
+ static final String READ_BUFFER_SIZE = "fs.obs.read.buffer.size";
+
+ /**
+ * Default value of {@link #READ_BUFFER_SIZE}.
+ */
+ static final int DEFAULT_READ_BUFFER_SIZE = 256 * 1024;
+
+ /**
+ * Write buffer size.
+ */
+ static final String WRITE_BUFFER_SIZE = "fs.obs.write.buffer.size";
+
+ /**
+ * Default value of {@link #WRITE_BUFFER_SIZE}.
+ */
+ static final int DEFAULT_WRITE_BUFFER_SIZE = 256 * 1024;
+
+ /**
+ * Canonical name.
+ */
+ static final String CNAME = "fs.obs.cname";
+
+ /**
+ * Default value of {@link #CNAME}.
+ */
+ static final boolean DEFAULT_CNAME = false;
+
+ /**
+ * Strict host name verification.
+ */
+ static final String STRICT_HOSTNAME_VERIFICATION
+ = "fs.obs.strict.hostname.verification";
+
+ /**
+ * Default value of {@link #STRICT_HOSTNAME_VERIFICATION}.
+ */
+ static final boolean DEFAULT_STRICT_HOSTNAME_VERIFICATION = false;
+
+ /**
+ * Size of object copy part pieces in bytes.
+ */
+ static final String COPY_PART_SIZE = "fs.obs.copypart.size";
+
+ /**
+ * Maximum value of {@link #COPY_PART_SIZE}.
+ */
+ static final long MAX_COPY_PART_SIZE = 5368709120L; // 5GB
+
+ /**
+ * Default value of {@link #COPY_PART_SIZE}.
+ */
+ static final long DEFAULT_COPY_PART_SIZE = 104857600L; // 100MB
+
+ /**
+ * Maximum number of copy part threads.
+ */
+ static final String MAX_COPY_PART_THREADS = "fs.obs.copypart.threads.max";
+
+ /**
+ * Default value of {@link #MAX_COPY_PART_THREADS}.
+ */
+ static final int DEFAULT_MAX_COPY_PART_THREADS = 40;
+
+ /**
+ * Number of core list threads.
+ */
+ static final String CORE_LIST_THREADS = "fs.obs.list.threads.core";
+
+ /**
+ * Default value of {@link #CORE_LIST_THREADS}.
+ */
+ static final int DEFAULT_CORE_LIST_THREADS = 30;
+
+ /**
+ * Maximum number of list threads.
+ */
+ static final String MAX_LIST_THREADS = "fs.obs.list.threads.max";
+
+ /**
+ * Default value of {@link #MAX_LIST_THREADS}.
+ */
+ static final int DEFAULT_MAX_LIST_THREADS = 60;
+
+ /**
+ * Capacity of list work queue.
+ */
+ static final String LIST_WORK_QUEUE_CAPACITY
+ = "fs.obs.list.workqueue.capacity";
+
+ /**
+ * Default value of {@link #LIST_WORK_QUEUE_CAPACITY}.
+ */
+ static final int DEFAULT_LIST_WORK_QUEUE_CAPACITY = 1024;
+
+ /**
+ * List parallel factor.
+ */
+ static final String LIST_PARALLEL_FACTOR = "fs.obs.list.parallel.factor";
+
+ /**
+ * Default value of {@link #LIST_PARALLEL_FACTOR}.
+ */
+ static final int DEFAULT_LIST_PARALLEL_FACTOR = 30;
+
+ /**
+ * Switch for the fast delete.
+ */
+ static final String TRASH_ENABLE = "fs.obs.trash.enable";
+
+ /**
+ * Enable obs content summary or not.
+ */
+ static final String OBS_CONTENT_SUMMARY_ENABLE
+ = "fs.obs.content.summary.enable";
+
+ /**
+ * Enable obs client dfs list or not.
+ */
+ static final String OBS_CLIENT_DFS_LIST_ENABLE
+ = "fs.obs.client.dfs.list.enable";
+
+ /**
+ * Default trash : false.
+ */
+ static final boolean DEFAULT_TRASH = false;
+
+ /**
+ * The fast delete recycle directory.
+ */
+ static final String TRASH_DIR = "fs.obs.trash.dir";
+
+ /**
+ * Encryption type is sse-kms or sse-c.
+ */
+ static final String SSE_TYPE = "fs.obs.server-side-encryption-type";
+
+ /**
+ * Kms key id for sse-kms, while key base64 encoded content for sse-c.
+ */
+ static final String SSE_KEY = "fs.obs.server-side-encryption-key";
+
+ /**
+ * Array first block size.
+ */
+ static final String FAST_UPLOAD_BUFFER_ARRAY_FIRST_BLOCK_SIZE
+ = "fs.obs.fast.upload.array.first.buffer";
+
+ /**
+ * The fast upload buffer array first block default size.
+ */
+ static final int FAST_UPLOAD_BUFFER_ARRAY_FIRST_BLOCK_SIZE_DEFAULT = 1024
+ * 1024;
+
+ /**
+ * Auth Type Negotiation Enable Switch.
+ */
+ static final String SDK_AUTH_TYPE_NEGOTIATION_ENABLE
+ = "fs.obs.authtype.negotiation.enable";
+
+ /**
+ * Default value of {@link #SDK_AUTH_TYPE_NEGOTIATION_ENABLE}.
+ */
+ static final boolean DEFAULT_SDK_AUTH_TYPE_NEGOTIATION_ENABLE = false;
+
+ /**
+ * Okhttp retryOnConnectionFailure switch.
+ */
+ static final String SDK_RETRY_ON_CONNECTION_FAILURE_ENABLE
+ = "fs.obs.connection.retry.enable";
+
+ /**
+ * Default value of {@link #SDK_RETRY_ON_CONNECTION_FAILURE_ENABLE}.
+ */
+ static final boolean DEFAULT_SDK_RETRY_ON_CONNECTION_FAILURE_ENABLE = true;
+
+ /**
+ * Sdk max retry times on unexpected end of stream. exception, default: -1,
+ * don't retry
+ */
+ static final String SDK_RETRY_TIMES_ON_UNEXPECTED_END_EXCEPTION
+ = "fs.obs.unexpectedend.retrytime";
+
+ /**
+ * Default value of {@link #SDK_RETRY_TIMES_ON_UNEXPECTED_END_EXCEPTION}.
+ */
+ static final int DEFAULT_SDK_RETRY_TIMES_ON_UNEXPECTED_END_EXCEPTION = -1;
+
+ /**
+ * Maximum sdk connection retry times, default : 2000.
+ */
+ static final int DEFAULT_MAX_SDK_CONNECTION_RETRY_TIMES = 2000;
+
+ /**
+ * Second to millisecond factor.
+ */
+ static final int SEC2MILLISEC_FACTOR = 1000;
+
+ private OBSConstants() {
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSDataBlocks.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSDataBlocks.java
new file mode 100644
index 00000000000..5e413e6841a
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSDataBlocks.java
@@ -0,0 +1,1020 @@
+/*
+ * 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.obs;
+
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.util.DirectBufferPool;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.Closeable;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Set of classes to support output streaming into blocks which are then
+ * uploaded as to OBS as a single PUT, or as part of a multipart request.
+ */
+final class OBSDataBlocks {
+
+ /**
+ * Class logger.
+ */
+ private static final Logger LOG = LoggerFactory.getLogger(
+ OBSDataBlocks.class);
+
+ private OBSDataBlocks() {
+ }
+
+ /**
+ * Validate args to a write command. These are the same validation checks
+ * expected for any implementation of {@code OutputStream.write()}.
+ *
+ * @param b byte array containing data
+ * @param off offset in array where to start
+ * @param len number of bytes to be written
+ * @throws NullPointerException for a null buffer
+ * @throws IndexOutOfBoundsException if indices are out of range
+ */
+ static void validateWriteArgs(final byte[] b, final int off,
+ final int len) {
+ Preconditions.checkNotNull(b);
+ if (off < 0 || off > b.length || len < 0 || off + len > b.length
+ || off + len < 0) {
+ throw new IndexOutOfBoundsException(
+ "write (b[" + b.length + "], " + off + ", " + len + ')');
+ }
+ }
+
+ /**
+ * Create a factory.
+ *
+ * @param owner factory owner
+ * @param name factory name -the option from {@link OBSConstants}.
+ * @return the factory, ready to be initialized.
+ * @throws IllegalArgumentException if the name is unknown.
+ */
+ static BlockFactory createFactory(final OBSFileSystem owner,
+ final String name) {
+ switch (name) {
+ case OBSConstants.FAST_UPLOAD_BUFFER_ARRAY:
+ return new ByteArrayBlockFactory(owner);
+ case OBSConstants.FAST_UPLOAD_BUFFER_DISK:
+ return new DiskBlockFactory(owner);
+ case OBSConstants.FAST_UPLOAD_BYTEBUFFER:
+ return new ByteBufferBlockFactory(owner);
+ default:
+ throw new IllegalArgumentException(
+ "Unsupported block buffer" + " \"" + name + '"');
+ }
+ }
+
+ /**
+ * Base class for block factories.
+ */
+ abstract static class BlockFactory {
+ /**
+ * OBS file system type.
+ */
+ private final OBSFileSystem owner;
+
+ protected BlockFactory(final OBSFileSystem obsFileSystem) {
+ this.owner = obsFileSystem;
+ }
+
+ /**
+ * Create a block.
+ *
+ * @param index index of block
+ * @param limit limit of the block.
+ * @return a new block.
+ * @throws IOException on any failure to create block
+ */
+ abstract DataBlock create(long index, int limit) throws IOException;
+
+ /**
+ * Owner.
+ *
+ * @return obsFileSystem instance
+ */
+ protected OBSFileSystem getOwner() {
+ return owner;
+ }
+ }
+
+ /**
+ * This represents a block being uploaded.
+ */
+ abstract static class DataBlock implements Closeable {
+
+ /**
+ * Data block index.
+ */
+ private final long index;
+
+ /**
+ * Dest state can be : writing/upload/closed.
+ */
+ private volatile DestState state = DestState.Writing;
+
+ protected DataBlock(final long dataIndex) {
+ this.index = dataIndex;
+ }
+
+ /**
+ * Atomically enter a state, verifying current state.
+ *
+ * @param current current state. null means "no check"
+ * @param next next state
+ * @throws IllegalStateException if the current state is not as expected
+ */
+ protected final synchronized void enterState(final DestState current,
+ final DestState next)
+ throws IllegalStateException {
+ verifyState(current);
+ LOG.debug("{}: entering state {}", this, next);
+ state = next;
+ }
+
+ /**
+ * Verify that the block is in the declared state.
+ *
+ * @param expected expected state.
+ * @throws IllegalStateException if the DataBlock is in the wrong state
+ */
+ protected final void verifyState(final DestState expected)
+ throws IllegalStateException {
+ if (expected != null && state != expected) {
+ throw new IllegalStateException(
+ "Expected stream state " + expected
+ + " -but actual state is " + state + " in " + this);
+ }
+ }
+
+ /**
+ * Current state.
+ *
+ * @return the current state.
+ */
+ protected final DestState getState() {
+ return state;
+ }
+
+ protected long getIndex() {
+ return index;
+ }
+
+ /**
+ * Return the current data size.
+ *
+ * @return the size of the data
+ */
+ abstract int dataSize();
+
+ /**
+ * Predicate to verify that the block has the capacity to write the given
+ * set of bytes.
+ *
+ * @param bytes number of bytes desired to be written.
+ * @return true if there is enough space.
+ */
+ abstract boolean hasCapacity(long bytes);
+
+ /**
+ * Predicate to check if there is data in the block.
+ *
+ * @return true if there is
+ */
+ boolean hasData() {
+ return dataSize() > 0;
+ }
+
+ /**
+ * The remaining capacity in the block before it is full.
+ *
+ * @return the number of bytes remaining.
+ */
+ abstract int remainingCapacity();
+
+ /**
+ * Write a series of bytes from the buffer, from the offset. Returns the
+ * number of bytes written. Only valid in the state {@code Writing}. Base
+ * class verifies the state but does no writing.
+ *
+ * @param buffer buffer
+ * @param offset offset
+ * @param length length of write
+ * @return number of bytes written
+ * @throws IOException trouble
+ */
+ int write(final byte[] buffer, final int offset, final int length)
+ throws IOException {
+ verifyState(DestState.Writing);
+ Preconditions.checkArgument(buffer != null, "Null buffer");
+ Preconditions.checkArgument(length >= 0, "length is negative");
+ Preconditions.checkArgument(offset >= 0, "offset is negative");
+ Preconditions.checkArgument(
+ !(buffer.length - offset < length),
+ "buffer shorter than amount of data to write");
+ return 0;
+ }
+
+ /**
+ * Flush the output. Only valid in the state {@code Writing}. In the base
+ * class, this is a no-op
+ *
+ * @throws IOException any IO problem.
+ */
+ void flush() throws IOException {
+ verifyState(DestState.Writing);
+ }
+
+ /**
+ * Switch to the upload state and return a stream for uploading. Base class
+ * calls {@link #enterState(DestState, DestState)} to manage the state
+ * machine.
+ *
+ * @return the stream
+ * @throws IOException trouble
+ */
+ Object startUpload() throws IOException {
+ LOG.debug("Start datablock[{}] upload", index);
+ enterState(DestState.Writing, DestState.Upload);
+ return null;
+ }
+
+ /**
+ * Enter the closed state.
+ *
+ * @return true if the class was in any other state, implying that the
+ * subclass should do its close operations
+ */
+ protected synchronized boolean enterClosedState() {
+ if (!state.equals(DestState.Closed)) {
+ enterState(null, DestState.Closed);
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (enterClosedState()) {
+ LOG.debug("Closed {}", this);
+ innerClose();
+ }
+ }
+
+ /**
+ * Inner close logic for subclasses to implement.
+ *
+ * @throws IOException on any failure to close
+ */
+ protected abstract void innerClose() throws IOException;
+
+ /**
+ * Destination state definition for a data block.
+ */
+ enum DestState {
+ /**
+ * destination state : writing.
+ */
+ Writing,
+ /**
+ * destination state : upload.
+ */
+ Upload,
+ /**
+ * destination state : closed.
+ */
+ Closed
+ }
+ }
+
+ /**
+ * Use byte arrays on the heap for storage.
+ */
+ static class ByteArrayBlockFactory extends BlockFactory {
+ ByteArrayBlockFactory(final OBSFileSystem owner) {
+ super(owner);
+ }
+
+ @Override
+ DataBlock create(final long index, final int limit) {
+ int firstBlockSize = super.owner.getConf()
+ .getInt(OBSConstants.FAST_UPLOAD_BUFFER_ARRAY_FIRST_BLOCK_SIZE,
+ OBSConstants
+ .FAST_UPLOAD_BUFFER_ARRAY_FIRST_BLOCK_SIZE_DEFAULT);
+ return new ByteArrayBlock(0, limit, firstBlockSize);
+ }
+ }
+
+ /**
+ * OBS specific byte array output stream.
+ */
+ static class OBSByteArrayOutputStream extends ByteArrayOutputStream {
+ OBSByteArrayOutputStream(final int size) {
+ super(size);
+ }
+
+ /**
+ * InputStream backed by the internal byte array.
+ *
+ * @return input stream
+ */
+ ByteArrayInputStream getInputStream() {
+ ByteArrayInputStream bin = new ByteArrayInputStream(this.buf, 0,
+ count);
+ this.reset();
+ this.buf = null;
+ return bin;
+ }
+ }
+
+ /**
+ * Stream to memory via a {@code ByteArrayOutputStream}.
+ *
+ *
This was taken from {@code OBSBlockOutputStream} and has the same
+ * problem which surfaced there: it can consume a lot of heap space
+ * proportional to the mismatch between writes to the stream and the JVM-wide
+ * upload bandwidth to the OBS endpoint. The memory consumption can be limited
+ * by tuning the filesystem settings to restrict the number of queued/active
+ * uploads.
+ */
+ static class ByteArrayBlock extends DataBlock {
+ /**
+ * Memory limit.
+ */
+ private final int limit;
+
+ /**
+ * Output stream.
+ */
+ private OBSByteArrayOutputStream buffer;
+
+ /**
+ * Cache data size so that it is consistent after the buffer is reset.
+ */
+ private Integer dataSize;
+
+ /**
+ * Block first size.
+ */
+ private int firstBlockSize;
+
+ /**
+ * Input stream.
+ */
+ private ByteArrayInputStream inputStream = null;
+
+ ByteArrayBlock(final long index, final int limitBlockSize,
+ final int blockSize) {
+ super(index);
+ this.limit = limitBlockSize;
+ this.buffer = new OBSByteArrayOutputStream(blockSize);
+ this.firstBlockSize = blockSize;
+ }
+
+ /**
+ * Returns the block first block size.
+ *
+ * @return the block first block size
+ */
+ @VisibleForTesting
+ public int firstBlockSize() {
+ return this.firstBlockSize;
+ }
+
+ /**
+ * Get the amount of data; if there is no buffer then the size is 0.
+ *
+ * @return the amount of data available to upload.
+ */
+ @Override
+ int dataSize() {
+ return dataSize != null ? dataSize : buffer.size();
+ }
+
+ @Override
+ InputStream startUpload() throws IOException {
+ super.startUpload();
+ dataSize = buffer.size();
+ inputStream = buffer.getInputStream();
+ return inputStream;
+ }
+
+ @Override
+ boolean hasCapacity(final long bytes) {
+ return dataSize() + bytes <= limit;
+ }
+
+ @Override
+ int remainingCapacity() {
+ return limit - dataSize();
+ }
+
+ @Override
+ int write(final byte[] b, final int offset, final int len)
+ throws IOException {
+ super.write(b, offset, len);
+ int written = Math.min(remainingCapacity(), len);
+ buffer.write(b, offset, written);
+ return written;
+ }
+
+ @Override
+ protected void innerClose() throws IOException {
+ if (buffer != null) {
+ buffer.close();
+ buffer = null;
+ }
+
+ if (inputStream != null) {
+ inputStream.close();
+ inputStream = null;
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "ByteArrayBlock{"
+ + "index="
+ + getIndex()
+ + ", state="
+ + getState()
+ + ", limit="
+ + limit
+ + ", dataSize="
+ + dataSize
+ + '}';
+ }
+ }
+
+ /**
+ * Stream via Direct ByteBuffers; these are allocated off heap via {@link
+ * DirectBufferPool}.
+ */
+ static class ByteBufferBlockFactory extends BlockFactory {
+
+ /**
+ * The directory buffer pool.
+ */
+ private static final DirectBufferPool BUFFER_POOL
+ = new DirectBufferPool();
+
+ /**
+ * Count of outstanding buffers.
+ */
+ private static final AtomicInteger BUFFERS_OUTSTANDING
+ = new AtomicInteger(0);
+
+ ByteBufferBlockFactory(final OBSFileSystem owner) {
+ super(owner);
+ }
+
+ @Override
+ ByteBufferBlock create(final long index, final int limit) {
+ return new ByteBufferBlock(index, limit);
+ }
+
+ public static ByteBuffer requestBuffer(final int limit) {
+ LOG.debug("Requesting buffer of size {}", limit);
+ BUFFERS_OUTSTANDING.incrementAndGet();
+ return BUFFER_POOL.getBuffer(limit);
+ }
+
+ public static void releaseBuffer(final ByteBuffer buffer) {
+ LOG.debug("Releasing buffer");
+ BUFFER_POOL.returnBuffer(buffer);
+ BUFFERS_OUTSTANDING.decrementAndGet();
+ }
+
+ /**
+ * Get count of outstanding buffers.
+ *
+ * @return the current buffer count
+ */
+ public int getOutstandingBufferCount() {
+ return BUFFERS_OUTSTANDING.get();
+ }
+
+ @Override
+ public String toString() {
+ return "ByteBufferBlockFactory{" + "buffersOutstanding="
+ + BUFFERS_OUTSTANDING + '}';
+ }
+ }
+
+ /**
+ * A DataBlock which requests a buffer from pool on creation; returns it when
+ * it is closed.
+ */
+ static class ByteBufferBlock extends DataBlock {
+ /**
+ * Set the buffer size.
+ */
+ private final int bufferSize;
+
+ /**
+ * Create block buffer.
+ */
+ private ByteBuffer blockBuffer;
+
+ /**
+ * Cache data size so that it is consistent after the buffer is reset.
+ */
+ private Integer dataSize;
+
+ /**
+ * Create input stream.
+ */
+ private ByteBufferInputStream inputStream;
+
+ /**
+ * Instantiate. This will request a ByteBuffer of the desired size.
+ *
+ * @param index block index
+ * @param initBufferSize buffer size
+ */
+ ByteBufferBlock(final long index, final int initBufferSize) {
+ super(index);
+ this.bufferSize = initBufferSize;
+ blockBuffer = ByteBufferBlockFactory.requestBuffer(initBufferSize);
+ }
+
+ /**
+ * Get the amount of data; if there is no buffer then the size is 0.
+ *
+ * @return the amount of data available to upload.
+ */
+ @Override
+ int dataSize() {
+ return dataSize != null ? dataSize : bufferCapacityUsed();
+ }
+
+ @Override
+ InputStream startUpload() throws IOException {
+ super.startUpload();
+ dataSize = bufferCapacityUsed();
+ // set the buffer up from reading from the beginning
+ blockBuffer.limit(blockBuffer.position());
+ blockBuffer.position(0);
+ inputStream = new ByteBufferInputStream(dataSize, blockBuffer);
+ return inputStream;
+ }
+
+ @Override
+ public boolean hasCapacity(final long bytes) {
+ return bytes <= remainingCapacity();
+ }
+
+ @Override
+ public int remainingCapacity() {
+ return blockBuffer != null ? blockBuffer.remaining() : 0;
+ }
+
+ private int bufferCapacityUsed() {
+ return blockBuffer.capacity() - blockBuffer.remaining();
+ }
+
+ @Override
+ int write(final byte[] b, final int offset, final int len)
+ throws IOException {
+ super.write(b, offset, len);
+ int written = Math.min(remainingCapacity(), len);
+ blockBuffer.put(b, offset, written);
+ return written;
+ }
+
+ /**
+ * Closing the block will release the buffer.
+ */
+ @Override
+ protected void innerClose() {
+ if (blockBuffer != null) {
+ ByteBufferBlockFactory.releaseBuffer(blockBuffer);
+ blockBuffer = null;
+ }
+ if (inputStream != null) {
+ inputStream.close();
+ inputStream = null;
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "ByteBufferBlock{"
+ + "index="
+ + getIndex()
+ + ", state="
+ + getState()
+ + ", dataSize="
+ + dataSize()
+ + ", limit="
+ + bufferSize
+ + ", remainingCapacity="
+ + remainingCapacity()
+ + '}';
+ }
+
+ /**
+ * Provide an input stream from a byte buffer; supporting {@link
+ * #mark(int)}, which is required to enable replay of failed PUT attempts.
+ */
+ class ByteBufferInputStream extends InputStream {
+
+ /**
+ * Set the input stream size.
+ */
+ private final int size;
+
+ /**
+ * Set the byte buffer.
+ */
+ private ByteBuffer byteBuffer;
+
+ ByteBufferInputStream(final int streamSize,
+ final ByteBuffer streamByteBuffer) {
+ LOG.debug("Creating ByteBufferInputStream of size {}",
+ streamSize);
+ this.size = streamSize;
+ this.byteBuffer = streamByteBuffer;
+ }
+
+ /**
+ * After the stream is closed, set the local reference to the byte buffer
+ * to null; this guarantees that future attempts to use stream methods
+ * will fail.
+ */
+ @Override
+ public synchronized void close() {
+ LOG.debug("ByteBufferInputStream.close() for {}",
+ ByteBufferBlock.super.toString());
+ byteBuffer = null;
+ }
+
+ /**
+ * Verify that the stream is open.
+ *
+ * @throws IOException if the stream is closed
+ */
+ private void verifyOpen() throws IOException {
+ if (byteBuffer == null) {
+ throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+ }
+ }
+
+ public synchronized int read() {
+ if (available() > 0) {
+ return byteBuffer.get() & OBSCommonUtils.BYTE_TO_INT_MASK;
+ } else {
+ return -1;
+ }
+ }
+
+ @Override
+ public synchronized long skip(final long offset)
+ throws IOException {
+ verifyOpen();
+ long newPos = position() + offset;
+ if (newPos < 0) {
+ throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK);
+ }
+ if (newPos > size) {
+ throw new EOFException(
+ FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
+ }
+ byteBuffer.position((int) newPos);
+ return newPos;
+ }
+
+ @Override
+ public synchronized int available() {
+ Preconditions.checkState(byteBuffer != null,
+ FSExceptionMessages.STREAM_IS_CLOSED);
+ return byteBuffer.remaining();
+ }
+
+ /**
+ * Get the current buffer position.
+ *
+ * @return the buffer position
+ */
+ public synchronized int position() {
+ return byteBuffer.position();
+ }
+
+ /**
+ * Check if there is data left.
+ *
+ * @return true if there is data remaining in the buffer.
+ */
+ public synchronized boolean hasRemaining() {
+ return byteBuffer.hasRemaining();
+ }
+
+ @Override
+ public synchronized void mark(final int readlimit) {
+ LOG.debug("mark at {}", position());
+ byteBuffer.mark();
+ }
+
+ @Override
+ public synchronized void reset() {
+ LOG.debug("reset");
+ byteBuffer.reset();
+ }
+
+ @Override
+ public boolean markSupported() {
+ return true;
+ }
+
+ /**
+ * Read in data.
+ *
+ * @param b destination buffer
+ * @param offset offset within the buffer
+ * @param length length of bytes to read
+ * @return read size
+ * @throws EOFException if the position is negative
+ * @throws IndexOutOfBoundsException if there isn't space for the amount
+ * of data requested.
+ * @throws IllegalArgumentException other arguments are invalid.
+ */
+ public synchronized int read(final byte[] b, final int offset,
+ final int length)
+ throws IOException {
+ Preconditions.checkArgument(length >= 0, "length is negative");
+ Preconditions.checkArgument(b != null, "Null buffer");
+ if (b.length - offset < length) {
+ throw new IndexOutOfBoundsException(
+ FSExceptionMessages.TOO_MANY_BYTES_FOR_DEST_BUFFER
+ + ": request length ="
+ + length
+ + ", with offset ="
+ + offset
+ + "; buffer capacity ="
+ + (b.length - offset));
+ }
+ verifyOpen();
+ if (!hasRemaining()) {
+ return -1;
+ }
+
+ int toRead = Math.min(length, available());
+ byteBuffer.get(b, offset, toRead);
+ return toRead;
+ }
+
+ @Override
+ public String toString() {
+ final StringBuilder sb = new StringBuilder(
+ "ByteBufferInputStream{");
+ sb.append("size=").append(size);
+ ByteBuffer buf = this.byteBuffer;
+ if (buf != null) {
+ sb.append(", available=").append(buf.remaining());
+ }
+ sb.append(", ").append(ByteBufferBlock.super.toString());
+ sb.append('}');
+ return sb.toString();
+ }
+ }
+ }
+
+ /**
+ * Buffer blocks to disk.
+ */
+ static class DiskBlockFactory extends BlockFactory {
+ /**
+ * Allocator the local directory.
+ */
+ private static LocalDirAllocator directoryAllocator;
+
+ DiskBlockFactory(final OBSFileSystem owner) {
+ super(owner);
+ }
+
+ /**
+ * Create a temp file and a {@link DiskBlock} instance to manage it.
+ *
+ * @param index block index
+ * @param limit limit of the block.
+ * @return the new block
+ * @throws IOException IO problems
+ */
+ @Override
+ DataBlock create(final long index, final int limit) throws IOException {
+ File destFile = createTmpFileForWrite(
+ String.format("obs-block-%04d-", index), limit,
+ getOwner().getConf());
+ return new DiskBlock(destFile, limit, index);
+ }
+
+ /**
+ * Demand create the directory allocator, then create a temporary file.
+ * {@link LocalDirAllocator#createTmpFileForWrite(String, long,
+ * Configuration)}.
+ *
+ * @param pathStr prefix for the temporary file
+ * @param size the size of the file that is going to be written
+ * @param conf the Configuration object
+ * @return a unique temporary file
+ * @throws IOException IO problems
+ */
+ static synchronized File createTmpFileForWrite(final String pathStr,
+ final long size, final Configuration conf)
+ throws IOException {
+ if (directoryAllocator == null) {
+ String bufferDir = conf.get(OBSConstants.BUFFER_DIR) != null
+ ? OBSConstants.BUFFER_DIR
+ : "hadoop.tmp.dir";
+ directoryAllocator = new LocalDirAllocator(bufferDir);
+ }
+ return directoryAllocator.createTmpFileForWrite(pathStr, size,
+ conf);
+ }
+ }
+
+ /**
+ * Stream to a file. This will stop at the limit; the caller is expected to
+ * create a new block.
+ */
+ static class DiskBlock extends DataBlock {
+
+ /**
+ * Create buffer file.
+ */
+ private final File bufferFile;
+
+ /**
+ * Buffer size limit.
+ */
+ private final int limit;
+
+ /**
+ * Verify block has closed or not.
+ */
+ private final AtomicBoolean closed = new AtomicBoolean(false);
+
+ /**
+ * Written bytes count.
+ */
+ private int bytesWritten;
+
+ /**
+ * Out put stream buffer.
+ */
+ private BufferedOutputStream out;
+
+ DiskBlock(final File destBufferFile, final int limitSize,
+ final long index)
+ throws FileNotFoundException {
+ super(index);
+ this.limit = limitSize;
+ this.bufferFile = destBufferFile;
+ out = new BufferedOutputStream(
+ new FileOutputStream(destBufferFile));
+ }
+
+ @Override
+ int dataSize() {
+ return bytesWritten;
+ }
+
+ @Override
+ boolean hasCapacity(final long bytes) {
+ return dataSize() + bytes <= limit;
+ }
+
+ @Override
+ int remainingCapacity() {
+ return limit - bytesWritten;
+ }
+
+ @Override
+ int write(final byte[] b, final int offset, final int len)
+ throws IOException {
+ super.write(b, offset, len);
+ int written = Math.min(remainingCapacity(), len);
+ out.write(b, offset, written);
+ bytesWritten += written;
+ return written;
+ }
+
+ @Override
+ File startUpload() throws IOException {
+ super.startUpload();
+ try {
+ out.flush();
+ } finally {
+ out.close();
+ out = null;
+ }
+ return bufferFile;
+ }
+
+ /**
+ * The close operation will delete the destination file if it still exists.
+ */
+ @Override
+ protected void innerClose() {
+ final DestState state = getState();
+ LOG.debug("Closing {}", this);
+ switch (state) {
+ case Writing:
+ if (bufferFile.exists()) {
+ // file was not uploaded
+ LOG.debug(
+ "Block[{}]: Deleting buffer file as upload "
+ + "did not start",
+ getIndex());
+ closeBlock();
+ }
+ break;
+
+ case Upload:
+ LOG.debug(
+ "Block[{}]: Buffer file {} exists close upload stream",
+ getIndex(), bufferFile);
+ break;
+
+ case Closed:
+ closeBlock();
+ break;
+
+ default:
+ // this state can never be reached, but checkstyle
+ // complains, so it is here.
+ }
+ }
+
+ /**
+ * Flush operation will flush to disk.
+ *
+ * @throws IOException IOE raised on FileOutputStream
+ */
+ @Override
+ void flush() throws IOException {
+ super.flush();
+ out.flush();
+ }
+
+ @Override
+ public String toString() {
+ return "FileBlock{index=" + getIndex() + ", destFile=" + bufferFile
+ + ", state=" + getState() + ", dataSize="
+ + dataSize() + ", limit=" + limit + '}';
+ }
+
+ /**
+ * Close the block. This will delete the block's buffer file if the block
+ * has not previously been closed.
+ */
+ void closeBlock() {
+ LOG.debug("block[{}]: closeBlock()", getIndex());
+ if (!closed.getAndSet(true)) {
+ if (!bufferFile.delete() && bufferFile.exists()) {
+ LOG.warn("delete({}) returned false",
+ bufferFile.getAbsoluteFile());
+ }
+ } else {
+ LOG.debug("block[{}]: skipping re-entrant closeBlock()",
+ getIndex());
+ }
+ }
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileStatus.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileStatus.java
new file mode 100644
index 00000000000..448115554f8
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileStatus.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.obs;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * File status for an OBS file.
+ *
+ *
The subclass is private as it should not be created directly.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+class OBSFileStatus extends FileStatus {
+ /**
+ * Create a directory status.
+ *
+ * @param path the path
+ * @param owner the owner
+ */
+ OBSFileStatus(final Path path, final String owner) {
+ super(0, true, 1, 0, 0, path);
+ setOwner(owner);
+ setGroup(owner);
+ }
+
+ /**
+ * Create a directory status.
+ *
+ * @param modificationTime modification time
+ * @param path the path
+ * @param owner the owner
+ */
+ OBSFileStatus(final Path path, final long modificationTime,
+ final String owner) {
+ super(0, true, 1, 0, modificationTime, path);
+ setOwner(owner);
+ setGroup(owner);
+ }
+
+ /**
+ * Create a directory status.
+ *
+ * @param modificationTime modification time
+ * @param accessTime access time
+ * @param path the path
+ * @param owner the owner
+ */
+ OBSFileStatus(final Path path, final long modificationTime,
+ final long accessTime,
+ final String owner) {
+ super(0, true, 1, 0, modificationTime, accessTime, null, owner, owner,
+ path);
+ }
+
+ /**
+ * A simple file.
+ *
+ * @param length file length
+ * @param modificationTime mod time
+ * @param path path
+ * @param blockSize block size
+ * @param owner owner
+ */
+ OBSFileStatus(
+ final long length, final long modificationTime, final Path path,
+ final long blockSize,
+ final String owner) {
+ super(length, false, 1, blockSize, modificationTime, path);
+ setOwner(owner);
+ setGroup(owner);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileSystem.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileSystem.java
new file mode 100644
index 00000000000..dd8be1e1fe9
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileSystem.java
@@ -0,0 +1,1562 @@
+/*
+ * 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.obs;
+
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService;
+import com.obs.services.ObsClient;
+import com.obs.services.exception.ObsException;
+import com.obs.services.model.AccessControlList;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Options.ChecksumOpt;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
+import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.EnumSet;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * The core OBS Filesystem implementation.
+ *
+ *
This subclass is marked as private as code should not be creating it
+ * directly; use {@link FileSystem#get(Configuration)} and variants to create
+ * one.
+ *
+ *
If cast to {@code OBSFileSystem}, extra methods and features may be
+ * accessed. Consider those private and unstable.
+ *
+ *
Because it prints some of the state of the instrumentation, the output of
+ * {@link #toString()} must also be considered unstable.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class OBSFileSystem extends FileSystem {
+ /**
+ * Class logger.
+ */
+ public static final Logger LOG = LoggerFactory.getLogger(
+ OBSFileSystem.class);
+
+ /**
+ * Flag indicating if the filesystem instance is closed.
+ */
+ private final AtomicBoolean closed = new AtomicBoolean(false);
+
+ /**
+ * URI of the filesystem.
+ */
+ private URI uri;
+
+ /**
+ * Current working directory of the filesystem.
+ */
+ private Path workingDir;
+
+ /**
+ * Short name of the user who instantiated the filesystem.
+ */
+ private String username;
+
+ /**
+ * OBS client instance.
+ */
+ private ObsClient obs;
+
+ /**
+ * Flag indicating if posix bucket is used.
+ */
+ private boolean enablePosix = false;
+
+ /**
+ * Flag indicating if multi-object delete recursion is enabled.
+ */
+ private boolean enableMultiObjectDeleteRecursion = true;
+
+ /**
+ * Flag indicating if OBS specific content summary is enabled.
+ */
+ private boolean obsContentSummaryEnable = true;
+
+ /**
+ * Flag indicating if OBS client specific depth first search (DFS) list is
+ * enabled.
+ */
+ private boolean obsClientDFSListEnable = true;
+
+ /**
+ * Bucket name.
+ */
+ private String bucket;
+
+ /**
+ * Max number of keys to get while paging through a directory listing.
+ */
+ private int maxKeys;
+
+ /**
+ * OBSListing instance.
+ */
+ private OBSListing obsListing;
+
+ /**
+ * Helper for an ongoing write operation.
+ */
+ private OBSWriteOperationHelper writeHelper;
+
+ /**
+ * Part size for multipart upload.
+ */
+ private long partSize;
+
+ /**
+ * Flag indicating if multi-object delete is enabled.
+ */
+ private boolean enableMultiObjectDelete;
+
+ /**
+ * Minimum number of objects in one multi-object delete call.
+ */
+ private int multiDeleteThreshold;
+
+ /**
+ * Maximum number of entries in one multi-object delete call.
+ */
+ private int maxEntriesToDelete;
+
+ /**
+ * Bounded thread pool for multipart upload.
+ */
+ private ListeningExecutorService boundedMultipartUploadThreadPool;
+
+ /**
+ * Bounded thread pool for copy.
+ */
+ private ThreadPoolExecutor boundedCopyThreadPool;
+
+ /**
+ * Bounded thread pool for delete.
+ */
+ private ThreadPoolExecutor boundedDeleteThreadPool;
+
+ /**
+ * Bounded thread pool for copy part.
+ */
+ private ThreadPoolExecutor boundedCopyPartThreadPool;
+
+ /**
+ * Bounded thread pool for list.
+ */
+ private ThreadPoolExecutor boundedListThreadPool;
+
+ /**
+ * List parallel factor.
+ */
+ private int listParallelFactor;
+
+ /**
+ * Read ahead range.
+ */
+ private long readAheadRange;
+
+ /**
+ * Flag indicating if {@link OBSInputStream#read(long, byte[], int, int)} will
+ * be transformed into {@link org.apache.hadoop.fs.FSInputStream#read(long,
+ * byte[], int, int)}.
+ */
+ private boolean readTransformEnable = true;
+
+ /**
+ * Factory for creating blocks.
+ */
+ private OBSDataBlocks.BlockFactory blockFactory;
+
+ /**
+ * Maximum Number of active blocks a single output stream can submit to {@link
+ * #boundedMultipartUploadThreadPool}.
+ */
+ private int blockOutputActiveBlocks;
+
+ /**
+ * Copy part size.
+ */
+ private long copyPartSize;
+
+ /**
+ * Flag indicating if fast delete is enabled.
+ */
+ private boolean enableTrash = false;
+
+ /**
+ * Trash directory for fast delete.
+ */
+ private String trashDir;
+
+ /**
+ * OBS redefined access control list.
+ */
+ private AccessControlList cannedACL;
+
+ /**
+ * Server-side encryption wrapper.
+ */
+ private SseWrapper sse;
+
+ /**
+ * Block size for {@link FileSystem#getDefaultBlockSize()}.
+ */
+ private long blockSize;
+
+ /**
+ * Initialize a FileSystem. Called after a new FileSystem instance is
+ * constructed.
+ *
+ * @param name a URI whose authority section names the host, port,
+ * etc. for this FileSystem
+ * @param originalConf the configuration to use for the FS. The
+ * bucket-specific options are patched over the base ones
+ * before any use is made of the config.
+ */
+ @Override
+ public void initialize(final URI name, final Configuration originalConf)
+ throws IOException {
+ uri = URI.create(name.getScheme() + "://" + name.getAuthority());
+ bucket = name.getAuthority();
+ // clone the configuration into one with propagated bucket options
+ Configuration conf = OBSCommonUtils.propagateBucketOptions(originalConf,
+ bucket);
+ OBSCommonUtils.patchSecurityCredentialProviders(conf);
+ super.initialize(name, conf);
+ setConf(conf);
+ try {
+
+ // Username is the current user at the time the FS was instantiated.
+ username = UserGroupInformation.getCurrentUser().getShortUserName();
+ workingDir = new Path("/user", username).makeQualified(this.uri,
+ this.getWorkingDirectory());
+
+ Class extends OBSClientFactory> obsClientFactoryClass =
+ conf.getClass(
+ OBSConstants.OBS_CLIENT_FACTORY_IMPL,
+ OBSConstants.DEFAULT_OBS_CLIENT_FACTORY_IMPL,
+ OBSClientFactory.class);
+ obs = ReflectionUtils.newInstance(obsClientFactoryClass, conf)
+ .createObsClient(name);
+ sse = new SseWrapper(conf);
+
+ OBSCommonUtils.verifyBucketExists(this);
+ enablePosix = OBSCommonUtils.getBucketFsStatus(obs, bucket);
+
+ maxKeys = OBSCommonUtils.intOption(conf,
+ OBSConstants.MAX_PAGING_KEYS,
+ OBSConstants.DEFAULT_MAX_PAGING_KEYS, 1);
+ obsListing = new OBSListing(this);
+ partSize = OBSCommonUtils.getMultipartSizeProperty(conf,
+ OBSConstants.MULTIPART_SIZE,
+ OBSConstants.DEFAULT_MULTIPART_SIZE);
+
+ // check but do not store the block size
+ blockSize = OBSCommonUtils.longBytesOption(conf,
+ OBSConstants.FS_OBS_BLOCK_SIZE,
+ OBSConstants.DEFAULT_FS_OBS_BLOCK_SIZE, 1);
+ enableMultiObjectDelete = conf.getBoolean(
+ OBSConstants.ENABLE_MULTI_DELETE, true);
+ maxEntriesToDelete = conf.getInt(
+ OBSConstants.MULTI_DELETE_MAX_NUMBER,
+ OBSConstants.DEFAULT_MULTI_DELETE_MAX_NUMBER);
+ enableMultiObjectDeleteRecursion = conf.getBoolean(
+ OBSConstants.MULTI_DELETE_RECURSION, true);
+ obsContentSummaryEnable = conf.getBoolean(
+ OBSConstants.OBS_CONTENT_SUMMARY_ENABLE, true);
+ readAheadRange = OBSCommonUtils.longBytesOption(conf,
+ OBSConstants.READAHEAD_RANGE,
+ OBSConstants.DEFAULT_READAHEAD_RANGE, 0);
+ readTransformEnable = conf.getBoolean(
+ OBSConstants.READ_TRANSFORM_ENABLE, true);
+ multiDeleteThreshold = conf.getInt(
+ OBSConstants.MULTI_DELETE_THRESHOLD,
+ OBSConstants.MULTI_DELETE_DEFAULT_THRESHOLD);
+
+ initThreadPools(conf);
+
+ writeHelper = new OBSWriteOperationHelper(this);
+
+ initCannedAcls(conf);
+
+ OBSCommonUtils.initMultipartUploads(this, conf);
+
+ String blockOutputBuffer = conf.getTrimmed(
+ OBSConstants.FAST_UPLOAD_BUFFER,
+ OBSConstants.FAST_UPLOAD_BUFFER_DISK);
+ partSize = OBSCommonUtils.ensureOutputParameterInRange(
+ OBSConstants.MULTIPART_SIZE, partSize);
+ blockFactory = OBSDataBlocks.createFactory(this, blockOutputBuffer);
+ blockOutputActiveBlocks =
+ OBSCommonUtils.intOption(conf,
+ OBSConstants.FAST_UPLOAD_ACTIVE_BLOCKS,
+ OBSConstants.DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS, 1);
+ LOG.debug(
+ "Using OBSBlockOutputStream with buffer = {}; block={};"
+ + " queue limit={}",
+ blockOutputBuffer,
+ partSize,
+ blockOutputActiveBlocks);
+
+ enableTrash = conf.getBoolean(OBSConstants.TRASH_ENABLE,
+ OBSConstants.DEFAULT_TRASH);
+ if (enableTrash) {
+ if (!isFsBucket()) {
+ String errorMsg = String.format(
+ "The bucket [%s] is not posix. not supported for "
+ + "trash.", bucket);
+ LOG.warn(errorMsg);
+ enableTrash = false;
+ trashDir = null;
+ } else {
+ trashDir = conf.get(OBSConstants.TRASH_DIR);
+ if (StringUtils.isEmpty(trashDir)) {
+ String errorMsg =
+ String.format(
+ "The trash feature(fs.obs.trash.enable) is "
+ + "enabled, but the "
+ + "configuration(fs.obs.trash.dir [%s]) "
+ + "is empty.",
+ trashDir);
+ LOG.error(errorMsg);
+ throw new ObsException(errorMsg);
+ }
+ trashDir = OBSCommonUtils.maybeAddBeginningSlash(trashDir);
+ trashDir = OBSCommonUtils.maybeAddTrailingSlash(trashDir);
+ }
+ }
+ } catch (ObsException e) {
+ throw OBSCommonUtils.translateException("initializing ",
+ new Path(name), e);
+ }
+ }
+
+ private void initThreadPools(final Configuration conf) {
+ long keepAliveTime = OBSCommonUtils.longOption(conf,
+ OBSConstants.KEEPALIVE_TIME,
+ OBSConstants.DEFAULT_KEEPALIVE_TIME, 0);
+
+ int maxThreads = conf.getInt(OBSConstants.MAX_THREADS,
+ OBSConstants.DEFAULT_MAX_THREADS);
+ if (maxThreads < 2) {
+ LOG.warn(OBSConstants.MAX_THREADS
+ + " must be at least 2: forcing to 2.");
+ maxThreads = 2;
+ }
+ int totalTasks = OBSCommonUtils.intOption(conf,
+ OBSConstants.MAX_TOTAL_TASKS,
+ OBSConstants.DEFAULT_MAX_TOTAL_TASKS, 1);
+ boundedMultipartUploadThreadPool =
+ BlockingThreadPoolExecutorService.newInstance(
+ maxThreads,
+ maxThreads + totalTasks,
+ keepAliveTime,
+ TimeUnit.SECONDS,
+ "obs-transfer-shared");
+
+ int maxDeleteThreads = conf.getInt(OBSConstants.MAX_DELETE_THREADS,
+ OBSConstants.DEFAULT_MAX_DELETE_THREADS);
+ if (maxDeleteThreads < 2) {
+ LOG.warn(OBSConstants.MAX_DELETE_THREADS
+ + " must be at least 2: forcing to 2.");
+ maxDeleteThreads = 2;
+ }
+ int coreDeleteThreads = (int) Math.ceil(maxDeleteThreads / 2.0);
+ boundedDeleteThreadPool =
+ new ThreadPoolExecutor(
+ coreDeleteThreads,
+ maxDeleteThreads,
+ keepAliveTime,
+ TimeUnit.SECONDS,
+ new LinkedBlockingQueue<>(),
+ BlockingThreadPoolExecutorService.newDaemonThreadFactory(
+ "obs-delete-transfer-shared"));
+ boundedDeleteThreadPool.allowCoreThreadTimeOut(true);
+
+ if (enablePosix) {
+ obsClientDFSListEnable = conf.getBoolean(
+ OBSConstants.OBS_CLIENT_DFS_LIST_ENABLE, true);
+ if (obsClientDFSListEnable) {
+ int coreListThreads = conf.getInt(
+ OBSConstants.CORE_LIST_THREADS,
+ OBSConstants.DEFAULT_CORE_LIST_THREADS);
+ int maxListThreads = conf.getInt(OBSConstants.MAX_LIST_THREADS,
+ OBSConstants.DEFAULT_MAX_LIST_THREADS);
+ int listWorkQueueCapacity = conf.getInt(
+ OBSConstants.LIST_WORK_QUEUE_CAPACITY,
+ OBSConstants.DEFAULT_LIST_WORK_QUEUE_CAPACITY);
+ listParallelFactor = conf.getInt(
+ OBSConstants.LIST_PARALLEL_FACTOR,
+ OBSConstants.DEFAULT_LIST_PARALLEL_FACTOR);
+ if (listParallelFactor < 1) {
+ LOG.warn(OBSConstants.LIST_PARALLEL_FACTOR
+ + " must be at least 1: forcing to 1.");
+ listParallelFactor = 1;
+ }
+ boundedListThreadPool =
+ new ThreadPoolExecutor(
+ coreListThreads,
+ maxListThreads,
+ keepAliveTime,
+ TimeUnit.SECONDS,
+ new LinkedBlockingQueue<>(listWorkQueueCapacity),
+ BlockingThreadPoolExecutorService
+ .newDaemonThreadFactory(
+ "obs-list-transfer-shared"));
+ boundedListThreadPool.allowCoreThreadTimeOut(true);
+ }
+ } else {
+ int maxCopyThreads = conf.getInt(OBSConstants.MAX_COPY_THREADS,
+ OBSConstants.DEFAULT_MAX_COPY_THREADS);
+ if (maxCopyThreads < 2) {
+ LOG.warn(OBSConstants.MAX_COPY_THREADS
+ + " must be at least 2: forcing to 2.");
+ maxCopyThreads = 2;
+ }
+ int coreCopyThreads = (int) Math.ceil(maxCopyThreads / 2.0);
+ boundedCopyThreadPool =
+ new ThreadPoolExecutor(
+ coreCopyThreads,
+ maxCopyThreads,
+ keepAliveTime,
+ TimeUnit.SECONDS,
+ new LinkedBlockingQueue<>(),
+ BlockingThreadPoolExecutorService.newDaemonThreadFactory(
+ "obs-copy-transfer-shared"));
+ boundedCopyThreadPool.allowCoreThreadTimeOut(true);
+
+ copyPartSize = OBSCommonUtils.longOption(conf,
+ OBSConstants.COPY_PART_SIZE,
+ OBSConstants.DEFAULT_COPY_PART_SIZE, 0);
+ if (copyPartSize > OBSConstants.MAX_COPY_PART_SIZE) {
+ LOG.warn(
+ "obs: {} capped to ~5GB (maximum allowed part size with "
+ + "current output mechanism)",
+ OBSConstants.COPY_PART_SIZE);
+ copyPartSize = OBSConstants.MAX_COPY_PART_SIZE;
+ }
+
+ int maxCopyPartThreads = conf.getInt(
+ OBSConstants.MAX_COPY_PART_THREADS,
+ OBSConstants.DEFAULT_MAX_COPY_PART_THREADS);
+ if (maxCopyPartThreads < 2) {
+ LOG.warn(OBSConstants.MAX_COPY_PART_THREADS
+ + " must be at least 2: forcing to 2.");
+ maxCopyPartThreads = 2;
+ }
+ int coreCopyPartThreads = (int) Math.ceil(maxCopyPartThreads / 2.0);
+ boundedCopyPartThreadPool =
+ new ThreadPoolExecutor(
+ coreCopyPartThreads,
+ maxCopyPartThreads,
+ keepAliveTime,
+ TimeUnit.SECONDS,
+ new LinkedBlockingQueue<>(),
+ BlockingThreadPoolExecutorService.newDaemonThreadFactory(
+ "obs-copy-part-transfer-shared"));
+ boundedCopyPartThreadPool.allowCoreThreadTimeOut(true);
+ }
+ }
+
+ /**
+ * Is posix bucket or not.
+ *
+ * @return is it posix bucket
+ */
+ boolean isFsBucket() {
+ return enablePosix;
+ }
+
+ /**
+ * Get read transform switch stat.
+ *
+ * @return is read transform enabled
+ */
+ boolean isReadTransformEnabled() {
+ return readTransformEnable;
+ }
+
+ /**
+ * Initialize bucket acl for upload, write operation.
+ *
+ * @param conf the configuration to use for the FS.
+ */
+ private void initCannedAcls(final Configuration conf) {
+ // No canned acl in obs
+ String cannedACLName = conf.get(OBSConstants.CANNED_ACL,
+ OBSConstants.DEFAULT_CANNED_ACL);
+ if (!cannedACLName.isEmpty()) {
+ switch (cannedACLName) {
+ case "Private":
+ case "PublicRead":
+ case "PublicReadWrite":
+ case "AuthenticatedRead":
+ case "LogDeliveryWrite":
+ case "BucketOwnerRead":
+ case "BucketOwnerFullControl":
+ cannedACL = new AccessControlList();
+ break;
+ default:
+ cannedACL = null;
+ }
+ } else {
+ cannedACL = null;
+ }
+ }
+
+ /**
+ * Get the bucket acl of user setting.
+ *
+ * @return bucket acl {@link AccessControlList}
+ */
+ AccessControlList getCannedACL() {
+ return cannedACL;
+ }
+
+ /**
+ * Return the protocol scheme for the FileSystem.
+ *
+ * @return "obs"
+ */
+ @Override
+ public String getScheme() {
+ return "obs";
+ }
+
+ /**
+ * Return a URI whose scheme and authority identify this FileSystem.
+ *
+ * @return the URI of this filesystem.
+ */
+ @Override
+ public URI getUri() {
+ return uri;
+ }
+
+ /**
+ * Return the default port for this FileSystem.
+ *
+ * @return -1 to indicate the port is undefined, which agrees with the
+ * contract of {@link URI#getPort()}
+ */
+ @Override
+ public int getDefaultPort() {
+ return OBSConstants.OBS_DEFAULT_PORT;
+ }
+
+ /**
+ * Return the OBS client used by this filesystem.
+ *
+ * @return OBS client
+ */
+ @VisibleForTesting
+ ObsClient getObsClient() {
+ return obs;
+ }
+
+ /**
+ * Return the read ahead range used by this filesystem.
+ *
+ * @return read ahead range
+ */
+ @VisibleForTesting
+ long getReadAheadRange() {
+ return readAheadRange;
+ }
+
+ /**
+ * Return the bucket of this filesystem.
+ *
+ * @return the bucket
+ */
+ String getBucket() {
+ return bucket;
+ }
+
+ /**
+ * Check that a Path belongs to this FileSystem. Unlike the superclass, this
+ * version does not look at authority, but only hostname.
+ *
+ * @param path the path to check
+ * @throws IllegalArgumentException if there is an FS mismatch
+ */
+ @Override
+ public void checkPath(final Path path) {
+ OBSLoginHelper.checkPath(getConf(), getUri(), path, getDefaultPort());
+ }
+
+ /**
+ * Canonicalize the given URI.
+ *
+ * @param rawUri the URI to be canonicalized
+ * @return the canonicalized URI
+ */
+ @Override
+ protected URI canonicalizeUri(final URI rawUri) {
+ return OBSLoginHelper.canonicalizeUri(rawUri, getDefaultPort());
+ }
+
+ /**
+ * Open an FSDataInputStream at the indicated Path.
+ *
+ * @param f the file path to open
+ * @param bufferSize the size of the buffer to be used
+ * @return the FSDataInputStream for the file
+ * @throws IOException on any failure to open the file
+ */
+ @Override
+ public FSDataInputStream open(final Path f, final int bufferSize)
+ throws IOException {
+ LOG.debug("Opening '{}' for reading.", f);
+ final FileStatus fileStatus = getFileStatus(f);
+ if (fileStatus.isDirectory()) {
+ throw new FileNotFoundException(
+ "Can't open " + f + " because it is a directory");
+ }
+
+ return new FSDataInputStream(
+ new OBSInputStream(bucket, OBSCommonUtils.pathToKey(this, f),
+ fileStatus.getLen(),
+ obs, statistics, readAheadRange, this));
+ }
+
+ /**
+ * Create an FSDataOutputStream at the indicated Path with write-progress
+ * reporting.
+ *
+ * @param f the file path to create
+ * @param permission the permission to set
+ * @param overwrite if a file with this name already exists, then if true,
+ * the file will be overwritten, and if false an error will
+ * be thrown
+ * @param bufferSize the size of the buffer to be used
+ * @param replication required block replication for the file
+ * @param blkSize the requested block size
+ * @param progress the progress reporter
+ * @throws IOException on any failure to create the file
+ * @see #setPermission(Path, FsPermission)
+ */
+ @Override
+ public FSDataOutputStream create(
+ final Path f,
+ final FsPermission permission,
+ final boolean overwrite,
+ final int bufferSize,
+ final short replication,
+ final long blkSize,
+ final Progressable progress)
+ throws IOException {
+ String key = OBSCommonUtils.pathToKey(this, f);
+ FileStatus status;
+ long objectLen = 0;
+ try {
+ // get the status or throw an exception
+ status = getFileStatus(f);
+ objectLen = status.getLen();
+ // if the thread reaches here, there is something at the path
+ if (status.isDirectory()) {
+ // path references a directory: automatic error
+ throw new FileAlreadyExistsException(f + " is a directory");
+ }
+ if (!overwrite) {
+ // path references a file and overwrite is disabled
+ throw new FileAlreadyExistsException(f + " already exists");
+ }
+ LOG.debug("create: Overwriting file {}", f);
+ } catch (FileNotFoundException e) {
+ // this means the file is not found
+ LOG.debug("create: Creating new file {}", f);
+ }
+ return new FSDataOutputStream(
+ new OBSBlockOutputStream(
+ this,
+ key,
+ objectLen,
+ new SemaphoredDelegatingExecutor(
+ boundedMultipartUploadThreadPool,
+ blockOutputActiveBlocks, true),
+ false),
+ null);
+ }
+
+ /**
+ * Return the part size for multipart upload used by {@link
+ * OBSBlockOutputStream}.
+ *
+ * @return the part size
+ */
+ long getPartSize() {
+ return partSize;
+ }
+
+ /**
+ * Return the block factory used by {@link OBSBlockOutputStream}.
+ *
+ * @return the block factory
+ */
+ OBSDataBlocks.BlockFactory getBlockFactory() {
+ return blockFactory;
+ }
+
+ /**
+ * Return the write helper used by {@link OBSBlockOutputStream}.
+ *
+ * @return the write helper
+ */
+ OBSWriteOperationHelper getWriteHelper() {
+ return writeHelper;
+ }
+
+ /**
+ * Create an FSDataOutputStream at the indicated Path with write-progress
+ * reporting.
+ *
+ * @param f the file name to create
+ * @param permission permission of
+ * @param flags {@link CreateFlag}s to use for this stream
+ * @param bufferSize the size of the buffer to be used
+ * @param replication required block replication for the file
+ * @param blkSize block size
+ * @param progress progress
+ * @param checksumOpt check sum option
+ * @throws IOException io exception
+ */
+ @Override
+ @SuppressWarnings("checkstyle:parameternumber")
+ public FSDataOutputStream create(
+ final Path f,
+ final FsPermission permission,
+ final EnumSet flags,
+ final int bufferSize,
+ final short replication,
+ final long blkSize,
+ final Progressable progress,
+ final ChecksumOpt checksumOpt)
+ throws IOException {
+ LOG.debug("create: Creating new file {}, flags:{}, isFsBucket:{}", f,
+ flags, isFsBucket());
+ if (null != flags && flags.contains(CreateFlag.APPEND)) {
+ if (!isFsBucket()) {
+ throw new UnsupportedOperationException(
+ "non-posix bucket. Append is not supported by "
+ + "OBSFileSystem");
+ }
+ String key = OBSCommonUtils.pathToKey(this, f);
+ FileStatus status;
+ long objectLen = 0;
+ try {
+ // get the status or throw an FNFE
+ status = getFileStatus(f);
+ objectLen = status.getLen();
+ // if the thread reaches here, there is something at the path
+ if (status.isDirectory()) {
+ // path references a directory: automatic error
+ throw new FileAlreadyExistsException(f + " is a directory");
+ }
+ } catch (FileNotFoundException e) {
+ LOG.debug("FileNotFoundException, create: Creating new file {}",
+ f);
+ }
+
+ return new FSDataOutputStream(
+ new OBSBlockOutputStream(
+ this,
+ key,
+ objectLen,
+ new SemaphoredDelegatingExecutor(
+ boundedMultipartUploadThreadPool,
+ blockOutputActiveBlocks, true),
+ true),
+ null);
+ } else {
+ return create(
+ f,
+ permission,
+ flags == null || flags.contains(CreateFlag.OVERWRITE),
+ bufferSize,
+ replication,
+ blkSize,
+ progress);
+ }
+ }
+
+ /**
+ * Open an FSDataOutputStream at the indicated Path with write-progress
+ * reporting. Same as create(), except fails if parent directory doesn't
+ * already exist.
+ *
+ * @param path the file path to create
+ * @param permission file permission
+ * @param flags {@link CreateFlag}s to use for this stream
+ * @param bufferSize the size of the buffer to be used
+ * @param replication required block replication for the file
+ * @param blkSize block size
+ * @param progress the progress reporter
+ * @throws IOException IO failure
+ */
+ @Override
+ public FSDataOutputStream createNonRecursive(
+ final Path path,
+ final FsPermission permission,
+ final EnumSet flags,
+ final int bufferSize,
+ final short replication,
+ final long blkSize,
+ final Progressable progress)
+ throws IOException {
+ Path parent = path.getParent();
+ if (parent != null && !getFileStatus(parent).isDirectory()) {
+ // expect this to raise an exception if there is no parent
+ throw new FileAlreadyExistsException("Not a directory: " + parent);
+ }
+ return create(
+ path,
+ permission,
+ flags.contains(CreateFlag.OVERWRITE),
+ bufferSize,
+ replication,
+ blkSize,
+ progress);
+ }
+
+ /**
+ * Append to an existing file (optional operation).
+ *
+ * @param f the existing file to be appended
+ * @param bufferSize the size of the buffer to be used
+ * @param progress for reporting progress if it is not null
+ * @throws IOException indicating that append is not supported
+ */
+ @Override
+ public FSDataOutputStream append(final Path f, final int bufferSize,
+ final Progressable progress)
+ throws IOException {
+ if (!isFsBucket()) {
+ throw new UnsupportedOperationException(
+ "non-posix bucket. Append is not supported "
+ + "by OBSFileSystem");
+ }
+ LOG.debug("append: Append file {}.", f);
+ String key = OBSCommonUtils.pathToKey(this, f);
+
+ // get the status or throw an FNFE
+ FileStatus status = getFileStatus(f);
+ long objectLen = status.getLen();
+ // if the thread reaches here, there is something at the path
+ if (status.isDirectory()) {
+ // path references a directory: automatic error
+ throw new FileAlreadyExistsException(f + " is a directory");
+ }
+
+ return new FSDataOutputStream(
+ new OBSBlockOutputStream(
+ this,
+ key,
+ objectLen,
+ new SemaphoredDelegatingExecutor(
+ boundedMultipartUploadThreadPool,
+ blockOutputActiveBlocks, true),
+ true),
+ null);
+ }
+
+ /**
+ * Check if a path exists.
+ *
+ * @param f source path
+ * @return true if the path exists
+ * @throws IOException IO failure
+ */
+ @Override
+ public boolean exists(final Path f) throws IOException {
+ try {
+ return getFileStatus(f) != null;
+ } catch (FileNotFoundException | FileConflictException e) {
+ return false;
+ }
+ }
+
+ /**
+ * Rename Path src to Path dst.
+ *
+ * @param src path to be renamed
+ * @param dst new path after rename
+ * @return true if rename is successful
+ * @throws IOException on IO failure
+ */
+ @Override
+ public boolean rename(final Path src, final Path dst) throws IOException {
+ long startTime = System.currentTimeMillis();
+ long threadId = Thread.currentThread().getId();
+ LOG.debug("Rename path {} to {} start", src, dst);
+ try {
+ if (enablePosix) {
+ return OBSPosixBucketUtils.renameBasedOnPosix(this, src, dst);
+ } else {
+ return OBSObjectBucketUtils.renameBasedOnObject(this, src, dst);
+ }
+ } catch (ObsException e) {
+ throw OBSCommonUtils.translateException(
+ "rename(" + src + ", " + dst + ")", src, e);
+ } catch (RenameFailedException e) {
+ LOG.error(e.getMessage());
+ return e.getExitCode();
+ } catch (FileNotFoundException e) {
+ LOG.error(e.toString());
+ return false;
+ } finally {
+ long endTime = System.currentTimeMillis();
+ LOG.debug(
+ "Rename path {} to {} finished, thread:{}, "
+ + "timeUsedInMilliSec:{}.", src, dst, threadId,
+ endTime - startTime);
+ }
+ }
+
+ /**
+ * Return maximum number of entries in one multi-object delete call.
+ *
+ * @return the maximum number of entries in one multi-object delete call
+ */
+ int getMaxEntriesToDelete() {
+ return maxEntriesToDelete;
+ }
+
+ /**
+ * Return list parallel factor.
+ *
+ * @return the list parallel factor
+ */
+ int getListParallelFactor() {
+ return listParallelFactor;
+ }
+
+ /**
+ * Return bounded thread pool for list.
+ *
+ * @return bounded thread pool for list
+ */
+ ThreadPoolExecutor getBoundedListThreadPool() {
+ return boundedListThreadPool;
+ }
+
+ /**
+ * Return a flag that indicates if OBS client specific depth first search
+ * (DFS) list is enabled.
+ *
+ * @return the flag
+ */
+ boolean isObsClientDFSListEnable() {
+ return obsClientDFSListEnable;
+ }
+
+ /**
+ * Return the {@link Statistics} instance used by this filesystem.
+ *
+ * @return the used {@link Statistics} instance
+ */
+ Statistics getSchemeStatistics() {
+ return statistics;
+ }
+
+ /**
+ * Return the minimum number of objects in one multi-object delete call.
+ *
+ * @return the minimum number of objects in one multi-object delete call
+ */
+ int getMultiDeleteThreshold() {
+ return multiDeleteThreshold;
+ }
+
+ /**
+ * Return a flag that indicates if multi-object delete is enabled.
+ *
+ * @return the flag
+ */
+ boolean isEnableMultiObjectDelete() {
+ return enableMultiObjectDelete;
+ }
+
+ /**
+ * Delete a Path. This operation is at least {@code O(files)}, with added
+ * overheads to enumerate the path. It is also not atomic.
+ *
+ * @param f the path to delete
+ * @param recursive if path is a directory and set to true, the directory is
+ * deleted else throws an exception. In case of a file the
+ * recursive can be set to either true or false
+ * @return true if delete is successful else false
+ * @throws IOException due to inability to delete a directory or file
+ */
+ @Override
+ public boolean delete(final Path f, final boolean recursive)
+ throws IOException {
+ try {
+ FileStatus status = getFileStatus(f);
+ LOG.debug("delete: path {} - recursive {}", status.getPath(),
+ recursive);
+
+ if (enablePosix) {
+ return OBSPosixBucketUtils.fsDelete(this, status, recursive);
+ }
+
+ return OBSObjectBucketUtils.objectDelete(this, status, recursive);
+ } catch (FileNotFoundException e) {
+ LOG.warn("Couldn't delete {} - does not exist", f);
+ return false;
+ } catch (ObsException e) {
+ throw OBSCommonUtils.translateException("delete", f, e);
+ }
+ }
+
+ /**
+ * Return a flag that indicates if fast delete is enabled.
+ *
+ * @return the flag
+ */
+ boolean isEnableTrash() {
+ return enableTrash;
+ }
+
+ /**
+ * Return trash directory for fast delete.
+ *
+ * @return the trash directory
+ */
+ String getTrashDir() {
+ return trashDir;
+ }
+
+ /**
+ * Return a flag that indicates if multi-object delete recursion is enabled.
+ *
+ * @return the flag
+ */
+ boolean isEnableMultiObjectDeleteRecursion() {
+ return enableMultiObjectDeleteRecursion;
+ }
+
+ /**
+ * List the statuses of the files/directories in the given path if the path is
+ * a directory.
+ *
+ * @param f given path
+ * @return the statuses of the files/directories in the given patch
+ * @throws FileNotFoundException when the path does not exist
+ * @throws IOException see specific implementation
+ */
+ @Override
+ public FileStatus[] listStatus(final Path f)
+ throws FileNotFoundException, IOException {
+ long startTime = System.currentTimeMillis();
+ long threadId = Thread.currentThread().getId();
+ try {
+ FileStatus[] statuses = OBSCommonUtils.innerListStatus(this, f,
+ false);
+ long endTime = System.currentTimeMillis();
+ LOG.debug(
+ "List status for path:{}, thread:{}, timeUsedInMilliSec:{}", f,
+ threadId, endTime - startTime);
+ return statuses;
+ } catch (ObsException e) {
+ throw OBSCommonUtils.translateException("listStatus", f, e);
+ }
+ }
+
+ /**
+ * This public interface is provided specially for Huawei MRS. List the
+ * statuses of the files/directories in the given path if the path is a
+ * directory. When recursive is true, iterator all objects in the given path
+ * and its sub directories.
+ *
+ * @param f given path
+ * @param recursive whether to iterator objects in sub direcotries
+ * @return the statuses of the files/directories in the given patch
+ * @throws FileNotFoundException when the path does not exist
+ * @throws IOException see specific implementation
+ */
+ public FileStatus[] listStatus(final Path f, final boolean recursive)
+ throws FileNotFoundException, IOException {
+ long startTime = System.currentTimeMillis();
+ long threadId = Thread.currentThread().getId();
+ try {
+ FileStatus[] statuses = OBSCommonUtils.innerListStatus(this, f,
+ recursive);
+ long endTime = System.currentTimeMillis();
+ LOG.debug(
+ "List status for path:{}, thread:{}, timeUsedInMilliSec:{}", f,
+ threadId, endTime - startTime);
+ return statuses;
+ } catch (ObsException e) {
+ throw OBSCommonUtils.translateException(
+ "listStatus with recursive flag["
+ + (recursive ? "true] " : "false] "), f, e);
+ }
+ }
+
+ /**
+ * Return the OBSListing instance used by this filesystem.
+ *
+ * @return the OBSListing instance
+ */
+ OBSListing getObsListing() {
+ return obsListing;
+ }
+
+ /**
+ * Return the current working directory for the given file system.
+ *
+ * @return the directory pathname
+ */
+ @Override
+ public Path getWorkingDirectory() {
+ return workingDir;
+ }
+
+ /**
+ * Set the current working directory for the file system. All relative paths
+ * will be resolved relative to it.
+ *
+ * @param newDir the new working directory
+ */
+ @Override
+ public void setWorkingDirectory(final Path newDir) {
+ workingDir = newDir;
+ }
+
+ /**
+ * Return the username of the filesystem.
+ *
+ * @return the short name of the user who instantiated the filesystem
+ */
+ String getUsername() {
+ return username;
+ }
+
+ /**
+ * Make the given path and all non-existent parents into directories. Has the
+ * semantics of Unix {@code 'mkdir -p'}. Existence of the directory hierarchy
+ * is not an error.
+ *
+ * @param path path to create
+ * @param permission to apply to f
+ * @return true if a directory was created
+ * @throws FileAlreadyExistsException there is a file at the path specified
+ * @throws IOException other IO problems
+ */
+ @Override
+ public boolean mkdirs(final Path path, final FsPermission permission)
+ throws IOException, FileAlreadyExistsException {
+ try {
+ return OBSCommonUtils.innerMkdirs(this, path);
+ } catch (ObsException e) {
+ throw OBSCommonUtils.translateException("mkdirs", path, e);
+ }
+ }
+
+ /**
+ * Return a file status object that represents the path.
+ *
+ * @param f the path we want information from
+ * @return a FileStatus object
+ * @throws FileNotFoundException when the path does not exist
+ * @throws IOException on other problems
+ */
+ @Override
+ public FileStatus getFileStatus(final Path f)
+ throws FileNotFoundException, IOException {
+ for (int retryTime = 1;
+ retryTime < OBSCommonUtils.MAX_RETRY_TIME; retryTime++) {
+ try {
+ return innerGetFileStatus(f);
+ } catch (FileNotFoundException | FileConflictException e) {
+ throw e;
+ } catch (IOException e) {
+ LOG.warn("Failed to get file status for [{}], retry time [{}], "
+ + "exception [{}]", f, retryTime, e);
+
+ try {
+ Thread.sleep(OBSCommonUtils.DELAY_TIME);
+ } catch (InterruptedException ie) {
+ throw e;
+ }
+ }
+ }
+
+ return innerGetFileStatus(f);
+ }
+
+ /**
+ * Inner implementation without retry for {@link #getFileStatus(Path)}.
+ *
+ * @param f the path we want information from
+ * @return a FileStatus object
+ * @throws IOException on IO failure
+ */
+ @VisibleForTesting
+ OBSFileStatus innerGetFileStatus(final Path f) throws IOException {
+ if (enablePosix) {
+ return OBSPosixBucketUtils.innerFsGetObjectStatus(this, f);
+ }
+
+ return OBSObjectBucketUtils.innerGetObjectStatus(this, f);
+ }
+
+ /**
+ * Return the {@link ContentSummary} of a given {@link Path}.
+ *
+ * @param f path to use
+ * @return the {@link ContentSummary}
+ * @throws FileNotFoundException if the path does not resolve
+ * @throws IOException IO failure
+ */
+ @Override
+ public ContentSummary getContentSummary(final Path f)
+ throws FileNotFoundException, IOException {
+ if (!obsContentSummaryEnable) {
+ return super.getContentSummary(f);
+ }
+
+ FileStatus status = getFileStatus(f);
+ if (status.isFile()) {
+ // f is a file
+ long length = status.getLen();
+ return new ContentSummary.Builder().length(length)
+ .fileCount(1).directoryCount(0).spaceConsumed(length).build();
+ }
+
+ // f is a directory
+ if (enablePosix) {
+ return OBSPosixBucketUtils.fsGetDirectoryContentSummary(this,
+ OBSCommonUtils.pathToKey(this, f));
+ } else {
+ return OBSObjectBucketUtils.getDirectoryContentSummary(this,
+ OBSCommonUtils.pathToKey(this, f));
+ }
+ }
+
+ /**
+ * Copy the {@code src} file on the local disk to the filesystem at the given
+ * {@code dst} name.
+ *
+ * @param delSrc whether to delete the src
+ * @param overwrite whether to overwrite an existing file
+ * @param src path
+ * @param dst path
+ * @throws FileAlreadyExistsException if the destination file exists and
+ * overwrite == false
+ * @throws IOException IO problem
+ */
+ @Override
+ public void copyFromLocalFile(final boolean delSrc, final boolean overwrite,
+ final Path src, final Path dst) throws FileAlreadyExistsException,
+ IOException {
+ try {
+ super.copyFromLocalFile(delSrc, overwrite, src, dst);
+ } catch (ObsException e) {
+ throw OBSCommonUtils.translateException(
+ "copyFromLocalFile(" + src + ", " + dst + ")", src, e);
+ }
+ }
+
+ /**
+ * Close the filesystem. This shuts down all transfers.
+ *
+ * @throws IOException IO problem
+ */
+ @Override
+ public void close() throws IOException {
+ LOG.debug("This Filesystem closed by user, clear resource.");
+ if (closed.getAndSet(true)) {
+ // already closed
+ return;
+ }
+
+ try {
+ super.close();
+ } finally {
+ OBSCommonUtils.shutdownAll(
+ boundedMultipartUploadThreadPool,
+ boundedCopyThreadPool,
+ boundedDeleteThreadPool,
+ boundedCopyPartThreadPool,
+ boundedListThreadPool);
+ }
+ }
+
+ /**
+ * Override {@code getCanonicalServiceName} and return {@code null} since
+ * delegation token is not supported.
+ */
+ @Override
+ public String getCanonicalServiceName() {
+ // Does not support Token
+ return null;
+ }
+
+ /**
+ * Return copy part size.
+ *
+ * @return copy part size
+ */
+ long getCopyPartSize() {
+ return copyPartSize;
+ }
+
+ /**
+ * Return bounded thread pool for copy part.
+ *
+ * @return the bounded thread pool for copy part
+ */
+ ThreadPoolExecutor getBoundedCopyPartThreadPool() {
+ return boundedCopyPartThreadPool;
+ }
+
+ /**
+ * Return bounded thread pool for copy.
+ *
+ * @return the bounded thread pool for copy
+ */
+ ThreadPoolExecutor getBoundedCopyThreadPool() {
+ return boundedCopyThreadPool;
+ }
+
+ /**
+ * Imitate HDFS to return the number of bytes that large input files should be
+ * optimally split into to minimize I/O time for compatibility.
+ *
+ * @deprecated use {@link #getDefaultBlockSize(Path)} instead
+ */
+ @Override
+ public long getDefaultBlockSize() {
+ return blockSize;
+ }
+
+ /**
+ * Imitate HDFS to return the number of bytes that large input files should be
+ * optimally split into to minimize I/O time. The given path will be used to
+ * locate the actual filesystem. The full path does not have to exist.
+ *
+ * @param f path of file
+ * @return the default block size for the path's filesystem
+ */
+ @Override
+ public long getDefaultBlockSize(final Path f) {
+ return blockSize;
+ }
+
+ /**
+ * Return a string that describes this filesystem instance.
+ *
+ * @return the string
+ */
+ @Override
+ public String toString() {
+ final StringBuilder sb = new StringBuilder("OBSFileSystem{");
+ sb.append("uri=").append(uri);
+ sb.append(", workingDir=").append(workingDir);
+ sb.append(", partSize=").append(partSize);
+ sb.append(", enableMultiObjectsDelete=")
+ .append(enableMultiObjectDelete);
+ sb.append(", maxKeys=").append(maxKeys);
+ if (cannedACL != null) {
+ sb.append(", cannedACL=").append(cannedACL.toString());
+ }
+ sb.append(", readAheadRange=").append(readAheadRange);
+ sb.append(", blockSize=").append(getDefaultBlockSize());
+ if (blockFactory != null) {
+ sb.append(", blockFactory=").append(blockFactory);
+ }
+ sb.append(", boundedMultipartUploadThreadPool=")
+ .append(boundedMultipartUploadThreadPool);
+ sb.append(", statistics {").append(statistics).append("}");
+ sb.append(", metrics {").append("}");
+ sb.append('}');
+ return sb.toString();
+ }
+
+ /**
+ * Return the maximum number of keys to get while paging through a directory
+ * listing.
+ *
+ * @return the maximum number of keys
+ */
+ int getMaxKeys() {
+ return maxKeys;
+ }
+
+ /**
+ * List the statuses and block locations of the files in the given path. Does
+ * not guarantee to return the iterator that traverses statuses of the files
+ * in a sorted order.
+ *
+ *
+ * If the path is a directory,
+ * if recursive is false, returns files in the directory;
+ * if recursive is true, return files in the subtree rooted at the path.
+ * If the path is a file, return the file's status and block locations.
+ *
+ *
+ * @param f a path
+ * @param recursive if the subdirectories need to be traversed recursively
+ * @return an iterator that traverses statuses of the files/directories in the
+ * given path
+ * @throws FileNotFoundException if {@code path} does not exist
+ * @throws IOException if any I/O error occurred
+ */
+ @Override
+ public RemoteIterator listFiles(final Path f,
+ final boolean recursive)
+ throws FileNotFoundException, IOException {
+ Path path = OBSCommonUtils.qualify(this, f);
+ LOG.debug("listFiles({}, {})", path, recursive);
+ try {
+ // lookup dir triggers existence check
+ final FileStatus fileStatus = getFileStatus(path);
+ if (fileStatus.isFile()) {
+ // simple case: File
+ LOG.debug("Path is a file");
+ return new OBSListing
+ .SingleStatusRemoteIterator(
+ OBSCommonUtils.toLocatedFileStatus(this, fileStatus));
+ } else {
+ LOG.debug(
+ "listFiles: doing listFiles of directory {} - recursive {}",
+ path, recursive);
+ // directory: do a bulk operation
+ String key = OBSCommonUtils.maybeAddTrailingSlash(
+ OBSCommonUtils.pathToKey(this, path));
+ String delimiter = recursive ? null : "/";
+ LOG.debug("Requesting all entries under {} with delimiter '{}'",
+ key, delimiter);
+ return obsListing.createLocatedFileStatusIterator(
+ obsListing.createFileStatusListingIterator(
+ path,
+ OBSCommonUtils.createListObjectsRequest(this, key,
+ delimiter),
+ OBSListing.ACCEPT_ALL,
+ new OBSListing.AcceptFilesOnly(path)));
+ }
+ } catch (ObsException e) {
+ throw OBSCommonUtils.translateException("listFiles", path, e);
+ }
+ }
+
+ /**
+ * List the statuses of the files/directories in the given path if the path is
+ * a directory. Return the file's status and block locations If the path is a
+ * file.
+ *
+ * If a returned status is a file, it contains the file's block locations.
+ *
+ * @param f is the path
+ * @return an iterator that traverses statuses of the files/directories in the
+ * given path
+ * @throws FileNotFoundException If f
does not exist
+ * @throws IOException If an I/O error occurred
+ */
+ @Override
+ public RemoteIterator listLocatedStatus(final Path f)
+ throws FileNotFoundException, IOException {
+ return listLocatedStatus(f,
+ OBSListing.ACCEPT_ALL);
+ }
+
+ /**
+ * List a directory. The returned results include its block location if it is
+ * a file The results are filtered by the given path filter
+ *
+ * @param f a path
+ * @param filter a path filter
+ * @return an iterator that traverses statuses of the files/directories in the
+ * given path
+ * @throws FileNotFoundException if f
does not exist
+ * @throws IOException if any I/O error occurred
+ */
+ @Override
+ public RemoteIterator listLocatedStatus(final Path f,
+ final PathFilter filter)
+ throws FileNotFoundException, IOException {
+ Path path = OBSCommonUtils.qualify(this, f);
+ LOG.debug("listLocatedStatus({}, {}", path, filter);
+ try {
+ // lookup dir triggers existence check
+ final FileStatus fileStatus = getFileStatus(path);
+ if (fileStatus.isFile()) {
+ // simple case: File
+ LOG.debug("Path is a file");
+ return new OBSListing.SingleStatusRemoteIterator(
+ filter.accept(path) ? OBSCommonUtils.toLocatedFileStatus(
+ this, fileStatus) : null);
+ } else {
+ // directory: trigger a lookup
+ String key = OBSCommonUtils.maybeAddTrailingSlash(
+ OBSCommonUtils.pathToKey(this, path));
+ return obsListing.createLocatedFileStatusIterator(
+ obsListing.createFileStatusListingIterator(
+ path,
+ OBSCommonUtils.createListObjectsRequest(this, key, "/"),
+ filter,
+ new OBSListing.AcceptAllButSelfAndS3nDirs(path)));
+ }
+ } catch (ObsException e) {
+ throw OBSCommonUtils.translateException("listLocatedStatus", path,
+ e);
+ }
+ }
+
+ /**
+ * Return server-side encryption wrapper used by this filesystem instance.
+ *
+ * @return the server-side encryption wrapper
+ */
+ SseWrapper getSse() {
+ return sse;
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFsDFSListing.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFsDFSListing.java
new file mode 100644
index 00000000000..bbf29df14f3
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFsDFSListing.java
@@ -0,0 +1,744 @@
+/*
+ * 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.obs;
+
+import com.obs.services.model.ListObjectsRequest;
+import com.obs.services.model.ObjectListing;
+import com.obs.services.model.ObjectMetadata;
+import com.obs.services.model.ObsObject;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Queue;
+import java.util.Stack;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+/**
+ * OBS depth first search listing implementation for posix bucket.
+ */
+class OBSFsDFSListing extends ObjectListing {
+ /**
+ * Class logger.
+ */
+ private static final Logger LOG = LoggerFactory.getLogger(
+ OBSFsDFSListing.class);
+
+ static void increaseLevelStats(final List levelStatsList,
+ final int level,
+ final boolean isDir) {
+ int currMaxLevel = levelStatsList.size() - 1;
+ if (currMaxLevel < level) {
+ for (int i = 0; i < level - currMaxLevel; i++) {
+ levelStatsList.add(new LevelStats(currMaxLevel + 1 + i));
+ }
+ }
+
+ if (isDir) {
+ levelStatsList.get(level).increaseDirNum();
+ } else {
+ levelStatsList.get(level).increaseFileNum();
+ }
+ }
+
+ static String fsDFSListNextBatch(final OBSFileSystem owner,
+ final Stack listStack,
+ final Queue resultQueue,
+ final String marker,
+ final int maxKeyNum,
+ final List objectSummaries,
+ final List levelStatsList) throws IOException {
+ // 0. check if marker matches with the peek of result queue when marker
+ // is given
+ if (marker != null) {
+ if (resultQueue.isEmpty()) {
+ throw new IllegalArgumentException(
+ "result queue is empty, but marker is not empty: "
+ + marker);
+ } else if (resultQueue.peek().getType()
+ == ListEntityType.LIST_TAIL) {
+ throw new RuntimeException(
+ "cannot put list tail (" + resultQueue.peek()
+ + ") into result queue");
+ } else if (!marker.equals(
+ resultQueue.peek().getType() == ListEntityType.COMMON_PREFIX
+ ? resultQueue.peek().getCommonPrefix()
+ : resultQueue.peek().getObjectSummary().getObjectKey())) {
+ throw new IllegalArgumentException("marker (" + marker
+ + ") does not match with result queue peek ("
+ + resultQueue.peek() + ")");
+ }
+ }
+
+ // 1. fetch some list results from local result queue
+ int resultNum = fetchListResultLocally(owner.getBucket(), resultQueue,
+ maxKeyNum, objectSummaries,
+ levelStatsList);
+
+ // 2. fetch more list results by doing one-level lists in parallel
+ fetchListResultRemotely(owner, listStack, resultQueue, maxKeyNum,
+ objectSummaries, levelStatsList, resultNum);
+
+ // 3. check if list operation ends
+ if (!listStack.empty() && resultQueue.isEmpty()) {
+ throw new RuntimeException(
+ "result queue is empty, but list stack is not empty: "
+ + listStack);
+ }
+
+ String nextMarker = null;
+ if (!resultQueue.isEmpty()) {
+ if (resultQueue.peek().getType() == ListEntityType.LIST_TAIL) {
+ throw new RuntimeException(
+ "cannot put list tail (" + resultQueue.peek()
+ + ") into result queue");
+ } else {
+ nextMarker =
+ resultQueue.peek().getType() == ListEntityType.COMMON_PREFIX
+ ? resultQueue
+ .peek().getCommonPrefix()
+ : resultQueue.peek().getObjectSummary().getObjectKey();
+ }
+ }
+ return nextMarker;
+ }
+
+ static void fetchListResultRemotely(final OBSFileSystem owner,
+ final Stack listStack,
+ final Queue resultQueue, final int maxKeyNum,
+ final List objectSummaries,
+ final List levelStatsList,
+ final int resultNum) throws IOException {
+ int newResultNum = resultNum;
+ while (!listStack.empty() && (newResultNum < maxKeyNum
+ || resultQueue.isEmpty())) {
+ List oneLevelListRequests = new ArrayList<>();
+ List> oneLevelListFutures = new ArrayList<>();
+ List levels = new ArrayList<>();
+ List oneLevelObjectListings = new ArrayList<>();
+ // a. submit some one-level list tasks in parallel
+ submitOneLevelListTasks(owner, listStack, maxKeyNum,
+ oneLevelListRequests, oneLevelListFutures, levels);
+
+ // b. wait these tasks to complete
+ waitForOneLevelListTasksFinished(oneLevelListRequests,
+ oneLevelListFutures, oneLevelObjectListings);
+
+ // c. put subdir/file into result commonPrefixes and
+ // objectSummaries;if the number of results reaches maxKeyNum,
+ // cache it into resultQueue for next list batch note: unlike
+ // standard DFS, we put subdir directly into result list to avoid
+ // caching it using more space
+ newResultNum = handleOneLevelListTaskResult(resultQueue, maxKeyNum,
+ objectSummaries, levelStatsList, newResultNum,
+ oneLevelListRequests, levels, oneLevelObjectListings);
+
+ // d. push subdirs and list continuing tail/end into list stack in
+ // reversed order,so that we can pop them from the stack in order
+ // later
+ addNewListStackEntities(listStack, oneLevelListRequests, levels,
+ oneLevelObjectListings);
+ }
+ }
+
+ @SuppressWarnings("checkstyle:parameternumber")
+ static int handleOneLevelListTaskResult(final Queue resultQueue,
+ final int maxKeyNum,
+ final List objectSummaries,
+ final List levelStatsList,
+ final int resultNum,
+ final List oneLevelListRequests,
+ final List levels,
+ final List oneLevelObjectListings) {
+ int newResultNum = resultNum;
+ for (int i = 0; i < oneLevelObjectListings.size(); i++) {
+ LOG.debug(
+ "one level listing with prefix=" + oneLevelListRequests.get(i)
+ .getPrefix()
+ + ", marker=" + (
+ oneLevelListRequests.get(i).getMarker() != null
+ ? oneLevelListRequests.get(i)
+ .getMarker()
+ : ""));
+
+ ObjectListing oneLevelObjectListing = oneLevelObjectListings.get(i);
+ LOG.debug("# of CommonPrefixes/Objects: {}/{}",
+ oneLevelObjectListing.getCommonPrefixes().size(),
+ oneLevelObjectListing.getObjects().size());
+
+ if (oneLevelObjectListing.getCommonPrefixes().isEmpty()
+ && oneLevelObjectListing.getObjects().isEmpty()) {
+ continue;
+ }
+
+ for (String commonPrefix
+ : oneLevelObjectListing.getCommonPrefixes()) {
+ if (commonPrefix.equals(
+ oneLevelListRequests.get(i).getPrefix())) {
+ // skip prefix itself
+ continue;
+ }
+
+ LOG.debug("common prefix: " + commonPrefix);
+ if (newResultNum < maxKeyNum) {
+ addCommonPrefixIntoObjectList(
+ oneLevelListRequests.get(i).getBucketName(),
+ objectSummaries,
+ commonPrefix);
+ increaseLevelStats(levelStatsList, levels.get(i), true);
+ newResultNum++;
+ } else {
+ resultQueue.add(
+ new ListEntity(commonPrefix, levels.get(i)));
+ }
+ }
+
+ for (ObsObject obj : oneLevelObjectListing.getObjects()) {
+ if (obj.getObjectKey()
+ .equals(oneLevelListRequests.get(i).getPrefix())) {
+ // skip prefix itself
+ continue;
+ }
+
+ LOG.debug("object: {}, size: {}", obj.getObjectKey(),
+ obj.getMetadata().getContentLength());
+ if (newResultNum < maxKeyNum) {
+ objectSummaries.add(obj);
+ increaseLevelStats(levelStatsList, levels.get(i),
+ obj.getObjectKey().endsWith("/"));
+ newResultNum++;
+ } else {
+ resultQueue.add(new ListEntity(obj, levels.get(i)));
+ }
+ }
+ }
+ return newResultNum;
+ }
+
+ static void waitForOneLevelListTasksFinished(
+ final List oneLevelListRequests,
+ final List> oneLevelListFutures,
+ final List oneLevelObjectListings)
+ throws IOException {
+ for (int i = 0; i < oneLevelListFutures.size(); i++) {
+ try {
+ oneLevelObjectListings.add(oneLevelListFutures.get(i).get());
+ } catch (InterruptedException e) {
+ LOG.warn("Interrupted while listing using DFS, prefix="
+ + oneLevelListRequests.get(i).getPrefix() + ", marker="
+ + (oneLevelListRequests.get(i).getMarker() != null
+ ? oneLevelListRequests.get(i).getMarker()
+ : ""));
+ throw new InterruptedIOException(
+ "Interrupted while listing using DFS, prefix="
+ + oneLevelListRequests.get(i).getPrefix() + ", marker="
+ + (oneLevelListRequests.get(i).getMarker() != null
+ ? oneLevelListRequests.get(i).getMarker()
+ : ""));
+ } catch (ExecutionException e) {
+ LOG.error("Exception while listing using DFS, prefix="
+ + oneLevelListRequests.get(i).getPrefix() + ", marker="
+ + (oneLevelListRequests.get(i).getMarker() != null
+ ? oneLevelListRequests.get(i).getMarker()
+ : ""),
+ e);
+ for (Future future : oneLevelListFutures) {
+ future.cancel(true);
+ }
+
+ throw OBSCommonUtils.extractException(
+ "Listing using DFS with exception, marker="
+ + (oneLevelListRequests.get(i).getMarker() != null
+ ? oneLevelListRequests.get(i).getMarker()
+ : ""),
+ oneLevelListRequests.get(i).getPrefix(), e);
+ }
+ }
+ }
+
+ static void submitOneLevelListTasks(final OBSFileSystem owner,
+ final Stack listStack, final int maxKeyNum,
+ final List oneLevelListRequests,
+ final List> oneLevelListFutures,
+ final List levels) {
+ for (int i = 0;
+ i < owner.getListParallelFactor() && !listStack.empty(); i++) {
+ ListEntity listEntity = listStack.pop();
+ if (listEntity.getType() == ListEntityType.LIST_TAIL) {
+ if (listEntity.getNextMarker() != null) {
+ ListObjectsRequest oneLevelListRequest
+ = new ListObjectsRequest();
+ oneLevelListRequest.setBucketName(owner.getBucket());
+ oneLevelListRequest.setPrefix(listEntity.getPrefix());
+ oneLevelListRequest.setMarker(listEntity.getNextMarker());
+ oneLevelListRequest.setMaxKeys(
+ Math.min(maxKeyNum, owner.getMaxKeys()));
+ oneLevelListRequest.setDelimiter("/");
+ oneLevelListRequests.add(oneLevelListRequest);
+ oneLevelListFutures.add(owner.getBoundedListThreadPool()
+ .submit(() -> OBSCommonUtils.commonContinueListObjects(
+ owner, oneLevelListRequest)));
+ levels.add(listEntity.getLevel());
+ }
+
+ // avoid adding list tasks in different levels later
+ break;
+ } else {
+ String oneLevelListPrefix =
+ listEntity.getType() == ListEntityType.COMMON_PREFIX
+ ? listEntity.getCommonPrefix()
+ : listEntity.getObjectSummary().getObjectKey();
+ ListObjectsRequest oneLevelListRequest = OBSCommonUtils
+ .createListObjectsRequest(owner, oneLevelListPrefix, "/",
+ maxKeyNum);
+ oneLevelListRequests.add(oneLevelListRequest);
+ oneLevelListFutures.add(owner.getBoundedListThreadPool()
+ .submit(() -> OBSCommonUtils.commonListObjects(owner,
+ oneLevelListRequest)));
+ levels.add(listEntity.getLevel() + 1);
+ }
+ }
+ }
+
+ static void addNewListStackEntities(final Stack listStack,
+ final List oneLevelListRequests,
+ final List levels,
+ final List oneLevelObjectListings) {
+ for (int i = oneLevelObjectListings.size() - 1; i >= 0; i--) {
+ ObjectListing oneLevelObjectListing = oneLevelObjectListings.get(i);
+
+ if (oneLevelObjectListing.getCommonPrefixes().isEmpty()
+ && oneLevelObjectListing.getObjects()
+ .isEmpty()) {
+ continue;
+ }
+
+ listStack.push(new ListEntity(oneLevelObjectListing.getPrefix(),
+ oneLevelObjectListing.isTruncated()
+ ? oneLevelObjectListing.getNextMarker()
+ : null,
+ levels.get(i)));
+
+ ListIterator commonPrefixListIterator
+ = oneLevelObjectListing.getCommonPrefixes()
+ .listIterator(oneLevelObjectListing.getCommonPrefixes().size());
+ while (commonPrefixListIterator.hasPrevious()) {
+ String commonPrefix = commonPrefixListIterator.previous();
+
+ if (commonPrefix.equals(
+ oneLevelListRequests.get(i).getPrefix())) {
+ // skip prefix itself
+ continue;
+ }
+
+ listStack.push(new ListEntity(commonPrefix, levels.get(i)));
+ }
+
+ ListIterator objectSummaryListIterator
+ = oneLevelObjectListing.getObjects()
+ .listIterator(oneLevelObjectListing.getObjects().size());
+ while (objectSummaryListIterator.hasPrevious()) {
+ ObsObject objectSummary = objectSummaryListIterator.previous();
+
+ if (objectSummary.getObjectKey()
+ .equals(oneLevelListRequests.get(i).getPrefix())) {
+ // skip prefix itself
+ continue;
+ }
+
+ if (objectSummary.getObjectKey().endsWith("/")) {
+ listStack.push(
+ new ListEntity(objectSummary, levels.get(i)));
+ }
+ }
+ }
+ }
+
+ static int fetchListResultLocally(final String bucketName,
+ final Queue resultQueue, final int maxKeyNum,
+ final List objectSummaries,
+ final List levelStatsList) {
+ int resultNum = 0;
+ while (!resultQueue.isEmpty() && resultNum < maxKeyNum) {
+ ListEntity listEntity = resultQueue.poll();
+ if (listEntity.getType() == ListEntityType.LIST_TAIL) {
+ throw new RuntimeException("cannot put list tail (" + listEntity
+ + ") into result queue");
+ } else if (listEntity.getType() == ListEntityType.COMMON_PREFIX) {
+ addCommonPrefixIntoObjectList(bucketName, objectSummaries,
+ listEntity.getCommonPrefix());
+ increaseLevelStats(levelStatsList, listEntity.getLevel(), true);
+ resultNum++;
+ } else {
+ objectSummaries.add(listEntity.getObjectSummary());
+ increaseLevelStats(levelStatsList, listEntity.getLevel(),
+ listEntity.getObjectSummary().getObjectKey().endsWith("/"));
+ resultNum++;
+ }
+ }
+ return resultNum;
+ }
+
+ static void addCommonPrefixIntoObjectList(final String bucketName,
+ final List objectSummaries,
+ final String commonPrefix) {
+ ObsObject objectSummary = new ObsObject();
+ ObjectMetadata objectMetadata = new ObjectMetadata();
+ objectMetadata.setContentLength(0L);
+ objectSummary.setBucketName(bucketName);
+ objectSummary.setObjectKey(commonPrefix);
+ objectSummary.setMetadata(objectMetadata);
+ objectSummaries.add(objectSummary);
+ }
+
+ static OBSFsDFSListing fsDFSListObjects(final OBSFileSystem owner,
+ final ListObjectsRequest request) throws IOException {
+ List objectSummaries = new ArrayList<>();
+ List commonPrefixes = new ArrayList<>();
+ String bucketName = owner.getBucket();
+ String prefix = request.getPrefix();
+ int maxKeyNum = request.getMaxKeys();
+ if (request.getDelimiter() != null) {
+ throw new IllegalArgumentException(
+ "illegal delimiter: " + request.getDelimiter());
+ }
+ if (request.getMarker() != null) {
+ throw new IllegalArgumentException(
+ "illegal marker: " + request.getMarker());
+ }
+
+ Stack listStack = new Stack<>();
+ Queue resultQueue = new LinkedList<>();
+ List levelStatsList = new ArrayList<>();
+
+ listStack.push(new ListEntity(prefix, 0));
+ increaseLevelStats(levelStatsList, 0, true);
+
+ String nextMarker = fsDFSListNextBatch(owner, listStack, resultQueue,
+ null, maxKeyNum, objectSummaries,
+ levelStatsList);
+
+ if (nextMarker == null) {
+ StringBuilder levelStatsStringBuilder = new StringBuilder();
+ levelStatsStringBuilder.append("bucketName=").append(bucketName)
+ .append(", prefix=").append(prefix).append(": ");
+ for (LevelStats levelStats : levelStatsList) {
+ levelStatsStringBuilder.append("level=")
+ .append(levelStats.getLevel())
+ .append(", dirNum=")
+ .append(levelStats.getDirNum())
+ .append(", fileNum=")
+ .append(levelStats.getFileNum())
+ .append("; ");
+ }
+ LOG.debug("[list level statistics info] "
+ + levelStatsStringBuilder.toString());
+ }
+
+ return new OBSFsDFSListing(request,
+ objectSummaries,
+ commonPrefixes,
+ nextMarker,
+ listStack,
+ resultQueue,
+ levelStatsList);
+ }
+
+ static OBSFsDFSListing fsDFSContinueListObjects(final OBSFileSystem owner,
+ final OBSFsDFSListing obsFsDFSListing)
+ throws IOException {
+ List objectSummaries = new ArrayList<>();
+ List commonPrefixes = new ArrayList<>();
+ String bucketName = owner.getBucket();
+ String prefix = obsFsDFSListing.getPrefix();
+ String marker = obsFsDFSListing.getNextMarker();
+ int maxKeyNum = obsFsDFSListing.getMaxKeys();
+ if (obsFsDFSListing.getDelimiter() != null) {
+ throw new IllegalArgumentException(
+ "illegal delimiter: " + obsFsDFSListing.getDelimiter());
+ }
+
+ Stack listStack = obsFsDFSListing.getListStack();
+ Queue resultQueue = obsFsDFSListing.getResultQueue();
+ List levelStatsList = obsFsDFSListing.getLevelStatsList();
+
+ String nextMarker = fsDFSListNextBatch(owner, listStack, resultQueue,
+ marker, maxKeyNum, objectSummaries,
+ levelStatsList);
+
+ if (nextMarker == null) {
+ StringBuilder levelStatsStringBuilder = new StringBuilder();
+ levelStatsStringBuilder.append("bucketName=").append(bucketName)
+ .append(", prefix=").append(prefix).append(": ");
+ for (LevelStats levelStats : levelStatsList) {
+ levelStatsStringBuilder.append("level=")
+ .append(levelStats.getLevel())
+ .append(", dirNum=")
+ .append(levelStats.getDirNum())
+ .append(", fileNum=")
+ .append(levelStats.getFileNum())
+ .append("; ");
+ }
+ LOG.debug("[list level statistics info] "
+ + levelStatsStringBuilder.toString());
+ }
+
+ return new OBSFsDFSListing(obsFsDFSListing,
+ objectSummaries,
+ commonPrefixes,
+ nextMarker,
+ listStack,
+ resultQueue,
+ levelStatsList);
+ }
+
+ /**
+ * List entity type definition.
+ */
+ enum ListEntityType {
+ /**
+ * Common prefix.
+ */
+ COMMON_PREFIX,
+ /**
+ * Object summary.
+ */
+ OBJECT_SUMMARY,
+ /**
+ * List tail.
+ */
+ LIST_TAIL
+ }
+
+ /**
+ * List entity for OBS depth first search listing.
+ */
+ static class ListEntity {
+ /**
+ * List entity type.
+ */
+ private ListEntityType type;
+
+ /**
+ * Entity level.
+ */
+ private final int level;
+
+ /**
+ * For COMMON_PREFIX.
+ */
+ private String commonPrefix = null;
+
+ /**
+ * For OBJECT_SUMMARY.
+ */
+ private ObsObject objectSummary = null;
+
+ /**
+ * For LIST_TAIL.
+ */
+ private String prefix = null;
+
+ /**
+ * Next marker.
+ */
+ private String nextMarker = null;
+
+ ListEntity(final String comPrefix, final int entityLevel) {
+ this.type = ListEntityType.COMMON_PREFIX;
+ this.commonPrefix = comPrefix;
+ this.level = entityLevel;
+ }
+
+ ListEntity(final ObsObject summary, final int entityLevel) {
+ this.type = ListEntityType.OBJECT_SUMMARY;
+ this.objectSummary = summary;
+ this.level = entityLevel;
+ }
+
+ ListEntity(final String pf, final String nextMk,
+ final int entityLevel) {
+ this.type = ListEntityType.LIST_TAIL;
+ this.prefix = pf;
+ this.nextMarker = nextMk;
+ this.level = entityLevel;
+ }
+
+ ListEntityType getType() {
+ return type;
+ }
+
+ int getLevel() {
+ return level;
+ }
+
+ String getCommonPrefix() {
+ return commonPrefix;
+ }
+
+ ObsObject getObjectSummary() {
+ return objectSummary;
+ }
+
+ public String getPrefix() {
+ return prefix;
+ }
+
+ String getNextMarker() {
+ return nextMarker;
+ }
+
+ @Override
+ public String toString() {
+ return "type: " + type
+ + ", commonPrefix: " + (commonPrefix != null
+ ? commonPrefix
+ : "")
+ + ", objectSummary: " + (objectSummary != null
+ ? objectSummary
+ : "")
+ + ", prefix: " + (prefix != null ? prefix : "")
+ + ", nextMarker: " + (nextMarker != null ? nextMarker : "");
+ }
+ }
+
+ /**
+ * Level statistics for OBS depth first search listing.
+ */
+ static class LevelStats {
+ /**
+ * Entity level.
+ */
+ private int level;
+
+ /**
+ * Directory num.
+ */
+ private long dirNum;
+
+ /**
+ * File num.
+ */
+ private long fileNum;
+
+ LevelStats(final int entityLevel) {
+ this.level = entityLevel;
+ this.dirNum = 0;
+ this.fileNum = 0;
+ }
+
+ void increaseDirNum() {
+ dirNum++;
+ }
+
+ void increaseFileNum() {
+ fileNum++;
+ }
+
+ int getLevel() {
+ return level;
+ }
+
+ long getDirNum() {
+ return dirNum;
+ }
+
+ long getFileNum() {
+ return fileNum;
+ }
+ }
+
+ /**
+ * Stack of entity list..
+ */
+ private Stack listStack;
+
+ /**
+ * Queue of entity list.
+ */
+ private Queue resultQueue;
+
+ /**
+ * List of levelStats.
+ */
+ private List levelStatsList;
+
+ OBSFsDFSListing(final ListObjectsRequest request,
+ final List objectSummaries,
+ final List commonPrefixes,
+ final String nextMarker,
+ final Stack listEntityStack,
+ final Queue listEntityQueue,
+ final List listLevelStats) {
+ super(objectSummaries,
+ commonPrefixes,
+ request.getBucketName(),
+ nextMarker != null,
+ request.getPrefix(),
+ null,
+ request.getMaxKeys(),
+ null,
+ nextMarker,
+ null);
+ this.listStack = listEntityStack;
+ this.resultQueue = listEntityQueue;
+ this.levelStatsList = listLevelStats;
+ }
+
+ OBSFsDFSListing(final OBSFsDFSListing obsFsDFSListing,
+ final List objectSummaries,
+ final List commonPrefixes,
+ final String nextMarker,
+ final Stack listEntityStack,
+ final Queue listEntityQueue,
+ final List listLevelStats) {
+ super(objectSummaries,
+ commonPrefixes,
+ obsFsDFSListing.getBucketName(),
+ nextMarker != null,
+ obsFsDFSListing.getPrefix(),
+ obsFsDFSListing.getNextMarker(),
+ obsFsDFSListing.getMaxKeys(),
+ null,
+ nextMarker,
+ null);
+ this.listStack = listEntityStack;
+ this.resultQueue = listEntityQueue;
+ this.levelStatsList = listLevelStats;
+ }
+
+ Stack getListStack() {
+ return listStack;
+ }
+
+ Queue getResultQueue() {
+ return resultQueue;
+ }
+
+ List getLevelStatsList() {
+ return levelStatsList;
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIOException.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIOException.java
new file mode 100644
index 00000000000..29a92c71919
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIOException.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.obs;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import com.obs.services.exception.ObsException;
+
+import java.io.IOException;
+
+/**
+ * IOException equivalent to {@link ObsException}.
+ */
+class OBSIOException extends IOException {
+ private static final long serialVersionUID = -1582681108285856259L;
+
+ /**
+ * Peration message.
+ */
+ private final String operation;
+
+ OBSIOException(final String operationMsg, final ObsException cause) {
+ super(cause);
+ Preconditions.checkArgument(operationMsg != null,
+ "Null 'operation' argument");
+ Preconditions.checkArgument(cause != null, "Null 'cause' argument");
+ this.operation = operationMsg;
+ }
+
+ public ObsException getCause() {
+ return (ObsException) super.getCause();
+ }
+
+ @Override
+ public String getMessage() {
+ return operation + ": " + getCause().getErrorMessage()
+ + ", detailMessage: " + super.getMessage();
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSInputStream.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSInputStream.java
new file mode 100644
index 00000000000..e94565a4d76
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSInputStream.java
@@ -0,0 +1,1047 @@
+/*
+ * 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.obs;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import com.obs.services.ObsClient;
+import com.obs.services.exception.ObsException;
+import com.obs.services.model.GetObjectRequest;
+import com.sun.istack.NotNull;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.ByteBufferReadable;
+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.nio.ByteBuffer;
+
+import static org.apache.hadoop.fs.obs.OBSCommonUtils.translateException;
+
+/**
+ * Input stream for an OBS object.
+ *
+ * As this stream seeks withing an object, it may close then re-open the
+ * stream. When this happens, any updated stream data may be retrieved, and,
+ * given the consistency model of Huawei OBS, outdated data may in fact be
+ * picked up.
+ *
+ *
As a result, the outcome of reading from a stream of an object which is
+ * actively manipulated during the read process is "undefined".
+ *
+ *
The class is marked as private as code should not be creating instances
+ * themselves. Any extra feature (e.g instrumentation) should be considered
+ * unstable.
+ *
+ *
Because it prints some of the state of the instrumentation, the output of
+ * {@link #toString()} must also be considered unstable.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+class OBSInputStream extends FSInputStream
+ implements CanSetReadahead, ByteBufferReadable {
+ /**
+ * Class logger.
+ */
+ public static final Logger LOG = LoggerFactory.getLogger(
+ OBSInputStream.class);
+
+ /**
+ * Read retry times.
+ */
+ private static final int READ_RETRY_TIME = 3;
+
+ /**
+ * Seek retry times.
+ */
+ private static final int SEEK_RETRY_TIME = 9;
+
+ /**
+ * Delay times.
+ */
+ private static final long DELAY_TIME = 10;
+
+ /**
+ * The statistics for OBS file system.
+ */
+ private final FileSystem.Statistics statistics;
+
+ /**
+ * Obs client.
+ */
+ private final ObsClient client;
+
+ /**
+ * Bucket name.
+ */
+ private final String bucket;
+
+ /**
+ * Bucket key.
+ */
+ private final String key;
+
+ /**
+ * Content length.
+ */
+ private final long contentLength;
+
+ /**
+ * Object uri.
+ */
+ private final String uri;
+
+ /**
+ * Obs file system instance.
+ */
+ private OBSFileSystem fs;
+
+ /**
+ * This is the public position; the one set in {@link #seek(long)} and
+ * returned in {@link #getPos()}.
+ */
+ private long streamCurrentPos;
+
+ /**
+ * Closed bit. Volatile so reads are non-blocking. Updates must be in a
+ * synchronized block to guarantee an atomic check and set
+ */
+ private volatile boolean closed;
+
+ /**
+ * Input stream.
+ */
+ private InputStream wrappedStream = null;
+
+ /**
+ * Read ahead range.
+ */
+ private long readAheadRange = OBSConstants.DEFAULT_READAHEAD_RANGE;
+
+ /**
+ * 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;
+
+ /**
+ * 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;
+
+ /**
+ * The start of the content range of the last request.
+ */
+ private long contentRangeStart;
+
+ OBSInputStream(
+ final String bucketName,
+ final String bucketKey,
+ final long fileStatusLength,
+ final ObsClient obsClient,
+ final FileSystem.Statistics stats,
+ final long readaheadRange,
+ final OBSFileSystem obsFileSystem) {
+ Preconditions.checkArgument(StringUtils.isNotEmpty(bucketName),
+ "No Bucket");
+ Preconditions.checkArgument(StringUtils.isNotEmpty(bucketKey),
+ "No Key");
+ Preconditions.checkArgument(fileStatusLength >= 0,
+ "Negative content length");
+ this.bucket = bucketName;
+ this.key = bucketKey;
+ this.contentLength = fileStatusLength;
+ this.client = obsClient;
+ this.statistics = stats;
+ this.uri = "obs://" + this.bucket + "/" + this.key;
+ this.fs = obsFileSystem;
+ setReadahead(readaheadRange);
+ }
+
+ /**
+ * Calculate the limit for a get request, based on input policy and state of
+ * object.
+ *
+ * @param targetPos position of the read
+ * @param length length of bytes requested; if less than zero
+ * "unknown"
+ * @param contentLength total length of file
+ * @param readahead current readahead value
+ * @return the absolute value of the limit of the request.
+ */
+ static long calculateRequestLimit(
+ final long targetPos, final long length, final long contentLength,
+ final long readahead) {
+ // cannot read past the end of the object
+ return Math.min(contentLength, length < 0 ? contentLength
+ : targetPos + Math.max(readahead, 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(final String reason, final long targetPos,
+ final long length)
+ throws IOException {
+ long startTime = System.currentTimeMillis();
+ long threadId = Thread.currentThread().getId();
+ if (wrappedStream != null) {
+ closeStream("reopen(" + reason + ")", contentRangeFinish);
+ }
+
+ contentRangeFinish =
+ calculateRequestLimit(targetPos, length, contentLength,
+ readAheadRange);
+
+ try {
+ GetObjectRequest request = new GetObjectRequest(bucket, key);
+ request.setRangeStart(targetPos);
+ request.setRangeEnd(contentRangeFinish);
+ if (fs.getSse().isSseCEnable()) {
+ request.setSseCHeader(fs.getSse().getSseCHeader());
+ }
+ wrappedStream = client.getObject(request).getObjectContent();
+ contentRangeStart = targetPos;
+ if (wrappedStream == null) {
+ throw new IOException(
+ "Null IO stream from reopen of (" + reason + ") " + uri);
+ }
+ } catch (ObsException e) {
+ throw translateException("Reopen at position " + targetPos, uri, e);
+ }
+
+ this.streamCurrentPos = targetPos;
+ long endTime = System.currentTimeMillis();
+ LOG.debug(
+ "reopen({}) for {} range[{}-{}], length={},"
+ + " streamPosition={}, nextReadPosition={}, thread={}, "
+ + "timeUsedInMilliSec={}",
+ uri,
+ reason,
+ targetPos,
+ contentRangeFinish,
+ length,
+ streamCurrentPos,
+ nextReadPos,
+ threadId,
+ endTime - startTime
+ );
+ }
+
+ @Override
+ public synchronized long getPos() {
+ return nextReadPos < 0 ? 0 : nextReadPos;
+ }
+
+ @Override
+ public synchronized void seek(final long targetPos) throws IOException {
+ checkNotClosed();
+
+ // Do not allow negative seek
+ if (targetPos < 0) {
+ throw new EOFException(
+ FSExceptionMessages.NEGATIVE_SEEK + " " + targetPos);
+ }
+
+ if (this.contentLength <= 0) {
+ return;
+ }
+
+ // Lazy seek
+ nextReadPos = targetPos;
+ }
+
+ /**
+ * Seek without raising any exception. This is for use in {@code finally}
+ * clauses
+ *
+ * @param positiveTargetPos a target position which must be positive.
+ */
+ private void seekQuietly(final long positiveTargetPos) {
+ try {
+ seek(positiveTargetPos);
+ } catch (IOException ioe) {
+ LOG.debug("Ignoring IOE on seek of {} to {}", uri,
+ positiveTargetPos, ioe);
+ }
+ }
+
+ /**
+ * Adjust the stream to a specific position.
+ *
+ * @param targetPos target seek position
+ * @throws IOException on any failure to seek
+ */
+ private void seekInStream(final long targetPos) throws IOException {
+ checkNotClosed();
+ if (wrappedStream == null) {
+ return;
+ }
+ // compute how much more to skip
+ long diff = targetPos - streamCurrentPos;
+ if (diff > 0) {
+ // forward seek -this is where data can be skipped
+
+ int available = wrappedStream.available();
+ // always seek at least as far as what is available
+ long forwardSeekRange = Math.max(readAheadRange, 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", uri, diff);
+ long skippedOnce = wrappedStream.skip(diff);
+ while (diff > 0 && skippedOnce > 0) {
+ streamCurrentPos += skippedOnce;
+ diff -= skippedOnce;
+ incrementBytesRead(skippedOnce);
+ skippedOnce = wrappedStream.skip(diff);
+ }
+
+ if (streamCurrentPos == targetPos) {
+ // all is well
+ return;
+ } else {
+ // log a warning; continue to attempt to re-open
+ LOG.info("Failed to seek on {} to {}. Current position {}",
+ uri, targetPos, streamCurrentPos);
+ }
+ }
+ } else if (diff == 0 && remainingInCurrentRequest() > 0) {
+ // targetPos == streamCurrentPos
+ // 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 streamCurrentPos
+ closeStream("seekInStream()", this.contentRangeFinish);
+ streamCurrentPos = targetPos;
+ }
+
+ @Override
+ public boolean seekToNewSource(final long targetPos) {
+ return false;
+ }
+
+ /**
+ * 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 on any failure to lazy seek
+ */
+ private void lazySeek(final long targetPos, final long len)
+ throws IOException {
+ for (int i = 0; i < SEEK_RETRY_TIME; i++) {
+ try {
+ // For lazy seek
+ seekInStream(targetPos);
+
+ // re-open at specific location if needed
+ if (wrappedStream == null) {
+ reopen("read from new offset", targetPos, len);
+ }
+
+ break;
+ } catch (IOException e) {
+ if (wrappedStream != null) {
+ closeStream("lazySeek() seekInStream has exception ",
+ this.contentRangeFinish);
+ }
+ Throwable cause = e.getCause();
+ if (cause instanceof ObsException) {
+ ObsException obsException = (ObsException) cause;
+ int status = obsException.getResponseCode();
+ switch (status) {
+ case OBSCommonUtils.UNAUTHORIZED_CODE:
+ case OBSCommonUtils.FORBIDDEN_CODE:
+ case OBSCommonUtils.NOT_FOUND_CODE:
+ case OBSCommonUtils.GONE_CODE:
+ case OBSCommonUtils.EOF_CODE:
+ throw e;
+ default:
+ break;
+ }
+ }
+
+ LOG.warn("IOException occurred in lazySeek, retry: {}", i, e);
+ if (i == SEEK_RETRY_TIME - 1) {
+ throw e;
+ }
+ try {
+ Thread.sleep(DELAY_TIME);
+ } catch (InterruptedException ie) {
+ throw e;
+ }
+ }
+ }
+ }
+
+ /**
+ * Increment the bytes read counter if there is a stats instance and the
+ * number of bytes read is more than zero.
+ *
+ * @param bytesRead number of bytes read
+ */
+ private void incrementBytesRead(final long bytesRead) {
+ if (statistics != null && bytesRead > 0) {
+ statistics.incrementBytesRead(bytesRead);
+ }
+ }
+
+ private void sleepInLock() throws InterruptedException {
+ long start = System.currentTimeMillis();
+ long now = start;
+ while (now - start < OBSInputStream.DELAY_TIME) {
+ wait(start + OBSInputStream.DELAY_TIME - now);
+ now = System.currentTimeMillis();
+ }
+ }
+
+ @Override
+ public synchronized int read() throws IOException {
+ long startTime = System.currentTimeMillis();
+ long threadId = Thread.currentThread().getId();
+ checkNotClosed();
+ if (this.contentLength == 0 || nextReadPos >= contentLength) {
+ return -1;
+ }
+
+ int byteRead = -1;
+ try {
+ lazySeek(nextReadPos, 1);
+ } catch (EOFException e) {
+ onReadFailure(e, 1);
+ return -1;
+ }
+
+ IOException exception = null;
+ for (int retryTime = 1; retryTime <= READ_RETRY_TIME; retryTime++) {
+ try {
+ byteRead = wrappedStream.read();
+ exception = null;
+ break;
+ } catch (EOFException e) {
+ onReadFailure(e, 1);
+ return -1;
+ } catch (IOException e) {
+ exception = e;
+ onReadFailure(e, 1);
+ LOG.warn(
+ "read of [{}] failed, retry time[{}], due to exception[{}]",
+ uri, retryTime, exception);
+ if (retryTime < READ_RETRY_TIME) {
+ try {
+ sleepInLock();
+ } catch (InterruptedException ie) {
+ LOG.error(
+ "read of [{}] failed, retry time[{}], due to "
+ + "exception[{}]",
+ uri, retryTime,
+ exception);
+ throw exception;
+ }
+ }
+ }
+ }
+
+ if (exception != null) {
+ LOG.error(
+ "read of [{}] failed, retry time[{}], due to exception[{}]",
+ uri, READ_RETRY_TIME, exception);
+ throw exception;
+ }
+
+ if (byteRead >= 0) {
+ streamCurrentPos++;
+ nextReadPos++;
+ }
+
+ if (byteRead >= 0) {
+ incrementBytesRead(1);
+ }
+
+ long endTime = System.currentTimeMillis();
+ LOG.debug(
+ "read-0arg uri:{}, contentLength:{}, position:{}, readValue:{}, "
+ + "thread:{}, timeUsedMilliSec:{}",
+ uri, contentLength, byteRead >= 0 ? nextReadPos - 1 : nextReadPos,
+ byteRead, threadId,
+ endTime - startTime);
+ return byteRead;
+ }
+
+ /**
+ * 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(final IOException ioe, final int length)
+ throws IOException {
+ LOG.debug(
+ "Got exception while trying to read from stream {}"
+ + " trying to recover: " + ioe, uri);
+ int i = 1;
+ while (true) {
+ try {
+ reopen("failure recovery", streamCurrentPos, length);
+ return;
+ } catch (OBSIOException e) {
+ LOG.warn(
+ "OBSIOException occurred in reopen for failure recovery, "
+ + "the {} retry time",
+ i, e);
+ if (i == READ_RETRY_TIME) {
+ throw e;
+ }
+ try {
+ Thread.sleep(DELAY_TIME);
+ } catch (InterruptedException ie) {
+ throw e;
+ }
+ }
+ i++;
+ }
+ }
+
+ @Override
+ public synchronized int read(final ByteBuffer byteBuffer)
+ throws IOException {
+ long startTime = System.currentTimeMillis();
+ long threadId = Thread.currentThread().getId();
+ LOG.debug("read byteBuffer: {}", byteBuffer.toString());
+ checkNotClosed();
+
+ int len = byteBuffer.remaining();
+ if (len == 0) {
+ return 0;
+ }
+
+ byte[] buf = new byte[len];
+
+ if (this.contentLength == 0 || nextReadPos >= contentLength) {
+ return -1;
+ }
+
+ try {
+ lazySeek(nextReadPos, len);
+ } catch (EOFException e) {
+ onReadFailure(e, len);
+ // the end of the file has moved
+ return -1;
+ }
+
+ int bytesRead = 0;
+ IOException exception = null;
+ for (int retryTime = 1; retryTime <= READ_RETRY_TIME; retryTime++) {
+ try {
+ bytesRead = tryToReadFromInputStream(wrappedStream, buf, 0,
+ len);
+ if (bytesRead == -1) {
+ return -1;
+ }
+ exception = null;
+ break;
+ } catch (EOFException e) {
+ onReadFailure(e, len);
+ return -1;
+ } catch (IOException e) {
+ exception = e;
+ onReadFailure(e, len);
+ LOG.warn(
+ "read len[{}] of [{}] failed, retry time[{}], "
+ + "due to exception[{}]",
+ len, uri, retryTime, exception);
+ if (retryTime < READ_RETRY_TIME) {
+ try {
+ sleepInLock();
+ } catch (InterruptedException ie) {
+ LOG.error(
+ "read len[{}] of [{}] failed, retry time[{}], "
+ + "due to exception[{}]",
+ len, uri, retryTime, exception);
+ throw exception;
+ }
+ }
+ }
+ }
+
+ if (exception != null) {
+ LOG.error(
+ "read len[{}] of [{}] failed, retry time[{}], "
+ + "due to exception[{}]",
+ len, uri, READ_RETRY_TIME, exception);
+ throw exception;
+ }
+
+ if (bytesRead > 0) {
+ streamCurrentPos += bytesRead;
+ nextReadPos += bytesRead;
+ byteBuffer.put(buf, 0, bytesRead);
+ }
+ incrementBytesRead(bytesRead);
+
+ long endTime = System.currentTimeMillis();
+ LOG.debug(
+ "Read-ByteBuffer uri:{}, contentLength:{}, destLen:{}, readLen:{}, "
+ + "position:{}, thread:{}, timeUsedMilliSec:{}",
+ uri, contentLength, len, bytesRead,
+ bytesRead >= 0 ? nextReadPos - bytesRead : nextReadPos, threadId,
+ endTime - startTime);
+ return bytesRead;
+ }
+
+ private int tryToReadFromInputStream(final InputStream in, final byte[] buf,
+ final int off, final int len) throws IOException {
+ int bytesRead = 0;
+ while (bytesRead < len) {
+ int bytes = in.read(buf, off + bytesRead, len - bytesRead);
+ if (bytes == -1) {
+ if (bytesRead == 0) {
+ return -1;
+ } else {
+ break;
+ }
+ }
+ bytesRead += bytes;
+ }
+
+ return bytesRead;
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ *
This updates the statistics on read operations started and whether or
+ * not the read operation "completed", that is: returned the exact number of
+ * bytes requested.
+ *
+ * @throws IOException if there are other problems
+ */
+ @Override
+ public synchronized int read(@NotNull final byte[] buf, final int off,
+ final int len) throws IOException {
+ long startTime = System.currentTimeMillis();
+ long threadId = Thread.currentThread().getId();
+ checkNotClosed();
+ validatePositionedReadArgs(nextReadPos, buf, off, len);
+ if (len == 0) {
+ return 0;
+ }
+
+ if (this.contentLength == 0 || nextReadPos >= contentLength) {
+ return -1;
+ }
+
+ try {
+ lazySeek(nextReadPos, len);
+ } catch (EOFException e) {
+ onReadFailure(e, len);
+ // the end of the file has moved
+ return -1;
+ }
+
+ int bytesRead = 0;
+ IOException exception = null;
+ for (int retryTime = 1; retryTime <= READ_RETRY_TIME; retryTime++) {
+ try {
+ bytesRead = tryToReadFromInputStream(wrappedStream, buf, off,
+ len);
+ if (bytesRead == -1) {
+ return -1;
+ }
+ exception = null;
+ break;
+ } catch (EOFException e) {
+ onReadFailure(e, len);
+ return -1;
+ } catch (IOException e) {
+ exception = e;
+ onReadFailure(e, len);
+ LOG.warn(
+ "read offset[{}] len[{}] of [{}] failed, retry time[{}], "
+ + "due to exception[{}]",
+ off, len, uri, retryTime, exception);
+ if (retryTime < READ_RETRY_TIME) {
+ try {
+ sleepInLock();
+ } catch (InterruptedException ie) {
+ LOG.error(
+ "read offset[{}] len[{}] of [{}] failed, "
+ + "retry time[{}], due to exception[{}]",
+ off, len, uri, retryTime, exception);
+ throw exception;
+ }
+ }
+ }
+ }
+
+ if (exception != null) {
+ LOG.error(
+ "read offset[{}] len[{}] of [{}] failed, retry time[{}], "
+ + "due to exception[{}]",
+ off, len, uri, READ_RETRY_TIME, exception);
+ throw exception;
+ }
+
+ if (bytesRead > 0) {
+ streamCurrentPos += bytesRead;
+ nextReadPos += bytesRead;
+ }
+ incrementBytesRead(bytesRead);
+
+ long endTime = System.currentTimeMillis();
+ LOG.debug(
+ "Read-3args uri:{}, contentLength:{}, destLen:{}, readLen:{}, "
+ + "position:{}, thread:{}, timeUsedMilliSec:{}",
+ uri, contentLength, len, bytesRead,
+ bytesRead >= 0 ? nextReadPos - bytesRead : nextReadPos, threadId,
+ endTime - startTime);
+ return bytesRead;
+ }
+
+ /**
+ * 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(
+ uri + ": " + FSExceptionMessages.STREAM_IS_CLOSED);
+ }
+ }
+
+ /**
+ * Close the stream. This triggers publishing of the stream statistics back to
+ * the filesystem statistics. This operation is synchronized, so that only one
+ * thread can attempt to close the connection; all later/blocked calls are
+ * no-ops.
+ *
+ * @throws IOException on any problem
+ */
+ @Override
+ public synchronized void close() throws IOException {
+ if (!closed) {
+ closed = true;
+ // close or abort the stream
+ closeStream("close() operation", this.contentRangeFinish);
+ // this is actually a no-op
+ super.close();
+ }
+ }
+
+ /**
+ * Close a stream: decide whether to abort or close, based on the length of
+ * the stream and the current position. If a close() is attempted and fails,
+ * the operation escalates to an abort.
+ *
+ *
This does not set the {@link #closed} flag.
+ *
+ * @param reason reason for stream being closed; used in messages
+ * @param length length of the stream
+ * @throws IOException on any failure to close stream
+ */
+ private synchronized void closeStream(final String reason,
+ final long length)
+ throws IOException {
+ if (wrappedStream != null) {
+ try {
+ wrappedStream.close();
+ } catch (IOException e) {
+ // exception escalates to an abort
+ LOG.debug("When closing {} stream for {}", uri, reason, e);
+ throw e;
+ }
+
+ LOG.debug(
+ "Stream {} : {}; streamPos={}, nextReadPos={},"
+ + " request range {}-{} length={}",
+ uri,
+ reason,
+ streamCurrentPos,
+ nextReadPos,
+ contentRangeStart,
+ contentRangeFinish,
+ length);
+ wrappedStream = null;
+ }
+ }
+
+ @Override
+ public synchronized int available() throws IOException {
+ checkNotClosed();
+
+ long remaining = remainingInFile();
+ if (remaining > Integer.MAX_VALUE) {
+ return Integer.MAX_VALUE;
+ }
+ return (int) remaining;
+ }
+
+ /**
+ * Bytes left in stream.
+ *
+ * @return how many bytes are left to read
+ */
+ @InterfaceAudience.Private
+ @InterfaceStability.Unstable
+ public synchronized long remainingInFile() {
+ return this.contentLength - this.streamCurrentPos;
+ }
+
+ /**
+ * 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.
+ */
+ @InterfaceAudience.Private
+ @InterfaceStability.Unstable
+ public synchronized long remainingInCurrentRequest() {
+ return this.contentRangeFinish - this.streamCurrentPos;
+ }
+
+ @Override
+ public boolean markSupported() {
+ return false;
+ }
+
+ /**
+ * String value includes statistics as well as stream state. Important:
+ * there are no guarantees as to the stability of this value.
+ *
+ * @return a string value for printing in logs/diagnostics
+ */
+ @Override
+ @InterfaceStability.Unstable
+ public String toString() {
+ synchronized (this) {
+ return "OBSInputStream{" + uri
+ + " wrappedStream=" + (wrappedStream != null
+ ? "open"
+ : "closed")
+ + " streamCurrentPos=" + streamCurrentPos
+ + " nextReadPos=" + nextReadPos
+ + " contentLength=" + contentLength
+ + " contentRangeStart=" + contentRangeStart
+ + " contentRangeFinish=" + contentRangeFinish
+ + " remainingInCurrentRequest=" + remainingInCurrentRequest()
+ + '}';
+ }
+ }
+
+ /**
+ * Subclass {@code readFully()} operation which only seeks at the start of the
+ * series of operations; seeking back at the end.
+ *
+ *
This is significantly higher performance if multiple read attempts
+ * are needed to fetch the data, as it does not break the HTTP connection.
+ *
+ *
To maintain thread safety requirements, this operation is
+ * synchronized for the duration of the sequence. {@inheritDoc}
+ */
+ @Override
+ public void readFully(final long position, final byte[] buffer,
+ final int offset,
+ final int length)
+ throws IOException {
+ long startTime = System.currentTimeMillis();
+ long threadId = Thread.currentThread().getId();
+ checkNotClosed();
+ validatePositionedReadArgs(position, buffer, offset, length);
+ if (length == 0) {
+ return;
+ }
+ int nread = 0;
+ synchronized (this) {
+ long oldPos = getPos();
+ try {
+ seek(position);
+ while (nread < length) {
+ int nbytes = read(buffer, offset + nread, length - nread);
+ if (nbytes < 0) {
+ throw new EOFException(
+ FSExceptionMessages.EOF_IN_READ_FULLY);
+ }
+ nread += nbytes;
+ }
+ } finally {
+ seekQuietly(oldPos);
+ }
+ }
+
+ long endTime = System.currentTimeMillis();
+ LOG.debug(
+ "ReadFully uri:{}, contentLength:{}, destLen:{}, readLen:{}, "
+ + "position:{}, thread:{}, timeUsedMilliSec:{}",
+ uri, contentLength, length, nread, position, threadId,
+ endTime - startTime);
+ }
+
+ /**
+ * Read bytes starting from the specified position.
+ *
+ * @param position start read from this position
+ * @param buffer read buffer
+ * @param offset offset into buffer
+ * @param length number of bytes to read
+ * @return actual number of bytes read
+ * @throws IOException on any failure to read
+ */
+ @Override
+ public int read(final long position, final byte[] buffer, final int offset,
+ final int length)
+ throws IOException {
+ int len = length;
+ checkNotClosed();
+ validatePositionedReadArgs(position, buffer, offset, len);
+ if (position < 0 || position >= contentLength) {
+ return -1;
+ }
+ if ((position + len) > contentLength) {
+ len = (int) (contentLength - position);
+ }
+
+ if (fs.isReadTransformEnabled()) {
+ return super.read(position, buffer, offset, len);
+ }
+
+ return randomReadWithNewInputStream(position, buffer, offset, len);
+ }
+
+ private int randomReadWithNewInputStream(final long position,
+ final byte[] buffer, final int offset, final int length)
+ throws IOException {
+ long startTime = System.currentTimeMillis();
+ long threadId = Thread.currentThread().getId();
+ int bytesRead = 0;
+ InputStream inputStream = null;
+ IOException exception = null;
+ GetObjectRequest request = new GetObjectRequest(bucket, key);
+ request.setRangeStart(position);
+ request.setRangeEnd(position + length);
+ if (fs.getSse().isSseCEnable()) {
+ request.setSseCHeader(fs.getSse().getSseCHeader());
+ }
+
+ for (int retryTime = 1; retryTime <= READ_RETRY_TIME; retryTime++) {
+ try {
+ inputStream = client.getObject(request).getObjectContent();
+ if (inputStream == null) {
+ break;
+ }
+ bytesRead = tryToReadFromInputStream(inputStream, buffer,
+ offset, length);
+ if (bytesRead == -1) {
+ return -1;
+ }
+
+ exception = null;
+ break;
+ } catch (ObsException | IOException e) {
+ if (e instanceof ObsException) {
+ exception = translateException(
+ "Read at position " + position, uri, (ObsException) e);
+ } else {
+ exception = (IOException) e;
+ }
+ LOG.warn(
+ "read position[{}] destLen[{}] offset[{}] readLen[{}] "
+ + "of [{}] failed, retry time[{}], due to "
+ + "exception[{}] e[{}]",
+ position, length, offset, bytesRead, uri, retryTime,
+ exception, e);
+ if (retryTime < READ_RETRY_TIME) {
+ try {
+ Thread.sleep(DELAY_TIME);
+ } catch (InterruptedException ie) {
+ LOG.error(
+ "read position[{}] destLen[{}] offset[{}] "
+ + "readLen[{}] of [{}] failed, retry time[{}], "
+ + "due to exception[{}] e[{}]",
+ position, length, offset, bytesRead, uri, retryTime,
+ exception, e);
+ throw exception;
+ }
+ }
+ } finally {
+ if (inputStream != null) {
+ inputStream.close();
+ }
+ }
+ }
+
+ if (inputStream == null || exception != null) {
+ LOG.error(
+ "read position[{}] destLen[{}] offset[{}] len[{}] failed, "
+ + "retry time[{}], due to exception[{}]",
+ position, length, offset, bytesRead, READ_RETRY_TIME,
+ exception);
+ throw new IOException("read failed of " + uri + ", inputStream is "
+ + (inputStream == null ? "null" : "not null"), exception);
+
+ }
+
+ long endTime = System.currentTimeMillis();
+ LOG.debug(
+ "Read-4args uri:{}, contentLength:{}, destLen:{}, readLen:{}, "
+ + "position:{}, thread:{}, timeUsedMilliSec:{}",
+ uri, contentLength, length, bytesRead, position, threadId,
+ endTime - startTime);
+ return bytesRead;
+ }
+
+ @Override
+ public synchronized void setReadahead(final Long newReadaheadRange) {
+ if (newReadaheadRange == null) {
+ this.readAheadRange = OBSConstants.DEFAULT_READAHEAD_RANGE;
+ } else {
+ Preconditions.checkArgument(newReadaheadRange >= 0,
+ "Negative readahead value");
+ this.readAheadRange = newReadaheadRange;
+ }
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSListing.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSListing.java
new file mode 100644
index 00000000000..4072feb2cac
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSListing.java
@@ -0,0 +1,656 @@
+/*
+ * 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.obs;
+
+import com.obs.services.exception.ObsException;
+import com.obs.services.model.ListObjectsRequest;
+import com.obs.services.model.ObjectListing;
+import com.obs.services.model.ObsObject;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+
+/**
+ * OBS listing implementation.
+ */
+class OBSListing {
+ /**
+ * A Path filter which accepts all filenames.
+ */
+ static final PathFilter ACCEPT_ALL =
+ new PathFilter() {
+ @Override
+ public boolean accept(final Path file) {
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ return "ACCEPT_ALL";
+ }
+ };
+
+ /**
+ * Class logger.
+ */
+ private static final Logger LOG = LoggerFactory.getLogger(OBSListing.class);
+
+ /**
+ * OBS File System instance.
+ */
+ private final OBSFileSystem owner;
+
+ OBSListing(final OBSFileSystem ownerFS) {
+ this.owner = ownerFS;
+ }
+
+ /**
+ * Create a FileStatus iterator against a path, with a given list object
+ * request.
+ *
+ * @param listPath path of the listing
+ * @param request initial request to make
+ * @param filter the filter on which paths to accept
+ * @param acceptor the class/predicate to decide which entries to accept in
+ * the listing based on the full file status.
+ * @return the iterator
+ * @throws IOException IO Problems
+ */
+ FileStatusListingIterator createFileStatusListingIterator(
+ final Path listPath,
+ final ListObjectsRequest request,
+ final PathFilter filter,
+ final FileStatusAcceptor acceptor)
+ throws IOException {
+ return new FileStatusListingIterator(
+ new ObjectListingIterator(listPath, request), filter, acceptor);
+ }
+
+ /**
+ * Create a located status iterator over a file status iterator.
+ *
+ * @param statusIterator an iterator over the remote status entries
+ * @return a new remote iterator
+ */
+ LocatedFileStatusIterator createLocatedFileStatusIterator(
+ final RemoteIterator statusIterator) {
+ return new LocatedFileStatusIterator(statusIterator);
+ }
+
+ /**
+ * Interface to implement by the logic deciding whether to accept a summary
+ * entry or path as a valid file or directory.
+ */
+ interface FileStatusAcceptor {
+
+ /**
+ * Predicate to decide whether or not to accept a summary entry.
+ *
+ * @param keyPath qualified path to the entry
+ * @param summary summary entry
+ * @return true if the entry is accepted (i.e. that a status entry should be
+ * generated.
+ */
+ boolean accept(Path keyPath, ObsObject summary);
+
+ /**
+ * Predicate to decide whether or not to accept a prefix.
+ *
+ * @param keyPath qualified path to the entry
+ * @param commonPrefix the prefix
+ * @return true if the entry is accepted (i.e. that a status entry should be
+ * generated.)
+ */
+ boolean accept(Path keyPath, String commonPrefix);
+ }
+
+ /**
+ * A remote iterator which only iterates over a single `LocatedFileStatus`
+ * value.
+ *
+ * If the status value is null, the iterator declares that it has no
+ * data. This iterator is used to handle
+ * {@link OBSFileSystem#listStatus(Path)}calls where the path handed in
+ * refers to a file, not a directory: this is
+ * the iterator returned.
+ */
+ static final class SingleStatusRemoteIterator
+ implements RemoteIterator {
+
+ /**
+ * The status to return; set to null after the first iteration.
+ */
+ private LocatedFileStatus status;
+
+ /**
+ * Constructor.
+ *
+ * @param locatedFileStatus status value: may be null, in which case the
+ * iterator is empty.
+ */
+ SingleStatusRemoteIterator(final LocatedFileStatus locatedFileStatus) {
+ this.status = locatedFileStatus;
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * @return true if there is a file status to return: this is always false
+ * for the second iteration, and may be false for the first.
+ */
+ @Override
+ public boolean hasNext() {
+ return status != null;
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * @return the non-null status element passed in when the instance was
+ * constructed, if it ha not already been retrieved.
+ * @throws NoSuchElementException if this is the second call, or it is the
+ * first call and a null
+ * {@link LocatedFileStatus}
+ * entry was passed to the constructor.
+ */
+ @Override
+ public LocatedFileStatus next() {
+ if (hasNext()) {
+ LocatedFileStatus s = this.status;
+ status = null;
+ return s;
+ } else {
+ throw new NoSuchElementException();
+ }
+ }
+ }
+
+ /**
+ * Accept all entries except the base path and those which map to OBS pseudo
+ * directory markers.
+ */
+ static class AcceptFilesOnly implements FileStatusAcceptor {
+ /**
+ * path to qualify.
+ */
+ private final Path qualifiedPath;
+
+ AcceptFilesOnly(final Path path) {
+ this.qualifiedPath = path;
+ }
+
+ /**
+ * Reject a summary entry if the key path is the qualified Path, or it ends
+ * with {@code "_$folder$"}.
+ *
+ * @param keyPath key path of the entry
+ * @param summary summary entry
+ * @return true if the entry is accepted (i.e. that a status entry should be
+ * generated.
+ */
+ @Override
+ public boolean accept(final Path keyPath, final ObsObject summary) {
+ return !keyPath.equals(qualifiedPath)
+ && !summary.getObjectKey()
+ .endsWith(OBSConstants.OBS_FOLDER_SUFFIX)
+ && !OBSCommonUtils.objectRepresentsDirectory(
+ summary.getObjectKey(),
+ summary.getMetadata().getContentLength());
+ }
+
+ /**
+ * Accept no directory paths.
+ *
+ * @param keyPath qualified path to the entry
+ * @param prefix common prefix in listing.
+ * @return false, always.
+ */
+ @Override
+ public boolean accept(final Path keyPath, final String prefix) {
+ return false;
+ }
+ }
+
+ /**
+ * Accept all entries except the base path and those which map to OBS pseudo
+ * directory markers.
+ */
+ static class AcceptAllButSelfAndS3nDirs implements FileStatusAcceptor {
+
+ /**
+ * Base path.
+ */
+ private final Path qualifiedPath;
+
+ /**
+ * Constructor.
+ *
+ * @param path an already-qualified path.
+ */
+ AcceptAllButSelfAndS3nDirs(final Path path) {
+ this.qualifiedPath = path;
+ }
+
+ /**
+ * Reject a summary entry if the key path is the qualified Path, or it ends
+ * with {@code "_$folder$"}.
+ *
+ * @param keyPath key path of the entry
+ * @param summary summary entry
+ * @return true if the entry is accepted (i.e. that a status entry should be
+ * generated.)
+ */
+ @Override
+ public boolean accept(final Path keyPath, final ObsObject summary) {
+ return !keyPath.equals(qualifiedPath) && !summary.getObjectKey()
+ .endsWith(OBSConstants.OBS_FOLDER_SUFFIX);
+ }
+
+ /**
+ * Accept all prefixes except the one for the base path, "self".
+ *
+ * @param keyPath qualified path to the entry
+ * @param prefix common prefix in listing.
+ * @return true if the entry is accepted (i.e. that a status entry should be
+ * generated.
+ */
+ @Override
+ public boolean accept(final Path keyPath, final String prefix) {
+ return !keyPath.equals(qualifiedPath);
+ }
+ }
+
+ /**
+ * Wraps up object listing into a remote iterator which will ask for more
+ * listing data if needed.
+ *
+ * This is a complex operation, especially the process to determine if
+ * there are more entries remaining. If there are no more results remaining in
+ * the (filtered) results of the current listing request, then another request
+ * is made
+ * and those results filtered before the iterator can declare that
+ * there is more data available.
+ *
+ *
The need to filter the results precludes the iterator from simply
+ * declaring that if the {@link ObjectListingIterator#hasNext()} is true then
+ * there are more results. Instead the next batch of results must be retrieved
+ * and filtered.
+ *
+ *
What does this mean? It means that remote requests to retrieve new
+ * batches of object listings are made in the {@link #hasNext()} call; the
+ * {@link #next()} call simply returns the filtered results of the last
+ * listing processed. However, do note that {@link #next()} calls {@link
+ * #hasNext()} during its operation. This is critical to ensure that a listing
+ * obtained through a sequence of {@link #next()} will complete with the same
+ * set of results as a classic {@code while(it.hasNext()} loop.
+ *
+ *
Thread safety: None.
+ */
+ class FileStatusListingIterator implements RemoteIterator {
+
+ /**
+ * Source of objects.
+ */
+ private final ObjectListingIterator source;
+
+ /**
+ * Filter of paths from API call.
+ */
+ private final PathFilter filter;
+
+ /**
+ * Filter of entries from file status.
+ */
+ private final FileStatusAcceptor acceptor;
+
+ /**
+ * Request batch size.
+ */
+ private int batchSize;
+
+ /**
+ * Iterator over the current set of results.
+ */
+ private ListIterator statusBatchIterator;
+
+ /**
+ * Create an iterator over file status entries.
+ *
+ * @param listPath the listing iterator from a listObjects call.
+ * @param pathFilter the filter on which paths to accept
+ * @param fileStatusAcceptor the class/predicate to decide which entries to
+ * accept in the listing based on the full file
+ * status.
+ * @throws IOException IO Problems
+ */
+ FileStatusListingIterator(
+ final ObjectListingIterator listPath, final PathFilter pathFilter,
+ final FileStatusAcceptor fileStatusAcceptor)
+ throws IOException {
+ this.source = listPath;
+ this.filter = pathFilter;
+ this.acceptor = fileStatusAcceptor;
+ // build the first set of results. This will not trigger any
+ // remote IO, assuming the source iterator is in its initial
+ // iteration
+ requestNextBatch();
+ }
+
+ /**
+ * Report whether or not there is new data available. If there is data in
+ * the local filtered list, return true. Else: request more data util that
+ * condition is met, or there is no more remote listing data.
+ *
+ * @return true if a call to {@link #next()} will succeed.
+ * @throws IOException on any failure to request next batch
+ */
+ @Override
+ public boolean hasNext() throws IOException {
+ return statusBatchIterator.hasNext() || requestNextBatch();
+ }
+
+ @Override
+ public FileStatus next() throws IOException {
+ if (!hasNext()) {
+ throw new NoSuchElementException();
+ }
+ return statusBatchIterator.next();
+ }
+
+ /**
+ * Try to retrieve another batch. Note that for the initial batch, {@link
+ * ObjectListingIterator} does not generate a request; it simply returns the
+ * initial set.
+ *
+ * @return true if a new batch was created.
+ * @throws IOException IO problems
+ */
+ private boolean requestNextBatch() throws IOException {
+ // look for more object listing batches being available
+ while (source.hasNext()) {
+ // if available, retrieve it and build the next status
+ if (buildNextStatusBatch(source.next())) {
+ // this batch successfully generated entries matching
+ // the filters/acceptors;
+ // declare that the request was successful
+ return true;
+ } else {
+ LOG.debug(
+ "All entries in batch were filtered...continuing");
+ }
+ }
+ // if this code is reached, it means that all remaining
+ // object lists have been retrieved, and there are no new entries
+ // to return.
+ return false;
+ }
+
+ /**
+ * Build the next status batch from a listing.
+ *
+ * @param objects the next object listing
+ * @return true if this added any entries after filtering
+ */
+ private boolean buildNextStatusBatch(final ObjectListing objects) {
+ // counters for debug logs
+ int added = 0;
+ int ignored = 0;
+ // list to fill in with results. Initial size will be list maximum.
+ List stats =
+ new ArrayList<>(
+ objects.getObjects().size() + objects.getCommonPrefixes()
+ .size());
+ // objects
+ for (ObsObject summary : objects.getObjects()) {
+ String key = summary.getObjectKey();
+ Path keyPath = OBSCommonUtils.keyToQualifiedPath(owner, key);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("{}: {}", keyPath,
+ OBSCommonUtils.stringify(summary));
+ }
+ // Skip over keys that are ourselves and old OBS _$folder$ files
+ if (acceptor.accept(keyPath, summary) && filter.accept(
+ keyPath)) {
+ FileStatus status =
+ OBSCommonUtils.createFileStatus(
+ keyPath, summary,
+ owner.getDefaultBlockSize(keyPath),
+ owner.getUsername());
+ LOG.debug("Adding: {}", status);
+ stats.add(status);
+ added++;
+ } else {
+ LOG.debug("Ignoring: {}", keyPath);
+ ignored++;
+ }
+ }
+
+ // prefixes: always directories
+ for (ObsObject prefix : objects.getExtenedCommonPrefixes()) {
+ String key = prefix.getObjectKey();
+ Path keyPath = OBSCommonUtils.keyToQualifiedPath(owner, key);
+ if (acceptor.accept(keyPath, key) && filter.accept(keyPath)) {
+ long lastModified =
+ prefix.getMetadata().getLastModified() == null
+ ? System.currentTimeMillis()
+ : OBSCommonUtils.dateToLong(
+ prefix.getMetadata().getLastModified());
+ FileStatus status = new OBSFileStatus(keyPath, lastModified,
+ lastModified, owner.getUsername());
+ LOG.debug("Adding directory: {}", status);
+ added++;
+ stats.add(status);
+ } else {
+ LOG.debug("Ignoring directory: {}", keyPath);
+ ignored++;
+ }
+ }
+
+ // finish up
+ batchSize = stats.size();
+ statusBatchIterator = stats.listIterator();
+ boolean hasNext = statusBatchIterator.hasNext();
+ LOG.debug(
+ "Added {} entries; ignored {}; hasNext={}; hasMoreObjects={}",
+ added,
+ ignored,
+ hasNext,
+ objects.isTruncated());
+ return hasNext;
+ }
+
+ /**
+ * Get the number of entries in the current batch.
+ *
+ * @return a number, possibly zero.
+ */
+ public int getBatchSize() {
+ return batchSize;
+ }
+ }
+
+ /**
+ * Wraps up OBS `ListObjects` requests in a remote iterator which will ask for
+ * more listing data if needed.
+ *
+ * That is:
+ *
+ *
1. The first invocation of the {@link #next()} call will return the
+ * results of the first request, the one created during the construction of
+ * the instance.
+ *
+ *
2. Second and later invocations will continue the ongoing listing,
+ * calling {@link OBSCommonUtils#continueListObjects} to request the next
+ * batch of results.
+ *
+ *
3. The {@link #hasNext()} predicate returns true for the initial call,
+ * where {@link #next()} will return the initial results. It declares that it
+ * has future results iff the last executed request was truncated.
+ *
+ *
Thread safety: none.
+ */
+ class ObjectListingIterator implements RemoteIterator {
+
+ /**
+ * The path listed.
+ */
+ private final Path listPath;
+
+ /**
+ * The most recent listing results.
+ */
+ private ObjectListing objects;
+
+ /**
+ * Indicator that this is the first listing.
+ */
+ private boolean firstListing = true;
+
+ /**
+ * Count of how many listings have been requested (including initial
+ * result).
+ */
+ private int listingCount = 1;
+
+ /**
+ * Maximum keys in a request.
+ */
+ private int maxKeys;
+
+ /**
+ * Constructor -calls {@link OBSCommonUtils#listObjects} on the request to
+ * populate the initial set of results/fail if there was a problem talking
+ * to the bucket.
+ *
+ * @param path path of the listing
+ * @param request initial request to make
+ * @throws IOException on any failure to list objects
+ */
+ ObjectListingIterator(final Path path,
+ final ListObjectsRequest request)
+ throws IOException {
+ this.listPath = path;
+ this.maxKeys = owner.getMaxKeys();
+ this.objects = OBSCommonUtils.listObjects(owner, request);
+ }
+
+ /**
+ * Declare that the iterator has data if it is either is the initial
+ * iteration or it is a later one and the last listing obtained was
+ * incomplete.
+ */
+ @Override
+ public boolean hasNext() {
+ return firstListing || objects.isTruncated();
+ }
+
+ /**
+ * Ask for the next listing. For the first invocation, this returns the
+ * initial set, with no remote IO. For later requests, OBS will be queried,
+ * hence the calls may block or fail.
+ *
+ * @return the next object listing.
+ * @throws IOException if a query made of OBS fails.
+ * @throws NoSuchElementException if there is no more data to list.
+ */
+ @Override
+ public ObjectListing next() throws IOException {
+ if (firstListing) {
+ // on the first listing, don't request more data.
+ // Instead just clear the firstListing flag so that it future
+ // calls will request new data.
+ firstListing = false;
+ } else {
+ try {
+ if (!objects.isTruncated()) {
+ // nothing more to request: fail.
+ throw new NoSuchElementException(
+ "No more results in listing of " + listPath);
+ }
+ // need to request a new set of objects.
+ LOG.debug("[{}], Requesting next {} objects under {}",
+ listingCount, maxKeys, listPath);
+ objects = OBSCommonUtils.continueListObjects(owner,
+ objects);
+ listingCount++;
+ LOG.debug("New listing status: {}", this);
+ } catch (ObsException e) {
+ throw OBSCommonUtils.translateException("listObjects()",
+ listPath, e);
+ }
+ }
+ return objects;
+ }
+
+ @Override
+ public String toString() {
+ return "Object listing iterator against "
+ + listPath
+ + "; listing count "
+ + listingCount
+ + "; isTruncated="
+ + objects.isTruncated();
+ }
+
+ }
+
+ /**
+ * Take a remote iterator over a set of {@link FileStatus} instances and
+ * return a remote iterator of {@link LocatedFileStatus} instances.
+ */
+ class LocatedFileStatusIterator
+ implements RemoteIterator {
+ /**
+ * File status.
+ */
+ private final RemoteIterator statusIterator;
+
+ /**
+ * Constructor.
+ *
+ * @param statusRemoteIterator an iterator over the remote status entries
+ */
+ LocatedFileStatusIterator(
+ final RemoteIterator statusRemoteIterator) {
+ this.statusIterator = statusRemoteIterator;
+ }
+
+ @Override
+ public boolean hasNext() throws IOException {
+ return statusIterator.hasNext();
+ }
+
+ @Override
+ public LocatedFileStatus next() throws IOException {
+ return OBSCommonUtils.toLocatedFileStatus(owner,
+ statusIterator.next());
+ }
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLoginHelper.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLoginHelper.java
new file mode 100644
index 00000000000..cd9853369af
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLoginHelper.java
@@ -0,0 +1,350 @@
+/*
+ * 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.obs;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URLDecoder;
+import java.util.Objects;
+
+import static org.apache.commons.lang3.StringUtils.equalsIgnoreCase;
+
+/**
+ * Helper for OBS login.
+ */
+final class OBSLoginHelper {
+ /**
+ * login warning.
+ */
+ public static final String LOGIN_WARNING =
+ "The Filesystem URI contains login details."
+ + " This is insecure and may be unsupported in future.";
+
+ /**
+ * plus warning.
+ */
+ public static final String PLUS_WARNING =
+ "Secret key contains a special character that should be URL encoded! "
+ + "Attempting to resolve...";
+
+ /**
+ * defined plus unencoded char.
+ */
+ public static final String PLUS_UNENCODED = "+";
+
+ /**
+ * defined plus encoded char.
+ */
+ public static final String PLUS_ENCODED = "%2B";
+
+ /**
+ * Class logger.
+ */
+ private static final Logger LOG = LoggerFactory.getLogger(
+ OBSLoginHelper.class);
+
+ private OBSLoginHelper() {
+ }
+
+ /**
+ * Build the filesystem URI. This can include stripping down of part of the
+ * URI.
+ *
+ * @param uri filesystem uri
+ * @return the URI to use as the basis for FS operation and qualifying paths.
+ * @throws IllegalArgumentException if the URI is in some way invalid.
+ */
+ public static URI buildFSURI(final URI uri) {
+ Objects.requireNonNull(uri, "null uri");
+ Objects.requireNonNull(uri.getScheme(), "null uri.getScheme()");
+ if (uri.getHost() == null && uri.getAuthority() != null) {
+ Objects.requireNonNull(
+ uri.getHost(),
+ "null uri host."
+ + " This can be caused by unencoded / in the "
+ + "password string");
+ }
+ Objects.requireNonNull(uri.getHost(), "null uri host.");
+ return URI.create(uri.getScheme() + "://" + uri.getHost());
+ }
+
+ /**
+ * Create a stripped down string value for error messages.
+ *
+ * @param pathUri URI
+ * @return a shortened schema://host/path value
+ */
+ public static String toString(final URI pathUri) {
+ return pathUri != null
+ ? String.format("%s://%s/%s", pathUri.getScheme(),
+ pathUri.getHost(), pathUri.getPath())
+ : "(null URI)";
+ }
+
+ /**
+ * Extract the login details from a URI, logging a warning if the URI contains
+ * these.
+ *
+ * @param name URI of the filesystem
+ * @return a login tuple, possibly empty.
+ */
+ public static Login extractLoginDetailsWithWarnings(final URI name) {
+ Login login = extractLoginDetails(name);
+ if (login.hasLogin()) {
+ LOG.warn(LOGIN_WARNING);
+ }
+ return login;
+ }
+
+ /**
+ * Extract the login details from a URI.
+ *
+ * @param name URI of the filesystem
+ * @return a login tuple, possibly empty.
+ */
+ public static Login extractLoginDetails(final URI name) {
+ try {
+ String authority = name.getAuthority();
+ if (authority == null) {
+ return Login.EMPTY;
+ }
+ int loginIndex = authority.indexOf('@');
+ if (loginIndex < 0) {
+ // no login
+ return Login.EMPTY;
+ }
+ String login = authority.substring(0, loginIndex);
+ int loginSplit = login.indexOf(':');
+ if (loginSplit > 0) {
+ String user = login.substring(0, loginSplit);
+ String encodedPassword = login.substring(loginSplit + 1);
+ if (encodedPassword.contains(PLUS_UNENCODED)) {
+ LOG.warn(PLUS_WARNING);
+ encodedPassword = encodedPassword.replaceAll(
+ "\\" + PLUS_UNENCODED, PLUS_ENCODED);
+ }
+ String password = URLDecoder.decode(encodedPassword, "UTF-8");
+ return new Login(user, password);
+ } else if (loginSplit == 0) {
+ // there is no user, just a password. In this case,
+ // there's no login
+ return Login.EMPTY;
+ } else {
+ return new Login(login, "");
+ }
+ } catch (UnsupportedEncodingException e) {
+ // this should never happen; translate it if it does.
+ throw new RuntimeException(e);
+ }
+ }
+
+ /**
+ * Canonicalize the given URI.
+ *
+ * This strips out login information.
+ *
+ * @param uri the URI to canonicalize
+ * @param defaultPort default port to use in canonicalized URI if the input
+ * URI has no port and this value is greater than 0
+ * @return a new, canonicalized URI.
+ */
+ public static URI canonicalizeUri(final URI uri, final int defaultPort) {
+ URI newUri = uri;
+ if (uri.getPort() == -1 && defaultPort > 0) {
+ // reconstruct the uri with the default port set
+ try {
+ newUri =
+ new URI(
+ newUri.getScheme(),
+ null,
+ newUri.getHost(),
+ defaultPort,
+ newUri.getPath(),
+ newUri.getQuery(),
+ newUri.getFragment());
+ } catch (URISyntaxException e) {
+ // Should never happen!
+ throw new AssertionError(
+ "Valid URI became unparseable: " + newUri);
+ }
+ }
+
+ return newUri;
+ }
+
+ /**
+ * Check the path, ignoring authentication details. See {@link
+ * OBSFileSystem#checkPath(Path)} for the operation of this.
+ *
+ *
Essentially
+ *
+ *
+ * - The URI is canonicalized.
+ *
- If the schemas match, the hosts are compared.
+ *
- If there is a mismatch between null/non-null host,
+ * the default FS values are used to patch in the host.
+ *
+ *
+ * That all originates in the core FS; the sole change here being to use
+ * {@link URI#getHost()}over {@link URI#getAuthority()}. Some of that code
+ * looks a relic of the code anti-pattern of using "hdfs:file.txt" to define
+ * the path without declaring the hostname. It's retained for compatibility.
+ *
+ * @param conf FS configuration
+ * @param fsUri the FS URI
+ * @param path path to check
+ * @param defaultPort default port of FS
+ */
+ public static void checkPath(final Configuration conf, final URI fsUri,
+ final Path path, final int defaultPort) {
+ URI pathUri = path.toUri();
+ String thatScheme = pathUri.getScheme();
+ if (thatScheme == null) {
+ // fs is relative
+ return;
+ }
+ URI thisUri = canonicalizeUri(fsUri, defaultPort);
+ String thisScheme = thisUri.getScheme();
+ // hostname and scheme are not case sensitive in these checks
+ if (equalsIgnoreCase(thisScheme, thatScheme)) { // schemes match
+ String thisHost = thisUri.getHost();
+ String thatHost = pathUri.getHost();
+ if (thatHost == null
+ && // path's host is null
+ thisHost != null) { // fs has a host
+ URI defaultUri = FileSystem.getDefaultUri(conf);
+ if (equalsIgnoreCase(thisScheme, defaultUri.getScheme())) {
+ pathUri
+ = defaultUri; // schemes match, so use this uri instead
+ } else {
+ pathUri = null; // can't determine auth of the path
+ }
+ }
+ if (pathUri != null) {
+ // canonicalize uri before comparing with this fs
+ pathUri = canonicalizeUri(pathUri, defaultPort);
+ thatHost = pathUri.getHost();
+ if (equalsIgnoreCase(thisHost, thatHost)) {
+ return;
+ }
+ }
+ }
+ // make sure the exception strips out any auth details
+ throw new IllegalArgumentException(
+ "Wrong FS " + OBSLoginHelper.toString(pathUri) + " -expected "
+ + fsUri);
+ }
+
+ /**
+ * Simple tuple of login details.
+ */
+ public static class Login {
+ /**
+ * Defined empty login instance.
+ */
+ public static final Login EMPTY = new Login();
+
+ /**
+ * Defined user name.
+ */
+ private final String user;
+
+ /**
+ * Defined password.
+ */
+ private final String password;
+
+ /**
+ * Login token.
+ */
+ private final String token;
+
+ /**
+ * Create an instance with no login details. Calls to {@link #hasLogin()}
+ * return false.
+ */
+ Login() {
+ this("", "");
+ }
+
+ Login(final String userName, final String passwd) {
+ this(userName, passwd, null);
+ }
+
+ Login(final String userName, final String passwd,
+ final String sessionToken) {
+ this.user = userName;
+ this.password = passwd;
+ this.token = sessionToken;
+ }
+
+ /**
+ * Predicate to verify login details are defined.
+ *
+ * @return true if the username is defined (not null, not empty).
+ */
+ public boolean hasLogin() {
+ return StringUtils.isNotEmpty(user);
+ }
+
+ /**
+ * Equality test matches user and password.
+ *
+ * @param o other object
+ * @return true if the objects are considered equivalent.
+ */
+ @Override
+ public boolean equals(final Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ Login that = (Login) o;
+ return Objects.equals(user, that.user) && Objects.equals(password,
+ that.password);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(user, password);
+ }
+
+ public String getUser() {
+ return user;
+ }
+
+ public String getPassword() {
+ return password;
+ }
+
+ public String getToken() {
+ return token;
+ }
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSObjectBucketUtils.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSObjectBucketUtils.java
new file mode 100644
index 00000000000..e632f61ca2d
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSObjectBucketUtils.java
@@ -0,0 +1,892 @@
+/*
+ * 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.obs;
+
+import com.obs.services.exception.ObsException;
+import com.obs.services.model.AbortMultipartUploadRequest;
+import com.obs.services.model.CompleteMultipartUploadRequest;
+import com.obs.services.model.CopyObjectRequest;
+import com.obs.services.model.CopyObjectResult;
+import com.obs.services.model.CopyPartRequest;
+import com.obs.services.model.CopyPartResult;
+import com.obs.services.model.DeleteObjectsRequest;
+import com.obs.services.model.GetObjectMetadataRequest;
+import com.obs.services.model.InitiateMultipartUploadRequest;
+import com.obs.services.model.InitiateMultipartUploadResult;
+import com.obs.services.model.KeyAndVersion;
+import com.obs.services.model.ListObjectsRequest;
+import com.obs.services.model.ObjectListing;
+import com.obs.services.model.ObjectMetadata;
+import com.obs.services.model.ObsObject;
+import com.obs.services.model.PartEtag;
+import com.obs.services.model.PutObjectRequest;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+/**
+ * Object bucket specific utils for {@link OBSFileSystem}.
+ */
+final class OBSObjectBucketUtils {
+ /**
+ * Class logger.
+ */
+ private static final Logger LOG = LoggerFactory.getLogger(
+ OBSObjectBucketUtils.class);
+
+ private OBSObjectBucketUtils() {
+
+ }
+
+ /**
+ * The inner rename operation.
+ *
+ * @param owner OBS File System instance
+ * @param src path to be renamed
+ * @param dst new path after rename
+ * @return boolean
+ * @throws RenameFailedException if some criteria for a state changing rename
+ * was not met. This means work didn't happen;
+ * it's not something which is reported upstream
+ * to the FileSystem APIs, for which the
+ * semantics of "false" are pretty vague.
+ * @throws FileNotFoundException there's no source file.
+ * @throws IOException on IO failure.
+ * @throws ObsException on failures inside the OBS SDK
+ */
+ static boolean renameBasedOnObject(final OBSFileSystem owner,
+ final Path src, final Path dst) throws RenameFailedException,
+ FileNotFoundException, IOException,
+ ObsException {
+ String srcKey = OBSCommonUtils.pathToKey(owner, src);
+ String dstKey = OBSCommonUtils.pathToKey(owner, dst);
+
+ if (srcKey.isEmpty()) {
+ LOG.error("rename: src [{}] is root directory", src);
+ throw new IOException(src + " is root directory");
+ }
+
+ // get the source file status; this raises a FNFE if there is no source
+ // file.
+ FileStatus srcStatus = owner.getFileStatus(src);
+
+ FileStatus dstStatus;
+ try {
+ dstStatus = owner.getFileStatus(dst);
+ // if there is no destination entry, an exception is raised.
+ // hence this code sequence can assume that there is something
+ // at the end of the path; the only detail being what it is and
+ // whether or not it can be the destination of the rename.
+ if (dstStatus.isDirectory()) {
+ String newDstKey = OBSCommonUtils.maybeAddTrailingSlash(dstKey);
+ String filename = srcKey.substring(
+ OBSCommonUtils.pathToKey(owner, src.getParent()).length()
+ + 1);
+ newDstKey = newDstKey + filename;
+ dstKey = newDstKey;
+ dstStatus = owner.getFileStatus(
+ OBSCommonUtils.keyToPath(dstKey));
+ if (dstStatus.isDirectory()) {
+ throw new RenameFailedException(src, dst,
+ "new destination is an existed directory")
+ .withExitCode(false);
+ } else {
+ throw new RenameFailedException(src, dst,
+ "new destination is an existed file")
+ .withExitCode(false);
+ }
+ } else {
+
+ if (srcKey.equals(dstKey)) {
+ LOG.warn(
+ "rename: src and dest refer to the same file or"
+ + " directory: {}",
+ dst);
+ return true;
+ } else {
+ throw new RenameFailedException(src, dst,
+ "destination is an existed file")
+ .withExitCode(false);
+ }
+ }
+ } catch (FileNotFoundException e) {
+ LOG.debug("rename: destination path {} not found", dst);
+
+ // Parent must exist
+ checkDestinationParent(owner, src, dst);
+ }
+
+ if (dstKey.startsWith(srcKey)
+ && dstKey.charAt(srcKey.length()) == Path.SEPARATOR_CHAR) {
+ LOG.error("rename: dest [{}] cannot be a descendant of src [{}]",
+ dst, src);
+ return false;
+ }
+
+ // Ok! Time to start
+ if (srcStatus.isFile()) {
+ LOG.debug("rename: renaming file {} to {}", src, dst);
+
+ renameFile(owner, srcKey, dstKey, srcStatus);
+ } else {
+ LOG.debug("rename: renaming directory {} to {}", src, dst);
+
+ // This is a directory to directory copy
+ dstKey = OBSCommonUtils.maybeAddTrailingSlash(dstKey);
+ srcKey = OBSCommonUtils.maybeAddTrailingSlash(srcKey);
+
+ renameFolder(owner, srcKey, dstKey);
+ }
+
+ if (src.getParent() != dst.getParent()) {
+ // deleteUnnecessaryFakeDirectories(dst.getParent());
+ createFakeDirectoryIfNecessary(owner, src.getParent());
+ }
+
+ return true;
+ }
+
+ private static void checkDestinationParent(final OBSFileSystem owner,
+ final Path src,
+ final Path dst) throws IOException {
+ Path parent = dst.getParent();
+ if (!OBSCommonUtils.pathToKey(owner, parent).isEmpty()) {
+ try {
+ FileStatus dstParentStatus = owner.getFileStatus(
+ dst.getParent());
+ if (!dstParentStatus.isDirectory()) {
+ throw new ParentNotDirectoryException(
+ "destination parent [" + dst.getParent()
+ + "] is not a directory");
+ }
+ } catch (FileNotFoundException e2) {
+ throw new RenameFailedException(src, dst,
+ "destination has no parent ");
+ }
+ }
+ }
+
+ /**
+ * Implement rename file.
+ *
+ * @param owner OBS File System instance
+ * @param srcKey source object key
+ * @param dstKey destination object key
+ * @param srcStatus source object status
+ * @throws IOException any problem with rename operation
+ */
+ private static void renameFile(final OBSFileSystem owner,
+ final String srcKey,
+ final String dstKey,
+ final FileStatus srcStatus)
+ throws IOException {
+ long startTime = System.nanoTime();
+
+ copyFile(owner, srcKey, dstKey, srcStatus.getLen());
+ objectDelete(owner, srcStatus, false);
+
+ if (LOG.isDebugEnabled()) {
+ long delay = System.nanoTime() - startTime;
+ LOG.debug("OBSFileSystem rename: "
+ + ", {src="
+ + srcKey
+ + ", dst="
+ + dstKey
+ + ", delay="
+ + delay
+ + "}");
+ }
+ }
+
+ static boolean objectDelete(final OBSFileSystem owner,
+ final FileStatus status,
+ final boolean recursive) throws IOException {
+ Path f = status.getPath();
+ String key = OBSCommonUtils.pathToKey(owner, f);
+
+ if (status.isDirectory()) {
+ LOG.debug("delete: Path is a directory: {} - recursive {}", f,
+ recursive);
+
+ key = OBSCommonUtils.maybeAddTrailingSlash(key);
+ if (!key.endsWith("/")) {
+ key = key + "/";
+ }
+
+ boolean isEmptyDir = OBSCommonUtils.isFolderEmpty(owner, key);
+ if (key.equals("/")) {
+ return OBSCommonUtils.rejectRootDirectoryDelete(
+ owner.getBucket(), isEmptyDir, recursive);
+ }
+
+ if (!recursive && !isEmptyDir) {
+ throw new PathIsNotEmptyDirectoryException(f.toString());
+ }
+
+ if (isEmptyDir) {
+ LOG.debug(
+ "delete: Deleting fake empty directory {} - recursive {}",
+ f, recursive);
+ OBSCommonUtils.deleteObject(owner, key);
+ } else {
+ LOG.debug(
+ "delete: Deleting objects for directory prefix {} "
+ + "- recursive {}",
+ f, recursive);
+ deleteNonEmptyDir(owner, recursive, key);
+ }
+
+ } else {
+ LOG.debug("delete: Path is a file");
+ OBSCommonUtils.deleteObject(owner, key);
+ }
+
+ Path parent = f.getParent();
+ if (parent != null) {
+ createFakeDirectoryIfNecessary(owner, parent);
+ }
+ return true;
+ }
+
+ /**
+ * Implement rename folder.
+ *
+ * @param owner OBS File System instance
+ * @param srcKey source folder key
+ * @param dstKey destination folder key
+ * @throws IOException any problem with rename folder
+ */
+ static void renameFolder(final OBSFileSystem owner, final String srcKey,
+ final String dstKey)
+ throws IOException {
+ long startTime = System.nanoTime();
+
+ List keysToDelete = new ArrayList<>();
+
+ createFakeDirectory(owner, dstKey);
+
+ ListObjectsRequest request = new ListObjectsRequest();
+ request.setBucketName(owner.getBucket());
+ request.setPrefix(srcKey);
+ request.setMaxKeys(owner.getMaxKeys());
+
+ ObjectListing objects = OBSCommonUtils.listObjects(owner, request);
+
+ List> copyfutures = new LinkedList<>();
+ while (true) {
+ for (ObsObject summary : objects.getObjects()) {
+ if (summary.getObjectKey().equals(srcKey)) {
+ // skip prefix itself
+ continue;
+ }
+
+ keysToDelete.add(new KeyAndVersion(summary.getObjectKey()));
+ String newDstKey = dstKey + summary.getObjectKey()
+ .substring(srcKey.length());
+ // copyFile(summary.getObjectKey(), newDstKey,
+ // summary.getMetadata().getContentLength());
+ copyfutures.add(
+ copyFileAsync(owner, summary.getObjectKey(), newDstKey,
+ summary.getMetadata().getContentLength()));
+
+ if (keysToDelete.size() == owner.getMaxEntriesToDelete()) {
+ waitAllCopyFinished(copyfutures);
+ copyfutures.clear();
+ }
+ }
+
+ if (!objects.isTruncated()) {
+ if (!keysToDelete.isEmpty()) {
+ waitAllCopyFinished(copyfutures);
+ copyfutures.clear();
+ }
+ break;
+ }
+ objects = OBSCommonUtils.continueListObjects(owner, objects);
+ }
+
+ keysToDelete.add(new KeyAndVersion(srcKey));
+
+ DeleteObjectsRequest deleteObjectsRequest = new DeleteObjectsRequest(
+ owner.getBucket());
+ deleteObjectsRequest.setKeyAndVersions(
+ keysToDelete.toArray(new KeyAndVersion[0]));
+ OBSCommonUtils.deleteObjects(owner, deleteObjectsRequest);
+
+ if (LOG.isDebugEnabled()) {
+ long delay = System.nanoTime() - startTime;
+ LOG.debug(
+ "OBSFileSystem rename: "
+ + ", {src="
+ + srcKey
+ + ", dst="
+ + dstKey
+ + ", delay="
+ + delay
+ + "}");
+ }
+ }
+
+ private static void waitAllCopyFinished(
+ final List> copyFutures)
+ throws IOException {
+ try {
+ for (Future copyFuture : copyFutures) {
+ copyFuture.get();
+ }
+ } catch (InterruptedException e) {
+ LOG.warn("Interrupted while copying objects (copy)");
+ throw new InterruptedIOException(
+ "Interrupted while copying objects (copy)");
+ } catch (ExecutionException e) {
+ for (Future future : copyFutures) {
+ future.cancel(true);
+ }
+
+ throw OBSCommonUtils.extractException(
+ "waitAllCopyFinished", copyFutures.toString(), e);
+ }
+ }
+
+ /**
+ * Request object metadata; increments counters in the process.
+ *
+ * @param owner OBS File System instance
+ * @param key key
+ * @return the metadata
+ */
+ protected static ObjectMetadata getObjectMetadata(final OBSFileSystem owner,
+ final String key) {
+ GetObjectMetadataRequest request = new GetObjectMetadataRequest();
+ request.setBucketName(owner.getBucket());
+ request.setObjectKey(key);
+ if (owner.getSse().isSseCEnable()) {
+ request.setSseCHeader(owner.getSse().getSseCHeader());
+ }
+ ObjectMetadata meta = owner.getObsClient().getObjectMetadata(request);
+ owner.getSchemeStatistics().incrementReadOps(1);
+ return meta;
+ }
+
+ /**
+ * Create a new object metadata instance. Any standard metadata headers are
+ * added here, for example: encryption.
+ *
+ * @param length length of data to set in header.
+ * @return a new metadata instance
+ */
+ static ObjectMetadata newObjectMetadata(final long length) {
+ final ObjectMetadata om = new ObjectMetadata();
+ if (length >= 0) {
+ om.setContentLength(length);
+ }
+ return om;
+ }
+
+ private static void deleteNonEmptyDir(final OBSFileSystem owner,
+ final boolean recursive, final String key) throws IOException {
+ String delimiter = recursive ? null : "/";
+ ListObjectsRequest request = OBSCommonUtils.createListObjectsRequest(
+ owner, key, delimiter);
+
+ ObjectListing objects = OBSCommonUtils.listObjects(owner, request);
+ List keys = new ArrayList<>(objects.getObjects().size());
+ while (true) {
+ for (ObsObject summary : objects.getObjects()) {
+ if (summary.getObjectKey().equals(key)) {
+ // skip prefix itself
+ continue;
+ }
+
+ keys.add(new KeyAndVersion(summary.getObjectKey()));
+ LOG.debug("Got object to delete {}", summary.getObjectKey());
+
+ if (keys.size() == owner.getMaxEntriesToDelete()) {
+ OBSCommonUtils.removeKeys(owner, keys, true, true);
+ }
+ }
+
+ if (!objects.isTruncated()) {
+ keys.add(new KeyAndVersion(key));
+ OBSCommonUtils.removeKeys(owner, keys, false, true);
+
+ break;
+ }
+ objects = OBSCommonUtils.continueListObjects(owner, objects);
+ }
+ }
+
+ static void createFakeDirectoryIfNecessary(final OBSFileSystem owner,
+ final Path f)
+ throws IOException, ObsException {
+
+ String key = OBSCommonUtils.pathToKey(owner, f);
+ if (!key.isEmpty() && !owner.exists(f)) {
+ LOG.debug("Creating new fake directory at {}", f);
+ createFakeDirectory(owner, key);
+ }
+ }
+
+ static void createFakeDirectory(final OBSFileSystem owner,
+ final String objectName)
+ throws ObsException, IOException {
+ String newObjectName = objectName;
+ newObjectName = OBSCommonUtils.maybeAddTrailingSlash(newObjectName);
+ createEmptyObject(owner, newObjectName);
+ }
+
+ // Used to create an empty file that represents an empty directory
+ private static void createEmptyObject(final OBSFileSystem owner,
+ final String objectName)
+ throws ObsException, IOException {
+ for (int retryTime = 1;
+ retryTime < OBSCommonUtils.MAX_RETRY_TIME; retryTime++) {
+ try {
+ innerCreateEmptyObject(owner, objectName);
+ return;
+ } catch (ObsException e) {
+ LOG.warn("Failed to create empty object [{}], retry time [{}], "
+ + "exception [{}]", objectName, retryTime, e);
+ try {
+ Thread.sleep(OBSCommonUtils.DELAY_TIME);
+ } catch (InterruptedException ie) {
+ throw e;
+ }
+ }
+ }
+
+ innerCreateEmptyObject(owner, objectName);
+ }
+
+ // Used to create an empty file that represents an empty directory
+ private static void innerCreateEmptyObject(final OBSFileSystem owner,
+ final String objectName)
+ throws ObsException, IOException {
+ final InputStream im =
+ new InputStream() {
+ @Override
+ public int read() {
+ return -1;
+ }
+ };
+
+ PutObjectRequest putObjectRequest = OBSCommonUtils
+ .newPutObjectRequest(owner, objectName, newObjectMetadata(0L), im);
+
+ long len;
+ if (putObjectRequest.getFile() != null) {
+ len = putObjectRequest.getFile().length();
+ } else {
+ len = putObjectRequest.getMetadata().getContentLength();
+ }
+
+ try {
+ owner.getObsClient().putObject(putObjectRequest);
+ owner.getSchemeStatistics().incrementWriteOps(1);
+ owner.getSchemeStatistics().incrementBytesWritten(len);
+ } finally {
+ im.close();
+ }
+ }
+
+ /**
+ * Copy a single object in the bucket via a COPY operation.
+ *
+ * @param owner OBS File System instance
+ * @param srcKey source object path
+ * @param dstKey destination object path
+ * @param size object size
+ * @throws InterruptedIOException the operation was interrupted
+ * @throws IOException Other IO problems
+ */
+ private static void copyFile(final OBSFileSystem owner, final String srcKey,
+ final String dstKey, final long size)
+ throws IOException, InterruptedIOException {
+ for (int retryTime = 1;
+ retryTime < OBSCommonUtils.MAX_RETRY_TIME; retryTime++) {
+ try {
+ innerCopyFile(owner, srcKey, dstKey, size);
+ return;
+ } catch (InterruptedIOException e) {
+ throw e;
+ } catch (IOException e) {
+ LOG.warn(
+ "Failed to copy file from [{}] to [{}] with size [{}], "
+ + "retry time [{}], exception [{}]", srcKey, dstKey,
+ size, retryTime, e);
+ try {
+ Thread.sleep(OBSCommonUtils.DELAY_TIME);
+ } catch (InterruptedException ie) {
+ throw e;
+ }
+ }
+ }
+
+ innerCopyFile(owner, srcKey, dstKey, size);
+ }
+
+ private static void innerCopyFile(final OBSFileSystem owner,
+ final String srcKey,
+ final String dstKey, final long size)
+ throws IOException {
+ LOG.debug("copyFile {} -> {} ", srcKey, dstKey);
+ try {
+ // 100MB per part
+ if (size > owner.getCopyPartSize()) {
+ // initial copy part task
+ InitiateMultipartUploadRequest request
+ = new InitiateMultipartUploadRequest(owner.getBucket(),
+ dstKey);
+ request.setAcl(owner.getCannedACL());
+ if (owner.getSse().isSseCEnable()) {
+ request.setSseCHeader(owner.getSse().getSseCHeader());
+ } else if (owner.getSse().isSseKmsEnable()) {
+ request.setSseKmsHeader(owner.getSse().getSseKmsHeader());
+ }
+ InitiateMultipartUploadResult result = owner.getObsClient()
+ .initiateMultipartUpload(request);
+
+ final String uploadId = result.getUploadId();
+ LOG.debug("Multipart copy file, uploadId: {}", uploadId);
+ // count the parts
+ long partCount = calPartCount(owner.getCopyPartSize(), size);
+
+ final List partEtags =
+ getCopyFilePartEtags(owner, srcKey, dstKey, size, uploadId,
+ partCount);
+ // merge the copy parts
+ CompleteMultipartUploadRequest completeMultipartUploadRequest =
+ new CompleteMultipartUploadRequest(owner.getBucket(),
+ dstKey, uploadId, partEtags);
+ owner.getObsClient()
+ .completeMultipartUpload(completeMultipartUploadRequest);
+ } else {
+ ObjectMetadata srcom = getObjectMetadata(owner, srcKey);
+ ObjectMetadata dstom = cloneObjectMetadata(srcom);
+ final CopyObjectRequest copyObjectRequest =
+ new CopyObjectRequest(owner.getBucket(), srcKey,
+ owner.getBucket(), dstKey);
+ copyObjectRequest.setAcl(owner.getCannedACL());
+ copyObjectRequest.setNewObjectMetadata(dstom);
+ if (owner.getSse().isSseCEnable()) {
+ copyObjectRequest.setSseCHeader(
+ owner.getSse().getSseCHeader());
+ copyObjectRequest.setSseCHeaderSource(
+ owner.getSse().getSseCHeader());
+ } else if (owner.getSse().isSseKmsEnable()) {
+ copyObjectRequest.setSseKmsHeader(
+ owner.getSse().getSseKmsHeader());
+ }
+ owner.getObsClient().copyObject(copyObjectRequest);
+ }
+
+ owner.getSchemeStatistics().incrementWriteOps(1);
+ } catch (ObsException e) {
+ throw OBSCommonUtils.translateException(
+ "copyFile(" + srcKey + ", " + dstKey + ")", srcKey, e);
+ }
+ }
+
+ static int calPartCount(final long partSize, final long cloudSize) {
+ // get user setting of per copy part size ,default is 100MB
+ // calculate the part count
+ long partCount = cloudSize % partSize == 0
+ ? cloudSize / partSize
+ : cloudSize / partSize + 1;
+ return (int) partCount;
+ }
+
+ static List getCopyFilePartEtags(final OBSFileSystem owner,
+ final String srcKey,
+ final String dstKey,
+ final long objectSize,
+ final String uploadId,
+ final long partCount)
+ throws IOException {
+ final List partEtags = Collections.synchronizedList(
+ new ArrayList<>());
+ final List> partCopyFutures = new ArrayList<>();
+ submitCopyPartTasks(owner, srcKey, dstKey, objectSize, uploadId,
+ partCount, partEtags, partCopyFutures);
+
+ // wait the tasks for completing
+ try {
+ for (Future> partCopyFuture : partCopyFutures) {
+ partCopyFuture.get();
+ }
+ } catch (InterruptedException e) {
+ LOG.warn("Interrupted while copying objects (copy)");
+ throw new InterruptedIOException(
+ "Interrupted while copying objects (copy)");
+ } catch (ExecutionException e) {
+ LOG.error("Multipart copy file exception.", e);
+ for (Future> future : partCopyFutures) {
+ future.cancel(true);
+ }
+
+ owner.getObsClient()
+ .abortMultipartUpload(
+ new AbortMultipartUploadRequest(owner.getBucket(), dstKey,
+ uploadId));
+
+ throw OBSCommonUtils.extractException(
+ "Multi-part copy with id '" + uploadId + "' from " + srcKey
+ + "to " + dstKey, dstKey, e);
+ }
+
+ // Make part numbers in ascending order
+ partEtags.sort(Comparator.comparingInt(PartEtag::getPartNumber));
+ return partEtags;
+ }
+
+ @SuppressWarnings("checkstyle:parameternumber")
+ private static void submitCopyPartTasks(final OBSFileSystem owner,
+ final String srcKey,
+ final String dstKey,
+ final long objectSize,
+ final String uploadId,
+ final long partCount,
+ final List partEtags,
+ final List> partCopyFutures) {
+ for (int i = 0; i < partCount; i++) {
+ final long rangeStart = i * owner.getCopyPartSize();
+ final long rangeEnd = (i + 1 == partCount)
+ ? objectSize - 1
+ : rangeStart + owner.getCopyPartSize() - 1;
+ final int partNumber = i + 1;
+ partCopyFutures.add(
+ owner.getBoundedCopyPartThreadPool().submit(() -> {
+ CopyPartRequest request = new CopyPartRequest();
+ request.setUploadId(uploadId);
+ request.setSourceBucketName(owner.getBucket());
+ request.setSourceObjectKey(srcKey);
+ request.setDestinationBucketName(owner.getBucket());
+ request.setDestinationObjectKey(dstKey);
+ request.setByteRangeStart(rangeStart);
+ request.setByteRangeEnd(rangeEnd);
+ request.setPartNumber(partNumber);
+ if (owner.getSse().isSseCEnable()) {
+ request.setSseCHeaderSource(
+ owner.getSse().getSseCHeader());
+ request.setSseCHeaderDestination(
+ owner.getSse().getSseCHeader());
+ }
+ CopyPartResult result = owner.getObsClient()
+ .copyPart(request);
+ partEtags.add(
+ new PartEtag(result.getEtag(), result.getPartNumber()));
+ LOG.debug(
+ "Multipart copy file, uploadId: {}, Part#{} done.",
+ uploadId, partNumber);
+ }));
+ }
+ }
+
+ /**
+ * Creates a copy of the passed {@link ObjectMetadata}. Does so without using
+ * the {@link ObjectMetadata#clone()} method, to avoid copying unnecessary
+ * headers.
+ *
+ * @param source the {@link ObjectMetadata} to copy
+ * @return a copy of {@link ObjectMetadata} with only relevant attributes
+ */
+ private static ObjectMetadata cloneObjectMetadata(
+ final ObjectMetadata source) {
+ // This approach may be too brittle, especially if
+ // in future there are new attributes added to ObjectMetadata
+ // that we do not explicitly call to set here
+ ObjectMetadata ret = newObjectMetadata(source.getContentLength());
+
+ if (source.getContentEncoding() != null) {
+ ret.setContentEncoding(source.getContentEncoding());
+ }
+ return ret;
+ }
+
+ static OBSFileStatus innerGetObjectStatus(final OBSFileSystem owner,
+ final Path f)
+ throws IOException {
+ final Path path = OBSCommonUtils.qualify(owner, f);
+ String key = OBSCommonUtils.pathToKey(owner, path);
+ LOG.debug("Getting path status for {} ({})", path, key);
+ if (!StringUtils.isEmpty(key)) {
+ try {
+ ObjectMetadata meta = getObjectMetadata(owner, key);
+
+ if (OBSCommonUtils.objectRepresentsDirectory(key,
+ meta.getContentLength())) {
+ LOG.debug("Found exact file: fake directory");
+ return new OBSFileStatus(path, owner.getUsername());
+ } else {
+ LOG.debug("Found exact file: normal file");
+ return new OBSFileStatus(meta.getContentLength(),
+ OBSCommonUtils.dateToLong(meta.getLastModified()),
+ path, owner.getDefaultBlockSize(path),
+ owner.getUsername());
+ }
+ } catch (ObsException e) {
+ if (e.getResponseCode() != OBSCommonUtils.NOT_FOUND_CODE) {
+ throw OBSCommonUtils.translateException("getFileStatus",
+ path, e);
+ }
+ }
+
+ if (!key.endsWith("/")) {
+ String newKey = key + "/";
+ try {
+ ObjectMetadata meta = getObjectMetadata(owner, newKey);
+
+ if (OBSCommonUtils.objectRepresentsDirectory(newKey,
+ meta.getContentLength())) {
+ LOG.debug("Found file (with /): fake directory");
+ return new OBSFileStatus(path, owner.getUsername());
+ } else {
+ LOG.debug(
+ "Found file (with /): real file? should not "
+ + "happen: {}",
+ key);
+
+ return new OBSFileStatus(meta.getContentLength(),
+ OBSCommonUtils.dateToLong(meta.getLastModified()),
+ path,
+ owner.getDefaultBlockSize(path),
+ owner.getUsername());
+ }
+ } catch (ObsException e) {
+ if (e.getResponseCode() != OBSCommonUtils.NOT_FOUND_CODE) {
+ throw OBSCommonUtils.translateException("getFileStatus",
+ newKey, e);
+ }
+ }
+ }
+ }
+
+ try {
+ boolean isEmpty = OBSCommonUtils.innerIsFolderEmpty(owner, key);
+ LOG.debug("Is dir ({}) empty? {}", path, isEmpty);
+ return new OBSFileStatus(path, owner.getUsername());
+ } catch (ObsException e) {
+ if (e.getResponseCode() != OBSCommonUtils.NOT_FOUND_CODE) {
+ throw OBSCommonUtils.translateException("getFileStatus", key,
+ e);
+ }
+ }
+
+ LOG.debug("Not Found: {}", path);
+ throw new FileNotFoundException("No such file or directory: " + path);
+ }
+
+ static ContentSummary getDirectoryContentSummary(final OBSFileSystem owner,
+ final String key) throws IOException {
+ String newKey = key;
+ newKey = OBSCommonUtils.maybeAddTrailingSlash(newKey);
+ long[] summary = {0, 0, 1};
+ LOG.debug("Summary key {}", newKey);
+ ListObjectsRequest request = new ListObjectsRequest();
+ request.setBucketName(owner.getBucket());
+ request.setPrefix(newKey);
+ Set directories = new TreeSet<>();
+ request.setMaxKeys(owner.getMaxKeys());
+ ObjectListing objects = OBSCommonUtils.listObjects(owner, request);
+ while (true) {
+ if (!objects.getCommonPrefixes().isEmpty() || !objects.getObjects()
+ .isEmpty()) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Found path as directory (with /): {}/{}",
+ objects.getCommonPrefixes().size(),
+ objects.getObjects().size());
+ }
+ for (String prefix : objects.getCommonPrefixes()) {
+ LOG.debug("Objects in folder [" + prefix + "]:");
+ getDirectories(prefix, newKey, directories);
+ }
+
+ for (ObsObject obj : objects.getObjects()) {
+ LOG.debug("Summary: {} {}", obj.getObjectKey(),
+ obj.getMetadata().getContentLength());
+ if (!obj.getObjectKey().endsWith("/")) {
+ summary[0] += obj.getMetadata().getContentLength();
+ summary[1] += 1;
+ }
+ getDirectories(obj.getObjectKey(), newKey, directories);
+ }
+ }
+ if (!objects.isTruncated()) {
+ break;
+ }
+ objects = OBSCommonUtils.continueListObjects(owner, objects);
+ }
+ summary[2] += directories.size();
+ LOG.debug(String.format(
+ "file size [%d] - file count [%d] - directory count [%d] - "
+ + "file path [%s]",
+ summary[0],
+ summary[1], summary[2], newKey));
+ return new ContentSummary.Builder().length(summary[0])
+ .fileCount(summary[1]).directoryCount(summary[2])
+ .spaceConsumed(summary[0]).build();
+ }
+
+ private static void getDirectories(final String key, final String sourceKey,
+ final Set directories) {
+ Path p = new Path(key);
+ Path sourcePath = new Path(sourceKey);
+ // directory must add first
+ if (key.endsWith("/") && p.compareTo(sourcePath) > 0) {
+ directories.add(p.toString());
+ }
+ while (p.compareTo(sourcePath) > 0) {
+ p = p.getParent();
+ if (p.isRoot() || p.compareTo(sourcePath) == 0) {
+ break;
+ }
+ directories.add(p.toString());
+ }
+ }
+
+ private static Future copyFileAsync(
+ final OBSFileSystem owner,
+ final String srcKey,
+ final String dstKey, final long size) {
+ return owner.getBoundedCopyThreadPool().submit(() -> {
+ copyFile(owner, srcKey, dstKey, size);
+ return null;
+ });
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSPosixBucketUtils.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSPosixBucketUtils.java
new file mode 100644
index 00000000000..d6afd456969
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSPosixBucketUtils.java
@@ -0,0 +1,745 @@
+/*
+ * 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.obs;
+
+import com.obs.services.exception.ObsException;
+import com.obs.services.model.KeyAndVersion;
+import com.obs.services.model.ListObjectsRequest;
+import com.obs.services.model.ObjectListing;
+import com.obs.services.model.ObsObject;
+import com.obs.services.model.fs.GetAttributeRequest;
+import com.obs.services.model.fs.NewFolderRequest;
+import com.obs.services.model.fs.ObsFSAttribute;
+import com.obs.services.model.fs.RenameRequest;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+
+/**
+ * Posix bucket specific utils for {@link OBSFileSystem}.
+ */
+final class OBSPosixBucketUtils {
+ /**
+ * Class logger.
+ */
+ private static final Logger LOG = LoggerFactory.getLogger(
+ OBSPosixBucketUtils.class);
+
+ private OBSPosixBucketUtils() {
+ }
+
+ /**
+ * Get the depth of an absolute path, that is the number of '/' in the path.
+ *
+ * @param key object key
+ * @return depth
+ */
+ static int fsGetObjectKeyDepth(final String key) {
+ int depth = 0;
+ for (int idx = key.indexOf('/');
+ idx >= 0; idx = key.indexOf('/', idx + 1)) {
+ depth++;
+ }
+ return key.endsWith("/") ? depth - 1 : depth;
+ }
+
+ /**
+ * Used to judge that an object is a file or folder.
+ *
+ * @param attr posix object attribute
+ * @return is posix folder
+ */
+ static boolean fsIsFolder(final ObsFSAttribute attr) {
+ final int ifDir = 0x004000;
+ int mode = attr.getMode();
+ // object mode is -1 when the object is migrated from
+ // object bucket to posix bucket.
+ // -1 is a file, not folder.
+ if (mode < 0) {
+ return false;
+ }
+
+ return (mode & ifDir) != 0;
+ }
+
+ /**
+ * The inner rename operation based on Posix bucket.
+ *
+ * @param owner OBS File System instance
+ * @param src source path to be renamed from
+ * @param dst destination path to be renamed to
+ * @return boolean
+ * @throws RenameFailedException if some criteria for a state changing rename
+ * was not met. This means work didn't happen;
+ * it's not something which is reported upstream
+ * to the FileSystem APIs, for which the
+ * semantics of "false" are pretty vague.
+ * @throws IOException on IO failure.
+ */
+ static boolean renameBasedOnPosix(final OBSFileSystem owner, final Path src,
+ final Path dst) throws IOException {
+ Path dstPath = dst;
+ String srcKey = OBSCommonUtils.pathToKey(owner, src);
+ String dstKey = OBSCommonUtils.pathToKey(owner, dstPath);
+
+ if (srcKey.isEmpty()) {
+ LOG.error("rename: src [{}] is root directory", src);
+ return false;
+ }
+
+ try {
+ FileStatus dstStatus = owner.getFileStatus(dstPath);
+ if (dstStatus.isDirectory()) {
+ String newDstString = OBSCommonUtils.maybeAddTrailingSlash(
+ dstPath.toString());
+ String filename = srcKey.substring(
+ OBSCommonUtils.pathToKey(owner, src.getParent())
+ .length() + 1);
+ dstPath = new Path(newDstString + filename);
+ dstKey = OBSCommonUtils.pathToKey(owner, dstPath);
+ LOG.debug(
+ "rename: dest is an existing directory and will be "
+ + "changed to [{}]", dstPath);
+
+ if (owner.exists(dstPath)) {
+ LOG.error("rename: failed to rename " + src + " to "
+ + dstPath
+ + " because destination exists");
+ return false;
+ }
+ } else {
+ if (srcKey.equals(dstKey)) {
+ LOG.warn(
+ "rename: src and dest refer to the same "
+ + "file or directory: {}", dstPath);
+ return true;
+ } else {
+ LOG.error("rename: failed to rename " + src + " to "
+ + dstPath
+ + " because destination exists");
+ return false;
+ }
+ }
+ } catch (FileNotFoundException e) {
+ // if destination does not exist, do not change the
+ // destination key, and just do rename.
+ LOG.debug("rename: dest [{}] does not exist", dstPath);
+ } catch (FileConflictException e) {
+ Path parent = dstPath.getParent();
+ if (!OBSCommonUtils.pathToKey(owner, parent).isEmpty()) {
+ FileStatus dstParentStatus = owner.getFileStatus(parent);
+ if (!dstParentStatus.isDirectory()) {
+ throw new ParentNotDirectoryException(
+ parent + " is not a directory");
+ }
+ }
+ }
+
+ if (dstKey.startsWith(srcKey) && (dstKey.equals(srcKey)
+ || dstKey.charAt(srcKey.length()) == Path.SEPARATOR_CHAR)) {
+ LOG.error("rename: dest [{}] cannot be a descendant of src [{}]",
+ dstPath, src);
+ return false;
+ }
+
+ return innerFsRenameWithRetry(owner, src, dstPath, srcKey, dstKey);
+ }
+
+ private static boolean innerFsRenameWithRetry(final OBSFileSystem owner,
+ final Path src,
+ final Path dst, final String srcKey, final String dstKey)
+ throws IOException {
+ boolean renameResult = true;
+ int retryTime = 1;
+ while (retryTime <= OBSCommonUtils.MAX_RETRY_TIME) {
+ try {
+ LOG.debug("rename: {}-st rename from [{}] to [{}] ...",
+ retryTime, srcKey, dstKey);
+ innerFsRenameFile(owner, srcKey, dstKey);
+ renameResult = true;
+ break;
+ } catch (FileNotFoundException e) {
+ if (owner.exists(dst)) {
+ LOG.warn(
+ "rename: successfully {}-st rename src [{}] "
+ + "to dest [{}] with SDK retry",
+ retryTime, src, dst, e);
+ renameResult = true;
+ } else {
+ LOG.error(
+ "rename: failed {}-st rename src [{}] to dest [{}]",
+ retryTime, src, dst, e);
+ renameResult = false;
+ }
+ break;
+ } catch (IOException e) {
+ if (retryTime == OBSCommonUtils.MAX_RETRY_TIME) {
+ LOG.error(
+ "rename: failed {}-st rename src [{}] to dest [{}]",
+ retryTime, src, dst, e);
+ throw e;
+ } else {
+ LOG.warn(
+ "rename: failed {}-st rename src [{}] to dest [{}]",
+ retryTime, src, dst, e);
+ if (owner.exists(dst) && owner.exists(src)) {
+ LOG.warn(
+ "rename: failed {}-st rename src [{}] to "
+ + "dest [{}] with SDK retry", retryTime, src,
+ dst, e);
+ renameResult = false;
+ break;
+ }
+
+ try {
+ Thread.sleep(OBSCommonUtils.DELAY_TIME);
+ } catch (InterruptedException ie) {
+ throw e;
+ }
+ }
+ }
+
+ retryTime++;
+ }
+
+ return renameResult;
+ }
+
+ /**
+ * Used to rename a source folder to a destination folder that is not existed
+ * before rename.
+ *
+ * @param owner OBS File System instance
+ * @param src source folder key
+ * @param dst destination folder key that not existed before rename
+ * @throws IOException any io exception
+ * @throws ObsException any obs operation exception
+ */
+ static void fsRenameToNewFolder(final OBSFileSystem owner, final String src,
+ final String dst)
+ throws IOException, ObsException {
+ LOG.debug("RenameFolder path {} to {}", src, dst);
+
+ try {
+ RenameRequest renameObjectRequest = new RenameRequest();
+ renameObjectRequest.setBucketName(owner.getBucket());
+ renameObjectRequest.setObjectKey(src);
+ renameObjectRequest.setNewObjectKey(dst);
+ owner.getObsClient().renameFolder(renameObjectRequest);
+ owner.getSchemeStatistics().incrementWriteOps(1);
+ } catch (ObsException e) {
+ throw OBSCommonUtils.translateException(
+ "renameFile(" + src + ", " + dst + ")", src, e);
+ }
+ }
+
+ static void innerFsRenameFile(final OBSFileSystem owner,
+ final String srcKey,
+ final String dstKey) throws IOException {
+ LOG.debug("RenameFile path {} to {}", srcKey, dstKey);
+
+ try {
+ final RenameRequest renameObjectRequest = new RenameRequest();
+ renameObjectRequest.setBucketName(owner.getBucket());
+ renameObjectRequest.setObjectKey(srcKey);
+ renameObjectRequest.setNewObjectKey(dstKey);
+ owner.getObsClient().renameFile(renameObjectRequest);
+ owner.getSchemeStatistics().incrementWriteOps(1);
+ } catch (ObsException e) {
+ if (e.getResponseCode() == OBSCommonUtils.NOT_FOUND_CODE) {
+ throw new FileNotFoundException(
+ "No such file or directory: " + srcKey);
+ }
+ if (e.getResponseCode() == OBSCommonUtils.CONFLICT_CODE) {
+ throw new FileConflictException(
+ "File conflicts during rename, " + e.getResponseStatus());
+ }
+ throw OBSCommonUtils.translateException(
+ "renameFile(" + srcKey + ", " + dstKey + ")", srcKey, e);
+ }
+ }
+
+ /**
+ * Used to rename a source object to a destination object which is not existed
+ * before rename.
+ *
+ * @param owner OBS File System instance
+ * @param srcKey source object key
+ * @param dstKey destination object key
+ * @throws IOException io exception
+ */
+ static void fsRenameToNewObject(final OBSFileSystem owner,
+ final String srcKey,
+ final String dstKey) throws IOException {
+ String newSrcKey = srcKey;
+ String newdstKey = dstKey;
+ newSrcKey = OBSCommonUtils.maybeDeleteBeginningSlash(newSrcKey);
+ newdstKey = OBSCommonUtils.maybeDeleteBeginningSlash(newdstKey);
+ if (newSrcKey.endsWith("/")) {
+ // Rename folder.
+ fsRenameToNewFolder(owner, newSrcKey, newdstKey);
+ } else {
+ // Rename file.
+ innerFsRenameFile(owner, newSrcKey, newdstKey);
+ }
+ }
+
+ // Delete a file.
+ private static int fsRemoveFile(final OBSFileSystem owner,
+ final String sonObjectKey,
+ final List files)
+ throws IOException {
+ files.add(new KeyAndVersion(sonObjectKey));
+ if (files.size() == owner.getMaxEntriesToDelete()) {
+ // batch delete files.
+ OBSCommonUtils.removeKeys(owner, files, true, false);
+ return owner.getMaxEntriesToDelete();
+ }
+ return 0;
+ }
+
+ // Recursively delete a folder that might be not empty.
+ static boolean fsDelete(final OBSFileSystem owner, final FileStatus status,
+ final boolean recursive)
+ throws IOException, ObsException {
+ long startTime = System.currentTimeMillis();
+ long threadId = Thread.currentThread().getId();
+ Path f = status.getPath();
+ String key = OBSCommonUtils.pathToKey(owner, f);
+
+ if (!status.isDirectory()) {
+ LOG.debug("delete: Path is a file");
+ trashObjectIfNeed(owner, key);
+ } else {
+ LOG.debug("delete: Path is a directory: {} - recursive {}", f,
+ recursive);
+ key = OBSCommonUtils.maybeAddTrailingSlash(key);
+ boolean isEmptyDir = OBSCommonUtils.isFolderEmpty(owner, key);
+ if (key.equals("")) {
+ return OBSCommonUtils.rejectRootDirectoryDelete(
+ owner.getBucket(), isEmptyDir, recursive);
+ }
+ if (!recursive && !isEmptyDir) {
+ LOG.warn("delete: Path is not empty: {} - recursive {}", f,
+ recursive);
+ throw new PathIsNotEmptyDirectoryException(f.toString());
+ }
+ if (isEmptyDir) {
+ LOG.debug(
+ "delete: Deleting fake empty directory {} - recursive {}",
+ f, recursive);
+ OBSCommonUtils.deleteObject(owner, key);
+ } else {
+ LOG.debug(
+ "delete: Deleting objects for directory prefix {} to "
+ + "delete - recursive {}", f, recursive);
+ trashFolderIfNeed(owner, key, f);
+ }
+ }
+
+ long endTime = System.currentTimeMillis();
+ LOG.debug("delete Path:{} thread:{}, timeUsedInMilliSec:{}", f,
+ threadId, endTime - startTime);
+ return true;
+ }
+
+ private static void trashObjectIfNeed(final OBSFileSystem owner,
+ final String key)
+ throws ObsException, IOException {
+ if (needToTrash(owner, key)) {
+ mkTrash(owner, key);
+ StringBuilder sb = new StringBuilder(owner.getTrashDir());
+ sb.append(key);
+ if (owner.exists(new Path(sb.toString()))) {
+ SimpleDateFormat df = new SimpleDateFormat("-yyyyMMddHHmmss");
+ sb.append(df.format(new Date()));
+ }
+ fsRenameToNewObject(owner, key, sb.toString());
+ LOG.debug("Moved: '" + key + "' to trash at: " + sb.toString());
+ } else {
+ OBSCommonUtils.deleteObject(owner, key);
+ }
+ }
+
+ private static void trashFolderIfNeed(final OBSFileSystem owner,
+ final String key,
+ final Path f) throws ObsException, IOException {
+ if (needToTrash(owner, key)) {
+ mkTrash(owner, key);
+ StringBuilder sb = new StringBuilder(owner.getTrashDir());
+ String subKey = OBSCommonUtils.maybeAddTrailingSlash(key);
+ sb.append(subKey);
+ if (owner.exists(new Path(sb.toString()))) {
+ SimpleDateFormat df = new SimpleDateFormat("-yyyyMMddHHmmss");
+ sb.insert(sb.length() - 1, df.format(new Date()));
+ }
+
+ String srcKey = OBSCommonUtils.maybeDeleteBeginningSlash(key);
+ String dstKey = OBSCommonUtils.maybeDeleteBeginningSlash(
+ sb.toString());
+ fsRenameToNewFolder(owner, srcKey, dstKey);
+ LOG.debug("Moved: '" + key + "' to trash at: " + sb.toString());
+ } else {
+ if (owner.isEnableMultiObjectDeleteRecursion()) {
+ long delNum = fsRecursivelyDeleteDir(owner, key, true);
+ LOG.debug("Recursively delete {} files/dirs when deleting {}",
+ delNum, key);
+ } else {
+ fsNonRecursivelyDelete(owner, f);
+ }
+ }
+ }
+
+ static long fsRecursivelyDeleteDir(final OBSFileSystem owner,
+ final String parentKey,
+ final boolean deleteParent) throws IOException {
+ long delNum = 0;
+ List subdirList = new ArrayList<>(
+ owner.getMaxEntriesToDelete());
+ List fileList = new ArrayList<>(
+ owner.getMaxEntriesToDelete());
+
+ ListObjectsRequest request = OBSCommonUtils.createListObjectsRequest(
+ owner, parentKey, "/", owner.getMaxKeys());
+ ObjectListing objects = OBSCommonUtils.listObjects(owner, request);
+ while (true) {
+ for (String commonPrefix : objects.getCommonPrefixes()) {
+ if (commonPrefix.equals(parentKey)) {
+ // skip prefix itself
+ continue;
+ }
+
+ delNum += fsRemoveSubdir(owner, commonPrefix, subdirList);
+ }
+
+ for (ObsObject sonObject : objects.getObjects()) {
+ String sonObjectKey = sonObject.getObjectKey();
+
+ if (sonObjectKey.equals(parentKey)) {
+ // skip prefix itself
+ continue;
+ }
+
+ if (!sonObjectKey.endsWith("/")) {
+ delNum += fsRemoveFile(owner, sonObjectKey, fileList);
+ } else {
+ delNum += fsRemoveSubdir(owner, sonObjectKey, subdirList);
+ }
+ }
+
+ if (!objects.isTruncated()) {
+ break;
+ }
+
+ objects = OBSCommonUtils.continueListObjects(owner, objects);
+ }
+
+ delNum += fileList.size();
+ OBSCommonUtils.removeKeys(owner, fileList, true, false);
+
+ delNum += subdirList.size();
+ OBSCommonUtils.removeKeys(owner, subdirList, true, false);
+
+ if (deleteParent) {
+ OBSCommonUtils.deleteObject(owner, parentKey);
+ delNum++;
+ }
+
+ return delNum;
+ }
+
+ private static boolean needToTrash(final OBSFileSystem owner,
+ final String key) {
+ String newKey = key;
+ newKey = OBSCommonUtils.maybeDeleteBeginningSlash(newKey);
+ if (owner.isEnableTrash() && newKey.startsWith(owner.getTrashDir())) {
+ return false;
+ }
+ return owner.isEnableTrash();
+ }
+
+ // Delete a sub dir.
+ private static int fsRemoveSubdir(final OBSFileSystem owner,
+ final String subdirKey,
+ final List subdirList)
+ throws IOException {
+ fsRecursivelyDeleteDir(owner, subdirKey, false);
+
+ subdirList.add(new KeyAndVersion(subdirKey));
+ if (subdirList.size() == owner.getMaxEntriesToDelete()) {
+ // batch delete subdirs.
+ OBSCommonUtils.removeKeys(owner, subdirList, true, false);
+ return owner.getMaxEntriesToDelete();
+ }
+
+ return 0;
+ }
+
+ private static void mkTrash(final OBSFileSystem owner, final String key)
+ throws ObsException, IOException {
+ String newKey = key;
+ StringBuilder sb = new StringBuilder(owner.getTrashDir());
+ newKey = OBSCommonUtils.maybeAddTrailingSlash(newKey);
+ sb.append(newKey);
+ sb.deleteCharAt(sb.length() - 1);
+ sb.delete(sb.lastIndexOf("/"), sb.length());
+ Path fastDeleteRecycleDirPath = new Path(sb.toString());
+ // keep the parent directory of the target path exists
+ if (!owner.exists(fastDeleteRecycleDirPath)) {
+ owner.mkdirs(fastDeleteRecycleDirPath);
+ }
+ }
+
+ // List all sub objects at first, delete sub objects in batch secondly.
+ private static void fsNonRecursivelyDelete(final OBSFileSystem owner,
+ final Path parent)
+ throws IOException, ObsException {
+ // List sub objects sorted by path depth.
+ FileStatus[] arFileStatus = OBSCommonUtils.innerListStatus(owner,
+ parent, true);
+ // Remove sub objects one depth by one depth to avoid that parents and
+ // children in a same batch.
+ fsRemoveKeys(owner, arFileStatus);
+ // Delete parent folder that should has become empty.
+ OBSCommonUtils.deleteObject(owner,
+ OBSCommonUtils.pathToKey(owner, parent));
+ }
+
+ // Remove sub objects of each depth one by one to avoid that parents and
+ // children in a same batch.
+ private static void fsRemoveKeys(final OBSFileSystem owner,
+ final FileStatus[] arFileStatus)
+ throws ObsException, IOException {
+ if (arFileStatus.length <= 0) {
+ // exit fast if there are no keys to delete
+ return;
+ }
+
+ String key;
+ for (FileStatus fileStatus : arFileStatus) {
+ key = OBSCommonUtils.pathToKey(owner, fileStatus.getPath());
+ OBSCommonUtils.blockRootDelete(owner.getBucket(), key);
+ }
+
+ fsRemoveKeysByDepth(owner, arFileStatus);
+ }
+
+ // Batch delete sub objects one depth by one depth to avoid that parents and
+ // children in a same
+ // batch.
+ // A batch deletion might be split into some concurrent deletions to promote
+ // the performance, but
+ // it
+ // can't make sure that an object is deleted before it's children.
+ private static void fsRemoveKeysByDepth(final OBSFileSystem owner,
+ final FileStatus[] arFileStatus)
+ throws ObsException, IOException {
+ if (arFileStatus.length <= 0) {
+ // exit fast if there is no keys to delete
+ return;
+ }
+
+ // Find all leaf keys in the list.
+ String key;
+ int depth = Integer.MAX_VALUE;
+ List leafKeys = new ArrayList<>(
+ owner.getMaxEntriesToDelete());
+ for (int idx = arFileStatus.length - 1; idx >= 0; idx--) {
+ if (leafKeys.size() >= owner.getMaxEntriesToDelete()) {
+ OBSCommonUtils.removeKeys(owner, leafKeys, true, false);
+ }
+
+ key = OBSCommonUtils.pathToKey(owner, arFileStatus[idx].getPath());
+
+ // Check file.
+ if (!arFileStatus[idx].isDirectory()) {
+ // A file must be a leaf.
+ leafKeys.add(new KeyAndVersion(key, null));
+ continue;
+ }
+
+ // Check leaf folder at current depth.
+ int keyDepth = fsGetObjectKeyDepth(key);
+ if (keyDepth == depth) {
+ // Any key at current depth must be a leaf.
+ leafKeys.add(new KeyAndVersion(key, null));
+ continue;
+ }
+ if (keyDepth < depth) {
+ // The last batch delete at current depth.
+ OBSCommonUtils.removeKeys(owner, leafKeys, true, false);
+ // Go on at the upper depth.
+ depth = keyDepth;
+ leafKeys.add(new KeyAndVersion(key, null));
+ continue;
+ }
+ LOG.warn(
+ "The objects list is invalid because it isn't sorted by"
+ + " path depth.");
+ throw new ObsException("System failure");
+ }
+
+ // The last batch delete at the minimum depth of all keys.
+ OBSCommonUtils.removeKeys(owner, leafKeys, true, false);
+ }
+
+ // Used to create a folder
+ static void fsCreateFolder(final OBSFileSystem owner,
+ final String objectName)
+ throws ObsException {
+ for (int retryTime = 1;
+ retryTime < OBSCommonUtils.MAX_RETRY_TIME; retryTime++) {
+ try {
+ innerFsCreateFolder(owner, objectName);
+ return;
+ } catch (ObsException e) {
+ LOG.warn("Failed to create folder [{}], retry time [{}], "
+ + "exception [{}]", objectName, retryTime, e);
+ try {
+ Thread.sleep(OBSCommonUtils.DELAY_TIME);
+ } catch (InterruptedException ie) {
+ throw e;
+ }
+ }
+ }
+
+ innerFsCreateFolder(owner, objectName);
+ }
+
+ private static void innerFsCreateFolder(final OBSFileSystem owner,
+ final String objectName)
+ throws ObsException {
+ final NewFolderRequest newFolderRequest = new NewFolderRequest(
+ owner.getBucket(), objectName);
+ newFolderRequest.setAcl(owner.getCannedACL());
+ long len = newFolderRequest.getObjectKey().length();
+ owner.getObsClient().newFolder(newFolderRequest);
+ owner.getSchemeStatistics().incrementWriteOps(1);
+ owner.getSchemeStatistics().incrementBytesWritten(len);
+ }
+
+ // Used to get the status of a file or folder in a file-gateway bucket.
+ static OBSFileStatus innerFsGetObjectStatus(final OBSFileSystem owner,
+ final Path f) throws IOException {
+ final Path path = OBSCommonUtils.qualify(owner, f);
+ String key = OBSCommonUtils.pathToKey(owner, path);
+ LOG.debug("Getting path status for {} ({})", path, key);
+
+ if (key.isEmpty()) {
+ LOG.debug("Found root directory");
+ return new OBSFileStatus(path, owner.getUsername());
+ }
+
+ try {
+ final GetAttributeRequest getAttrRequest = new GetAttributeRequest(
+ owner.getBucket(), key);
+ ObsFSAttribute meta = owner.getObsClient()
+ .getAttribute(getAttrRequest);
+ owner.getSchemeStatistics().incrementReadOps(1);
+ if (fsIsFolder(meta)) {
+ LOG.debug("Found file (with /): fake directory");
+ return new OBSFileStatus(path,
+ OBSCommonUtils.dateToLong(meta.getLastModified()),
+ owner.getUsername());
+ } else {
+ LOG.debug(
+ "Found file (with /): real file? should not happen: {}",
+ key);
+ return new OBSFileStatus(
+ meta.getContentLength(),
+ OBSCommonUtils.dateToLong(meta.getLastModified()),
+ path,
+ owner.getDefaultBlockSize(path),
+ owner.getUsername());
+ }
+ } catch (ObsException e) {
+ if (e.getResponseCode() == OBSCommonUtils.NOT_FOUND_CODE) {
+ LOG.debug("Not Found: {}", path);
+ throw new FileNotFoundException(
+ "No such file or directory: " + path);
+ }
+ if (e.getResponseCode() == OBSCommonUtils.CONFLICT_CODE) {
+ throw new FileConflictException(
+ "file conflicts: " + e.getResponseStatus());
+ }
+ throw OBSCommonUtils.translateException("getFileStatus", path, e);
+ }
+ }
+
+ static ContentSummary fsGetDirectoryContentSummary(
+ final OBSFileSystem owner,
+ final String key) throws IOException {
+ String newKey = key;
+ newKey = OBSCommonUtils.maybeAddTrailingSlash(newKey);
+ long[] summary = {0, 0, 1};
+ LOG.debug("Summary key {}", newKey);
+ ListObjectsRequest request = new ListObjectsRequest();
+ request.setBucketName(owner.getBucket());
+ request.setPrefix(newKey);
+ request.setMaxKeys(owner.getMaxKeys());
+ ObjectListing objects = OBSCommonUtils.listObjects(owner, request);
+ while (true) {
+ if (!objects.getCommonPrefixes().isEmpty() || !objects.getObjects()
+ .isEmpty()) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Found path as directory (with /): {}/{}",
+ objects.getCommonPrefixes().size(),
+ objects.getObjects().size());
+ }
+ for (String prefix : objects.getCommonPrefixes()) {
+ if (!prefix.equals(newKey)) {
+ summary[2]++;
+ }
+ }
+
+ for (ObsObject obj : objects.getObjects()) {
+ if (!obj.getObjectKey().endsWith("/")) {
+ summary[0] += obj.getMetadata().getContentLength();
+ summary[1] += 1;
+ } else if (!obj.getObjectKey().equals(newKey)) {
+ summary[2]++;
+ }
+ }
+ }
+ if (!objects.isTruncated()) {
+ break;
+ }
+ objects = OBSCommonUtils.continueListObjects(owner, objects);
+ }
+ LOG.debug(String.format(
+ "file size [%d] - file count [%d] - directory count [%d] - "
+ + "file path [%s]",
+ summary[0], summary[1], summary[2], newKey));
+ return new ContentSummary.Builder().length(summary[0])
+ .fileCount(summary[1]).directoryCount(summary[2])
+ .spaceConsumed(summary[0]).build();
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSWriteOperationHelper.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSWriteOperationHelper.java
new file mode 100644
index 00000000000..5cc3008f1dc
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSWriteOperationHelper.java
@@ -0,0 +1,310 @@
+/*
+ * 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.obs;
+
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import com.obs.services.ObsClient;
+import com.obs.services.exception.ObsException;
+import com.obs.services.model.AbortMultipartUploadRequest;
+import com.obs.services.model.CompleteMultipartUploadRequest;
+import com.obs.services.model.CompleteMultipartUploadResult;
+import com.obs.services.model.InitiateMultipartUploadRequest;
+import com.obs.services.model.ObjectMetadata;
+import com.obs.services.model.PartEtag;
+import com.obs.services.model.PutObjectRequest;
+import com.obs.services.model.PutObjectResult;
+import com.obs.services.model.UploadPartRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Helper for an ongoing write operation.
+ *
+ * It hides direct access to the OBS API from the output stream, and is a
+ * location where the object upload process can be evolved/enhanced.
+ *
+ *
Features
+ *
+ *
+ * - Methods to create and submit requests to OBS, so avoiding all direct
+ * interaction with the OBS APIs.
+ *
- Some extra preflight checks of arguments, so failing fast on errors.
+ *
- Callbacks to let the FS know of events in the output stream upload
+ * process.
+ *
+ *
+ * Each instance of this state is unique to a single output stream.
+ */
+class OBSWriteOperationHelper {
+ /**
+ * Class logger.
+ */
+ public static final Logger LOG = LoggerFactory.getLogger(
+ OBSWriteOperationHelper.class);
+
+ /**
+ * Part number of the multipart task.
+ */
+ static final int PART_NUMBER = 10000;
+
+ /**
+ * Owning filesystem.
+ */
+ private final OBSFileSystem owner;
+
+ /**
+ * Bucket of the owner FS.
+ */
+ private final String bucket;
+
+ /**
+ * Define obs client.
+ */
+ private final ObsClient obs;
+
+ protected OBSWriteOperationHelper(final OBSFileSystem fs) {
+ this.owner = fs;
+ this.bucket = fs.getBucket();
+ this.obs = fs.getObsClient();
+ }
+
+ /**
+ * Create a {@link PutObjectRequest} request. If {@code length} is set, the
+ * metadata is configured with the size of the upload.
+ *
+ * @param destKey key of object
+ * @param inputStream source data
+ * @param length size, if known. Use -1 for not known
+ * @return the request
+ */
+ PutObjectRequest newPutRequest(final String destKey,
+ final InputStream inputStream,
+ final long length) {
+ return OBSCommonUtils.newPutObjectRequest(owner, destKey,
+ newObjectMetadata(length), inputStream);
+ }
+
+ /**
+ * Create a {@link PutObjectRequest} request to upload a file.
+ *
+ * @param destKey object key for request
+ * @param sourceFile source file
+ * @return the request
+ */
+ PutObjectRequest newPutRequest(final String destKey,
+ final File sourceFile) {
+ int length = (int) sourceFile.length();
+ return OBSCommonUtils.newPutObjectRequest(owner, destKey,
+ newObjectMetadata(length), sourceFile);
+ }
+
+ /**
+ * Callback on a successful write.
+ *
+ * @param destKey object key
+ */
+ void writeSuccessful(final String destKey) {
+ LOG.debug("Finished write to {}", destKey);
+ }
+
+ /**
+ * Create a new object metadata instance. Any standard metadata headers are
+ * added here, for example: encryption.
+ *
+ * @param length size, if known. Use -1 for not known
+ * @return a new metadata instance
+ */
+ public ObjectMetadata newObjectMetadata(final long length) {
+ return OBSObjectBucketUtils.newObjectMetadata(length);
+ }
+
+ /**
+ * Start the multipart upload process.
+ *
+ * @param destKey object key
+ * @return the upload result containing the ID
+ * @throws IOException IO problem
+ */
+ String initiateMultiPartUpload(final String destKey) throws IOException {
+ LOG.debug("Initiating Multipart upload");
+ final InitiateMultipartUploadRequest initiateMPURequest =
+ new InitiateMultipartUploadRequest(bucket, destKey);
+ initiateMPURequest.setAcl(owner.getCannedACL());
+ initiateMPURequest.setMetadata(newObjectMetadata(-1));
+ if (owner.getSse().isSseCEnable()) {
+ initiateMPURequest.setSseCHeader(owner.getSse().getSseCHeader());
+ } else if (owner.getSse().isSseKmsEnable()) {
+ initiateMPURequest.setSseKmsHeader(
+ owner.getSse().getSseKmsHeader());
+ }
+ try {
+ return obs.initiateMultipartUpload(initiateMPURequest)
+ .getUploadId();
+ } catch (ObsException ace) {
+ throw OBSCommonUtils.translateException("Initiate MultiPartUpload",
+ destKey, ace);
+ }
+ }
+
+ /**
+ * Complete a multipart upload operation.
+ *
+ * @param destKey Object key
+ * @param uploadId multipart operation Id
+ * @param partETags list of partial uploads
+ * @return the result
+ * @throws ObsException on problems.
+ */
+ CompleteMultipartUploadResult completeMultipartUpload(
+ final String destKey, final String uploadId,
+ final List partETags)
+ throws ObsException {
+ Preconditions.checkNotNull(uploadId);
+ Preconditions.checkNotNull(partETags);
+ Preconditions.checkArgument(!partETags.isEmpty(),
+ "No partitions have been uploaded");
+ LOG.debug("Completing multipart upload {} with {} parts", uploadId,
+ partETags.size());
+ // a copy of the list is required, so that the OBS SDK doesn't
+ // attempt to sort an unmodifiable list.
+ return obs.completeMultipartUpload(
+ new CompleteMultipartUploadRequest(bucket, destKey, uploadId,
+ new ArrayList<>(partETags)));
+ }
+
+ /**
+ * Abort a multipart upload operation.
+ *
+ * @param destKey object key
+ * @param uploadId multipart operation Id
+ * @throws ObsException on problems. Immediately execute
+ */
+ void abortMultipartUpload(final String destKey, final String uploadId)
+ throws ObsException {
+ LOG.debug("Aborting multipart upload {}", uploadId);
+ obs.abortMultipartUpload(
+ new AbortMultipartUploadRequest(bucket, destKey, uploadId));
+ }
+
+ /**
+ * Create request for uploading one part of a multipart task.
+ *
+ * @param destKey destination object key
+ * @param uploadId upload id
+ * @param partNumber part number
+ * @param size data size
+ * @param sourceFile source file to be uploaded
+ * @return part upload request
+ */
+ UploadPartRequest newUploadPartRequest(
+ final String destKey,
+ final String uploadId,
+ final int partNumber,
+ final int size,
+ final File sourceFile) {
+ Preconditions.checkNotNull(uploadId);
+
+ Preconditions.checkArgument(sourceFile != null, "Data source");
+ Preconditions.checkArgument(size > 0, "Invalid partition size %s",
+ size);
+ Preconditions.checkArgument(
+ partNumber > 0 && partNumber <= PART_NUMBER);
+
+ LOG.debug("Creating part upload request for {} #{} size {}", uploadId,
+ partNumber, size);
+ UploadPartRequest request = new UploadPartRequest();
+ request.setUploadId(uploadId);
+ request.setBucketName(bucket);
+ request.setObjectKey(destKey);
+ request.setPartSize((long) size);
+ request.setPartNumber(partNumber);
+ request.setFile(sourceFile);
+ if (owner.getSse().isSseCEnable()) {
+ request.setSseCHeader(owner.getSse().getSseCHeader());
+ }
+ return request;
+ }
+
+ /**
+ * Create request for uploading one part of a multipart task.
+ *
+ * @param destKey destination object key
+ * @param uploadId upload id
+ * @param partNumber part number
+ * @param size data size
+ * @param uploadStream upload stream for the part
+ * @return part upload request
+ */
+ UploadPartRequest newUploadPartRequest(
+ final String destKey,
+ final String uploadId,
+ final int partNumber,
+ final int size,
+ final InputStream uploadStream) {
+ Preconditions.checkNotNull(uploadId);
+
+ Preconditions.checkArgument(uploadStream != null, "Data source");
+ Preconditions.checkArgument(size > 0, "Invalid partition size %s",
+ size);
+ Preconditions.checkArgument(
+ partNumber > 0 && partNumber <= PART_NUMBER);
+
+ LOG.debug("Creating part upload request for {} #{} size {}", uploadId,
+ partNumber, size);
+ UploadPartRequest request = new UploadPartRequest();
+ request.setUploadId(uploadId);
+ request.setBucketName(bucket);
+ request.setObjectKey(destKey);
+ request.setPartSize((long) size);
+ request.setPartNumber(partNumber);
+ request.setInput(uploadStream);
+ if (owner.getSse().isSseCEnable()) {
+ request.setSseCHeader(owner.getSse().getSseCHeader());
+ }
+ return request;
+ }
+
+ public String toString(final String destKey) {
+ return "{bucket=" + bucket + ", key='" + destKey + '\'' + '}';
+ }
+
+ /**
+ * PUT an object directly (i.e. not via the transfer manager).
+ *
+ * @param putObjectRequest the request
+ * @return the upload initiated
+ * @throws IOException on problems
+ */
+ PutObjectResult putObject(final PutObjectRequest putObjectRequest)
+ throws IOException {
+ try {
+ return OBSCommonUtils.putObjectDirect(owner, putObjectRequest);
+ } catch (ObsException e) {
+ throw OBSCommonUtils.translateException("put",
+ putObjectRequest.getObjectKey(), e);
+ }
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RenameFailedException.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RenameFailedException.java
new file mode 100644
index 00000000000..b7f7965ebe2
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RenameFailedException.java
@@ -0,0 +1,57 @@
+/*
+ * 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.obs;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+
+/**
+ * Exception to indicate a specific rename failure. The exit code defines the
+ * value returned by {@link OBSFileSystem#rename(Path, Path)}.
+ */
+class RenameFailedException extends PathIOException {
+ /**
+ * Exit code to be returned.
+ */
+ private boolean exitCode = false;
+
+ RenameFailedException(final Path src, final Path optionalDest,
+ final String error) {
+ super(src.toString(), error);
+ setOperation("rename");
+ if (optionalDest != null) {
+ setTargetPath(optionalDest.toString());
+ }
+ }
+
+ public boolean getExitCode() {
+ return exitCode;
+ }
+
+ /**
+ * Set the exit code.
+ *
+ * @param code exit code to raise
+ * @return the exception
+ */
+ public RenameFailedException withExitCode(final boolean code) {
+ this.exitCode = code;
+ return this;
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/SseWrapper.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/SseWrapper.java
new file mode 100644
index 00000000000..d14479c2d85
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/SseWrapper.java
@@ -0,0 +1,87 @@
+/*
+ * 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.obs;
+
+import static org.apache.hadoop.fs.obs.OBSConstants.SSE_KEY;
+import static org.apache.hadoop.fs.obs.OBSConstants.SSE_TYPE;
+
+import com.obs.services.model.SseCHeader;
+import com.obs.services.model.SseKmsHeader;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Wrapper for Server-Side Encryption (SSE).
+ */
+class SseWrapper {
+ /**
+ * SSE-KMS: Server-Side Encryption with Key Management Service.
+ */
+ private static final String SSE_KMS = "sse-kms";
+
+ /**
+ * SSE-C: Server-Side Encryption with Customer-Provided Encryption Keys.
+ */
+ private static final String SSE_C = "sse-c";
+
+ /**
+ * SSE-C header.
+ */
+ private SseCHeader sseCHeader;
+
+ /**
+ * SSE-KMS header.
+ */
+ private SseKmsHeader sseKmsHeader;
+
+ @SuppressWarnings("deprecation")
+ SseWrapper(final Configuration conf) {
+ String sseType = conf.getTrimmed(SSE_TYPE);
+ if (null != sseType) {
+ String sseKey = conf.getTrimmed(SSE_KEY);
+ if (sseType.equalsIgnoreCase(SSE_C) && null != sseKey) {
+ sseCHeader = new SseCHeader();
+ sseCHeader.setSseCKeyBase64(sseKey);
+ sseCHeader.setAlgorithm(
+ com.obs.services.model.ServerAlgorithm.AES256);
+ } else if (sseType.equalsIgnoreCase(SSE_KMS)) {
+ sseKmsHeader = new SseKmsHeader();
+ sseKmsHeader.setEncryption(
+ com.obs.services.model.ServerEncryption.OBS_KMS);
+ sseKmsHeader.setKmsKeyId(sseKey);
+ }
+ }
+ }
+
+ boolean isSseCEnable() {
+ return sseCHeader != null;
+ }
+
+ boolean isSseKmsEnable() {
+ return sseKmsHeader != null;
+ }
+
+ SseCHeader getSseCHeader() {
+ return sseCHeader;
+ }
+
+ SseKmsHeader getSseKmsHeader() {
+ return sseKmsHeader;
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/package-info.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/package-info.java
new file mode 100644
index 00000000000..9e198d32057
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/package-info.java
@@ -0,0 +1,29 @@
+/*
+ * 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 for supporting
+ * HuaweiCloud
+ * Object Storage Service (OBS) as a backend filesystem in Hadoop.
+ *
+ * OBS supports two kinds of buckets: object bucket and posix bucket. Posix
+ * bucket provides more POSIX-like semantics than object bucket, and is
+ * recommended for Hadoop. Object bucket is deprecated for Hadoop.
+ */
+
+package org.apache.hadoop.fs.obs;
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
new file mode 100644
index 00000000000..e77425ab529
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.hadoop.fs.obs.OBSFileSystem
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/site/markdown/index.md b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/site/markdown/index.md
new file mode 100644
index 00000000000..723da89e2be
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/site/markdown/index.md
@@ -0,0 +1,370 @@
+
+
+# OBSA: HuaweiCloud OBS Adapter for Hadoop Support
+
+
+
+## Introduction
+
+The `hadoop-huaweicloud` module provides support for integration with the
+[HuaweiCloud Object Storage Service (OBS)](https://www.huaweicloud.com/en-us/product/obs.html).
+This support comes via the JAR file `hadoop-huaweicloud.jar`.
+
+## Features
+
+* Read and write data stored in a HuaweiCloud OBS account.
+* Reference file system paths using URLs using the `obs` scheme.
+* Present a hierarchical file system view by implementing the standard Hadoop `FileSystem` interface.
+* Support multipart upload for a large file.
+* Can act as a source of data in a MapReduce job, or a sink.
+* Uses HuaweiCloud OBS’s Java SDK with support for latest OBS features and authentication schemes.
+* Tested for scale.
+
+## Limitations
+
+Partial or no support for the following operations :
+
+* Symbolic link operations.
+* Proxy users.
+* File truncate.
+* File concat.
+* File checksum.
+* File replication factor.
+* Extended Attributes(XAttrs) operations.
+* Snapshot operations.
+* Storage policy.
+* Quota.
+* POSIX ACL.
+* Delegation token operations.
+
+## Getting Started
+
+### Packages
+
+OBSA depends upon two JARs, alongside `hadoop-common` and its dependencies.
+
+* `hadoop-huaweicloud` JAR.
+* `esdk-obs-java` JAR.
+
+The versions of `hadoop-common` and `hadoop-huaweicloud` must be identical.
+
+To import the libraries into a Maven build, add `hadoop-huaweicloud` JAR to the
+build dependencies; it will pull in a compatible `esdk-obs-java` JAR.
+
+The `hadoop-huaweicloud` JAR *does not* declare any dependencies other than that
+dependencies unique to it, the OBS SDK JAR. This is simplify excluding/tuning
+Hadoop dependency JARs in downstream applications. The `hadoop-client` or
+`hadoop-common` dependency must be declared.
+
+
+```xml
+
+
+ 3.4.0
+
+
+
+
+ org.apache.hadoop
+ hadoop-client
+ ${hadoop.version}
+
+
+ org.apache.hadoop
+ hadoop-huaweicloud
+ ${hadoop.version}
+
+
+```
+### Accessing OBS URLs
+Before access a URL, OBS implementation classes of Filesystem/AbstractFileSystem and
+a region endpoint where a bucket is located shoud be configured as follows:
+```xml
+
+ fs.obs.impl
+ org.apache.hadoop.fs.obs.OBSFileSystem
+ The OBS implementation class of the Filesystem.
+
+
+
+ fs.AbstractFileSystem.obs.impl
+ org.apache.hadoop.fs.obs.OBS
+ The OBS implementation class of the AbstractFileSystem.
+
+
+
+ fs.obs.endpoint
+ obs.region.myhuaweicloud.com
+ OBS region endpoint where a bucket is located.
+
+```
+
+OBS URLs can then be accessed as follows:
+
+```
+obs:///path
+```
+The scheme `obs` identifies a URL on a Hadoop-compatible file system `OBSFileSystem`
+backed by HuaweiCloud OBS.
+For example, the following
+[FileSystem Shell](../hadoop-project-dist/hadoop-common/FileSystemShell.html)
+commands demonstrate access to a bucket named `mybucket`.
+```bash
+hadoop fs -mkdir obs://mybucket/testDir
+
+hadoop fs -put testFile obs://mybucket/testDir/testFile
+
+hadoop fs -cat obs://mybucket/testDir/testFile
+test file content
+```
+
+For details on how to create a bucket, see
+[**Help Center > Object Storage Service > Getting Started> Basic Operation Procedure**](https://support.huaweicloud.com/intl/en-us/qs-obs/obs_qs_0003.html)
+
+### Authenticating with OBS
+Except when interacting with public OBS buckets, the OBSA client
+needs the credentials needed to interact with buckets.
+The client supports multiple authentication mechanisms. The simplest authentication mechanisms is
+to provide OBS access key and secret key as follows.
+```xml
+
+ fs.obs.access.key
+ OBS access key.
+ Omit for provider-based authentication.
+
+
+
+ fs.obs.secret.key
+ OBS secret key.
+ Omit for provider-based authentication.
+
+```
+
+**Do not share access key, secret key, and session token. They must be kept secret.**
+
+Custom implementations
+of `com.obs.services.IObsCredentialsProvider` (see [**Creating an Instance of ObsClient**](https://support.huaweicloud.com/intl/en-us/sdk-java-devg-obs/en-us_topic_0142815570.html)) or
+`org.apache.hadoop.fs.obs.BasicSessionCredential` may also be used for authentication.
+
+```xml
+
+ fs.obs.security.provider
+
+ Class name of security provider class which implements
+ com.obs.services.IObsCredentialsProvider, which will
+ be used to construct an OBS client instance as an input parameter.
+
+
+
+
+ fs.obs.credentials.provider
+
+ lass nameCof credential provider class which implements
+ org.apache.hadoop.fs.obs.BasicSessionCredential,
+ which must override three APIs: getOBSAccessKeyId(),
+ getOBSSecretKey(), and getSessionToken().
+
+
+```
+
+## General OBSA Client Configuration
+
+All OBSA client options are configured with options with the prefix `fs.obs.`.
+
+```xml
+
+ fs.obs.connection.ssl.enabled
+ false
+ Enable or disable SSL connections to OBS.
+
+
+
+ fs.obs.connection.maximum
+ 1000
+ Maximum number of simultaneous connections to OBS.
+
+
+
+ fs.obs.connection.establish.timeout
+ 120000
+ Socket connection setup timeout in milliseconds.
+
+
+
+ fs.obs.connection.timeout
+ 120000
+ Socket connection timeout in milliseconds.
+
+
+
+ fs.obs.idle.connection.time
+ 30000
+ Socket idle connection time.
+
+
+
+ fs.obs.max.idle.connections
+ 1000
+ Maximum number of socket idle connections.
+
+
+
+ fs.obs.socket.send.buffer
+ 256 * 1024
+ Socket send buffer to be used in OBS SDK. Represented in bytes.
+
+
+
+ fs.obs.socket.recv.buffer
+ 256 * 1024
+ Socket receive buffer to be used in OBS SDK. Represented in bytes.
+
+
+
+ fs.obs.threads.keepalivetime
+ 60
+ Number of seconds a thread can be idle before being
+ terminated in thread pool.
+
+
+
+ fs.obs.threads.max
+ 20
+ Maximum number of concurrent active (part)uploads,
+ which each use a thread from thread pool.
+
+
+
+ fs.obs.max.total.tasks
+ 20
+ Number of (part)uploads allowed to the queue before
+ blocking additional uploads.
+
+
+
+ fs.obs.delete.threads.max
+ 20
+ Max number of delete threads.
+
+
+
+ fs.obs.multipart.size
+ 104857600
+ Part size for multipart upload.
+
+
+
+
+ fs.obs.multiobjectdelete.maximum
+ 1000
+ Max number of objects in one multi-object delete call.
+
+
+
+
+ fs.obs.fast.upload.buffer
+ disk
+ Which buffer to use. Default is `disk`, value may be
+ `disk` | `array` | `bytebuffer`.
+
+
+
+
+ fs.obs.buffer.dir
+ dir1,dir2,dir3
+ Comma separated list of directories that will be used to buffer file
+ uploads to. This option takes effect only when the option 'fs.obs.fast.upload.buffer'
+ is set to 'disk'.
+
+
+
+
+ fs.obs.fast.upload.active.blocks
+ 4
+ Maximum number of blocks a single output stream can have active
+ (uploading, or queued to the central FileSystem instance's pool of queued
+ operations).
+
+
+
+
+ fs.obs.readahead.range
+ 1024 * 1024
+ Bytes to read ahead during a seek() before closing and
+ re-opening the OBS HTTP connection.
+
+
+
+ fs.obs.read.transform.enable
+ true
+ Flag indicating if socket connections can be reused by
+ position read. Set `false` only for HBase.
+
+
+
+ fs.obs.list.threads.core
+ 30
+ Number of core list threads.
+
+
+
+ fs.obs.list.threads.max
+ 60
+ Maximum number of list threads.
+
+
+
+ fs.obs.list.workqueue.capacity
+ 1024
+ Capacity of list work queue.
+
+
+
+ fs.obs.list.parallel.factor
+ 30
+ List parallel factor.
+
+
+
+ fs.obs.trash.enable
+ false
+ Switch for the fast delete.
+
+
+
+ fs.obs.trash.dir
+ The fast delete recycle directory.
+
+
+
+ fs.obs.block.size
+ 128 * 1024 * 1024
+ Default block size for OBS FileSystem.
+
+
+```
+
+## Testing the hadoop-huaweicloud Module
+The `hadoop-huaweicloud` module includes a full suite of unit tests.
+Most of the tests will run against the HuaweiCloud OBS. To run these
+tests, please create `src/test/resources/auth-keys.xml` with OBS account
+information mentioned in the above sections and the following properties.
+
+```xml
+
+ fs.contract.test.fs.obs
+ obs://obsfilesystem-bucket
+
+```
\ No newline at end of file
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/site/resources/css/site.css b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/site/resources/css/site.css
new file mode 100644
index 00000000000..7315db31e53
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/site/resources/css/site.css
@@ -0,0 +1,29 @@
+/*
+* 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.
+*/
+#banner {
+ height: 93px;
+ background: none;
+}
+
+#bannerLeft img {
+ margin-left: 30px;
+ margin-top: 10px;
+}
+
+#bannerRight img {
+ margin: 17px;
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSContract.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSContract.java
new file mode 100644
index 00000000000..ab9d6dae4cc
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSContract.java
@@ -0,0 +1,72 @@
+/*
+ * 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.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
+
+/**
+ * The contract of OBS: only enabled if the test bucket is provided.
+ */
+public class OBSContract extends AbstractBondedFSContract {
+
+ public static final String CONTRACT_XML = "contract/obs.xml";
+
+ private static final String CONTRACT_ENABLE_KEY =
+ "fs.obs.test.contract.enable";
+
+ private static final boolean CONTRACT_ENABLE_DEFAULT = false;
+
+ public OBSContract(Configuration conf) {
+ super(conf);
+ //insert the base features
+ addConfResource(CONTRACT_XML);
+ }
+
+ @Override
+ public String getScheme() {
+ return "obs";
+ }
+
+ @Override
+ public Path getTestPath() {
+ return OBSTestUtils.createTestPath(super.getTestPath());
+ }
+
+ public synchronized static boolean isContractTestEnabled() {
+ Configuration conf = null;
+ boolean isContractTestEnabled = true;
+
+ if (conf == null) {
+ conf = getConfiguration();
+ }
+ String fileSystem = conf.get(OBSTestConstants.TEST_FS_OBS_NAME);
+ if (fileSystem == null || fileSystem.trim().length() == 0) {
+ isContractTestEnabled = false;
+ }
+ return isContractTestEnabled;
+ }
+
+ public synchronized static Configuration getConfiguration() {
+ Configuration newConf = new Configuration();
+ newConf.addResource(CONTRACT_XML);
+ return newConf;
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestConstants.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestConstants.java
new file mode 100644
index 00000000000..4fcff35b9c9
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestConstants.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.obs;
+
+/**
+ * Constants for OBS Testing.
+ */
+
+final class OBSTestConstants {
+
+ private OBSTestConstants(){
+ }
+
+ /**
+ * Name of the test filesystem.
+ */
+ static final String TEST_FS_OBS_NAME = "fs.contract.test.fs.obs";
+
+ /**
+ * Fork ID passed down from maven if the test is running in parallel.
+ */
+ static final String TEST_UNIQUE_FORK_ID = "test.unique.fork.id";
+
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestUtils.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestUtils.java
new file mode 100644
index 00000000000..9496617256a
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestUtils.java
@@ -0,0 +1,119 @@
+/*
+ * 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.obs;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.junit.internal.AssumptionViolatedException;
+
+import java.io.IOException;
+import java.net.URI;
+
+import static org.apache.hadoop.fs.obs.OBSTestConstants.*;
+import static org.apache.hadoop.fs.obs.OBSConstants.*;
+
+/**
+ * Utilities for the OBS tests.
+ */
+public final class OBSTestUtils {
+
+ /**
+ * Create the test filesystem.
+ *
+ * If the test.fs.obs.name property is not set, this will trigger a JUnit
+ * failure.
+ *
+ * Multipart purging is enabled.
+ *
+ * @param conf configuration
+ * @return the FS
+ * @throws IOException IO Problems
+ * @throws AssumptionViolatedException if the FS is not named
+ */
+ public static OBSFileSystem createTestFileSystem(Configuration conf)
+ throws IOException {
+ return createTestFileSystem(conf, false);
+ }
+
+ /**
+ * Create the test filesystem with or without multipart purging
+ *
+ * If the test.fs.obs.name property is not set, this will trigger a JUnit
+ * failure.
+ *
+ * @param conf configuration
+ * @param purge flag to enable Multipart purging
+ * @return the FS
+ * @throws IOException IO Problems
+ * @throws AssumptionViolatedException if the FS is not named
+ */
+ @SuppressWarnings("deprecation")
+ public static OBSFileSystem createTestFileSystem(Configuration conf,
+ boolean purge)
+ throws IOException {
+
+ String fsname = conf.getTrimmed(TEST_FS_OBS_NAME, "");
+
+ boolean liveTest = !StringUtils.isEmpty(fsname);
+ URI testURI = null;
+ if (liveTest) {
+ testURI = URI.create(fsname);
+ liveTest = testURI.getScheme().equals(OBSConstants.OBS_SCHEME);
+ }
+ if (!liveTest) {
+ // This doesn't work with our JUnit 3 style test cases, so instead we'll
+ // make this whole class not run by default
+ throw new AssumptionViolatedException(
+ "No test filesystem in " + TEST_FS_OBS_NAME);
+ }
+ OBSFileSystem fs1 = new OBSFileSystem();
+ //enable purging in tests
+ if (purge) {
+ conf.setBoolean(PURGE_EXISTING_MULTIPART, true);
+ // but a long delay so that parallel multipart tests don't
+ // suddenly start timing out
+ conf.setInt(PURGE_EXISTING_MULTIPART_AGE, 30 * 60);
+ }
+ fs1.initialize(testURI, conf);
+ return fs1;
+ }
+
+ /**
+ * Create a test path, using the value of
+ * {@link OBSTestConstants#TEST_UNIQUE_FORK_ID}
+ * if it is set.
+ *
+ * @param defVal default value
+ * @return a path
+ */
+ public static Path createTestPath(Path defVal) {
+ String testUniqueForkId = System.getProperty(
+ OBSTestConstants.TEST_UNIQUE_FORK_ID);
+ return testUniqueForkId == null ? defVal :
+ new Path("/" + testUniqueForkId, "test");
+ }
+
+ /**
+ * This class should not be instantiated.
+ */
+ private OBSTestUtils() {
+ }
+
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractAppend.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractAppend.java
new file mode 100644
index 00000000000..a4fb8153e7c
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractAppend.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.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractAppendTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.Assume;
+
+/**
+ * Append test cases on obs file system.
+ */
+public class TestOBSContractAppend extends AbstractContractAppendTest {
+
+ @Override
+ protected AbstractFSContract createContract(final Configuration conf) {
+ return new OBSContract(conf);
+ }
+
+ @Override
+ public void testRenameFileBeingAppended() {
+ Assume.assumeTrue("unsupport.", false);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractCreate.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractCreate.java
new file mode 100644
index 00000000000..d3966a13b95
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractCreate.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.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.Assume;
+
+/**
+ * Create test cases on obs file system.
+ */
+public class TestOBSContractCreate extends AbstractContractCreateTest {
+
+ @Override
+ protected AbstractFSContract createContract(final Configuration conf) {
+ return new OBSContract(conf);
+ }
+
+ @Override
+ public void testCreatedFileIsImmediatelyVisible() {
+ Assume.assumeTrue("unsupport.", false);
+ }
+
+ @Override
+ public void testCreatedFileIsVisibleOnFlush() {
+ Assume.assumeTrue("unsupport", false);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractDelete.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractDelete.java
new file mode 100644
index 00000000000..9dd67ad779b
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractDelete.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Delete test cases on obs file system.
+ */
+public class TestOBSContractDelete extends AbstractContractDeleteTest {
+
+ @Override
+ protected AbstractFSContract createContract(final Configuration conf) {
+ return new OBSContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractGetFileStatus.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractGetFileStatus.java
new file mode 100644
index 00000000000..15ffd97e090
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractGetFileStatus.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Get file status test cases on obs file system.
+ */
+public class TestOBSContractGetFileStatus extends
+ AbstractContractGetFileStatusTest {
+
+ @Override
+ protected AbstractFSContract createContract(
+ final Configuration conf) {
+ return new OBSContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractMkdir.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractMkdir.java
new file mode 100644
index 00000000000..e06ad860e21
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractMkdir.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Mkdir test cases on obs file system.
+ */
+public class TestOBSContractMkdir extends AbstractContractMkdirTest {
+
+ @Override
+ protected AbstractFSContract createContract(final Configuration conf) {
+ return new OBSContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractOpen.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractOpen.java
new file mode 100644
index 00000000000..c8641dfd627
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractOpen.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Open test cases on obs file system.
+ */
+public class TestOBSContractOpen extends AbstractContractOpenTest {
+
+ @Override
+ protected AbstractFSContract createContract(final Configuration conf) {
+ return new OBSContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractRename.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractRename.java
new file mode 100644
index 00000000000..25502a23f27
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractRename.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.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.Assume;
+
+/**
+ * Rename test cases on obs file system.
+ */
+public class TestOBSContractRename extends AbstractContractRenameTest {
+
+ @Override
+ protected AbstractFSContract createContract(final Configuration conf) {
+ return new OBSContract(conf);
+ }
+
+ @Override
+ public void testRenameFileUnderFileSubdir() {
+ Assume.assumeTrue("unsupport.", false);
+ }
+
+ @Override
+ public void testRenameFileUnderFile() {
+ Assume.assumeTrue("unsupport.", false);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractRootDir.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractRootDir.java
new file mode 100644
index 00000000000..ba961a300ef
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractRootDir.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Root directory test cases on obs file system.
+ */
+public class TestOBSContractRootDir extends AbstractContractRootDirectoryTest {
+
+ @Override
+ protected AbstractFSContract createContract(final Configuration conf) {
+ return new OBSContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractSeek.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractSeek.java
new file mode 100644
index 00000000000..48751ea6696
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractSeek.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Seek test cases on obs file system.
+ */
+public class TestOBSContractSeek extends AbstractContractSeekTest {
+
+ @Override
+ protected AbstractFSContract createContract(final Configuration conf) {
+ return new OBSContract(conf);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFSMainOperations.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFSMainOperations.java
new file mode 100644
index 00000000000..b62023b6424
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFSMainOperations.java
@@ -0,0 +1,93 @@
+/*
+ * 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.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.TestFSMainOperationsLocalFileSystem;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+
+/**
+ *
+ * A collection of tests for the {@link FileSystem}. This test should be used
+ * for testing an instance of FileSystem that has been initialized to a specific
+ * default FileSystem such a LocalFileSystem, HDFS,OBS, etc.
+ *
+ *
+ * To test a given {@link FileSystem} implementation create a subclass of this
+ * test and override {@link #setUp()} to initialize the fSys
{@link
+ * FileSystem} instance variable.
+ *
+ * Since this a junit 4 you can also do a single setup before the start of any
+ * tests. E.g.
+ *
+ *
+ *
+ */
+public class TestOBSFSMainOperations extends
+ TestFSMainOperationsLocalFileSystem {
+
+ @Override
+ @Before
+ public void setUp() throws Exception {
+ skipTestCheck();
+ Configuration conf = new Configuration();
+ conf.addResource(OBSContract.CONTRACT_XML);
+ fSys = OBSTestUtils.createTestFileSystem(conf);
+ }
+
+ @Override
+ public void testWorkingDirectory() {
+ Assume.assumeTrue("unspport.", false);
+ }
+
+ @Override
+ public void testListStatusThrowsExceptionForUnreadableDir() {
+ Assume.assumeTrue("unspport.", false);
+ }
+
+ @Override
+ public void testRenameDirectoryToItself() {
+ Assume.assumeTrue("unspport.", false);
+ }
+
+ @Override
+ public void testGlobStatusThrowsExceptionForUnreadableDir() {
+ Assume.assumeTrue("unspport.", false);
+ }
+
+ @Override
+ public void testRenameFileToItself() {
+ Assume.assumeTrue("unspport.", false);
+ }
+
+ @Override
+ @After
+ public void tearDown() throws Exception {
+ if(fSys != null) {
+ super.tearDown();
+ }
+ }
+
+ public void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextCreateMkdir.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextCreateMkdir.java
new file mode 100644
index 00000000000..7860f356aa3
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextCreateMkdir.java
@@ -0,0 +1,75 @@
+/*
+ * 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.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegateToFileSystem;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileContextCreateMkdirBaseTest;
+import org.apache.hadoop.fs.FileContextTestHelper;
+import org.apache.hadoop.fs.FileSystem;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+
+import java.net.URI;
+import java.util.UUID;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+/**
+ * File context create mkdir test cases on obs file system.
+ */
+public class TestOBSFileContextCreateMkdir extends
+ FileContextCreateMkdirBaseTest {
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+
+ @SuppressFBWarnings("ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD")
+ @Override
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration();
+ conf.addResource(OBSContract.CONTRACT_XML);
+ String fileSystem = conf.get(OBSTestConstants.TEST_FS_OBS_NAME);
+ if (fileSystem == null || fileSystem.trim().length() == 0) {
+ throw new Exception("Default file system not configured.");
+ }
+
+ URI uri = new URI(fileSystem);
+ FileSystem fs = OBSTestUtils.createTestFileSystem(conf);
+ if (fc == null) {
+ this.fc = FileContext.getFileContext(new DelegateToFileSystem(uri, fs,
+ conf, fs.getScheme(), false) {
+ }, conf);
+ }
+ super.setUp();
+ }
+
+ @Override
+ protected FileContextTestHelper createFileContextHelper() {
+ // On Windows, root directory path is created from local running
+ // directory.
+ // obs does not support ':' as part of the path which results in
+ // failure.
+ return new FileContextTestHelper(UUID.randomUUID().toString());
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextMainOperations.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextMainOperations.java
new file mode 100644
index 00000000000..ef6d31215f7
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextMainOperations.java
@@ -0,0 +1,77 @@
+/*
+ * 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.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegateToFileSystem;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileContextMainOperationsBaseTest;
+import org.apache.hadoop.fs.FileSystem;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.net.URI;
+
+/**
+ * Rename test cases on obs file system.
+ */
+public class TestOBSFileContextMainOperations extends
+ FileContextMainOperationsBaseTest {
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @edu.umd.cs.findbugs.annotations.SuppressFBWarnings(
+ "ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD")
+ @Override
+ public void setUp() throws Exception {
+ Configuration conf = new Configuration();
+ conf.addResource(OBSContract.CONTRACT_XML);
+ String fileSystem = conf.get(OBSTestConstants.TEST_FS_OBS_NAME);
+ if (fileSystem == null || fileSystem.trim().length() == 0) {
+ throw new Exception("Default file system not configured.");
+ }
+
+ URI uri = new URI(fileSystem);
+ FileSystem fs = OBSTestUtils.createTestFileSystem(conf);
+ fc = FileContext.getFileContext(new DelegateToFileSystem(uri, fs,
+ conf, fs.getScheme(), false) {
+ }, conf);
+ super.setUp();
+ }
+
+ @Override
+ protected boolean listCorruptedBlocksSupported() {
+ return false;
+ }
+
+ @Override
+ @Test
+ public void testSetVerifyChecksum() {
+ Assume.assumeTrue("unsupport.", false);
+ }
+
+ @Override
+ public void testMkdirsFailsForSubdirectoryOfExistingFile() {
+ Assume.assumeTrue("unsupport.", false);
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextURI.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextURI.java
new file mode 100644
index 00000000000..b3f523092a9
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextURI.java
@@ -0,0 +1,88 @@
+/*
+ * 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.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegateToFileSystem;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileContextURIBase;
+import org.apache.hadoop.fs.FileSystem;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+
+import java.net.URI;
+
+/**
+ *
+ * A collection of tests for the {@link FileContext} to test path names passed
+ * as URIs. This test should be used for testing an instance of FileContext that
+ * has been initialized to a specific default FileSystem such a LocalFileSystem,
+ * HDFS,OBS, etc, and where path names are passed that are URIs in a different
+ * FileSystem.
+ *
+ *
+ *
+ * To test a given {@link FileSystem} implementation create a subclass of this
+ * test and override {@link #setUp()} to initialize the fc1
and
+ * fc2
+ *
+ * The tests will do operations on fc1 that use a URI in fc2
+ *
+ * {@link FileContext} instance variable.
+ *
+ */
+public class TestOBSFileContextURI extends FileContextURIBase {
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Override
+ public void setUp() throws Exception {
+ Configuration conf = new Configuration();
+ conf.addResource(OBSContract.CONTRACT_XML);
+ String fileSystem = conf.get(OBSTestConstants.TEST_FS_OBS_NAME);
+ if (fileSystem == null || fileSystem.trim().length() == 0) {
+ throw new Exception("Default file system not configured.");
+ }
+
+ URI uri = new URI(fileSystem);
+ FileSystem fs = OBSTestUtils.createTestFileSystem(conf);
+ fc1 = FileContext.getFileContext(new DelegateToFileSystem(uri, fs,
+ conf, fs.getScheme(), false) {
+ }, conf);
+
+ fc2 = FileContext.getFileContext(new DelegateToFileSystem(uri, fs,
+ conf, fs.getScheme(), false) {
+ }, conf);
+ super.setUp();
+ }
+
+ @Override
+ public void testMkdirsFailsForSubdirectoryOfExistingFile() {
+ Assume.assumeTrue("unsupport.", false);
+ }
+
+ @Override
+ public void testFileStatus() {
+ Assume.assumeTrue("unsupport.", false);
+ }
+
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextUtil.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextUtil.java
new file mode 100644
index 00000000000..1404e06a452
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextUtil.java
@@ -0,0 +1,68 @@
+/*
+ * 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.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegateToFileSystem;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileContextUtilBase;
+import org.apache.hadoop.fs.FileSystem;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+
+import java.net.URI;
+
+/**
+ *
+ * A collection of Util tests for the {@link FileContext#util()}. This test
+ * should be used for testing an instance of {@link FileContext#util()} that has
+ * been initialized to a specific default FileSystem such a LocalFileSystem,
+ * HDFS,OBS, etc.
+ *
+ *
+ * To test a given {@link FileSystem} implementation create a subclass of this
+ * test and override {@link #setUp()} to initialize the fc
{@link
+ * FileContext} instance variable.
+ *
+ *
+ */
+public class TestOBSFileContextUtil extends FileContextUtilBase {
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Override
+ public void setUp() throws Exception {
+ Configuration conf = new Configuration();
+ conf.addResource(OBSContract.CONTRACT_XML);
+ String fileSystem = conf.get(OBSTestConstants.TEST_FS_OBS_NAME);
+ if (fileSystem == null || fileSystem.trim().length() == 0) {
+ throw new Exception("Default file system not configured.");
+ }
+
+ URI uri = new URI(fileSystem);
+ FileSystem fs = OBSTestUtils.createTestFileSystem(conf);
+ fc = FileContext.getFileContext(new DelegateToFileSystem(uri, fs,
+ conf, fs.getScheme(), false) {
+ }, conf);
+ super.setUp();
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileSystemContract.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileSystemContract.java
new file mode 100644
index 00000000000..defd3ba40f2
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileSystemContract.java
@@ -0,0 +1,59 @@
+/*
+ * 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.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystemContractBaseTest;
+import org.junit.Assume;
+import org.junit.Before;
+
+
+/**
+ * Tests a live OBS system. If your keys and bucket aren't specified, all tests
+ * are marked as passed.
+ *
+ * This uses BlockJUnit4ClassRunner because FileSystemContractBaseTest from
+ * TestCase which uses the old Junit3 runner that doesn't ignore assumptions
+ * properly making it impossible to skip the tests if we don't have a valid
+ * bucket.
+ **/
+public class TestOBSFileSystemContract extends FileSystemContractBaseTest {
+
+ @Before
+ public void setUp() throws Exception {
+ skipTestCheck();
+ Configuration conf = new Configuration();
+ conf.addResource(OBSContract.CONTRACT_XML);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ }
+
+ @Override
+ public void testMkdirsWithUmask() {
+ Assume.assumeTrue("unspport.", false);
+ }
+
+ @Override
+ public void testRenameRootDirForbidden() {
+ Assume.assumeTrue("unspport.", false);
+ }
+
+ public void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+}
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/contract/obs.xml b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/contract/obs.xml
new file mode 100644
index 00000000000..30b2cf04234
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/contract/obs.xml
@@ -0,0 +1,139 @@
+
+
+
+
+ fs.contract.test.root-tests-enabled
+ true
+
+
+
+ fs.contract.test.supports-concat
+ true
+
+
+
+ fs.contract.rename-returns-false-if-source-missing
+ true
+
+
+
+ fs.contract.test.random-seek-count
+ 10
+
+
+
+ fs.contract.is-case-sensitive
+ true
+
+
+
+ fs.contract.rename-returns-true-if-dest-exists
+ false
+
+
+
+ fs.contract.rename-returns-true-if-source-missing
+ false
+
+
+
+ fs.contract.rename-creates-dest-dirs
+ false
+
+
+
+ fs.contract.rename-remove-dest-if-empty-dir
+ false
+
+
+
+ fs.contract.supports-settimes
+ true
+
+
+
+ fs.contract.supports-append
+ true
+
+
+
+ fs.contract.supports-atomic-directory-delete
+ true
+
+
+
+ fs.contract.supports-atomic-rename
+ true
+
+
+
+ fs.contract.supports-block-locality
+ true
+
+
+
+ fs.contract.supports-concat
+ true
+
+
+
+ fs.contract.supports-seek
+ true
+
+
+
+ fs.contract.supports-seek-on-closed-file
+ true
+
+
+
+ fs.contract.rejects-seek-past-eof
+ true
+
+
+
+ fs.contract.supports-available-on-closed-file
+ true
+
+
+
+ fs.contract.supports-strict-exceptions
+ false
+
+
+
+ fs.contract.supports-unix-permissions
+ true
+
+
+
+ fs.contract.rename-overwrites-dest
+ false
+
+
+
+ fs.contract.supports-append
+ true
+
+
+
+ fs.contract.supports-getfilestatus
+ true
+
+
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/core-site.xml b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/core-site.xml
new file mode 100644
index 00000000000..2058293646e
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/core-site.xml
@@ -0,0 +1,136 @@
+
+
+
+
+
+
+
+
+ hadoop.tmp.dir
+ target/build/test
+ A base for other temporary directories.
+ true
+
+
+
+ hadoop.security.authentication
+ simple
+
+
+
+ fs.obs.impl
+ org.apache.hadoop.fs.obs.OBSFileSystem
+ The implementation class of the obs Filesystem
+
+
+ fs.obs.connection.establish.timeout
+ 60000
+
+
+ fs.obs.connection.timeout
+ 60000
+
+
+ fs.obs.idle.connection.time
+ 30000
+
+
+ fs.obs.max.idle.connections
+ 10
+
+
+ fs.obs.connection.maximum
+ 1000
+
+
+ fs.obs.attempts.maximum
+ 5
+
+
+ fs.obs.upload.stream.retry.buffer.size
+ 524288
+
+
+ fs.obs.read.buffer.size
+ 8192
+
+
+ fs.obs.write.buffer.size
+ 8192
+
+
+ fs.obs.socket.recv.buffer
+ -1
+
+
+ fs.obs.socket.send.buffer
+ -1
+
+
+ fs.obs.keep.alive
+ true
+
+
+ fs.obs.validate.certificate
+ false
+
+
+ fs.obs.verify.response.content.type
+ true
+
+
+ fs.obs.strict.hostname.verification
+ false
+
+
+ fs.obs.cname
+ false
+
+
+
+ fs.obs.test.local.path
+ /uplod_file
+
+
+
+ fs.obs.fast.upload
+ true
+
+
+ fs.obs.multipart.size
+ 10485760
+
+
+ fs.obs.experimental.input.fadvise
+ random
+
+
+
+
+
+
+
+
diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/log4j.properties b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/log4j.properties
new file mode 100644
index 00000000000..6c0829f4ee6
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/log4j.properties
@@ -0,0 +1,23 @@
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=error,stdout
+log4j.threshold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n
+
+log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
+
+# for debugging low level obs operations, uncomment this line
+log4j.logger.org.apache.hadoop.fs.obs=ERROR
diff --git a/hadoop-cloud-storage-project/pom.xml b/hadoop-cloud-storage-project/pom.xml
index da0d88a8117..8df6bb41e90 100644
--- a/hadoop-cloud-storage-project/pom.xml
+++ b/hadoop-cloud-storage-project/pom.xml
@@ -32,6 +32,7 @@
hadoop-cloud-storage
hadoop-cos
+ hadoop-huaweicloud
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index f3fa47505aa..ff19abce837 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -676,6 +676,12 @@
${hadoop.version}
+
+ org.apache.hadoop
+ hadoop-huaweicloud
+ ${hadoop.version}
+
+
org.apache.hadoop
hadoop-kms
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index 86949b0404c..50f75873bf8 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -179,6 +179,7 @@
href="hadoop-azure-datalake/index.html"/>
+