objMeta) {
+ FsPermission fsPermission = null;
+ try {
+ fsPermission = Optional.ofNullable(objMeta.get("permission")).map(Object::toString)
+ .map(Short::valueOf).map(FsPermission::new).orElse(null);
+ } catch (NumberFormatException e) {
+ LOG.debug("File {} permission is invalid, use default permission.", status.getPath());
}
+
+ status.setAccessControlAttr(
+ Optional.ofNullable(objMeta.get("user")).map(Object::toString).orElse(fs.getShortUserName()),
+ Optional.ofNullable(objMeta.get("group")).map(Object::toString).orElse(fs.getShortUserName()),
+ fsPermission
+ );
}
}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSConstants.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSConstants.java
index b06940b..9669a30 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSConstants.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSConstants.java
@@ -430,6 +430,13 @@ public final class OBSConstants {
*/
static final String USER_AGENT_PREFIX = "fs.obs.user.agent.prefix";
+ public static final String DELEGATION_TOKEN_ONLY = "fs.obs.delegation.token.only";
+
+ public static final boolean DEFAULT_DELEGATION_TOKEN_ONLY = false;
+
+ public static final String DELEGATION_TOKEN_PROVIDERS = "fs.obs.delegation.token.providers";
+
+ public static final String DEFAULT_DELEGATION_TOKEN_PROVIDERS = "";
/**
* what read policy to use. Default is {@link #READAHEAD_POLICY_PRIMARY} Value:
* {@value}
@@ -443,6 +450,17 @@ public final class OBSConstants {
@InterfaceStability.Unstable
public static final String READAHEAD_POLICY_ADVANCE = "advance";
+ @InterfaceStability.Unstable
+ public static final String READAHEAD_POLICY_MEMARTSCC = "memArtsCC";
+
+ public static final String MEMARTSCC_LOCALITY_ENABLE = "fs.obs.memartscc.locality.enable";
+
+ public static final String CACHE_CONFIG_PREFIX = "fs.obs.cache.config.prefix";
+
+ public static final String DEFAULT_CACHE_CONFIG_PREFIX = "fs.obs.memartscc.config";
+
+ public static final boolean DEFAULT_MEMARTSCC_LOCALITY_ENABLE = false;
+
/**
* Read ahead buffer size to prevent connection re-establishments.
*/
@@ -453,10 +471,70 @@ public final class OBSConstants {
*/
public static final long DEFAULT_READAHEAD_RANGE = 1024 * 1024;
+ /**
+ * the prefetch range sent to memartscc
+ */
+ public static final String MEMARTSCC_READAHEAD_RANGE = "fs.obs.memartscc.readahead.range";
+
+ public static final long DEFAULT_MEMARTSCC_READAHEAD_RANGE = 8 * 1024 * 1024;
+
+ public static final String MEMARTSCC_BUFFER_SIZE = "fs.obs.memartscc.buffer.size";
+
+ public static final int DEFAULT_MEMARTSCC_BUFFER_SIZE = 8192;
+
+ public static final String MEMARTSCC_DIRECTBUFFER_SIZE = "fs.obs.memartscc.directbuffer.size";
+
+ public static final int DEFAULT_MEMARTSCC_DIRECTBUFFER_SIZE = 1024 * 1024;
+
+ public static final String MEMARTSCC_AKSK_REFRESH_INTERVAL = "fs.obs.memartscc.aksk.refresh.interval";
+
+ public static final int DEFAULT_MEMARTSCC_AKSK_REFRESH_INTERVAL = 60; // 60 sec
+
+ public static final String MEMARTSCC_CACHE_IMPL = "fs.obs.memartscc.cache.impl";
+
public static final String READAHEAD_MAX_NUM = "fs.obs.readahead.max.number";
public static final int DEFAULT_READAHEAD_MAX_NUM = 4;
+ public static final String MEMARTSCC_INPUTSTREAM_BUFFER_TYPE = "fs.obs.memartscc.inputstream.buffer.type";
+
+ public static final String MEMARTSCC_INPUTSTREAM_BUFFER_TYPE_BIND = "bind";
+
+ public static final String MEMARTSCC_INPUTSTREAM_BUFFER_TYPE_POOL = "pool";
+
+ public static final String MEMARTSCC_INPUTSTREAM_BUFFER_POOL_MAX_SIZE = "fs.obs.memartscc.inputstream.buffer.pool.maxsize";
+
+ public static final int MEMARTSCC_INPUTSTREAM_BUFFER_POOL_DEFAULT_MAX_SIZE = 128;
+
+ public static final String MEMARTSCC_INPUTSTREAM_BUFFER_BORROW_TIMEOUT = "fs.obs.memartscc.inputstream.buffer.poll.timeout";
+
+ public static final int MEMARTSCC_INPUTSTREAM_BUFFER_BORROW_DEFAULT_TIMEOUT = 5000; // ms
+
+ public static final String MEMARTSCC_PYSPARK_OPTIMIZED = "fs.obs.memartscc.pyspark.optimized";
+
+ public static final boolean DEFAULT_MEMARTSCC_PYSPARK_OPTIMIZED = true;
+
+ public static final String MEMARTSCC_TRAFFIC_REPORT_ENABLE = "fs.obs.memartscc.inputstream.statistics.report.enable";
+
+ public static final boolean DEFAULT_MEMARTSCC_TRAFFIC_REPORT_ENABLE = false;
+
+ /**
+ * The interval of reporting traffic statistics to CC SDK, unit: seconds
+ */
+ public static final String MEMARTSCC_TRAFFIC_REPORT_INTERVAL = "fs.obs.memartscc.inputstream.statistics.report.interval";
+
+ public static final long MEMARTSCC_TRAFFIC_REPORT_DEFAULT_INTERVAL = 30;
+
+ /*
+ * memartscc duplications num, set to 1 for now
+ * */
+ public static final int MAX_DUPLICATION_NUM = 1;
+
+ /*
+ * memartscc errorcode getShardInfo success
+ * */
+ public static final int GET_SHARD_INFO_SUCCESS = 0;
+
/**
* Flag indicating if
* {@link OBSInputStream#read(long, byte[], int, int)}
@@ -466,6 +544,21 @@ public final class OBSConstants {
*/
public static final String READAHEAD_TRANSFORM_ENABLE = "fs.obs.read.transform.enable";
+ /**
+ * obs file system permission mode settings, only take effect on posix file system.
+ */
+ public static final String PERMISSIONS_MODE = "fs.obs.permissions.mode";
+
+ /**
+ * default permission mode, doesn't support permissions.
+ */
+ public static final String DEFAULT_PERMISSIONS_MODE = "none";
+
+ /**
+ * disguise permission mode, support file owner, group, permission attribute.
+ */
+ public static final String PERMISSIONS_MODE_DISGUISE = "disguise";
+
/**
* OBS client factory implementation class.
*/
@@ -654,29 +747,47 @@ public final class OBSConstants {
static final int DEFAULT_LIST_PARALLEL_FACTOR = 30;
/**
- * Switch for the fast delete.
+ * Multi list contentsummary parallel factor
*/
- static final String TRASH_ENABLE = "fs.obs.trash.enable";
+ static final String MULTILISTCS_PARALLEL_FACTOR = "fs.obs.multilistcs.parallel.factor";
/**
- * Enable obs content summary or not.
+ * Default value of {@link #MULTILISTCS_PARALLEL_FACTOR}.
*/
- static final String OBS_CONTENT_SUMMARY_ENABLE = "fs.obs.content.summary.enable";
+ static final int DEFAULT_MULTILISTCS_PARALLEL_FACTOR = 30;
/**
- * Enable obs client dfs list or not.
+ * Switch for the fast delete.
*/
- static final String OBS_CLIENT_DFS_LIST_ENABLE = "fs.obs.client.dfs.list.enable";
+ static final String FAST_DELETE_ENABLE = "fs.obs.trash.enable";
/**
* Default trash : false.
*/
- static final boolean DEFAULT_TRASH = false;
+ static final boolean DEFAULT_FAST_DELETE_ENABLE = false;
/**
* The fast delete recycle directory.
*/
- static final String TRASH_DIR = "fs.obs.trash.dir";
+ static final String FAST_DELETE_DIR = "fs.obs.trash.dir";
+
+ /**
+ * Enable obs content summary or not.
+ */
+ static final String OBS_CONTENT_SUMMARY_ENABLE = "fs.obs.content.summary.enable";
+
+ static final String OBS_CONTENT_SUMMARY_VERSION = "fs.obs.content.summary.version";
+
+ static final String OBS_CONTENT_SUMMARY_VERSION_V1 = "1";
+
+ static final String OBS_CONTENT_SUMMARY_VERSION_V2 = "2";
+
+ static final int OBS_CONTENT_SUMMARY_FALLBACK_THRESHOLD = 1000;
+
+ /**
+ * Enable obs client dfs list or not.
+ */
+ static final String OBS_CLIENT_DFS_LIST_ENABLE = "fs.obs.client.dfs.list.enable";
/**
* Encryption type is sse-kms or sse-c.
@@ -744,12 +855,38 @@ public final class OBSConstants {
*/
static final boolean DEFAULT_GET_CANONICAL_SERVICE_NAME_ENABLE = false;
- static final String MAX_TIME_IN_MILLISECOND_TO_RETRY = "fs.obs.retry.maxtime";
+ static final String RETRY_MAXTIME = "fs.obs.retry.maxtime";
- /**
- * Default value of {@link #MAX_TIME_IN_MILLISECOND_TO_RETRY}
- */
- static final long DEFAULT_TIME_IN_MILLISECOND_TO_RETRY = 180000;
+ static final long DEFAULT_RETRY_MAXTIME = 180000;
+
+ static final String RETRY_SLEEP_BASETIME = "fs.obs.retry.sleep.basetime";
+
+ static final long DEFAULT_RETRY_SLEEP_BASETIME = 50;
+
+ static final String RETRY_SLEEP_MAXTIME = "fs.obs.retry.sleep.maxtime";
+
+ static final long DEFAULT_RETRY_SLEEP_MAXTIME = 30000;
+
+
+ static final String RETRY_QOS_MAXTIME = "fs.obs.retry.qos.maxtime";
+
+ static final long DEFAULT_RETRY_QOS_MAXTIME = 180000;
+
+ static final String RETRY_QOS_SLEEP_BASETIME = "fs.obs.retry.qos.sleep.basetime";
+
+ static final long DEFAULT_RETRY_QOS_SLEEP_BASETIME = 1000;
+
+ static final String RETRY_QOS_SLEEP_MAXTIME = "fs.obs.retry.qos.sleep.maxtime";
+
+ static final long DEFAULT_RETRY_QOS_SLEEP_MAXTIME = 30000;
+
+ public static final String RETRY_LIMIT = "fs.obs.retry.limit";
+
+ public static final int DEFAULT_RETRY_LIMIT = 7;
+
+ public static final String RETRY_QOS_LIMIT = "fs.obs.retry.qos.limit";
+
+ public static final int DEFAULT_RETRY_QOS_LIMIT = 7;
/**
* File visibility after create interface switch.
@@ -764,10 +901,10 @@ public final class OBSConstants {
public static final String AUTHORIZER_PROVIDER = "fs.obs.authorize.provider";
- public static final String AUTHORIZE_FAIL_FALLBACK= "fs.obs.authorize.fail.fallback";
+ public static final String AUTHORIZE_FAIL_FALLBACK = "fs.obs.authorize.fail.fallback";
public static final boolean DEFAULT_AUTHORIZE_FAIL_FALLBACK = false;
- public static final String AUTHORIZE_EXCEPTION_FALLBACK= "fs.obs.authorize.exception.fallback";
+ public static final String AUTHORIZE_EXCEPTION_FALLBACK = "fs.obs.authorize.exception.fallback";
public static final boolean DEFAULT_AUTHORIZE_EXCEPTION_FALLBACK = true;
/**
@@ -798,6 +935,48 @@ public final class OBSConstants {
static final String OUTPUT_STREAM_HFLUSH_POLICY_EMPTY = "empty"; // downgrade hflush/hsync to empty func, which means calling hflush/hsync will do nothing
+ static final String OUTPUT_STREAM_ATTACH_MD5 = "fs.obs.outputstream.attach.md5";
+
+ static final Boolean DEFAULT_OUTPUT_STREAM_ATTACH_MD5 = false;
+
+ /**
+ * Use which type to validate consistency of uploaded block data. Default value is {@link #FAST_UPLOAD_CHECKSUM_TYPE_NONE}.
+ * Normally replace {@link #OUTPUT_STREAM_ATTACH_MD5}.
+ * Recommend {@link #FAST_UPLOAD_CHECKSUM_TYPE_SHA256} for more secure.
+ */
+ static final String FAST_UPLOAD_CHECKSUM_TYPE = "fs.obs.fast.upload.checksum.type";
+
+ static final String FAST_UPLOAD_CHECKSUM_TYPE_NONE = "none";
+
+ static final String FAST_UPLOAD_CHECKSUM_TYPE_MD5 = "md5";
+
+ static final String FAST_UPLOAD_CHECKSUM_TYPE_SHA256 = "sha256";
+
+ static final String OUTPUT_STREAM_DISK_FORCE_FLUSH = "fs.obs.outputstream.disk.force.flush";
+
+ static final Boolean DEFAULT_OUTPUT_STREAM_DISK_FORCE_FLUSH = true;
+
+ static final String FAST_DELETE_VERSION = "fs.obs.fast.delete.version";
+
+ static final String FAST_DELETE_VERSION_V1 = "1";
+
+ static final String FAST_DELETE_VERSION_V2 = "2";
+
+ static final String FAST_DELETE_VERSION_V2_CHECKPOINT_FORMAT = "yyyyMMddHH";
+
+ /**
+ * Determines the HDFS trash behavior. Default value is {@link #HDFS_TRASH_VERSION_V1}.
+ */
+ static final String HDFS_TRASH_VERSION = "fs.obs.hdfs.trash.version";
+
+ static final String HDFS_TRASH_VERSION_V1 = "1";
+
+ static final String HDFS_TRASH_VERSION_V2 = "2";
+
+ static final String HDFS_TRASH_PREFIX = "fs.obs.hdfs.trash.prefix";
+
+ static final String DEFAULT_HDFS_TRASH_PREFIX = "/user/.Trash";
+
private OBSConstants() {
}
}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSDataBlocks.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSDataBlocks.java
index bb2b061..0ddb1b5 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSDataBlocks.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSDataBlocks.java
@@ -23,6 +23,8 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
import org.apache.hadoop.util.DirectBufferPool;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -38,6 +40,10 @@
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Base64;
+import java.util.Locale;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
@@ -55,20 +61,14 @@ final class OBSDataBlocks {
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 + ')');
+ boolean offsetInvalid = off < 0 || off > b.length;
+ boolean lenInvalid = len < 0;
+ boolean endposInvalid = off + len > b.length || off + len < 0;
+ boolean invalid = offsetInvalid || lenInvalid || endposInvalid;
+ if (invalid) {
+ throw new IndexOutOfBoundsException(String.format(Locale.ROOT, "write (b[%d], %d, %d)", b.length, off, len));
}
}
@@ -93,6 +93,39 @@ static BlockFactory createFactory(final OBSFileSystem owner, final String name)
}
}
+ enum ChecksumType {
+ NONE("NONE") {
+ @Override
+ public String encode(byte[] bytes) {
+ throw new UnsupportedOperationException("Not supported when checksum type is none");
+ }
+ },
+ MD5("MD5") {
+ @Override
+ public String encode(byte[] bytes) {
+ return Base64.getEncoder().encodeToString(bytes);
+ }
+ },
+ SHA256("SHA-256") {
+ @Override
+ public String encode(byte[] bytes) {
+ return OBSCommonUtils.toHex(bytes);
+ }
+ };
+
+ private String algorithm;
+
+ ChecksumType(String algorithm) {
+ this.algorithm = algorithm;
+ }
+
+ public String getAlgorithm() {
+ return algorithm;
+ }
+
+ public abstract String encode(byte[] bytes);
+ }
+
/**
* Base class for block factories.
*/
@@ -116,19 +149,33 @@ protected BlockFactory(final OBSFileSystem obsFileSystem) {
*/
abstract DataBlock create(long index, int limit) throws IOException;
- /**
- * Owner.
- *
- * @return obsFileSystem instance
- */
protected OBSFileSystem getOwner() {
return owner;
}
+
+ protected boolean calcMd5() {
+ return this.getOwner()
+ .getConf()
+ .getBoolean(OBSConstants.OUTPUT_STREAM_ATTACH_MD5, OBSConstants.DEFAULT_OUTPUT_STREAM_ATTACH_MD5);
+ }
+
+ protected ChecksumType determineChecksumType() throws IOException {
+ String type = this.getOwner()
+ .getConf()
+ .get(OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE, OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE_NONE);
+ if (OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE_SHA256.equals(type)) {
+ return ChecksumType.SHA256;
+ } else if (OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE_MD5.equals(type) || calcMd5()) {
+ return ChecksumType.MD5;
+ } else if (OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE_NONE.equals(type)) {
+ return ChecksumType.NONE;
+ } else {
+ throw new IOException(String.format("Unsupported fast upload checksum type '%s'", type));
+ }
+ }
+
}
- /**
- * This represents a block being uploaded.
- */
abstract static class DataBlock implements Closeable {
/**
@@ -136,13 +183,38 @@ abstract static class DataBlock implements Closeable {
*/
private final long index;
+ private ChecksumType checksumType;
+
+ private MessageDigest digest;
+
+ private String checksum;
+
/**
* Dest state can be : writing/upload/closed.
*/
private volatile DestState state = DestState.Writing;
- protected DataBlock(final long dataIndex) {
+ protected DataBlock(final long dataIndex, final ChecksumType checksumType) {
this.index = dataIndex;
+ if (checksumType == null) {
+ this.checksumType = ChecksumType.NONE;
+ } else {
+ this.checksumType = checksumType;
+ }
+
+ if (checksumType != ChecksumType.NONE) {
+ LOG.debug("init data block digest state to calculate checksum. checksumType: {}", checksumType);
+ initDigestState();
+ }
+ }
+
+ private void initDigestState() {
+ try {
+ digest = MessageDigest.getInstance(checksumType.getAlgorithm());
+ } catch (NoSuchAlgorithmException e) {
+ LOG.warn("load digest algorithm failed", e);
+ this.checksumType = ChecksumType.NONE;
+ }
}
/**
@@ -160,12 +232,6 @@ protected final synchronized void enterState(final DestState current, final Dest
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(
@@ -173,11 +239,6 @@ protected final void verifyState(final DestState expected) throws IllegalStateEx
}
}
- /**
- * Current state.
- *
- * @return the current state.
- */
protected final DestState getState() {
return state;
}
@@ -186,49 +247,16 @@ 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");
@@ -236,46 +264,43 @@ int write(final byte[] buffer, final int offset, final int length) throws IOExce
Preconditions.checkArgument(offset >= 0, "offset is negative");
Preconditions.checkArgument(!(buffer.length - offset < length),
"buffer shorter than amount of data to write");
+ if (checksumType != ChecksumType.NONE) {
+ digest.update(buffer, offset, length);
+ }
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);
+ if (checksumType != ChecksumType.NONE) {
+ finalDigest();
+ this.digest = null;
+ }
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;
+ private void finalDigest() {
+ if (digest == null) {
+ LOG.warn("digest is null");
}
+ if (!getState().equals(DestState.Upload)) {
+ throw new IllegalStateException("finalDigest() should in Upload state");
+ }
+ byte[] bytes = digest.digest();
+ checksum = checksumType.encode(bytes);
+ }
+
+ public ChecksumType getChecksumType() {
+ return checksumType;
+ }
+
+ public String getChecksum() {
+ return checksum;
}
@Override
@@ -286,62 +311,43 @@ public void close() throws IOException {
}
}
- /**
- * 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.
- */
+ protected synchronized boolean enterClosedState() {
+ if (!state.equals(DestState.Closed)) {
+ enterState(null, DestState.Closed);
+ return true;
+ } else {
+ return false;
+ }
+ }
+
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) {
+ DataBlock create(final long index, final int limit) throws IOException {
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);
+ return new ByteArrayBlock(0, limit, firstBlockSize, determineChecksumType());
}
}
- /**
- * 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();
@@ -350,77 +356,30 @@ ByteArrayInputStream getInputStream() {
}
}
- /**
- * 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);
+ ByteArrayBlock(final long index, final int limitBlockSize, final int blockSize, final ChecksumType checksumType) {
+ super(index, checksumType);
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;
@@ -431,10 +390,15 @@ int remainingCapacity() {
return limit - dataSize();
}
+ @VisibleForTesting
+ public int firstBlockSize() {
+ return this.firstBlockSize;
+ }
+
@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);
+ super.write(b, offset, written);
buffer.write(b, offset, written);
return written;
}
@@ -452,6 +416,14 @@ protected void innerClose() throws IOException {
}
}
+ @Override
+ InputStream startUpload() throws IOException {
+ super.startUpload();
+ dataSize = buffer.size();
+ inputStream = buffer.getInputStream();
+ return inputStream;
+ }
+
@Override
public String toString() {
return "ByteArrayBlock{" + "index=" + getIndex() + ", state=" + getState() + ", limit=" + limit
@@ -459,20 +431,10 @@ public String toString() {
}
}
- /**
- * 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) {
@@ -480,8 +442,8 @@ static class ByteBufferBlockFactory extends BlockFactory {
}
@Override
- ByteBufferBlock create(final long index, final int limit) {
- return new ByteBufferBlock(index, limit);
+ ByteBufferBlock create(final long index, final int limit) throws IOException {
+ return new ByteBufferBlock(index, limit, determineChecksumType());
}
public static ByteBuffer requestBuffer(final int limit) {
@@ -496,104 +458,41 @@ public static void releaseBuffer(final ByteBuffer 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 + '}';
+ return String.format(Locale.ROOT, "ByteBufferBlockFactory{ buffersOutstanding = %d }", BUFFERS_OUTSTANDING.get());
}
}
- /**
- * 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);
+ ByteBufferBlock(final long index, final int initBufferSize, final ChecksumType checksumType) {
+ super(index, checksumType);
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() {
+ private int capacityUsed() {
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;
+ int dataSize() {
+ return dataSize != null ? dataSize : capacityUsed();
}
- /**
- * Closing the block will release the buffer.
- */
@Override
protected void innerClose() {
if (blockBuffer != null) {
@@ -606,27 +505,34 @@ protected void innerClose() {
}
}
+ @Override
+ public int remainingCapacity() {
+ return blockBuffer != null ? blockBuffer.remaining() : 0;
+ }
+
+ @Override
+ public boolean hasCapacity(final long bytes) {
+ return bytes <= remainingCapacity();
+ }
+
@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.
- */
+ @Override
+ int write(final byte[] b, final int offset, final int len) throws IOException {
+ int written = Math.min(remainingCapacity(), len);
+ super.write(b, offset, written);
+ blockBuffer.put(b, offset, written);
+ return written;
+ }
+
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) {
@@ -635,48 +541,38 @@ class ByteBufferInputStream extends InputStream {
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) {
+ long pos = position() + offset;
+ if (pos < 0) {
throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK);
}
- if (newPos > size) {
+ if (pos > size) {
throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
}
- byteBuffer.position((int) newPos);
- return newPos;
+ byteBuffer.position((int) pos);
+ return pos;
+ }
+
+ @Override
+ public synchronized void close() {
+ LOG.debug("ByteBufferInputStream.close() for {}", ByteBufferBlock.super.toString());
+ byteBuffer = null;
+ }
+
+ public synchronized int read() {
+ if (available() > 0) {
+ return byteBuffer.get() & OBSCommonUtils.BYTE_TO_INT_MASK;
+ } else {
+ return -1;
+ }
}
@Override
@@ -685,53 +581,20 @@ public synchronized int available() {
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");
@@ -750,107 +613,99 @@ public synchronized int read(final byte[] b, final int offset, final int length)
return toRead;
}
+ @Override
+ public synchronized void mark(final int readLimit) {
+ LOG.debug("mark at {}", position());
+ byteBuffer.mark();
+ }
+
@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());
+ if (this.byteBuffer != null) {
+ sb.append(", available=").append(this.byteBuffer.remaining());
}
- sb.append(", ").append(ByteBufferBlock.super.toString());
- sb.append('}');
+ sb.append(", ").append(ByteBufferBlock.super.toString()).append('}');
return sb.toString();
}
+
+ @Override
+ public boolean markSupported() {
+ return true;
+ }
+ }
+
+ @Override
+ InputStream startUpload() throws IOException {
+ super.startUpload();
+ dataSize = capacityUsed();
+ blockBuffer.limit(blockBuffer.position());
+ blockBuffer.position(0);
+ inputStream = new ByteBufferInputStream(dataSize, blockBuffer);
+ return inputStream;
}
}
- /**
- * Buffer blocks to disk.
- */
static class DiskBlockFactory extends BlockFactory {
- /**
- * Allocator the local directory.
- */
- private static OBSLocalDirAllocator directoryAllocator;
+
+ private static volatile 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
- */
+ protected boolean diskForce() {
+ return this.getOwner()
+ .getConf()
+ .getBoolean(OBSConstants.OUTPUT_STREAM_DISK_FORCE_FLUSH , OBSConstants.DEFAULT_OUTPUT_STREAM_DISK_FORCE_FLUSH);
+ }
+
@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);
+ return new DiskBlock(destFile, limit, index, determineChecksumType(), diskForce());
}
- /**
- * Demand create the directory allocator, then create a temporary file.
- * {@link OBSLocalDirAllocator#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 File createTmpFileForWrite(final String pathStr, final long size, final Configuration conf)
+ 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 OBSLocalDirAllocator(bufferDir);
+ ? OBSConstants.BUFFER_DIR
+ : "hadoop.tmp.dir";
+ directoryAllocator = new LocalDirAllocator(bufferDir);
}
- return directoryAllocator.createTmpFileForWrite(pathStr, size, conf);
+ Path path = directoryAllocator.getLocalPathForWrite(pathStr, size, conf);
+ File dir = new File(path.getParent().toUri().getPath());
+ String prefix = path.getName();
+ return File.createTempFile(prefix, null, dir);
}
}
- /**
- * 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;
+ private boolean diskForce;
+
+ private BufferedOutputStream outputStream;
- DiskBlock(final File destBufferFile, final int limitSize, final long index) throws FileNotFoundException {
- super(index);
+ private FileOutputStream fileoutputstream;
+
+ DiskBlock(final File destBufferFile, final int limitSize, final long index, final ChecksumType checksumType,
+ final boolean diskForce) throws FileNotFoundException {
+ super(index, checksumType);
this.limit = limitSize;
+ this.diskForce = diskForce;
this.bufferFile = destBufferFile;
- out = new BufferedOutputStream(new FileOutputStream(destBufferFile));
+ this.fileoutputstream = new FileOutputStream(destBufferFile);
+ outputStream = new BufferedOutputStream(fileoutputstream);
}
@Override
@@ -859,8 +714,12 @@ int dataSize() {
}
@Override
- boolean hasCapacity(final long bytes) {
- return dataSize() + bytes <= limit;
+ int write(final byte[] b, final int offset, final int len) throws IOException {
+ int writeLen = Math.min(remainingCapacity(), len);
+ super.write(b, offset, writeLen);
+ outputStream.write(b, offset, writeLen);
+ bytesWritten += writeLen;
+ return writeLen;
}
@Override
@@ -868,67 +727,56 @@ 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;
+ 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());
}
- 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();
- }
+ case Closed:
+ closeBlock();
break;
case Upload:
LOG.debug("Block[{}]: Buffer file {} exists —close upload stream", getIndex(), bufferFile);
break;
- case Closed:
- closeBlock();
+ case Writing:
+ if (bufferFile.exists()) {
+ LOG.debug("Block[{}]: Deleting buffer file as upload did not start", getIndex());
+ 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();
+ File startUpload() throws IOException {
+ super.startUpload();
+ try {
+ outputStream.flush();
+ if (diskForce) {
+ fileoutputstream.getChannel().force(true);
+ }
+
+ } finally {
+ outputStream.close();
+ outputStream = null;
+ }
+ return bufferFile;
}
@Override
@@ -937,19 +785,15 @@ public String toString() {
+ ", 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());
- }
+ @Override
+ void flush() throws IOException {
+ super.flush();
+ outputStream.flush();
+ }
+
+ @Override
+ boolean hasCapacity(final long bytes) {
+ return dataSize() + bytes <= limit;
}
}
}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/FileConflictException.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileConflictException.java
similarity index 54%
rename from hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/FileConflictException.java
rename to hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileConflictException.java
index 6b4212a..767804d 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/FileConflictException.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileConflictException.java
@@ -5,8 +5,9 @@
/**
* OBS file conflict exception.
*/
-class FileConflictException extends IOException {
+class OBSFileConflictException extends IOException implements WithErrCode {
private static final long serialVersionUID = -897856973823710492L;
+ private String errCode;
/**
* Constructs a FileConflictException
with the specified detail
@@ -14,9 +15,18 @@ class FileConflictException extends IOException {
* {@link java.lang.Throwable#getMessage}
* method of class java.lang.Throwable
.
*
- * @param s the detail message.
+ * @param message the detail message.
*/
- FileConflictException(final String s) {
- super(s);
+ OBSFileConflictException(final String message) {
+ super(message);
+ }
+
+ public void setErrCode(String errCode) {
+ this.errCode = errCode;
+ }
+
+ @Override
+ public String getErrCode() {
+ return this.errCode;
}
}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileStatus.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileStatus.java
index 30dca8c..f0b84af 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileStatus.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileStatus.java
@@ -22,6 +22,7 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
/**
* File status for an OBS file.
@@ -30,7 +31,10 @@
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
-class OBSFileStatus extends FileStatus {
+public class OBSFileStatus extends FileStatus {
+
+ private String etag;
+
/**
* Create a directory status.
*
@@ -83,4 +87,30 @@ class OBSFileStatus extends FileStatus {
setOwner(owner);
setGroup(owner);
}
+
+ /**
+ * A simple file with etag.
+ *
+ * @param length file length
+ * @param modificationTime mod time
+ * @param path path
+ * @param blockSize block size
+ * @param owner owner
+ * @param etag etag
+ */
+ OBSFileStatus(final long length, final long modificationTime, final Path path, final long blockSize,
+ final String owner, final String etag) {
+ this(length, modificationTime, path, blockSize, owner);
+ this.etag = etag;
+ }
+
+ public String getEtag() {
+ return this.etag;
+ }
+
+ protected void setAccessControlAttr(final String owner, final String group, final FsPermission fsPermission) {
+ super.setOwner(owner);
+ super.setGroup(group);
+ super.setPermission(fsPermission);
+ }
}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileSystem.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileSystem.java
index eb7dea2..be8eea0 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileSystem.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileSystem.java
@@ -23,12 +23,25 @@
import com.obs.services.ObsClient;
import com.obs.services.exception.ObsException;
import com.obs.services.model.AccessControlList;
+import com.obs.services.model.select.CsvInput;
+import com.obs.services.model.select.CsvOutput;
+import com.obs.services.model.select.ExpressionType;
+import com.obs.services.model.select.FileHeaderInfo;
+import com.obs.services.model.select.InputSerialization;
+import com.obs.services.model.select.JsonInput;
+import com.obs.services.model.select.JsonOutput;
+import com.obs.services.model.select.JsonType;
+import com.obs.services.model.select.OrcInput;
+import com.obs.services.model.select.OutputSerialization;
+import com.obs.services.model.select.ScanRange;
+import com.obs.services.model.select.SelectObjectRequest;
+import com.obs.services.model.select.SelectObjectResult;
-import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataInputStream;
@@ -43,15 +56,20 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.obs.input.OBSInputStream;
import org.apache.hadoop.fs.obs.input.InputPolicyFactory;
import org.apache.hadoop.fs.obs.input.InputPolicys;
-import org.apache.hadoop.fs.obs.input.OBSInputStream;
+import org.apache.hadoop.fs.obs.input.ObsSelectInputStream;
+import org.apache.hadoop.fs.obs.memartscc.CcGetShardParam;
+import org.apache.hadoop.fs.obs.memartscc.MemArtsCCClient;
+import org.apache.hadoop.fs.obs.memartscc.ObjectShard;
import org.apache.hadoop.fs.obs.security.AccessType;
import org.apache.hadoop.fs.obs.security.AuthorizeProvider;
-import org.apache.hadoop.fs.obs.security.DelegationTokenCapability;
import org.apache.hadoop.fs.obs.security.OBSAuthorizationException;
+import org.apache.hadoop.fs.obs.security.ObsDelegationTokenManger;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Progressable;
@@ -63,9 +81,16 @@
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
import java.net.URI;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
import java.util.EnumSet;
import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
import java.util.Map;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
@@ -87,13 +112,13 @@
//CHECKSTYLE:OFF
@InterfaceAudience.Private
@InterfaceStability.Evolving
-public class OBSFileSystem extends FileSystem {
+public class OBSFileSystem extends FileSystem implements OpenFileWithJobConf {
//CHECKSTYLE:ON
/**
* Class logger.
*/
- public static final Logger LOG = LoggerFactory.getLogger(OBSFileSystem.class);
+ private static final Logger LOG = LoggerFactory.getLogger(OBSFileSystem.class);
/**
* Flag indicating if the filesystem instance is closed.
@@ -119,11 +144,17 @@ public class OBSFileSystem extends FileSystem {
private int invokeCountThreshold = 0;
+ private ObsDelegationTokenManger obsDelegationTokenManger;
/**
* OBS client instance.
*/
private ObsClient obs;
+ /**
+ * memartscc client instance.
+ */
+ private MemArtsCCClient memArtsCCClient;
+
/**
* Metrics consumer.
*/
@@ -248,12 +279,18 @@ public class OBSFileSystem extends FileSystem {
/**
* Flag indicating if fast delete is enabled.
*/
- private boolean enableTrash = false;
+ private boolean enableFastDelete = false;
+
+ private String fastDeleteVersion;
/**
* Trash directory for fast delete.
*/
- private String trashDir;
+ private String fastDeleteDir;
+
+ private String hdfsTrashVersion;
+
+ private String hdfsTrashPrefix;
/**
* OBS redefined access control list.
@@ -282,10 +319,23 @@ public class OBSFileSystem extends FileSystem {
* currently being written by this client. Note that a file can only be
* written by a single filesystem.
*/
- private Map filesBeingWritten = new HashMap<>();
+ private final Map filesBeingWritten = new HashMap<>();
private boolean enableFileVisibilityAfterCreate = false;
+ private String readPolicy = OBSConstants.READAHEAD_POLICY_PRIMARY;
+
+ private boolean localityEnabled;
+
+ private TrafficStatisticsReporter statsReporter;
+
+ private TrafficStatistics trafficStatistics;
+
+ /**
+ * obs file system permissions mode.
+ */
+ private String permissionsMode;
+
/**
* Close all {@link FSDataOutputStream} opened by the owner {@link
* OBSFileSystem}.
@@ -347,6 +397,23 @@ boolean isFileBeingWritten(String file) {
public void initialize(final URI name, final Configuration originalConf) throws IOException {
uri = URI.create(name.getScheme() + "://" + name.getAuthority());
bucket = name.getAuthority();
+
+ // Delegation token only mode.
+ if (originalConf.getBoolean(OBSConstants.DELEGATION_TOKEN_ONLY, OBSConstants.DEFAULT_DELEGATION_TOKEN_ONLY)
+ && OBSCommonUtils.isStringEmpty(bucket)) {
+ LOG.debug("Delegation-token-only mode");
+ obsDelegationTokenManger = OBSCommonUtils.initDelegationTokenManger(this, uri, originalConf);
+ if (originalConf.getTrimmed(OBSConstants.READAHEAD_POLICY,
+ OBSConstants.READAHEAD_POLICY_PRIMARY).equals(OBSConstants.READAHEAD_POLICY_MEMARTSCC)) {
+ memArtsCCClient = new MemArtsCCClient(bucket, enablePosix);
+ boolean initRes = memArtsCCClient.initializeDtOnly(originalConf);
+ if (!initRes) {
+ LOG.warn("MemArtsCCClient dt-only initialize failed!");
+ }
+ }
+ return;
+ }
+
// clone the configuration into one with propagated bucket options
Configuration conf = OBSCommonUtils.propagateBucketOptions(originalConf, bucket);
OBSCommonUtils.patchSecurityCredentialProviders(conf);
@@ -367,6 +434,8 @@ public void initialize(final URI name, final Configuration originalConf) throws
OBSConstants.OBS_CLIENT_FACTORY_IMPL, OBSConstants.DEFAULT_OBS_CLIENT_FACTORY_IMPL,
OBSClientFactory.class);
obs = ReflectionUtils.newInstance(obsClientFactoryClass, conf).createObsClient(name);
+ OBSCommonUtils.init(this, conf);
+
sse = new SseWrapper(conf);
Class extends BasicMetricsConsumer> metricsConsumerClass = conf.getClass(
@@ -382,8 +451,12 @@ public void initialize(final URI name, final Configuration originalConf) throws
}
}
- OBSCommonUtils.verifyBucketExists(this);
- enablePosix = OBSCommonUtils.getBucketFsStatus(obs, bucket);
+ try {
+ enablePosix = OBSCommonUtils.getBucketFsStatus(obs, bucket);
+ } catch (IOException e) {
+ OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.initialize, e);
+ throw e;
+ }
maxKeys = OBSCommonUtils.intOption(conf, OBSConstants.MAX_PAGING_KEYS, OBSConstants.DEFAULT_MAX_PAGING_KEYS,
1);
@@ -421,64 +494,96 @@ public void initialize(final URI name, final Configuration originalConf) throws
LOG.debug("Using OBSBlockOutputStream with buffer = {}; block={};" + " queue limit={}", blockOutputBuffer,
partSize, blockOutputActiveBlocks);
- String readPolicy = conf.getTrimmed(OBSConstants.READAHEAD_POLICY, OBSConstants.READAHEAD_POLICY_PRIMARY);
+ readPolicy = conf.getTrimmed(OBSConstants.READAHEAD_POLICY, OBSConstants.READAHEAD_POLICY_PRIMARY);
inputPolicyFactory = InputPolicys.createFactory(readPolicy);
- enableTrash = conf.getBoolean(OBSConstants.TRASH_ENABLE, OBSConstants.DEFAULT_TRASH);
- if (enableTrash) {
+ // initialize MemArtsCC
+ if (readPolicy.equals(OBSConstants.READAHEAD_POLICY_MEMARTSCC)) {
+ if (initMemArtsCC(conf, name)) {
+ // Traffic Report
+ initTrafficReport(conf);
+ }
+ }
+
+ obsDelegationTokenManger = OBSCommonUtils.initDelegationTokenManger(this, uri, conf);
+ localityEnabled = conf.getBoolean(OBSConstants.MEMARTSCC_LOCALITY_ENABLE, OBSConstants.DEFAULT_MEMARTSCC_LOCALITY_ENABLE);
+
+ hdfsTrashVersion = conf.get(OBSConstants.HDFS_TRASH_VERSION, OBSConstants.HDFS_TRASH_VERSION_V1);
+ hdfsTrashPrefix = conf.get(OBSConstants.HDFS_TRASH_PREFIX, OBSConstants.DEFAULT_HDFS_TRASH_PREFIX);
+ enableFastDelete = conf.getBoolean(OBSConstants.FAST_DELETE_ENABLE, OBSConstants.DEFAULT_FAST_DELETE_ENABLE);
+ if (enableFastDelete) {
if (!isFsBucket()) {
String errorMsg = String.format("The bucket [%s] is not posix. not supported for " + "trash.",
bucket);
LOG.warn(errorMsg);
- enableTrash = false;
- trashDir = null;
+ enableFastDelete = false;
+ fastDeleteDir = null;
} else {
- trashDir = conf.get(OBSConstants.TRASH_DIR);
- if (StringUtils.isEmpty(trashDir)) {
+ String fastDelete = "FastDelete";
+ String defaultTrashDir = OBSCommonUtils.maybeAddTrailingSlash(
+ this.getTrashRoot(new Path(fastDelete)).toUri().getPath()) + fastDelete;
+ fastDeleteDir = conf.get(OBSConstants.FAST_DELETE_DIR, defaultTrashDir);
+ if (OBSCommonUtils.isStringEmpty(fastDeleteDir)) {
String errorMsg = String.format(
- "The trash feature(fs.obs.trash.enable) is " + "enabled, but the "
- + "configuration(fs.obs.trash.dir [%s]) " + "is empty.", trashDir);
+ "The fast delete feature(fs.obs.trash.enable) is " + "enabled, but the "
+ + "configuration(fs.obs.trash.dir [%s]) " + "is empty.", fastDeleteDir);
LOG.error(errorMsg);
throw new ObsException(errorMsg);
}
- trashDir = OBSCommonUtils.maybeAddBeginningSlash(trashDir);
- trashDir = OBSCommonUtils.maybeAddTrailingSlash(trashDir);
+ fastDeleteDir = OBSCommonUtils.maybeAddBeginningSlash(fastDeleteDir);
+ fastDeleteDir = OBSCommonUtils.maybeAddTrailingSlash(fastDeleteDir);
+ fastDeleteVersion = conf.get(OBSConstants.FAST_DELETE_VERSION, OBSConstants.FAST_DELETE_VERSION_V1);
}
}
- OBSCommonUtils.setMaxTimeInMillisecondsToRetry(conf.getLong(OBSConstants.MAX_TIME_IN_MILLISECOND_TO_RETRY,
- OBSConstants.DEFAULT_TIME_IN_MILLISECOND_TO_RETRY));
- enableCanonicalServiceName =
- conf.getBoolean(OBSConstants.GET_CANONICAL_SERVICE_NAME_ENABLE,
- OBSConstants.DEFAULT_GET_CANONICAL_SERVICE_NAME_ENABLE);
+
+ OBSCommonUtils.setRetryTime(conf.getLong(OBSConstants.RETRY_MAXTIME, OBSConstants.DEFAULT_RETRY_MAXTIME),
+ conf.getLong(OBSConstants.RETRY_SLEEP_BASETIME, OBSConstants.DEFAULT_RETRY_SLEEP_BASETIME),
+ conf.getLong(OBSConstants.RETRY_SLEEP_MAXTIME, OBSConstants.DEFAULT_RETRY_SLEEP_MAXTIME));
enableFileVisibilityAfterCreate =
conf.getBoolean(OBSConstants.FILE_VISIBILITY_AFTER_CREATE_ENABLE,
OBSConstants.DEFAULT_FILE_VISIBILITY_AFTER_CREATE_ENABLE);
enableCanonicalServiceName = conf.getBoolean(OBSConstants.GET_CANONICAL_SERVICE_NAME_ENABLE,
OBSConstants.DEFAULT_GET_CANONICAL_SERVICE_NAME_ENABLE);
-
+ permissionsMode = conf.get(OBSConstants.PERMISSIONS_MODE, OBSConstants.DEFAULT_PERMISSIONS_MODE);
this.authorizer = initAuthorizeProvider(conf);
} catch (ObsException e) {
+ LOG.error("initializing OBSFilesystem fail", e);
throw OBSCommonUtils.translateException("initializing ", new Path(name), e);
}
LOG.info("Finish initializing filesystem instance for uri: {}", uri);
}
+ private void initTrafficReport(final Configuration conf) {
+ boolean reportEnable = conf.getBoolean(OBSConstants.MEMARTSCC_TRAFFIC_REPORT_ENABLE, OBSConstants.DEFAULT_MEMARTSCC_TRAFFIC_REPORT_ENABLE);
+ LOG.debug("get report enable from config:{}", reportEnable);
+ if (!reportEnable) {
+ return;
+ }
+
+ trafficStatistics = new TrafficStatistics();
+ long interval = getConf().
+ getLong(OBSConstants.MEMARTSCC_TRAFFIC_REPORT_INTERVAL, OBSConstants.MEMARTSCC_TRAFFIC_REPORT_DEFAULT_INTERVAL);
+ LOG.debug("get report traffic interval from config: interval:{}", interval);
+ statsReporter = new TrafficStatisticsReporter(trafficStatistics, memArtsCCClient, interval);
+ statsReporter.startReport();
+ }
+
private AuthorizeProvider initAuthorizeProvider(Configuration conf) throws IOException {
- AuthorizeProvider authorizer = null;
+ AuthorizeProvider authorizeProvider = null;
Class> authClassName = conf.getClass(OBSConstants.AUTHORIZER_PROVIDER, null);
if (authClassName != null) {
try {
LOG.info("authorize provider is " + authClassName.getName());
- authorizer = (AuthorizeProvider)authClassName.newInstance();
- authorizer.init(conf);
+ authorizeProvider = (AuthorizeProvider)authClassName.newInstance();
+ authorizeProvider.init(conf);
} catch (Exception e) {
LOG.error(String.format("init %s failed", OBSConstants.AUTHORIZER_PROVIDER), e);
throw new IOException(String.format("init %s failed", OBSConstants.AUTHORIZER_PROVIDER), e);
}
}
- return authorizer;
+ return authorizeProvider;
}
private void checkPermission(Path path, AccessType accessType) throws IOException {
@@ -497,7 +602,7 @@ private void checkPermission(Path path, AccessType accessType) throws IOExceptio
long st = System.currentTimeMillis();
result = authorizer.isAuthorized(this.bucket, key, accessType);
long et = System.currentTimeMillis();
- if (LOG.isDebugEnabled()){
+ if (LOG.isDebugEnabled()) {
LOG.debug("authorizing:[user: {}], [action: {}], "
+ "[bucket: {}], [path: {}], [result: {}], [cost: {}]",
currentUser,
@@ -530,6 +635,19 @@ private void checkPermission(Path path, AccessType accessType) throws IOExceptio
}
}
+ private boolean initMemArtsCC(final Configuration conf, final URI name) {
+ String obsBucketName = getBucket();
+ memArtsCCClient = new MemArtsCCClient(obsBucketName, enablePosix);
+ if (!memArtsCCClient.initialize(name, conf)) {
+ LOG.warn("fallback to 'primary' read policy");
+ // fall back to 'primary' read policy
+ inputPolicyFactory = InputPolicys.createFactory(OBSConstants.READAHEAD_POLICY_PRIMARY);
+ // do not set memArtsCCClient to null
+ return false;
+ }
+ return true;
+ }
+
private void initThreadPools(final Configuration conf) {
long keepAliveTime = OBSCommonUtils.longOption(conf, OBSConstants.KEEPALIVE_TIME,
OBSConstants.DEFAULT_KEEPALIVE_TIME, 0);
@@ -715,6 +833,16 @@ public ObsClient getObsClient() {
return obs;
}
+ /**
+ * Return the MemArtsCC Client used by this filesystem.
+ *
+ * @return MemArtsCCClient
+ */
+ @VisibleForTesting
+ public MemArtsCCClient getMemArtsCCClient() {
+ return memArtsCCClient;
+ }
+
/**
* Return the read ahead range used by this filesystem.
*
@@ -757,6 +885,32 @@ protected URI canonicalizeUri(final URI rawUri) {
return OBSLoginHelper.canonicalizeUri(rawUri, getDefaultPort());
}
+ /**
+ * Open an FSDataInputStream at the indicated Path with the given configuration.
+ *
+ * @param f
+ * @param jobConf
+ * @return
+ * @throws IOException
+ */
+ @Override
+ public FSDataInputStream open(Path f, Configuration jobConf) throws IOException {
+ String sqlExpr = jobConf.get("mapreduce.job.input.file.option.fs.obs.select.sql");
+ if (sqlExpr != null) {
+ String key = OBSCommonUtils.pathToKey(this, f);
+ String scanStart = jobConf.get("mapreduce.job.input.file.option.fs.obs.select.scan.start");
+ String scanEnd = jobConf.get("mapreduce.job.input.file.option.fs.obs.select.scan.end");
+ LOG.info("OBSFileSystem.open(): bucket: {}; key: {}; sql: {}; range: [{}, {}]%n",
+ bucket, key,
+ sqlExpr,
+ scanStart != null ? scanStart : "n/a",
+ scanEnd != null ? scanEnd : "n/a");
+
+ return select(f, sqlExpr, jobConf);
+ }
+ return open(f);
+ }
+
/**
* Open an FSDataInputStream at the indicated Path.
*
@@ -768,43 +922,146 @@ protected URI canonicalizeUri(final URI rawUri) {
@Override
public FSDataInputStream open(final Path f, final int bufferSize) throws IOException {
checkOpen();
- long startTime = System.currentTimeMillis();
- long endTime;
LOG.debug("Opening '{}' for reading.", f);
- final FileStatus fileStatus;
+ final OBSFileStatus fileStatus;
try {
- fileStatus = OBSCommonUtils.innerGetFileStatusWithRetry(this, f);
- } catch (FileConflictException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.OPEN, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
+ fileStatus = OBSCommonUtils.getFileStatusWithRetry(this, f);
+ } catch (OBSFileConflictException e) {
throw new AccessControlException(e);
}
if (fileStatus.isDirectory()) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.OPEN, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
throw new FileNotFoundException("Can't open " + f + " because it is a directory");
}
checkPermission(f, AccessType.READ);
FSInputStream fsInputStream = inputPolicyFactory.create(this, bucket, OBSCommonUtils.pathToKey(this, f),
- fileStatus.getLen(), statistics, boundedMultipartUploadThreadPool);
- FSDataInputStream fsDataInputStream = new FSDataInputStream(fsInputStream);
+ fileStatus.getLen(), statistics, boundedMultipartUploadThreadPool, fileStatus);
+ return new FSDataInputStream(fsInputStream);
+ }
+
+ private FSDataInputStream select(Path f, String sqlExpr, Configuration jobConf) throws IOException {
+ final FileStatus fileStatus = getFileStatus(f);
+ if (fileStatus.isDirectory()) {
+ throw new FileNotFoundException("Can't open " + f + " because it is a directory");
+ }
+
+ String key = OBSCommonUtils.pathToKey(this, f);
+
+ final String obsSelectJobPath = "mapreduce.job.input.file.option.fs.obs.select.";
+ String fileFormat = jobConf.get(obsSelectJobPath + "format");
+ if (fileFormat == null) {
+ throw new IllegalArgumentException("file format is missing");
+ }
+
+ if (!fileFormat.equals("orc") &&
+ !fileFormat.equals("csv") &&
+ !fileFormat.equals("json")) {
+ throw new IllegalArgumentException("invalid file format '" + fileFormat + "', it must be one of { 'csv', 'json', 'orc' }");
+ }
+
+ SelectObjectRequest selectRequest = new SelectObjectRequest()
+ .withExpression(sqlExpr)
+ .withBucketName(bucket)
+ .withKey(key)
+ .withExpressionType(ExpressionType.SQL);
+
+ if (!fileFormat.equals("json")) {
+ // prepare input & output for CSV & ORC
+ CsvOutput output = new CsvOutput();
+ String delimiter = jobConf.get(obsSelectJobPath + "output.csv.delimiter");
+ if (delimiter != null) {
+ if (delimiter.length() != 1) {
+ throw new IllegalArgumentException("Invalid output delimiter " + delimiter);
+ }
+
+ output.withFieldDelimiter(delimiter.charAt(0));
+ }
+
+ // prepare the request
+ if (fileFormat.equals("csv")) {
+ CsvInput input = new CsvInput();
+
+ delimiter = jobConf.get(obsSelectJobPath + "input.csv.delimiter");
+ if (delimiter != null) {
+ if (delimiter.length() != 1)
+ throw new IllegalArgumentException("Invalid input delimiter " + delimiter);
+
+ input.withFieldDelimiter(delimiter.charAt(0));
+ }
+
+ String headerInfo = jobConf.get(obsSelectJobPath + "input.csv.header");
+ if (headerInfo != null) {
+ boolean found = false;
+ for (FileHeaderInfo enumEntry : FileHeaderInfo.values()) {
+ found = (enumEntry.toString().equals(headerInfo));
+ if (found) {
+ input.withFileHeaderInfo(enumEntry);
+ break;
+ }
+ }
+
+ if (!found) {
+ throw new IllegalArgumentException("Invalid header " + headerInfo);
+ }
+ }
+
+ selectRequest.withInputSerialization(
+ new InputSerialization()
+ .withCsv(input)
+ ).withOutputSerialization(
+ new OutputSerialization()
+ .withCsv(output)
+ );
+
+ } else {
+ // it is ORC
+ selectRequest.withInputSerialization(
+ new InputSerialization()
+ .withOrc(
+ new OrcInput()
+ )
+ ).withOutputSerialization(
+ new OutputSerialization()
+ .withCsv(output)
+ );
+ }
+ } else {
+ // it is JSON
+ JsonType type = jobConf.get(obsSelectJobPath + "input.json.type").equals("lines") ?
+ JsonType.LINES : JsonType.DOCUMENT;
+
+ selectRequest.withInputSerialization(
+ new InputSerialization()
+ .withJson(
+ new JsonInput()
+ .withType(type)
+ )
+ ).withOutputSerialization(
+ new OutputSerialization()
+ .withJson(
+ new JsonOutput()
+ )
+ );
+ }
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.OPEN, true, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
+ // Set Scan Range properties.
+ String scanStart = jobConf.get(obsSelectJobPath + "scan.start");
+ String scanEnd = jobConf.get(obsSelectJobPath + "scan.end");
+ if (scanStart != null && scanEnd != null) {
+ selectRequest.withScanRange(
+ new ScanRange()
+ .withStart(Long.parseLong(scanStart))
+ .withEnd(Long.parseLong(scanEnd)));
}
- return fsDataInputStream;
+
+ LOG.info("OBSFileSystem.select(): bucket: {}; key: {}; sql: {}; range: [{}, {}]\n",
+ bucket, key, sqlExpr,
+ scanStart != null ? scanStart : "n/a",
+ scanEnd != null ? scanEnd : "n/a");
+
+ SelectObjectResult selectResult = obs.selectObjectContent(selectRequest);
+ return new FSDataInputStream(
+ new ObsSelectInputStream(bucket, key, selectResult));
}
/**
@@ -830,45 +1087,21 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi
checkOpen();
String key = OBSCommonUtils.pathToKey(this, f);
final FileStatus status;
- long startTime = System.currentTimeMillis();
- long endTime;
boolean exist = true;
try {
// get the status or throw an exception
try {
- status = OBSCommonUtils.innerGetFileStatusWithRetry(this, f);
- } catch (FileConflictException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.OVERWRITE, BasicMetricsConsumer.MetricRecord.CREATE, false,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
-
+ status = OBSCommonUtils.getFileStatusWithRetry(this, f);
+ } catch (OBSFileConflictException e) {
throw new ParentNotDirectoryException(e.getMessage());
}
// if the thread reaches here, there is something at the path
if (status.isDirectory()) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.OVERWRITE, BasicMetricsConsumer.MetricRecord.CREATE, false,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
// path references a directory: automatic error
throw new FileAlreadyExistsException(f + " is a directory");
}
if (!overwrite) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.OVERWRITE, BasicMetricsConsumer.MetricRecord.CREATE, false,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
// path references a file and overwrite is disabled
throw new FileAlreadyExistsException(f + " already exists");
}
@@ -895,13 +1128,6 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi
filesBeingWritten.put(key, outputStream);
}
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.OVERWRITE, BasicMetricsConsumer.MetricRecord.CREATE, true,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
return outputStream;
}
@@ -952,8 +1178,6 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi
final int bufferSize, final short replication, final long blkSize, final Progressable progress,
final ChecksumOpt checksumOpt) throws IOException {
checkOpen();
- long startTime = System.currentTimeMillis();
- long endTime;
LOG.debug("create: Creating new file {}, flags:{}, isFsBucket:{}", f, flags, isFsBucket());
OBSCommonUtils.checkCreateFlag(flags);
FSDataOutputStream outputStream;
@@ -968,27 +1192,13 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi
try {
// get the status or throw an FNFE
try {
- status = OBSCommonUtils.innerGetFileStatusWithRetry(this, f);
- } catch (FileConflictException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.FLAGS, BasicMetricsConsumer.MetricRecord.CREATE, false,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
+ status = OBSCommonUtils.getFileStatusWithRetry(this, f);
+ } catch (OBSFileConflictException e) {
throw new ParentNotDirectoryException(e.getMessage());
}
// if the thread reaches here, there is something at the path
if (status.isDirectory()) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.FLAGS, BasicMetricsConsumer.MetricRecord.CREATE, false,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
// path references a directory: automatic error
throw new FileAlreadyExistsException(f + " is a directory");
}
@@ -1010,26 +1220,10 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi
synchronized (filesBeingWritten) {
filesBeingWritten.put(key, outputStream);
}
-
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.FLAGS, BasicMetricsConsumer.MetricRecord.CREATE, true,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
return outputStream;
} else {
outputStream = create(f, permission, flags == null || flags.contains(CreateFlag.OVERWRITE), bufferSize,
replication, blkSize, progress);
-
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.FLAGS, BasicMetricsConsumer.MetricRecord.CREATE, true,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
return outputStream;
}
}
@@ -1053,28 +1247,13 @@ public FSDataOutputStream createNonRecursive(final Path path, final FsPermission
final EnumSet flags, final int bufferSize, final short replication, final long blkSize,
final Progressable progress) throws IOException {
checkOpen();
- long startTime = System.currentTimeMillis();
- long endTime;
OBSCommonUtils.checkCreateFlag(flags);
if (path.getParent() != null && !this.exists(path.getParent())) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.CREATE_NR, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
throw new FileNotFoundException(path.toString() + " parent directory not exist.");
}
- FSDataOutputStream fsDataOutputStream = create(path, permission, flags.contains(CreateFlag.OVERWRITE),
+ return create(path, permission, flags.contains(CreateFlag.OVERWRITE),
bufferSize, replication, blkSize, progress);
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.CREATE_NR, true, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
- return fsDataOutputStream;
}
/**
@@ -1089,8 +1268,6 @@ public FSDataOutputStream createNonRecursive(final Path path, final FsPermission
public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress)
throws IOException {
checkOpen();
- long startTime = System.currentTimeMillis();
- long endTime;
if (!isFsBucket()) {
throw new UnsupportedOperationException("non-posix bucket. Append is not supported " + "by OBSFileSystem");
}
@@ -1100,26 +1277,14 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr
// get the status or throw an FNFE
FileStatus status;
try {
- status = OBSCommonUtils.innerGetFileStatusWithRetry(this, f);
- } catch (FileConflictException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.APPEND, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
+ status = OBSCommonUtils.getFileStatusWithRetry(this, f);
+ } catch (OBSFileConflictException e) {
throw new AccessControlException(e);
}
long objectLen = status.getLen();
// if the thread reaches here, there is something at the path
if (status.isDirectory()) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.APPEND, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
// path references a directory: automatic error
throw new FileAlreadyExistsException(f + " is a directory");
}
@@ -1137,35 +1302,9 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr
synchronized (filesBeingWritten) {
filesBeingWritten.put(key, outputStream);
}
-
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.APPEND, true, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
return outputStream;
}
- /**
- * Truncate the file in the indicated path to the indicated size.
- *
- * - Fails if path is a directory.
- * - Fails if path does not exist.
- * - Fails if path is not closed.
- * - Fails if new size is greater than current size.
- *
- *
- * @param f The path to the file to be truncated
- * @param newLength The size the file is to be truncated to
- * @return true
if the file has been truncated to the desired
- * newLength
and is immediately available to be reused for
- * write operations such as append
, or
- * false
if a background process of adjusting the length of
- * the last block has been started, and clients should wait for it to
- * complete before proceeding with further file updates.
- * @throws IOException IO failure
- */
@Override
public boolean truncate(Path f, long newLength) throws IOException {
checkOpen();
@@ -1179,8 +1318,8 @@ public boolean truncate(Path f, long newLength) throws IOException {
FileStatus status;
try {
- status = OBSCommonUtils.innerGetFileStatusWithRetry(this, f);
- } catch (FileConflictException e) {
+ status = OBSCommonUtils.getFileStatusWithRetry(this, f);
+ } catch (OBSFileConflictException e) {
throw new AccessControlException(e);
}
@@ -1192,7 +1331,7 @@ public boolean truncate(Path f, long newLength) throws IOException {
if (isFileBeingWritten(key)) {
// AlreadyBeingCreatedException (on HDFS NameNode) is transformed
// into IOException (on HDFS Client)
- throw new AlreadyBeingCreatedException("Cannot truncate " + f + " that is being written.");
+ throw new OBSAlreadyBeingCreatedException("Cannot truncate " + f + " that is being written.");
}
// Truncate length check.
@@ -1249,51 +1388,29 @@ public boolean rename(final Path src, final Path dst) throws IOException {
checkPermission(src, AccessType.WRITE);
checkPermission(dst,AccessType.WRITE);
boolean success = OBSPosixBucketUtils.renameBasedOnPosix(this, src, dst);
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.RENAME, true, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
+ OBSCommonUtils.setMetricsNormalInfo(this, OBSOperateAction.rename, startTime);
return success;
} else {
checkPermission(src, AccessType.WRITE);
checkPermission(dst, AccessType.WRITE);
boolean success = OBSObjectBucketUtils.renameBasedOnObject(this, src, dst);
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.RENAME, true, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
+ OBSCommonUtils.setMetricsNormalInfo(this, OBSOperateAction.rename, startTime);
return success;
}
} catch (ObsException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.RENAME, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
+ OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.rename, e);
throw OBSCommonUtils.translateException("rename(" + src + ", " + dst + ")", src, e);
- } catch (RenameFailedException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.RENAME, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
+ } catch (OBSRenameFailedException e) {
+ OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.rename, e);
LOG.error(e.getMessage());
return e.getExitCode();
} catch (FileNotFoundException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.RENAME, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
- LOG.error(e.toString());
+ OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.rename, e);
+ LOG.error("file not found when rename(" + src + ", " + dst + ")");
return false;
+ } catch (IOException e) {
+ OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.rename, e);
+ throw e;
} finally {
endTime = System.currentTimeMillis();
LOG.debug("Rename path {} to {} finished, thread:{}, " + "timeUsedInMilliSec:{}.", src, dst, threadId,
@@ -1301,6 +1418,47 @@ public boolean rename(final Path src, final Path dst) throws IOException {
}
}
+ @Override
+ public Path getTrashRoot(Path path) {
+ if (OBSConstants.HDFS_TRASH_VERSION_V2.equals(hdfsTrashVersion)) {
+ return this.makeQualified(new Path(hdfsTrashPrefix, getUsername()));
+ }
+
+ return super.getTrashRoot(path);
+ }
+
+ @Override
+ public Collection getTrashRoots(boolean allUsers) {
+ if (OBSConstants.HDFS_TRASH_VERSION_V2.equals(hdfsTrashVersion)) {
+ Path trashPrefix = new Path(hdfsTrashPrefix);
+ List ret = new ArrayList<>();
+ try {
+ if (!exists(trashPrefix)) {
+ return ret;
+ }
+
+ if (allUsers) {
+ FileStatus[] candidates = listStatus(trashPrefix);
+ for (FileStatus fs : candidates) {
+ ret.add(fs);
+ }
+ return ret;
+ }
+
+ Path userTrash = new Path(trashPrefix, getUsername());
+ if (exists(userTrash)) {
+ ret.add(getFileStatus(userTrash));
+ }
+ } catch (IOException e) {
+ LOG.warn("Cannot get all trash roots", e);
+ }
+
+ return ret;
+ }
+
+ return super.getTrashRoots(allUsers);
+ }
+
/**
* Return maximum number of entries in one multi-object delete call.
*
@@ -1381,56 +1539,30 @@ boolean isEnableMultiObjectDelete() {
public boolean delete(final Path f, final boolean recursive) throws IOException {
checkOpen();
long startTime = System.currentTimeMillis();
- long endTime;
try {
- FileStatus status = OBSCommonUtils.innerGetFileStatusWithRetry(this, f);
+ FileStatus status = OBSCommonUtils.getFileStatusWithRetry(this, f);
LOG.debug("delete: path {} - recursive {}", status.getPath(), recursive);
if (enablePosix) {
checkPermission(f,AccessType.WRITE);
boolean success = OBSPosixBucketUtils.fsDelete(this, status, recursive);
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.DELETE, true, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
+ OBSCommonUtils.setMetricsNormalInfo(this, OBSOperateAction.delete, startTime);
return success;
}
checkPermission(f,AccessType.WRITE);
- boolean success = OBSObjectBucketUtils.objectDelete(this, status, recursive);
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.DELETE, true, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
- return success;
+ return OBSObjectBucketUtils.objectDelete(this, status, recursive);
} catch (FileNotFoundException e) {
LOG.warn("Couldn't delete {} - does not exist", f);
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.DELETE, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
return false;
- } catch (FileConflictException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.DELETE, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
+ } catch (OBSFileConflictException e) {
+ OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.delete, e);
throw new AccessControlException(e);
} catch (ObsException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.DELETE, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
+ OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.delete, e);
throw OBSCommonUtils.translateException("delete", f, e);
+ } catch (IOException e) {
+ OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.delete, e);
+ throw e;
}
}
@@ -1439,8 +1571,8 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
*
* @return the flag
*/
- boolean isEnableTrash() {
- return enableTrash;
+ boolean isEnableFastDelete() {
+ return enableFastDelete;
}
/**
@@ -1448,8 +1580,15 @@ boolean isEnableTrash() {
*
* @return the trash directory
*/
- String getTrashDir() {
- return trashDir;
+ String getFastDeleteDir() {
+ if (fastDeleteVersion.equals(OBSConstants.FAST_DELETE_VERSION_V2)) {
+ SimpleDateFormat dateFmt = new SimpleDateFormat(OBSConstants.FAST_DELETE_VERSION_V2_CHECKPOINT_FORMAT);
+ String checkpointStr = dateFmt.format(new Date());
+ String checkpointDir = String.format(Locale.ROOT, "%s%s/",
+ OBSCommonUtils.maybeAddTrailingSlash(fastDeleteDir), checkpointStr);
+ return checkpointDir;
+ }
+ return fastDeleteDir;
}
/**
@@ -1469,28 +1608,31 @@ public FileStatus[] listStatus(final Path f) throws FileNotFoundException, IOExc
long threadId = Thread.currentThread().getId();
long endTime;
try {
- FileStatus[] statuses = OBSCommonUtils.innerListStatus(this, f, false);
+ FileStatus[] statuses = OBSCommonUtils.listStatus(this, f, false);
endTime = System.currentTimeMillis();
LOG.debug("List status for path:{}, thread:{}, timeUsedInMilliSec:{}", f, threadId, endTime - startTime);
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.NONRECURSIVE, BasicMetricsConsumer.MetricRecord.LIST_STATUS, true,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
return statuses;
} catch (ObsException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.NONRECURSIVE, BasicMetricsConsumer.MetricRecord.LIST_STATUS,
- false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
throw OBSCommonUtils.translateException("listStatus", f, e);
}
}
+ @Override
+ public Path getHomeDirectory() {
+ return this.makeQualified(new Path("/user/" + getUsername()));
+ }
+
+ static String getUsername() {
+ String user;
+ try {
+ user = UserGroupInformation.getCurrentUser().getShortUserName();
+ } catch(IOException ex) {
+ LOG.error("get user fail,fallback to system property user.name", ex);
+ user = System.getProperty("user.name");
+ }
+ return user;
+ }
+
/**
* 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
@@ -1510,24 +1652,11 @@ public FileStatus[] listStatus(final Path f, final boolean recursive) throws Fil
long threadId = Thread.currentThread().getId();
long endTime;
try {
- FileStatus[] statuses = OBSCommonUtils.innerListStatus(this, f, recursive);
+ FileStatus[] statuses = OBSCommonUtils.listStatus(this, f, recursive);
endTime = System.currentTimeMillis();
LOG.debug("List status for path:{}, thread:{}, timeUsedInMilliSec:{}", f, threadId, endTime - startTime);
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.RECURSIVE, BasicMetricsConsumer.MetricRecord.LIST_STATUS, true,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
return statuses;
} catch (ObsException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.RECURSIVE, BasicMetricsConsumer.MetricRecord.LIST_STATUS, false,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
throw OBSCommonUtils.translateException(
"listStatus with recursive flag[" + (recursive ? "true] " : "false] "), f, e);
}
@@ -1592,28 +1721,70 @@ public boolean mkdirs(final Path path, final FsPermission permission)
throws IOException, FileAlreadyExistsException {
checkOpen();
checkPermission(path,AccessType.WRITE);
- long startTime = System.currentTimeMillis();
- long endTime;
try {
- boolean success = OBSCommonUtils.innerMkdirs(this, path);
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.MKDIRS, true, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
- return success;
+ return OBSCommonUtils.mkdirs(this, path);
} catch (ObsException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.MKDIRS, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
throw OBSCommonUtils.translateException("mkdirs", path, e);
}
}
+ @Override
+ public BlockLocation[] getFileBlockLocations(FileStatus file, long start, long len) throws IOException {
+ if (!readPolicy.equals(OBSConstants.READAHEAD_POLICY_MEMARTSCC)
+ || this.getMemArtsCCClient() == null || !localityEnabled) {
+ // only support for MemArtsCC
+ return super.getFileBlockLocations(file, start, len);
+ }
+ if (file == null || start < 0L || len < 0L) {
+ return super.getFileBlockLocations(file, start, len);
+ } else if (file.isDirectory()) {
+ return super.getFileBlockLocations(file, start, len);
+ } else if (file.getLen() <= start) {
+ return new BlockLocation[0];
+ } else {
+ MemArtsCCClient ccClient = this.getMemArtsCCClient();
+ CcGetShardParam shardParam = buildCcShardParam(file, start, len);
+ int result = ccClient.getObjectShardInfo(shardParam);
+
+ if (result != OBSConstants.GET_SHARD_INFO_SUCCESS) {
+ LOG.error("Get memartscc shard info failed! ret code = {}", result);
+ return new BlockLocation[0];
+ }
+
+ ObjectShard[] objectShards = shardParam.getObjectShard();
+ int numOfBlocks = shardParam.getValidShardNum();
+ BlockLocation[] locations = new BlockLocation[numOfBlocks];
+ for(int i = 0; i < numOfBlocks; ++i) {
+ long offset = objectShards[i].getStart();
+ long length = objectShards[i].getEnd() - offset;
+ String[] hosts = objectShards[i].getHosts();
+ for (int j = 0; j < hosts.length; j++) {
+ InetAddress addr = InetAddress.getByName(hosts[j]);
+ String hostName = addr.getHostName();
+ hosts[j] = hostName;
+ }
+ locations[i] = new BlockLocation(hosts, hosts, offset, length);
+ }
+ return locations;
+ }
+ }
+
+ private CcGetShardParam buildCcShardParam(FileStatus file, long start, long len) {
+ String obsBucketName = getBucket();
+ long end = start + len;
+ String objKey = OBSCommonUtils.pathToKey(this, file.getPath());
+
+ int shardNum = (int) (len / this.blockSize + 1);
+ ObjectShard[] objectShards = new ObjectShard[shardNum];
+ for (int i = 0; i < shardNum; i++) {
+ String[] hosts = new String[OBSConstants.MAX_DUPLICATION_NUM];
+ objectShards[i] = new ObjectShard(hosts);
+ }
+
+ return new CcGetShardParam(start, end, obsBucketName, enablePosix, objKey,
+ objectShards, shardNum, 0);
+ }
+
/**
* Return a file status object that represents the path.
*
@@ -1626,26 +1797,16 @@ public boolean mkdirs(final Path path, final FsPermission permission)
public FileStatus getFileStatus(final Path f) throws FileNotFoundException, IOException {
checkOpen();
long startTime = System.currentTimeMillis();
- long endTime;
try {
- FileStatus fileStatus = OBSCommonUtils.innerGetFileStatusWithRetry(this, f);
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.GET_FILE_STATUS, true, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
+ FileStatus fileStatus = OBSCommonUtils.getFileStatusWithRetry(this, f);
+ OBSCommonUtils.setMetricsNormalInfo(this, OBSOperateAction.getFileStatus, startTime);
return fileStatus;
- } catch (FileConflictException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.GET_FILE_STATUS, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
+ } catch (OBSFileConflictException e) {
+ FileNotFoundException fileNotFoundException = new FileNotFoundException(e.getMessage());
+ OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.getFileStatus, fileNotFoundException);
// For super user, convert AccessControlException
// to null on NameNode then to FileNotFoundException on Client
- throw new FileNotFoundException(e.getMessage());
+ throw fileNotFoundException;
}
}
@@ -1658,13 +1819,10 @@ public FileStatus getFileStatus(final Path f) throws FileNotFoundException, IOEx
*/
@VisibleForTesting
OBSFileStatus innerGetFileStatus(final Path f) throws IOException {
- OBSFileStatus fileStatus;
if (enablePosix) {
- fileStatus = OBSPosixBucketUtils.innerFsGetObjectStatus(this, f);
- return fileStatus;
+ return OBSPosixBucketUtils.innerFsGetObjectStatus(this, f);
}
- fileStatus = OBSObjectBucketUtils.innerGetObjectStatus(this, f);
- return fileStatus;
+ return OBSObjectBucketUtils.innerGetObjectStatus(this, f);
}
/**
@@ -1678,8 +1836,6 @@ OBSFileStatus innerGetFileStatus(final Path f) throws IOException {
@Override
public ContentSummary getContentSummary(final Path f) throws FileNotFoundException, IOException {
checkOpen();
- long startTime = System.currentTimeMillis();
- long endTime;
ContentSummary contentSummary;
if (!obsContentSummaryEnable) {
return super.getContentSummary(f);
@@ -1687,14 +1843,8 @@ public ContentSummary getContentSummary(final Path f) throws FileNotFoundExcepti
FileStatus status;
try {
- status = OBSCommonUtils.innerGetFileStatusWithRetry(this, f);
- } catch (FileConflictException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.GET_CONTENT_SUMMARY, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
+ status = OBSCommonUtils.getFileStatusWithRetry(this, f);
+ } catch (OBSFileConflictException e) {
throw new AccessControlException(e);
}
@@ -1706,34 +1856,41 @@ public ContentSummary getContentSummary(final Path f) throws FileNotFoundExcepti
.directoryCount(0)
.spaceConsumed(length)
.build();
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.GET_CONTENT_SUMMARY, true, endTime - startTime);
-
- OBSCommonUtils.setMetricsInfo(this, record);
- }
return contentSummary;
}
// f is a directory
if (enablePosix) {
- contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummary(this, OBSCommonUtils.pathToKey(this, f));
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.GET_CONTENT_SUMMARY, true, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
+ contentSummary = null;
+ if (this.getConf()
+ .get(OBSConstants.OBS_CONTENT_SUMMARY_VERSION, OBSConstants.OBS_CONTENT_SUMMARY_VERSION_V2)
+ .equals(OBSConstants.OBS_CONTENT_SUMMARY_VERSION_V2)) {
+ boolean fallback = false;
+ try {
+ contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(this, status);
+ if (contentSummary.getFileCount() + contentSummary.getDirectoryCount()
+ < OBSConstants.OBS_CONTENT_SUMMARY_FALLBACK_THRESHOLD) {
+ fallback = true;
+ }
+ } catch (OBSMethodNotAllowedException e) {
+ LOG.debug("bucket[{}] not support fsGetDirectoryContentSummaryV2, fallback to V1, path={}, cause {}"
+ , bucket, f.toString(), e.getMessage());
+ fallback = true;
+ } catch (Exception e) {
+ LOG.warn("fsGetDirectoryContentSummaryV2 failed with exception, fallback to V1, path={}, cause {}"
+ , f.toString(), e.getMessage());
+ fallback = true;
+ }
+
+ if (!fallback) {
+ return contentSummary;
+ }
+ LOG.debug("fallback to getContentSummaryV1, path={}", f.toString());
}
+ contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummary(this, OBSCommonUtils.pathToKey(this, f));
return contentSummary;
} else {
contentSummary = OBSObjectBucketUtils.getDirectoryContentSummary(this, OBSCommonUtils.pathToKey(this, f));
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.GET_CONTENT_SUMMARY, true, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
return contentSummary;
}
}
@@ -1754,23 +1911,9 @@ public ContentSummary getContentSummary(final Path f) throws FileNotFoundExcepti
public void copyFromLocalFile(final boolean delSrc, final boolean overwrite, final Path src, final Path dst)
throws FileAlreadyExistsException, IOException {
checkOpen();
- long startTime = System.currentTimeMillis();
- long endTime;
try {
super.copyFromLocalFile(delSrc, overwrite, src, dst);
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.COPYFROMLOCAL, true, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
} catch (ObsException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.COPYFROMLOCAL, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
throw OBSCommonUtils.translateException("copyFromLocalFile(" + src + ", " + dst + ")", src, e);
}
}
@@ -1785,16 +1928,11 @@ public void close() throws IOException {
// already closed
return;
}
- long startTime = System.currentTimeMillis();
closeAllFilesBeingWritten();
-
closed = true;
- long endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.FS, BasicMetricsConsumer.MetricRecord.CLOSE, true,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
+
+ if (statsReporter != null) {
+ statsReporter.shutdownReport();
}
try {
@@ -1802,37 +1940,44 @@ public void close() throws IOException {
if (metricsConsumer != null) {
metricsConsumer.close();
}
- obs.close();
+ if (obs != null) {
+ obs.close();
+ }
} finally {
OBSCommonUtils.shutdownAll(boundedMultipartUploadThreadPool, boundedCopyThreadPool, boundedDeleteThreadPool,
boundedCopyPartThreadPool, boundedListThreadPool);
}
+ if (memArtsCCClient != null) {
+ memArtsCCClient.close();
+ }
+
LOG.info("Finish closing filesystem instance for uri: {}", uri);
}
+
+
@Override
public String getCanonicalServiceName() {
- if (authorizer != null && authorizer instanceof DelegationTokenCapability) {
- LOG.debug("getting CanonicalServiceName");
- return ((DelegationTokenCapability)authorizer).getCanonicalServiceName();
- } else if (enableCanonicalServiceName) {
- // Does not support Token, only enable for HBase BulkLoad
- return getScheme() + "://" + bucket;
- }
- return null;
+ return obsDelegationTokenManger != null
+ ? obsDelegationTokenManger.getCanonicalServiceName()
+ : getScheme() + "://" + bucket;
+ }
+
+ @Override
+ public Token>[] addDelegationTokens(String renewer, Credentials credentials) throws IOException {
+ LOG.info("add delegation tokens for renewer {}", renewer);
+ return obsDelegationTokenManger != null
+ ? obsDelegationTokenManger.addDelegationTokens(renewer, credentials)
+ : super.addDelegationTokens(renewer, credentials);
}
@Override
public Token> getDelegationToken(String renewer) throws IOException {
- if (authorizer != null && authorizer instanceof DelegationTokenCapability) {
- long st = System.currentTimeMillis();
- Token> delegationToken = ((DelegationTokenCapability) authorizer).getDelegationToken(renewer);
- long et = System.currentTimeMillis();
- LOG.debug("getDelegationToken:[renewer: {}], [cost: {}]", renewer, et - st);
- return delegationToken;
- }
- return super.getDelegationToken(renewer);
+ LOG.info("get delegation tokens for renewer {}", renewer);
+ return obsDelegationTokenManger != null
+ ? obsDelegationTokenManger.getDelegationToken(renewer)
+ : super.getDelegationToken(renewer);
}
/**
@@ -1949,8 +2094,6 @@ public RemoteIterator listFiles(final Path f, final boolean r
throws FileNotFoundException, IOException {
checkOpen();
checkPermission(f,AccessType.READ);
- long startTime = System.currentTimeMillis();
- long endTime;
RemoteIterator locatedFileStatus;
Path path = OBSCommonUtils.qualify(this, f);
@@ -1959,26 +2102,13 @@ public RemoteIterator listFiles(final Path f, final boolean r
// lookup dir triggers existence check
final FileStatus fileStatus;
try {
- fileStatus = OBSCommonUtils.innerGetFileStatusWithRetry(this, path);
- } catch (FileConflictException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.LIST_FILES, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
+ fileStatus = OBSCommonUtils.getFileStatusWithRetry(this, path);
+ } catch (OBSFileConflictException e) {
throw new AccessControlException(e);
}
if (fileStatus.isFile()) {
locatedFileStatus = new OBSListing.SingleStatusRemoteIterator(
OBSCommonUtils.toLocatedFileStatus(this, fileStatus));
-
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.LIST_FILES, true, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
// simple case: File
LOG.debug("Path is a file");
return locatedFileStatus;
@@ -1992,22 +2122,9 @@ public RemoteIterator listFiles(final Path f, final boolean r
obsListing.createFileStatusListingIterator(path,
OBSCommonUtils.createListObjectsRequest(this, key, delimiter),
org.apache.hadoop.fs.obs.OBSListing.ACCEPT_ALL, new OBSListing.AcceptFilesOnly(path)));
-
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.LIST_FILES, true, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
return locatedFileStatus;
}
} catch (ObsException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.LIST_FILES, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
throw OBSCommonUtils.translateException("listFiles", path, e);
}
}
@@ -2049,21 +2166,13 @@ public RemoteIterator listLocatedStatus(final Path f, final P
checkPermission(f,AccessType.READ);
Path path = OBSCommonUtils.qualify(this, f);
LOG.debug("listLocatedStatus({}, {}", path, filter);
- long startTime = System.currentTimeMillis();
- long endTime;
RemoteIterator locatedFileStatusRemoteList;
try {
// lookup dir triggers existence check
final FileStatus fileStatus;
try {
- fileStatus = OBSCommonUtils.innerGetFileStatusWithRetry(this, path);
- } catch (FileConflictException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.LIST_LOCATED_STS, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
+ fileStatus = OBSCommonUtils.getFileStatusWithRetry(this, path);
+ } catch (OBSFileConflictException e) {
throw new AccessControlException(e);
}
@@ -2072,12 +2181,6 @@ public RemoteIterator listLocatedStatus(final Path f, final P
LOG.debug("Path is a file");
locatedFileStatusRemoteList = new OBSListing.SingleStatusRemoteIterator(
filter.accept(path) ? OBSCommonUtils.toLocatedFileStatus(this, fileStatus) : null);
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.LIST_LOCATED_STS, true, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
return locatedFileStatusRemoteList;
} else {
// directory: trigger a lookup
@@ -2085,22 +2188,10 @@ public RemoteIterator listLocatedStatus(final Path f, final P
locatedFileStatusRemoteList = obsListing.createLocatedFileStatusIterator(
obsListing.createFileStatusListingIterator(path,
OBSCommonUtils.createListObjectsRequest(this, key, "/"), filter,
- new OBSListing.AcceptAllButSelfAndS3nDirs(path)));
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.LIST_LOCATED_STS, true, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
+ new OBSListing.AcceptAllButSelfAndOBSDirs(path)));
return locatedFileStatusRemoteList;
}
} catch (ObsException e) {
- endTime = System.currentTimeMillis();
- if (getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.LIST_LOCATED_STS, false, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(this, record);
- }
throw OBSCommonUtils.translateException("listLocatedStatus", path, e);
}
}
@@ -2136,4 +2227,52 @@ public boolean getMetricSwitch() {
int getInvokeCountThreshold() {
return invokeCountThreshold;
}
+
+ public TrafficStatistics getTrafficStatistics() {
+ return trafficStatistics;
+ }
+
+ /**
+ * Get disguise permission mode support stat.
+ *
+ * @return is disguise permission mode supported
+ */
+ boolean supportDisguisePermissionsMode() {
+ return OBSConstants.PERMISSIONS_MODE_DISGUISE.equals(permissionsMode) && enablePosix;
+ }
+
+ /**
+ * Set permission for given file.
+ *
+ * @param f the file to set permission info
+ * @param permission file's permission
+ * @throws IOException If an I/O error occurred
+ */
+ @Override
+ public void setPermission(final Path f, final FsPermission permission) throws IOException {
+ checkOpen();
+ checkPermission(f, AccessType.WRITE);
+ if (supportDisguisePermissionsMode()) {
+ LOG.debug("Set file {} permission to {}", f, permission);
+ OBSPosixBucketUtils.fsSetPermission(this, f, permission);
+ }
+ }
+
+ /**
+ * Set owner and group for given file.
+ *
+ * @param f the file to set owner and group info
+ * @param username file's owner
+ * @param groupname file's group
+ * @throws IOException If an I/O error occurred
+ */
+ @Override
+ public void setOwner(final Path f, final String username, final String groupname) throws IOException {
+ checkOpen();
+ checkPermission(f, AccessType.WRITE);
+ if (supportDisguisePermissionsMode()) {
+ LOG.debug("Set file {} owner to {} and group to {}", f , username, groupname);
+ OBSPosixBucketUtils.fsSetOwner(this, f, username, groupname);
+ }
+ }
}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFsDFSListing.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFsDFSListing.java
index 39abc1b..10b804a 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFsDFSListing.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFsDFSListing.java
@@ -51,7 +51,7 @@ static String fsDFSListNextBatch(final OBSFileSystem owner, final Stack resultQueue, fin
continue;
}
- for (ObsObject extenedCommonPrefixes : oneLevelObjectListing.getExtenedCommonPrefixes()) {
+ for (ObsObject extenedCommonPrefixes : oneLevelObjectListing.getExtendCommonPrefixes()) {
if (extenedCommonPrefixes.getObjectKey().equals(oneLevelListRequests.get(i).getPrefix())) {
// skip prefix itself
continue;
@@ -285,9 +285,9 @@ static int fetchListResultLocally(final Queue resultQueue, final int
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");
+ throw new IllegalStateException("cannot put list tail (" + listEntity + ") into result queue");
} else if (listEntity.getType() == ListEntityType.COMMON_PREFIX) {
- throw new RuntimeException("cannot put common prefix (" + listEntity + ") into result queue");
+ throw new IllegalStateException("cannot put common prefix (" + listEntity + ") into result queue");
} else {
objectSummaries.add(listEntity.getObjectSummary());
increaseLevelStats(levelStatsList, listEntity.getLevel(),
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSHDFSFileSystem.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSHDFSFileSystem.java
index 069cd7f..d84c837 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSHDFSFileSystem.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSHDFSFileSystem.java
@@ -112,7 +112,7 @@ private List> getMountList(final Configuration config, fina
if (pathPrefix.endsWith(Path.SEPARATOR) && !pathPrefix.trim().equals(Path.SEPARATOR)) {
pathPrefix = pathPrefix.substring(0, pathPrefix.length() - 1);
}
- mountList.add(new Pair(pathPrefix, si.getValue()));
+ mountList.add(new Pair<>(pathPrefix, si.getValue()));
}
}
}
@@ -264,9 +264,10 @@ private Path transferToWrappedPath(Path path, TransferedPath newPath) {
public void initialize(URI theUri, Configuration conf) throws IOException {
this.wrapperConf = new Configuration(conf);
wrapperConf.set("fs.hdfs.impl", DistributedFileSystem.class.getName());
+ wrapperConf.setBoolean("fs.hdfs.impl.disable.cache", true);
super.initialize(theUri, conf);
- underHDFS = (DistributedFileSystem) FileSystem.newInstance(theUri, wrapperConf);
+ underHDFS = (DistributedFileSystem) FileSystem.get(theUri, wrapperConf);
final String authority = theUri.getAuthority();
mountMap = new HashMap<>();
@@ -531,9 +532,9 @@ public FileStatus[] listStatus(Path p) throws IOException {
@Override
public RemoteIterator listFiles(Path f, boolean recursive)
- throws FileNotFoundException, IOException {
+ throws IOException {
TransferedPath newPath = transferToNewPath(f);
- return new WrappedRemoteIterator(newPath.getFS().listFiles(newPath.toPath(), recursive),
+ return new WrappedRemoteIterator<>(newPath.getFS().listFiles(newPath.toPath(), recursive),
fileStatus -> {
Path originPath = transferToWrappedPath(fileStatus.getPath(), newPath);
fileStatus.setPath(originPath);
@@ -544,7 +545,7 @@ public RemoteIterator listFiles(Path f, boolean recursive)
@Override
public RemoteIterator listLocatedStatus(Path p, final PathFilter filter) throws IOException {
TransferedPath newPath = transferToNewPath(p);
- return new WrappedRemoteIterator(newPath.getFS().listLocatedStatus(newPath.toPath()),
+ return new WrappedRemoteIterator<>(newPath.getFS().listLocatedStatus(newPath.toPath()),
fileStatus -> {
Path originPath = transferToWrappedPath(fileStatus.getPath(), newPath);
fileStatus.setPath(originPath);
@@ -555,7 +556,7 @@ public RemoteIterator listLocatedStatus(Path p, final PathFil
@Override
public RemoteIterator listStatusIterator(Path p) throws IOException {
TransferedPath newPath = transferToNewPath(p);
- return new WrappedRemoteIterator(newPath.getFS().listStatusIterator(newPath.toPath()),
+ return new WrappedRemoteIterator<>(newPath.getFS().listStatusIterator(newPath.toPath()),
fileStatus -> {
Path originPath = transferToWrappedPath(fileStatus.getPath(), newPath);
fileStatus.setPath(originPath);
@@ -617,7 +618,7 @@ public FsStatus getStatus(Path p) throws IOException {
@Override
public RemoteIterator listCorruptFileBlocks(Path path) throws IOException {
TransferedPath newPath = transferToNewPath(path);
- return new WrappedRemoteIterator(newPath.getFS().listCorruptFileBlocks(newPath.toPath()),
+ return new WrappedRemoteIterator<>(newPath.getFS().listCorruptFileBlocks(newPath.toPath()),
p -> transferToWrappedPath(p, newPath));
}
@@ -802,7 +803,7 @@ public Path getTrashRoot(Path path) {
}
}
- private static class WrappedRemoteIterator implements RemoteIterator {
+ private static class WrappedRemoteIterator implements RemoteIterator {
private final RemoteIterator origin;
@@ -819,7 +820,7 @@ public boolean hasNext() throws IOException {
}
@Override
- public Object next() throws IOException {
+ public T next() throws IOException {
return convertFunc.apply(origin.next());
}
}
@@ -919,6 +920,8 @@ public Path toPath() {
}
static class UncheckException extends RuntimeException {
+ static final long serialVersionUID = 5746198432791324945L;
+
public UncheckException(IOException origin) {
super(origin);
}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIOException.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIOException.java
index 2d56d5f..c78f697 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIOException.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIOException.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.fs.obs;
-import com.google.common.base.Preconditions;
import com.obs.services.exception.ObsException;
import java.io.IOException;
@@ -26,27 +25,25 @@
/**
* IOException equivalent to {@link ObsException}.
*/
-public class OBSIOException extends IOException {
+public class OBSIOException extends IOException implements WithErrCode {
private static final long serialVersionUID = -1582681108285856259L;
-
- /**
- * Peration message.
- */
- private final String operation;
+ private String errCode;
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;
+ super(operationMsg,cause);
}
- public ObsException getCause() {
+ public synchronized ObsException getCause() {
return (ObsException) super.getCause();
}
+ public void setErrCode(String errCode) {
+ this.errCode = errCode;
+ }
+
@Override
- public String getMessage() {
- return operation + ": " + getCause().getErrorMessage() + ", detailMessage: " + super.getMessage();
+ public String getErrCode() {
+ return this.errCode;
}
+
}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIllegalArgumentException.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIllegalArgumentException.java
new file mode 100644
index 0000000..e97b36c
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIllegalArgumentException.java
@@ -0,0 +1,27 @@
+package org.apache.hadoop.fs.obs;
+
+import java.io.IOException;
+
+/**
+ * description
+ *
+ * @since 2022-02-07
+ */
+public class OBSIllegalArgumentException extends IOException implements WithErrCode {
+
+ private static final long serialVersionUID = 2188013092663783231L;
+ private String errCode;
+
+ OBSIllegalArgumentException(final String message) {
+ super(message);
+ }
+
+ public void setErrCode(String errCode) {
+ this.errCode = errCode;
+ }
+
+ @Override
+ public String getErrCode() {
+ return this.errCode;
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSInvoker.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSInvoker.java
new file mode 100644
index 0000000..3933828
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSInvoker.java
@@ -0,0 +1,115 @@
+/*
+ * 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 org.apache.hadoop.io.retry.RetryPolicy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+
+public class OBSInvoker {
+ private static final Logger LOG = LoggerFactory.getLogger(OBSInvoker.class);
+
+ public static final RetryCallback LOG_EVENT = new RetryCallback() {
+ @Override
+ public void onFailure(String text,
+ IOException exception,
+ int retries,
+ boolean idempotent) {
+ LOG.debug("retry #{}, {}", retries, exception);
+ }
+ };
+
+ private final RetryPolicyWithMaxTime retryPolicy;
+ private final RetryCallback retryCallback;
+ private final OBSFileSystem fs;
+
+ public OBSInvoker(OBSFileSystem fs, RetryPolicyWithMaxTime retryPolicy, RetryCallback callback) {
+ this.retryPolicy = retryPolicy;
+ this.retryCallback = callback;
+ this.fs = fs;
+ }
+
+ public T retryByMaxTime(OBSOperateAction action, String path, OBSCallable operation, boolean idempotent)
+ throws IOException {
+ return retryByMaxTime(action, path, operation, idempotent, retryCallback);
+ }
+
+
+ public T retryByMaxTime(OBSOperateAction action, String path, OBSCallable operation, boolean idempotent, RetryCallback retrying)
+ throws IOException {
+ int retryCount = 0;
+ long startTime = System.currentTimeMillis();
+ IOException translated = null;
+ RetryPolicy.RetryAction retryAction;
+ boolean shouldRetry;
+ do {
+ try {
+ return operation.call();
+ } catch (IOException e) {
+ translated = e;
+ } catch (ObsException e) {
+ translated = OBSCommonUtils.translateException(action.toString(), path, e);
+ }
+
+ OBSCommonUtils.putQosMetric(fs, action, translated);
+
+ String text = action + " on " + path;
+
+ try {
+ retryAction = retryPolicy.shouldRetryByMaxTime(startTime, translated, retryCount, 0,
+ idempotent);
+ shouldRetry = retryAction.action.equals(
+ RetryPolicy.RetryAction.RETRY.action);
+ if (shouldRetry) {
+ retryCount++;
+ retrying.onFailure(text, translated, retryCount, idempotent);
+ Thread.sleep(retryAction.delayMillis);
+ }
+ } catch (InterruptedException e) {
+ translated = new InterruptedIOException(text + ",interrupted in retry process");
+ translated.initCause(e);
+ shouldRetry = false;
+ Thread.currentThread().interrupt();
+ } catch (Exception e) {
+ translated = new IOException(text + ",unexpect exception in retry process", e);
+ shouldRetry = false;
+ }
+ } while (shouldRetry);
+
+ if (retryCount != 0) {
+ LOG.error("retry {} times fail: {}", retryCount, translated.toString());
+ }
+ throw translated;
+ }
+
+
+ @FunctionalInterface
+ public interface RetryCallback {
+ void onFailure(
+ String text,
+ IOException exception,
+ int retries,
+ boolean idempotent);
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSListing.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSListing.java
index 3b84a49..308d697 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSListing.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSListing.java
@@ -41,29 +41,20 @@
* 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;
+ public String toString() {
+ return "ACCEPT_ALL";
}
@Override
- public String toString() {
- return "ACCEPT_ALL";
+ public boolean accept(final Path file) {
+ return true;
}
};
- /**
- * Class logger.
- */
private static final Logger LOG = LoggerFactory.getLogger(OBSListing.class);
- /**
- * OBS File System instance.
- */
private final OBSFileSystem owner;
OBSListing(final OBSFileSystem ownerFS) {
@@ -98,30 +89,27 @@ LocatedFileStatusIterator createLocatedFileStatusIterator(final RemoteIterator {
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();
@@ -380,21 +360,13 @@ public FileStatus next() throws IOException {
* @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;
}
@@ -403,8 +375,9 @@ private boolean requestNextBatch() throws IOException {
*
* @param objects the next object listing
* @return true if this added any entries after filtering
+ * @throws IOException If an I/O error occurred
*/
- private boolean buildNextStatusBatch(final ObjectListing objects) {
+ private boolean buildNextStatusBatch(final ObjectListing objects) throws IOException {
// counters for debug logs
int added = 0;
int ignored = 0;
@@ -420,7 +393,7 @@ private boolean buildNextStatusBatch(final ObjectListing objects) {
// 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.getShortUserName());
+ owner.getDefaultBlockSize(keyPath), owner);
LOG.debug("Adding: {}", status);
stats.add(status);
added++;
@@ -431,31 +404,31 @@ private boolean buildNextStatusBatch(final ObjectListing objects) {
}
// prefixes: always directories
- for (ObsObject prefix : objects.getExtenedCommonPrefixes()) {
+ for (ObsObject prefix : objects.getExtendCommonPrefixes()) {
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,
+ OBSFileStatus status = new OBSFileStatus(keyPath, lastModified, lastModified,
owner.getShortUserName());
- LOG.debug("Adding directory: {}", status);
+ if (owner.supportDisguisePermissionsMode()) {
+ OBSCommonUtils.setAccessControlAttrForFileStatus(owner, status,
+ OBSCommonUtils.getObjectMetadata(owner, key));
+ }
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;
+ boolean ret = statusBatchIterator.hasNext();
+ LOG.debug("Added {}; ignored {}; hasNext={}; hasMoreObjects={}",
+ added, ignored, ret, objects.isTruncated());
+ return ret;
}
/**
@@ -531,25 +504,11 @@ class ObjectListingIterator implements RemoteIterator {
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) {
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLocalDirAllocator.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLocalDirAllocator.java
deleted file mode 100644
index a2c21be..0000000
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLocalDirAllocator.java
+++ /dev/null
@@ -1,601 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.obs;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.DF;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.util.DiskChecker;
-import org.apache.hadoop.util.DiskChecker.DiskErrorException;
-import org.apache.hadoop.util.StringUtils;
-
-import java.io.File;
-import java.io.IOException;
-import java.security.SecureRandom;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Random;
-import java.util.TreeMap;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * this class copy from hadoop-common module.
- * in flink or other long run application,OBSAllocatorPerContext#createTmpFileForWrite method invoke
- * File.deleteOnExit,and will cause JVM OOM.
- *
- * JDK-4872014:The File.deleteOnExit API is flawed and causes JVM crashes
- * on long running servers (or even short running if the API is used enough). See bug 4513817.
- */
-
-public class OBSLocalDirAllocator {
-
- //A Map from the config item names like "mapred.local.dir"
- //to the instance of the AllocatorPerContext. This
- //is a static object to make sure there exists exactly one instance per JVM
- private static Map contexts = new TreeMap();
-
- private String contextCfgItemName;
-
- /**
- * Used when size of file to be allocated is unknown.
- */
- public static final int SIZE_UNKNOWN = -1;
-
- /**
- * Create an allocator object
- *
- * @param contextCfgItemName
- */
- public OBSLocalDirAllocator(String contextCfgItemName) {
- this.contextCfgItemName = contextCfgItemName;
- }
-
- /**
- * This method must be used to obtain the dir allocation context for a
- * particular value of the context name. The context name must be an item
- * defined in the Configuration object for which we want to control the
- * dir allocations (e.g., mapred.local.dir
). The method will
- * create a context for that name if it doesn't already exist.
- */
- private OBSAllocatorPerContext obtainContext(String contextCfgItemName) {
- synchronized (contexts) {
- OBSAllocatorPerContext l = contexts.get(contextCfgItemName);
- if (l == null) {
- contexts.put(contextCfgItemName, l = new OBSAllocatorPerContext(contextCfgItemName));
- }
- return l;
- }
- }
-
- /**
- * Get a path from the local FS. This method should be used if the size of
- * the file is not known apriori. We go round-robin over the set of disks
- * (via the configured dirs) and return the first complete path where
- * we could create the parent directory of the passed path.
- *
- * @param pathStr the requested path (this will be created on the first
- * available disk)
- * @param conf the Configuration object
- * @return the complete path to the file on a local disk
- * @throws IOException
- */
- public Path getLocalPathForWrite(String pathStr, Configuration conf) throws IOException {
- return getLocalPathForWrite(pathStr, SIZE_UNKNOWN, conf);
- }
-
- /**
- * Get a path from the local FS. Pass size as
- * SIZE_UNKNOWN if not known apriori. We
- * round-robin over the set of disks (via the configured dirs) and return
- * the first complete path which has enough space
- *
- * @param pathStr the requested path (this will be created on the first
- * available disk)
- * @param size the size of the file that is going to be written
- * @param conf the Configuration object
- * @return the complete path to the file on a local disk
- * @throws IOException
- */
- public Path getLocalPathForWrite(String pathStr, long size, Configuration conf) throws IOException {
- return getLocalPathForWrite(pathStr, size, conf, true);
- }
-
- /**
- * Get a path from the local FS. Pass size as
- * SIZE_UNKNOWN if not known apriori. We
- * round-robin over the set of disks (via the configured dirs) and return
- * the first complete path which has enough space
- *
- * @param pathStr the requested path (this will be created on the first
- * available disk)
- * @param size the size of the file that is going to be written
- * @param conf the Configuration object
- * @param checkWrite ensure that the path is writable
- * @return the complete path to the file on a local disk
- * @throws IOException
- */
- public Path getLocalPathForWrite(String pathStr, long size, Configuration conf, boolean checkWrite)
- throws IOException {
- OBSAllocatorPerContext context = obtainContext(contextCfgItemName);
- return context.getLocalPathForWrite(pathStr, size, conf, checkWrite);
- }
-
- /**
- * Get a path from the local FS for reading. We search through all the
- * configured dirs for the file's existence and return the complete
- * path to the file when we find one
- *
- * @param pathStr the requested file (this will be searched)
- * @param conf the Configuration object
- * @return the complete path to the file on a local disk
- * @throws IOException
- */
- public Path getLocalPathToRead(String pathStr, Configuration conf) throws IOException {
- OBSAllocatorPerContext context = obtainContext(contextCfgItemName);
- return context.getLocalPathToRead(pathStr, conf);
- }
-
- /**
- * Get all of the paths that currently exist in the working directories.
- *
- * @param pathStr the path underneath the roots
- * @param conf the configuration to look up the roots in
- * @return all of the paths that exist under any of the roots
- * @throws IOException
- */
- public Iterable getAllLocalPathsToRead(String pathStr, Configuration conf) throws IOException {
- OBSAllocatorPerContext context;
- synchronized (this) {
- context = obtainContext(contextCfgItemName);
- }
- return context.getAllLocalPathsToRead(pathStr, conf);
- }
-
- /**
- * Creates a temporary file in the local FS. Pass size as -1 if not known
- * apriori. We round-robin over the set of disks (via the configured dirs)
- * and select the first complete path which has enough space. A file is
- * created on this directory. The file is guaranteed to go away when the
- * JVM exits.
- *
- * @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
- */
- public File createTmpFileForWrite(String pathStr, long size, Configuration conf) throws IOException {
- OBSAllocatorPerContext context = obtainContext(contextCfgItemName);
- return context.createTmpFileForWrite(pathStr, size, conf);
- }
-
- /**
- * Method to check whether a context is valid
- *
- * @param contextCfgItemName
- * @return true/false
- */
- public static boolean isContextValid(String contextCfgItemName) {
- synchronized (contexts) {
- return contexts.containsKey(contextCfgItemName);
- }
- }
-
- /**
- * Removes the context from the context config items
- *
- * @param contextCfgItemName
- */
- @Deprecated
- @InterfaceAudience.LimitedPrivate({"MapReduce"})
- public static void removeContext(String contextCfgItemName) {
- synchronized (contexts) {
- contexts.remove(contextCfgItemName);
- }
- }
-
- /**
- * We search through all the configured dirs for the file's existence
- * and return true when we find
- *
- * @param pathStr the requested file (this will be searched)
- * @param conf the Configuration object
- * @return true if files exist. false otherwise
- * @throws IOException
- */
- public boolean ifExists(String pathStr, Configuration conf) {
- OBSAllocatorPerContext context = obtainContext(contextCfgItemName);
- return context.ifExists(pathStr, conf);
- }
-
- /**
- * Get the current directory index for the given configuration item.
- *
- * @return the current directory index for the given configuration item.
- */
- int getCurrentDirectoryIndex() {
- OBSAllocatorPerContext context = obtainContext(contextCfgItemName);
- return context.getCurrentDirectoryIndex();
- }
-
- private static class OBSAllocatorPerContext {
-
- private final Log log = LogFactory.getLog(OBSAllocatorPerContext.class);
-
- private Random dirIndexRandomizer = new Random();
-
- private String contextCfgItemName;
-
- // NOTE: the context must be accessed via a local reference as it
- // may be updated at any time to reference a different context
- private AtomicReference currentContext;
-
- private static class Context {
- private AtomicInteger dirNumLastAccessed = new AtomicInteger(0);
-
- private FileSystem localFS;
-
- private DF[] dirDF;
-
- private Path[] localDirs;
-
- private String savedLocalDirs;
-
- public int getAndIncrDirNumLastAccessed() {
- return getAndIncrDirNumLastAccessed(1);
- }
-
- public int getAndIncrDirNumLastAccessed(int delta) {
- if (localDirs.length < 2 || delta == 0) {
- return dirNumLastAccessed.get();
- }
- int oldval;
- int newval;
- do {
- oldval = dirNumLastAccessed.get();
- newval = (oldval + delta) % localDirs.length;
- } while (!dirNumLastAccessed.compareAndSet(oldval, newval));
- return oldval;
- }
- }
-
- public OBSAllocatorPerContext(String contextCfgItemName) {
- this.contextCfgItemName = contextCfgItemName;
- this.currentContext = new AtomicReference(new Context());
- }
-
- /**
- * This method gets called everytime before any read/write to make sure
- * that any change to localDirs is reflected immediately.
- */
- private Context confChanged(Configuration conf) throws IOException {
- Context ctx = currentContext.get();
- String newLocalDirs = conf.get(contextCfgItemName);
- if (null == newLocalDirs) {
- throw new IOException(contextCfgItemName + " not configured");
- }
- if (!newLocalDirs.equals(ctx.savedLocalDirs)) {
- ctx = confChanged2(conf, newLocalDirs);
- }
-
- return ctx;
- }
-
- private Context confChanged2(Configuration conf, String newLocalDirs) throws IOException {
- Context ctx = new Context();
- String[] dirStrings = StringUtils.getTrimmedStrings(newLocalDirs);
- ctx.localFS = FileSystem.getLocal(conf);
- int numDirs = dirStrings.length;
- ArrayList dirs = new ArrayList(numDirs);
- ArrayList dfList = new ArrayList(numDirs);
- for (int i = 0; i < numDirs; i++) {
- try {
- // filter problematic directories
- Path tmpDir = new Path(dirStrings[i]);
- if (ctx.localFS.mkdirs(tmpDir) || ctx.localFS.exists(tmpDir)) {
- try {
- File tmpFile = tmpDir.isAbsolute()
- ? new File(ctx.localFS.makeQualified(tmpDir).toUri())
- : new File(dirStrings[i]);
-
- DiskChecker.checkDir(tmpFile);
- dirs.add(new Path(tmpFile.getPath()));
- dfList.add(new DF(tmpFile, 30000));
- } catch (DiskErrorException de) {
- log.warn(dirStrings[i] + " is not writable\n", de);
- }
- } else {
- log.warn("Failed to create " + dirStrings[i]);
- }
- } catch (IOException ie) {
- log.warn("Failed to create " + dirStrings[i] + ": " + ie.getMessage() + "\n", ie);
- } //ignore
- }
- ctx.localDirs = dirs.toArray(new Path[dirs.size()]);
- ctx.dirDF = dfList.toArray(new DF[dirs.size()]);
- ctx.savedLocalDirs = newLocalDirs;
-
- if (dirs.size() > 0) {
- // randomize the first disk picked in the round-robin selection
- ctx.dirNumLastAccessed.set(dirIndexRandomizer.nextInt(dirs.size()));
- }
-
- currentContext.set(ctx);
-
- return ctx;
- }
-
- private Path createPath(Path dir, String path, boolean checkWrite) throws IOException {
- Path file = new Path(dir, path);
- if (checkWrite) {
- //check whether we are able to create a directory here. If the disk
- //happens to be RDONLY we will fail
- try {
- DiskChecker.checkDir(new File(file.getParent().toUri().getPath()));
- return file;
- } catch (DiskErrorException d) {
- log.warn("Disk Error Exception: ", d);
- return null;
- }
- }
- return file;
- }
-
- /**
- * Get the current directory index.
- *
- * @return the current directory index.
- */
- int getCurrentDirectoryIndex() {
- return currentContext.get().dirNumLastAccessed.get();
- }
-
- /**
- * Get a path from the local FS. If size is known, we go
- * round-robin over the set of disks (via the configured dirs) and return
- * the first complete path which has enough space.
- *
- * If size is not known, use roulette selection -- pick directories
- * with probability proportional to their available space.
- */
- public Path getLocalPathForWrite(String pathStr, long size, Configuration conf, boolean checkWrite)
- throws IOException {
- Context ctx = confChanged(conf);
- int numDirs = ctx.localDirs.length;
- int numDirsSearched = 0;
- //remove the leading slash from the path (to make sure that the uri
- //resolution results in a valid path on the dir being checked)
- if (pathStr.startsWith("/")) {
- pathStr = pathStr.substring(1);
- }
- Path returnPath = null;
-
- if (size == SIZE_UNKNOWN) { //do roulette selection: pick dir with probability
- //proportional to available size
- long[] availableOnDisk = new long[ctx.dirDF.length];
- long totalAvailable = 0;
-
- //build the "roulette wheel"
- for (int i = 0; i < ctx.dirDF.length; ++i) {
- availableOnDisk[i] = ctx.dirDF[i].getAvailable();
- totalAvailable += availableOnDisk[i];
- }
-
- if (totalAvailable == 0) {
- throw new DiskErrorException("No space available in any of the local directories.");
- }
-
- // Keep rolling the wheel till we get a valid path
- SecureRandom r = new SecureRandom();
- // Random r = new Random();
- while (numDirsSearched < numDirs && returnPath == null) {
- long randomPosition = (r.nextLong() >>> 1) % totalAvailable;
- int dir = 0;
- while (randomPosition > availableOnDisk[dir]) {
- randomPosition -= availableOnDisk[dir];
- dir++;
- }
- ctx.dirNumLastAccessed.set(dir);
- returnPath = createPath(ctx.localDirs[dir], pathStr, checkWrite);
- if (returnPath == null) {
- totalAvailable -= availableOnDisk[dir];
- availableOnDisk[dir] = 0; // skip this disk
- numDirsSearched++;
- }
- }
- } else {
- int dirNum = ctx.getAndIncrDirNumLastAccessed();
- while (numDirsSearched < numDirs) {
- long capacity = ctx.dirDF[dirNum].getAvailable();
- if (capacity > size) {
- returnPath = createPath(ctx.localDirs[dirNum], pathStr, checkWrite);
- if (returnPath != null) {
- ctx.getAndIncrDirNumLastAccessed(numDirsSearched);
- break;
- }
- }
- dirNum++;
- dirNum = dirNum % numDirs;
- numDirsSearched++;
- }
- }
- if (returnPath != null) {
- return returnPath;
- }
-
- //no path found
- throw new DiskErrorException("Could not find any valid local " + "directory for " + pathStr);
- }
-
- /**
- * Creates a file on the local FS. Pass size as
- * {@link OBSLocalDirAllocator#SIZE_UNKNOWN} if not known apriori. We
- * round-robin over the set of disks (via the configured dirs) and return
- * a file on the first path which has enough space. The file is guaranteed
- * to go away when the JVM exits.
- */
- public File createTmpFileForWrite(String pathStr, long size, Configuration conf) throws IOException {
-
- // find an appropriate directory
- Path path = getLocalPathForWrite(pathStr, size, conf, true);
- File dir = new File(path.getParent().toUri().getPath());
- String prefix = path.getName();
-
- // create a temp file on this directory
- File result = File.createTempFile(prefix, null, dir);
-
- return result;
- }
-
- /**
- * Get a path from the local FS for reading. We search through all the
- * configured dirs for the file's existence and return the complete
- * path to the file when we find one
- */
- public Path getLocalPathToRead(String pathStr, Configuration conf) throws IOException {
- Context ctx = confChanged(conf);
- int numDirs = ctx.localDirs.length;
- int numDirsSearched = 0;
- //remove the leading slash from the path (to make sure that the uri
- //resolution results in a valid path on the dir being checked)
- if (pathStr.startsWith("/")) {
- pathStr = pathStr.substring(1);
- }
- while (numDirsSearched < numDirs) {
- Path file = new Path(ctx.localDirs[numDirsSearched], pathStr);
- if (ctx.localFS.exists(file)) {
- return file;
- }
- numDirsSearched++;
- }
-
- //no path found
- throw new DiskErrorException(
- "Could not find " + pathStr + " in any of" + " the configured local directories");
- }
-
- private static class PathIterator implements Iterator, Iterable {
- private final FileSystem fs;
-
- private final String pathStr;
-
- private int i = 0;
-
- private final Path[] rootDirs;
-
- private Path next = null;
-
- private PathIterator(FileSystem fs, String pathStr, Path[] rootDirs) throws IOException {
- this.fs = fs;
- this.pathStr = pathStr;
- this.rootDirs = rootDirs;
- advance();
- }
-
- @Override
- public boolean hasNext() {
- return next != null;
- }
-
- private void advance() throws IOException {
- while (i < rootDirs.length) {
- next = new Path(rootDirs[i++], pathStr);
- if (fs.exists(next)) {
- return;
- }
- }
- next = null;
- }
-
- @Override
- public Path next() {
- final Path result = next;
- try {
- advance();
- } catch (IOException ie) {
- throw new RuntimeException("Can't check existence of " + next, ie);
- }
- if (result == null) {
- throw new NoSuchElementException();
- }
- return result;
- }
-
- @Override
- public void remove() {
- throw new UnsupportedOperationException("read only iterator");
- }
-
- @Override
- public Iterator iterator() {
- return this;
- }
- }
-
- /**
- * Get all of the paths that currently exist in the working directories.
- *
- * @param pathStr the path underneath the roots
- * @param conf the configuration to look up the roots in
- * @return all of the paths that exist under any of the roots
- * @throws IOException
- */
- Iterable getAllLocalPathsToRead(String pathStr, Configuration conf) throws IOException {
- Context ctx = confChanged(conf);
- if (pathStr.startsWith("/")) {
- pathStr = pathStr.substring(1);
- }
- return new PathIterator(ctx.localFS, pathStr, ctx.localDirs);
- }
-
- /**
- * We search through all the configured dirs for the file's existence
- * and return true when we find one
- */
- public boolean ifExists(String pathStr, Configuration conf) {
- Context ctx = currentContext.get();
- try {
- int numDirs = ctx.localDirs.length;
- int numDirsSearched = 0;
- //remove the leading slash from the path (to make sure that the uri
- //resolution results in a valid path on the dir being checked)
- if (pathStr.startsWith("/")) {
- pathStr = pathStr.substring(1);
- }
- while (numDirsSearched < numDirs) {
- Path file = new Path(ctx.localDirs[numDirsSearched], pathStr);
- if (ctx.localFS.exists(file)) {
- return true;
- }
- numDirsSearched++;
- }
- } catch (IOException e) {
- log.error(e);
- }
- return false;
- }
- }
-}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLoginHelper.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLoginHelper.java
index f7d31a4..7e9a14a 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLoginHelper.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLoginHelper.java
@@ -18,9 +18,6 @@
package org.apache.hadoop.fs.obs;
-import static org.apache.commons.lang.StringUtils.equalsIgnoreCase;
-
-import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -151,7 +148,7 @@ public static Login extractLoginDetails(final URI name) {
}
} catch (UnsupportedEncodingException e) {
// this should never happen; translate it if it does.
- throw new RuntimeException(e);
+ throw new IllegalArgumentException(e);
}
}
@@ -216,13 +213,13 @@ public static void checkPath(final Configuration conf, final URI fsUri, final Pa
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
+ if (OBSCommonUtils.stringEqualsIgnoreCase(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())) {
+ if (OBSCommonUtils.stringEqualsIgnoreCase(thisScheme, defaultUri.getScheme())) {
pathUri = defaultUri; // schemes match, so use this uri instead
} else {
pathUri = null; // can't determine auth of the path
@@ -232,7 +229,7 @@ public static void checkPath(final Configuration conf, final URI fsUri, final Pa
// canonicalize uri before comparing with this fs
pathUri = canonicalizeUri(pathUri, defaultPort);
thatHost = pathUri.getHost();
- if (equalsIgnoreCase(thisHost, thatHost)) {
+ if (OBSCommonUtils.stringEqualsIgnoreCase(thisHost, thatHost)) {
return;
}
}
@@ -289,7 +286,7 @@ public static class Login {
* @return true if the username is defined (not null, not empty).
*/
public boolean hasLogin() {
- return StringUtils.isNotEmpty(user);
+ return OBSCommonUtils.isStringNotEmpty(user);
}
/**
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSMethodNotAllowedException.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSMethodNotAllowedException.java
new file mode 100644
index 0000000..b45c522
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSMethodNotAllowedException.java
@@ -0,0 +1,22 @@
+package org.apache.hadoop.fs.obs;
+
+import java.io.IOException;
+
+public class OBSMethodNotAllowedException extends IOException implements WithErrCode {
+
+ private static final long serialVersionUID = 2461327923217975442L;
+ private String errCode;
+
+ OBSMethodNotAllowedException(final String message) {
+ super(message);
+ }
+
+ public void setErrCode(String errCode) {
+ this.errCode = errCode;
+ }
+
+ @Override
+ public String getErrCode() {
+ return this.errCode;
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSObjectBucketUtils.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSObjectBucketUtils.java
index e1cba55..3ea1e4c 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSObjectBucketUtils.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSObjectBucketUtils.java
@@ -19,7 +19,6 @@
import com.obs.services.model.PartEtag;
import com.obs.services.model.PutObjectRequest;
-import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.ParentNotDirectoryException;
@@ -34,16 +33,17 @@
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.Comparator;
import java.util.Set;
import java.util.TreeSet;
+import java.util.Locale;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
/**
- * Object bucket specific utils for {@link OBSFileSystem}.
+ * Object bucket specific utils
*/
final class OBSObjectBucketUtils {
/**
@@ -62,7 +62,7 @@ private OBSObjectBucketUtils() {
* @param src path to be renamed
* @param dst new path after rename
* @return boolean
- * @throws RenameFailedException if some criteria for a state changing
+ * @throws OBSRenameFailedException 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,
@@ -73,7 +73,7 @@ private OBSObjectBucketUtils() {
* @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 {
+ throws OBSRenameFailedException, FileNotFoundException, IOException, ObsException {
String srcKey = OBSCommonUtils.pathToKey(owner, src);
String dstKey = OBSCommonUtils.pathToKey(owner, dst);
@@ -84,11 +84,11 @@ static boolean renameBasedOnObject(final OBSFileSystem owner, final Path src, fi
// get the source file status; this raises a FNFE if there is no source
// file.
- FileStatus srcStatus = OBSCommonUtils.innerGetFileStatusWithRetry(owner, src);
+ FileStatus srcStatus = OBSCommonUtils.getFileStatusWithRetry(owner, src);
FileStatus dstStatus;
try {
- dstStatus = OBSCommonUtils.innerGetFileStatusWithRetry(owner, dst);
+ dstStatus = OBSCommonUtils.getFileStatusWithRetry(owner, 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
@@ -98,12 +98,12 @@ static boolean renameBasedOnObject(final OBSFileSystem owner, final Path src, fi
String filename = srcKey.substring(OBSCommonUtils.pathToKey(owner, src.getParent()).length() + 1);
newDstKey = newDstKey + filename;
dstKey = newDstKey;
- dstStatus = OBSCommonUtils.innerGetFileStatusWithRetry(owner, OBSCommonUtils.keyToPath(dstKey));
+ dstStatus = OBSCommonUtils.getFileStatusWithRetry(owner, OBSCommonUtils.keyToPath(dstKey));
if (dstStatus.isDirectory()) {
- throw new RenameFailedException(src, dst, "new destination is an existed directory").withExitCode(
+ throw new OBSRenameFailedException(src, dst, "new destination is an existed directory").withExitCode(
false);
} else {
- throw new RenameFailedException(src, dst, "new destination is an existed file").withExitCode(false);
+ throw new OBSRenameFailedException(src, dst, "new destination is an existed file").withExitCode(false);
}
} else {
@@ -111,7 +111,7 @@ static boolean renameBasedOnObject(final OBSFileSystem owner, final Path src, fi
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);
+ throw new OBSRenameFailedException(src, dst, "destination is an existed file").withExitCode(false);
}
}
} catch (FileNotFoundException e) {
@@ -153,13 +153,13 @@ private static void checkDestinationParent(final OBSFileSystem owner, final Path
Path parent = dst.getParent();
if (!OBSCommonUtils.pathToKey(owner, parent).isEmpty()) {
try {
- FileStatus dstParentStatus = OBSCommonUtils.innerGetFileStatusWithRetry(owner, dst.getParent());
+ FileStatus dstParentStatus = OBSCommonUtils.getFileStatusWithRetry(owner, 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 ");
+ throw new OBSRenameFailedException(src, dst, "destination has no parent ");
}
}
}
@@ -403,11 +403,7 @@ static void createFakeDirectory(final OBSFileSystem owner, final String objectNa
// Used to create an empty file that represents an empty directory
static void createEmptyObject(final OBSFileSystem owner, final String objectName) throws IOException {
- long delayMs;
- int retryTime = 0;
- long startTime = System.currentTimeMillis();
- IOException lastException = null;
- while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
+ OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.createEmptyObject, objectName, () -> {
InputStream im = null;
try {
im = new InputStream() {
@@ -421,36 +417,13 @@ public int read() {
owner.getObsClient().putObject(putObjectRequest);
owner.getSchemeStatistics().incrementWriteOps(1);
owner.getSchemeStatistics().incrementBytesWritten(putObjectRequest.getMetadata().getContentLength());
- return;
- } catch (ObsException e) {
- LOG.debug("create empty obj failed with [{}], " + "retry time [{}] - request id [{}] - "
- + "error code [{}] - error message [{}]", e.getResponseCode(), retryTime, e.getErrorRequestId(),
- e.getErrorCode(), e.getErrorMessage());
-
- IOException ioException = OBSCommonUtils.translateException("innerCreateEmptyObject", objectName, e);
- if (!(ioException instanceof OBSIOException)) {
- throw ioException;
- }
-
- lastException = ioException;
-
- delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime);
- retryTime++;
- if (System.currentTimeMillis() - startTime + delayMs
- < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
- try {
- Thread.sleep(delayMs);
- } catch (InterruptedException ie) {
- throw ioException;
- }
- }
} finally {
if (im != null) {
im.close();
}
}
- }
- throw lastException;
+ return null;
+ }, true);
}
/**
@@ -464,34 +437,12 @@ public int read() {
* @throws IOException Other IO problems
*/
static void copyFile(final OBSFileSystem owner, final String srcKey, final String dstKey, final long size)
- throws IOException, InterruptedIOException {
- long delayMs;
- int retryTime = 0;
- long startTime = System.currentTimeMillis();
- while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
- try {
- innerCopyFile(owner, srcKey, dstKey, size);
- return;
- } catch (InterruptedIOException e) {
- throw e;
- } catch (OBSIOException e) {
- String errMsg = String.format("Failed to copy file from %s to " + "%s with size %s, retry time %s",
- srcKey, dstKey, size, retryTime);
- LOG.debug(errMsg, e);
- delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime);
- retryTime++;
- if (System.currentTimeMillis() - startTime + delayMs
- < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
- try {
- Thread.sleep(delayMs);
- } catch (InterruptedException ie) {
- LOG.error(errMsg, e);
- throw e;
- }
- }
- }
- }
- innerCopyFile(owner, srcKey, dstKey, size);
+ throws IOException {
+ String path = srcKey + " to " + dstKey;
+ OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.copyFile, path, () -> {
+ innerCopyFile(owner, srcKey, dstKey, size);
+ return null;
+ }, true);
}
private static void innerCopyFile(final OBSFileSystem owner, final String srcKey, final String dstKey,
@@ -572,8 +523,7 @@ static List getCopyFilePartEtags(final OBSFileSystem owner, final Stri
owner.getObsClient()
.abortMultipartUpload(new AbortMultipartUploadRequest(owner.getBucket(), dstKey, uploadId));
- throw OBSCommonUtils.extractException(
- "Multi-part copy with id '" + uploadId + "' from " + srcKey + "to " + dstKey, dstKey, e);
+ throw OBSCommonUtils.extractException("copy part from " + srcKey + " to " + dstKey, dstKey, e);
}
// Make part numbers in ascending order
@@ -634,7 +584,7 @@ static OBSFileStatus innerGetObjectStatus(final OBSFileSystem owner, final Path
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)) {
+ if (OBSCommonUtils.isStringNotEmpty(key)) {
try {
ObjectMetadata meta = getObjectMetadata(owner, key);
@@ -644,7 +594,7 @@ static OBSFileStatus innerGetObjectStatus(final OBSFileSystem owner, final Path
} else {
LOG.debug("Found exact file: normal file");
return new OBSFileStatus(meta.getContentLength(), OBSCommonUtils.dateToLong(meta.getLastModified()),
- path, owner.getDefaultBlockSize(path), owner.getShortUserName());
+ path, owner.getDefaultBlockSize(path), owner.getShortUserName(), meta.getEtag());
}
} catch (ObsException e) {
if (e.getResponseCode() != OBSCommonUtils.NOT_FOUND_CODE) {
@@ -665,7 +615,7 @@ static OBSFileStatus innerGetObjectStatus(final OBSFileSystem owner, final Path
return new OBSFileStatus(meta.getContentLength(),
OBSCommonUtils.dateToLong(meta.getLastModified()), path, owner.getDefaultBlockSize(path),
- owner.getShortUserName());
+ owner.getShortUserName(), meta.getEtag());
}
} catch (ObsException e) {
if (e.getResponseCode() != OBSCommonUtils.NOT_FOUND_CODE) {
@@ -676,7 +626,7 @@ static OBSFileStatus innerGetObjectStatus(final OBSFileSystem owner, final Path
}
try {
- boolean isEmpty = OBSCommonUtils.innerIsFolderEmpty(owner, key);
+ boolean isEmpty = OBSCommonUtils.isFolderEmpty(owner, key);
LOG.debug("Is dir ({}) empty? {}", path, isEmpty);
return new OBSFileStatus(path, owner.getShortUserName());
} catch (ObsException e) {
@@ -727,7 +677,7 @@ static ContentSummary getDirectoryContentSummary(final OBSFileSystem owner, fina
}
summary[2] += directories.size();
LOG.debug(
- String.format("file size [%d] - file count [%d] - directory count [%d] - " + "file path [%s]", summary[0],
+ String.format(Locale.ROOT,"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])
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSOperateAction.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSOperateAction.java
new file mode 100644
index 0000000..23432fd
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSOperateAction.java
@@ -0,0 +1,32 @@
+package org.apache.hadoop.fs.obs;
+
+public enum OBSOperateAction {
+ getFileStatus,
+ rename,
+ delete,
+ getBucketFsStatus,
+ listObjects,
+ isFolderEmpty,
+ onReadFailure,
+ createEmptyObject,
+ createFolder,
+ truncate,
+ readOneByte,
+ readByteBuff,
+ readbytes,
+ readrandom,
+ readfully,
+ lazySeek,
+ putObject,
+ write,
+ uploadPart,
+ initMultiPartUpload,
+ completeMultipartUpload,
+ copyFile,
+ appendFile,
+ getContentSummaryFs,
+ listContentSummaryFs,
+ initialize,
+ getObjectMetadata,
+ setObjectMetadata
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSPosixBucketUtils.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSPosixBucketUtils.java
index 035070c..a79d1bd 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSPosixBucketUtils.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSPosixBucketUtils.java
@@ -5,39 +5,68 @@
import com.obs.services.model.ListObjectsRequest;
import com.obs.services.model.ObjectListing;
import com.obs.services.model.ObsObject;
+import com.obs.services.model.SetObjectMetadataRequest;
+import com.obs.services.model.fs.ContentSummaryFsRequest;
+import com.obs.services.model.fs.ContentSummaryFsResult;
+import com.obs.services.model.fs.DirContentSummary;
+import com.obs.services.model.fs.DirSummary;
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 com.obs.services.model.fs.ListContentSummaryFsRequest;
+import com.obs.services.model.fs.ListContentSummaryFsResult;
+import org.apache.hadoop.conf.Configuration;
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.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.security.AccessControlException;
+import com.google.common.collect.ImmutableMap;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.FileNotFoundException;
import java.io.IOException;
+import java.io.InterruptedIOException;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Date;
import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
/**
- * Posix bucket specific utils for {@link OBSFileSystem}.
+ * Posix bucket specific utils
*/
final class OBSPosixBucketUtils {
- /**
- * Class logger.
- */
private static final Logger LOG = LoggerFactory.getLogger(OBSPosixBucketUtils.class);
+ private static RetryPolicyWithMaxTime retryPolicy;
+ private static RetryPolicyWithMaxTime obsRetryPolicy;
+
private OBSPosixBucketUtils() {
}
+ public static void init(Configuration conf) {
+ retryPolicy = new OBSRetryPolicy.ExponentialBackoffRetryPolicy(
+ conf.getInt(OBSConstants.RETRY_LIMIT, OBSConstants.DEFAULT_RETRY_LIMIT),
+ conf.getLong(OBSConstants.RETRY_MAXTIME, OBSConstants.DEFAULT_RETRY_MAXTIME),
+ conf.getLong(OBSConstants.RETRY_SLEEP_BASETIME, OBSConstants.DEFAULT_RETRY_SLEEP_BASETIME),
+ conf.getLong(OBSConstants.RETRY_SLEEP_MAXTIME, OBSConstants.DEFAULT_RETRY_SLEEP_MAXTIME));
+ obsRetryPolicy = new OBSRetryPolicy(conf);
+ }
+
/**
* Used to judge that an object is a file or folder.
*
@@ -64,7 +93,7 @@ static boolean fsIsFolder(final ObsFSAttribute attr) {
* @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
+ * @throws OBSRenameFailedException 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,
@@ -83,7 +112,7 @@ static boolean renameBasedOnPosix(final OBSFileSystem owner, final Path src, fin
}
try {
- FileStatus dstStatus = OBSCommonUtils.innerGetFileStatusWithRetry(owner, dstPath);
+ FileStatus dstStatus = OBSCommonUtils.getFileStatusWithRetry(owner, dstPath);
if (dstStatus.isDirectory()) {
String newDstString = OBSCommonUtils.maybeAddTrailingSlash(dstPath.toString());
String filename = srcKey.substring(OBSCommonUtils.pathToKey(owner, src.getParent()).length() + 1);
@@ -108,7 +137,7 @@ static boolean renameBasedOnPosix(final OBSFileSystem owner, final Path src, fin
// 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) {
+ } catch (OBSFileConflictException e) {
throw new ParentNotDirectoryException(e.getMessage());
}
@@ -125,65 +154,70 @@ static boolean innerFsRenameWithRetry(final OBSFileSystem owner, final Path src,
final String srcKey, final String dstKey) throws IOException {
String newSrcKey = srcKey;
String newDstKey = dstKey;
- IOException lastException;
- long delayMs;
- int retryTime = 0;
+ int retries = 0;
long startTime = System.currentTimeMillis();
- do {
+ while (true) {
boolean isRegularDirPath = newSrcKey.endsWith("/") && newDstKey.endsWith("/");
try {
- LOG.debug("rename: {}-st rename from [{}] to [{}] ...", retryTime, newSrcKey, newDstKey);
+ LOG.debug("rename: {}-st rename from [{}] to [{}] ...", retries, newSrcKey, newDstKey);
innerFsRenameFile(owner, newSrcKey, newDstKey);
return true;
} catch (FileNotFoundException e) {
if (owner.exists(dst)) {
- LOG.debug("rename: successfully {}-st rename src [{}] " + "to dest [{}] with SDK retry", retryTime,
- src, dst, e);
+ LOG.debug("file not found when rename. rename: successfully {}-st rename src [{}] " + "to dest [{}] with SDK retry", retries,
+ src, dst);
return true;
} else {
- LOG.error("rename: failed {}-st rename src [{}] to dest [{}]", retryTime, src, dst, e);
+ LOG.error("file not found when rename. rename: failed {}-st rename src [{}] to dest [{}]", retries, src, dst);
return false;
}
- } catch (IOException e) {
- if (e instanceof AccessControlException && isRegularDirPath) {
+ } catch (AccessControlException e) {
+ if (isRegularDirPath) {
throw e;
}
-
try {
- FileStatus srcFileStatus = OBSCommonUtils.innerGetFileStatusWithRetry(owner, src);
+ FileStatus srcFileStatus = OBSCommonUtils.getFileStatusWithRetry(owner, src);
if (srcFileStatus.isDirectory()) {
newSrcKey = OBSCommonUtils.maybeAddTrailingSlash(newSrcKey);
newDstKey = OBSCommonUtils.maybeAddTrailingSlash(newDstKey);
- } else if (e instanceof AccessControlException) {
+ continue;
+ } else {
throw e;
}
- } catch (FileConflictException e1) {
+ } catch (OBSFileConflictException e1) {
+ throw new AccessControlException(e);
+ }
+ } catch (IOException e) {
+ try {
+ OBSCommonUtils.getFileStatusWithRetry(owner, src);
+ } catch (OBSFileConflictException e1) {
throw new AccessControlException(e);
}
-
- lastException = e;
- 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,
+ LOG.warn("rename: failed {}-st rename src [{}] to " + "dest [{}] with SDK retry", retries, src,
dst, e);
return false;
}
- delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime);
- retryTime++;
- if (System.currentTimeMillis() - startTime + delayMs
- < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
- try {
- Thread.sleep(delayMs);
- } catch (InterruptedException ie) {
- throw e;
- }
+ OBSCommonUtils.putQosMetric(owner, OBSOperateAction.rename, e);
+
+ RetryPolicy.RetryAction rc;
+ try {
+ rc = obsRetryPolicy.shouldRetryByMaxTime(startTime, e, retries++, 0, true);
+ } catch (Exception e1) {
+ throw new IOException("unexpected exception ", e1);
+ }
+ if (rc.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
+ throw e;
+ }
+ try {
+ LOG.warn("retry #{}, {}", retries, e);
+ Thread.sleep(rc.delayMillis);
+ } catch (InterruptedException e1) {
+ throw (IOException)new InterruptedIOException("interrupted").initCause(e1);
}
}
- } while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY);
-
- LOG.error("rename: failed {}-st rename src [{}] to dest [{}]", retryTime, src, dst, lastException);
- throw lastException;
+ }
}
static void innerFsRenameFile(final OBSFileSystem owner, final String srcKey, final String dstKey)
@@ -258,7 +292,7 @@ static boolean fsDelete(final OBSFileSystem owner, final FileStatus status, fina
LOG.debug("delete: Deleting fake empty directory {} - recursive {}", f, recursive);
try {
OBSCommonUtils.deleteObject(owner, key);
- } catch (FileConflictException e) {
+ } catch (OBSFileConflictException e) {
LOG.warn("delete emtryDir[{}] has conflict exception, "
+ "will retry.", key, e);
trashFolderIfNeed(owner, key);
@@ -283,39 +317,34 @@ private static void trashObjectIfNeed(final OBSFileSystem owner, final String ke
}
mkTrash(owner, key);
- String destKeyWithNoSuffix = owner.getTrashDir() + key;
+ String destKeyWithNoSuffix = owner.getFastDeleteDir() + key;
String destKey = destKeyWithNoSuffix;
SimpleDateFormat df = new SimpleDateFormat("-yyyyMMddHHmmssSSS");
if (owner.exists(new Path(destKey))) {
destKey = destKeyWithNoSuffix + df.format(new Date());
}
- // add timestamp when rename failed to avoid multi clients rename sources to the same target
+
+ LOG.debug("Moved file : '" + key + "' to trash at: " + destKey);
+ int retries = 0;
long startTime = System.currentTimeMillis();
- int retryTime = 0;
- long delayMs;
while (!fsRenameToNewObject(owner, key, destKey)) {
- LOG.debug("Move file [{}] to [{}] failed, retryTime[{}].", key,
- destKey, retryTime);
-
- delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime);
- if (System.currentTimeMillis() - startTime + delayMs
- > OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
- LOG.error("Failed rename file [{}] to [{}] after "
- + "retryTime[{}].", key, destKey, retryTime);
- throw new IOException("Failed to rename " + key + " to " + destKey);
- } else {
- try {
- Thread.sleep(delayMs);
- } catch (InterruptedException ie) {
- LOG.error("Failed rename file [{}] to [{}] after "
- + "retryTime[{}].", key, destKey, retryTime);
- throw new IOException("Failed to rename " + key + " to " + destKey);
- }
+ RetryPolicy.RetryAction rc;
+ try {
+ rc = retryPolicy.shouldRetryByMaxTime(startTime, null, retries++, 0, true);
+ } catch (Exception e1) {
+ throw new IOException("unexpected exception ", e1);
+ }
+ if (rc.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
+ throw new IOException("failed to rename " + key + " to " + destKey);
+ }
+ try {
+ LOG.warn("retry #{}, fast delete {} to {}", retries, key, destKey);
+ Thread.sleep(rc.delayMillis);
+ destKey = destKeyWithNoSuffix + df.format(new Date());
+ } catch (InterruptedException e1) {
+ throw (IOException)new InterruptedIOException("interrupted").initCause(e1);
}
- destKey = destKeyWithNoSuffix + df.format(new Date());
- retryTime++;
}
- LOG.debug("Moved file : '" + key + "' to trash at: " + destKey);
}
private static void trashFolderIfNeed(final OBSFileSystem owner, final String key)
@@ -326,7 +355,7 @@ private static void trashFolderIfNeed(final OBSFileSystem owner, final String ke
}
mkTrash(owner, key);
- StringBuilder sb = new StringBuilder(owner.getTrashDir());
+ StringBuilder sb = new StringBuilder(owner.getFastDeleteDir());
SimpleDateFormat df = new SimpleDateFormat("-yyyyMMddHHmmssSSS");
int endIndex = key.endsWith("/") ? key.length() - 1 : key.length();
sb.append(key, 0, endIndex);
@@ -339,64 +368,56 @@ private static void trashFolderIfNeed(final OBSFileSystem owner, final String ke
String srcKey = OBSCommonUtils.maybeAddTrailingSlash(key);
String dstKey = OBSCommonUtils.maybeAddTrailingSlash(destKey);
- // add timestamp when rename failed to avoid multi clients rename sources to the same target
+ LOG.debug("Moved folder : '" + key + "' to trash: " + destKey);
+ int retries = 0;
long startTime = System.currentTimeMillis();
- int retryTime = 0;
- long delayMs;
while (!fsRenameToNewObject(owner, srcKey, dstKey)) {
- LOG.debug("Move folder [{}] to [{}] failed, retryTime[{}].", key,
- destKey, retryTime);
-
- delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime);
- if (System.currentTimeMillis() - startTime + delayMs
- > OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
- LOG.error("Failed rename folder [{}] to [{}] after "
- + "retryTime[{}].", key, destKey, retryTime);
- throw new IOException("Failed to rename " + key + " to " + destKey);
- } else {
- try {
- Thread.sleep(delayMs);
- } catch (InterruptedException ie) {
- LOG.error("Failed rename folder [{}] to [{}] after "
- + "retryTime[{}].", key, destKey, retryTime);
- throw new IOException("Failed to rename " + key + " to " + destKey);
- }
+ RetryPolicy.RetryAction rc;
+ try {
+ rc = retryPolicy.shouldRetryByMaxTime(startTime, null, retries++, 0, true);
+ } catch (Exception e1) {
+ throw new IOException("unexpected exception ", e1);
+ }
+ if (rc.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
+ throw new IOException("failed to rename " + key + " to " + destKey);
+ }
+ try {
+ LOG.warn("retry #{}, fast delete {} to {}", retries, key, destKey);
+ Thread.sleep(rc.delayMillis);
+ destKey = destKeyWithNoSuffix + df.format(new Date());
+ dstKey = OBSCommonUtils.maybeAddTrailingSlash(destKey);
+ } catch (InterruptedException e1) {
+ throw (IOException)new InterruptedIOException("interrupted").initCause(e1);
}
- destKey = destKeyWithNoSuffix + df.format(new Date());
- dstKey = OBSCommonUtils.maybeAddTrailingSlash(destKey);
- retryTime++;
}
- LOG.debug("Moved folder : '" + key + "' to trash at: " + destKey);
}
static void fsRecursivelyDeleteDirWithRetry(final OBSFileSystem owner,
final String key, boolean deleteParent) throws IOException {
+ int retries = 0;
long startTime = System.currentTimeMillis();
- long delayMs;
- int retryTime = 0;
- while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
+ while (true) {
try {
- long delNum = fsRecursivelyDeleteDir(owner, key, deleteParent);
- LOG.debug("Recursively delete {} files/dirs when deleting {}",
- delNum, key);
+ fsRecursivelyDeleteDir(owner, key, deleteParent);
return;
- } catch (FileConflictException e) {
- LOG.warn("Recursively delete [{}] has conflict exception, "
- + "retryTime[{}].", key, e);
- delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime);
- retryTime++;
- if (System.currentTimeMillis() - startTime + delayMs
- < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
- try {
- Thread.sleep(delayMs);
- } catch (InterruptedException ie) {
- throw e;
- }
+ } catch (OBSFileConflictException e) {
+ RetryPolicy.RetryAction rc;
+ try {
+ LOG.warn("retry #{}, {}", retries, e);
+ rc = retryPolicy.shouldRetryByMaxTime(startTime, e, retries++, 0, true);
+ } catch (Exception e1) {
+ throw new IOException("Unexpected exception ", e1);
+ }
+ if (rc.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
+ throw e;
+ }
+ try {
+ Thread.sleep(rc.delayMillis);
+ } catch (InterruptedException e1) {
+ throw (IOException)new InterruptedIOException("interrupted").initCause(e1);
}
}
}
-
- fsRecursivelyDeleteDir(owner, key, deleteParent);
}
static long fsRecursivelyDeleteDir(final OBSFileSystem owner, final String parentKey, final boolean deleteParent)
@@ -456,13 +477,13 @@ static long fsRecursivelyDeleteDir(final OBSFileSystem owner, final String paren
private static boolean needToTrash(final OBSFileSystem owner, final String key) {
String newKey = key;
newKey = OBSCommonUtils.maybeDeleteBeginningSlash(newKey);
- if (owner.isEnableTrash()) {
- String trashPathKey = OBSCommonUtils.pathToKey(owner, new Path(owner.getTrashDir()));
+ if (owner.isEnableFastDelete()) {
+ String trashPathKey = OBSCommonUtils.pathToKey(owner, new Path(owner.getFastDeleteDir()));
if (newKey.startsWith(trashPathKey)) {
return false;
}
}
- return owner.isEnableTrash();
+ return owner.isEnableFastDelete();
}
// Delete a sub dir.
@@ -482,7 +503,7 @@ private static int fsRemoveSubdir(final OBSFileSystem owner, final String subdir
private static void mkTrash(final OBSFileSystem owner, final String key) throws ObsException, IOException {
String newKey = key;
- StringBuilder sb = new StringBuilder(owner.getTrashDir());
+ StringBuilder sb = new StringBuilder(owner.getFastDeleteDir());
newKey = OBSCommonUtils.maybeAddTrailingSlash(newKey);
sb.append(newKey);
sb.deleteCharAt(sb.length() - 1);
@@ -500,39 +521,13 @@ static void fsCreateFolder(final OBSFileSystem owner, final String objectName) t
final NewFolderRequest newFolderRequest = new NewFolderRequest(owner.getBucket(), newObjectName);
newFolderRequest.setAcl(owner.getCannedACL());
long len = newFolderRequest.getObjectKey().length();
- IOException lastException = null;
- long delayMs;
- int retryTime = 0;
- long startTime = System.currentTimeMillis();
- while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
- try {
- owner.getObsClient().newFolder(newFolderRequest);
- owner.getSchemeStatistics().incrementWriteOps(1);
- owner.getSchemeStatistics().incrementBytesWritten(len);
- return;
- } catch (ObsException e) {
- LOG.debug("Failed to create folder [{}], retry time [{}], " + "exception [{}]", newObjectName,
- retryTime, e);
+ OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.createFolder, objectName, () -> {
+ owner.getObsClient().newFolder(newFolderRequest);
+ return null;
+ },true);
- IOException ioException = OBSCommonUtils.translateException("innerFsCreateFolder", newObjectName, e);
- if (!(ioException instanceof OBSIOException)) {
- throw ioException;
- }
- lastException = ioException;
-
- delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime);
- retryTime++;
- if (System.currentTimeMillis() - startTime + delayMs
- < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
- try {
- Thread.sleep(delayMs);
- } catch (InterruptedException ie) {
- throw ioException;
- }
- }
- }
- }
- throw lastException;
+ owner.getSchemeStatistics().incrementWriteOps(1);
+ owner.getSchemeStatistics().incrementBytesWritten(len);
}
// Used to get the status of a file or folder in a file-gateway bucket.
@@ -549,16 +544,22 @@ static OBSFileStatus innerFsGetObjectStatus(final OBSFileSystem owner, final Pat
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()),
+ OBSFileStatus status = new OBSFileStatus(path, OBSCommonUtils.dateToLong(meta.getLastModified()),
owner.getShortUserName());
- } 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.getShortUserName());
+ OBSCommonUtils.setAccessControlAttrForFileStatus(owner, status, meta.getAllMetadata());
+ return status;
}
+
+ LOG.debug("Found file (with /): real file? should not happen: {}", key);
+ OBSFileStatus status = new OBSFileStatus(meta.getContentLength(),
+ OBSCommonUtils.dateToLong(meta.getLastModified()), path, owner.getDefaultBlockSize(path),
+ owner.getShortUserName(), meta.getEtag());
+ OBSCommonUtils.setAccessControlAttrForFileStatus(owner, status, meta.getAllMetadata());
+ return status;
} catch (ObsException e) {
throw OBSCommonUtils.translateException("getFileStatus", path, e);
}
@@ -601,7 +602,7 @@ static ContentSummary fsGetDirectoryContentSummary(final OBSFileSystem owner, fi
objects = OBSCommonUtils.continueListObjects(owner, objects);
}
LOG.debug(
- String.format("file size [%d] - file count [%d] - directory count [%d] - " + "file path [%s]", summary[0],
+ String.format(Locale.ROOT,"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])
@@ -612,41 +613,425 @@ static ContentSummary fsGetDirectoryContentSummary(final OBSFileSystem owner, fi
static void innerFsTruncateWithRetry(final OBSFileSystem owner, final Path f, final long newLength)
throws IOException {
- long delayMs;
- int retryTime = 0;
- long startTime = System.currentTimeMillis();
- while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
+ LOG.debug("truncate {} to newLength {}", f, newLength);
+ String key = OBSCommonUtils.pathToKey(owner, f);
+ OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.truncate, key, () -> {
+ owner.getObsClient().truncateObject(owner.getBucket(), key, newLength);
+ return null;
+ }, true);
+ owner.getSchemeStatistics().incrementWriteOps(1);
+ }
+
+ static ContentSummary fsGetDirectoryContentSummaryV2(final OBSFileSystem owner, final FileStatus fileStatus)
+ throws IOException {
+ FsDirContentSummaryCounter summaryCounter = new FsDirContentSummaryCounter(owner, fileStatus);
+ return summaryCounter.getContentSummary();
+ }
+
+ private static class FsDirContentSummaryCounter {
+ public static final int MULTI_LIST_CS_MAX_DIR = 10; // multiListCS allow up to 10 dir in one batch
+
+ public static final int MIN_RETRY_TIME = 3;
+
+ private final OBSFileSystem owner;
+
+ private final LinkedBlockingDeque deque;
+
+ private final Counter counter;
+
+ private final FileStatus fileStatus;
+
+ private final int parallelFactor;
+
+ public FsDirContentSummaryCounter(OBSFileSystem owner, FileStatus fileStatus) {
+ this.owner = owner;
+ this.deque = new LinkedBlockingDeque<>();
+ this.counter = new Counter();
+ this.fileStatus = fileStatus;
+ this.parallelFactor = owner.getConf().getInt(OBSConstants.MULTILISTCS_PARALLEL_FACTOR,
+ OBSConstants.DEFAULT_MULTILISTCS_PARALLEL_FACTOR);
+ }
+
+ /**
+ * get content summary of path, path should be a dir
+ *
+ * @return
+ */
+ public ContentSummary getContentSummary() throws IOException {
+ if (!fileStatus.isDirectory()) {
+ throw new IllegalArgumentException("the input should be a dir");
+ }
+ this.counter.increase(1, 0, 0); // count root itself
+ LOG.debug("counter increase (1, 0, 0) for root itself");
+
+ Path path = fileStatus.getPath();
+ String key = OBSCommonUtils.pathToKey(owner, path);
+ if (path.isRoot()) {
+ key = "/"; // special case of 'root'
+ }
+ ContentSummaryFsRequest getCSReq = new ContentSummaryFsRequest();
+ getCSReq.setBucketName(this.owner.getBucket());
+ getCSReq.setDirName(key);
+
+ // 1. getCS of root dir
try {
- innerFsTruncate(owner, f, newLength);
+ ContentSummaryFsResult getCSRes = OBSCommonUtils.getOBSInvoker().retryByMaxTime(
+ OBSOperateAction.getContentSummaryFs, key, () -> this.owner.getObsClient().getContentSummaryFs(getCSReq),
+ true);
+ DirSummary summary = getCSRes.getContentSummary();
+ this.counter.increase(summary.getDirCount(), summary.getFileCount(), summary.getFileSize());
+ LOG.debug("counter increase ({}, {}, {}) for [{}]",
+ summary.getDirCount(), summary.getFileCount(), summary.getFileSize(), key);
+ if (summary.getDirCount() != 0) {
+ enqueue(new Node(key, null, summary.getInode()));
+ }
+ } catch (ObsException e) {
+ if (e.getResponseCode() == OBSCommonUtils.NOT_ALLOWED_CODE) {
+ throw new UnsupportedOperationException("unsupported getContentSummaryFs");
+ }
+ throw OBSCommonUtils.translateException("getContentSummaryFs", path, e);
+ }
+
+ countExhaustive();
+
+ return this.counter.getContentSummary();
+ }
+
+ private void countExhaustive() throws IOException {
+ while (true) {
+ List> futures = new ArrayList<>();
+
+ // 2. try to get node for queue
+ try {
+ for (int i = 0; i < parallelFactor; i++) {
+ BatchNodes nodes = tryToGetBatchNodes();
+ if (nodes.size() == 0) { // queue is empty, break out
+ break;
+ } else {
+ // 3. submit task
+ futures.add(submitTask(nodes));
+ }
+ }
+ } catch (InterruptedException e) {
+ LOG.warn("getContentSummaryV2 for [{}] failed because get nodes from queue failed, {}",
+ fileStatus.getPath().toString(), e.getMessage());
+ throw new IOException(String.format(
+ "getContentSummaryV2 for [%s] failed because get nodes from queue failed",
+ fileStatus.getPath().toString()), e);
+ }
+
+ // 4. check finish
+ if (futures.size() == 0) {
+ if (deque.size() != 0) {
+ continue;
+ }
+ break;
+ }
+
+ // 5. wait for batch task finished
+ waitBatchTaskFinish(futures);
+ }
+ }
+
+ private BatchNodes tryToGetBatchNodes() throws InterruptedException, IOException {
+ BatchNodes ret = new BatchNodes();
+ int size = this.deque.size();
+ for (int i = 0; i < size; i++) {
+ if (ret.size() == MULTI_LIST_CS_MAX_DIR) {
+ break;
+ }
+ Node n = this.deque.pollFirst(1, TimeUnit.SECONDS);
+ if (n != null && verifyNodeRetryState(n)) {
+ ret.add(n);
+ }
+ }
+ return ret;
+ }
+
+ // verify node should add to request
+ private boolean verifyNodeRetryState(Node n) throws IOException {
+ final int retryState = n.retryState();
+ switch (retryState) {
+ case Node.RETRY_STATE_TRIGGER:
+ return true;
+ case Node.RETRY_STATE_DISCARD:
+ LOG.error("node[key={} marker={}] failed {} times, due to {}",
+ n.getPath(), n.getMarker(), n.getRetryNum(), n.getRetryMsg());
+ throw new IllegalStateException(String.format(Locale.ROOT,"node[key=%s marker=%s] failed %d times, due to %s",
+ n.getPath(), n.getMarker(), n.getRetryNum(), n.getRetryMsg()));
+ case Node.RETRY_STATE_DELAY:
+ try {
+ Thread.sleep(100);
+ } catch (InterruptedException ie) {
+ LOG.error("failed to retry node[key={}, marker={}], retry time[{}], due to {}",
+ n.getPath(), n.getMarker(), n.getRetryNum(), n.getRetryMsg());
+ throw new IllegalStateException(n.getRetryMsg());
+ }
+ if (!deque.offerLast(n)) { // put retry node to the end of queue
+ LOG.warn("node [{}, {}, {}, {}] enqueue failed, may be queue is full",
+ n.getPath(), n.getInode(), n.getMarker(), n.getRetryNum());
+ throw new IllegalStateException(
+ String.format(Locale.ROOT,"node [%s, %d, %s, %d] enqueue failed, may be queue is full",
+ n.getPath(), n.getInode(), n.getMarker(), n.getRetryNum()));
+ }
+ return false;
+
+ default:
+ throw new IllegalStateException("unreachable code");
+ }
+ }
+
+ private Future> submitTask(BatchNodes nodes) {
+ return owner.getBoundedListThreadPool().submit(
+ () -> {
+ List dirs = transformToDirLayer(nodes);
+ ListContentSummaryFsRequest req = new ListContentSummaryFsRequest();
+ req.setBucketName(owner.getBucket());
+ req.setMaxKeys(owner.getMaxKeys());
+ req.setDirLayers(dirs);
+ ListContentSummaryFsResult res;
+ res = OBSCommonUtils.getOBSInvoker().retryByMaxTime(
+ OBSOperateAction.listContentSummaryFs, "",
+ () -> owner.getObsClient().listContentSummaryFs(req),
+ true);
+ handleListContentSummaryFsResult(res, nodes);
+ return "";
+ }
+ );
+ }
+
+ private void enqueue(Node n) {
+ if (!deque.offerFirst(n)) {
+ LOG.warn("node [{}, {}, {}, {}] enqueue failed, may be queue is full",
+ n.getPath(), n.getInode(), n.getMarker(), n.getRetryNum());
+ throw new IllegalStateException(
+ String.format(Locale.ROOT,"node [%s, %d, %s, %d] enqueue failed, may be queue is full",
+ n.getPath(), n.getInode(), n.getMarker(), n.getRetryNum()));
+ }
+ }
+
+ private List transformToDirLayer(BatchNodes nodes) {
+ List ret = new ArrayList<>();
+ for (Node node : nodes) {
+ ListContentSummaryFsRequest.DirLayer dir = new ListContentSummaryFsRequest.DirLayer();
+ dir.setKey(node.getPath());
+ dir.setMarker(node.getMarker());
+ dir.setInode(node.getInode());
+ LOG.debug("transform node ({}, {}, {}, {})",
+ node.getPath(), node.getMarker(), node.getInode(), node.getRetryNum());
+ ret.add(dir);
+ }
+ return ret;
+ }
+
+ private void handleListContentSummaryFsResult(ListContentSummaryFsResult res, BatchNodes nodes) {
+ // handle errors
+ List errs = res.getErrorResults();
+ if (errs != null) {
+ for (ListContentSummaryFsResult.ErrorResult err : errs) {
+ LOG.debug("listContentSummary return error contents: {}, {}, {}", err.getKey(), err.getErrorCode(),
+ err.getMessage());
+ retryErrorResult(err, nodes);
+ }
+ }
+
+ List dirs = res.getDirContentSummaries();
+ if (dirs != null) {
+ for (DirContentSummary dir : dirs) {
+ if (dir.isTruncated()) {
+ // put self with nextMarker to the queue
+ enqueue(new Node(dir.getKey(), dir.getNextMarker(), dir.getInode()));
+ }
+ // handle subdirs
+ for (DirSummary subDir : dir.getSubDir()) {
+ counter.increase(subDir.getDirCount(), subDir.getFileCount(), subDir.getFileSize());
+ LOG.debug("counter increase ({}, {}, {}) for [{}, {}]",
+ subDir.getDirCount(), subDir.getFileCount(), subDir.getFileSize(),
+ subDir.getName(), subDir.getInode());
+ if (subDir.getDirCount() != 0) { // enqueue when dir has sub dir
+ enqueue(new Node(subDir.getName(), null, subDir.getInode()));
+ }
+ }
+ }
+ }
+ }
+
+ private void retryErrorResult(ListContentSummaryFsResult.ErrorResult err, BatchNodes nodes) {
+ String statusCode = err.getStatusCode();
+ if (statusCode == null) {
+ LOG.warn("statusCode is null, {}", err);
return;
- } catch (OBSIOException e) {
- OBSFileSystem.LOG.debug(
- "Failed to truncate [{}] to newLength" + " [{}], retry time [{}], exception [{}]", f, newLength,
- retryTime, e);
+ }
+ // 408 409 429 5XX should retry
+ boolean needRetry = statusCode.equals("408") || statusCode.equals("409") || statusCode.equals("429");
+ needRetry = needRetry || statusCode.startsWith("5"); // 5xx
+
+ if (needRetry) {
+ for (final Node node : nodes) {
+ if (node.getPath().equals(err.getKey()) /*&& node.getMarker().equals(err.getMarker())*/) {
+ node.increaseRetryNum(
+ String.format("%s %s %s", err.getStatusCode(), err.getErrorCode(), err.getMessage()));
+ enqueue(node);
+ return;
+ }
+ }
+ throw new IllegalStateException(String.format("Unreachable code: could not find err[key=%s marker=] from nodes, err=%s",
+ err.getKey(), err.getMessage()));
+ }
+ }
- delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime);
- retryTime++;
+ private void waitBatchTaskFinish(List> futures) throws IOException {
+ for (Future> fRes : futures) {
try {
- Thread.sleep(delayMs);
- } catch (InterruptedException ie) {
- throw e;
+ fRes.get();
+ } catch (InterruptedException e) {
+ LOG.warn("Interrupted while listContentSummary, {}", e.getMessage());
+ throw new InterruptedIOException(String.format("Interrupted while listContentSummary, %s", e));
+ } catch (ExecutionException e) {
+ fRes.cancel(true);
+ throw OBSCommonUtils.extractException("ListContentSummary with exception", "", e);
+ }
+ }
+ }
+
+ private static class BatchNodes extends ArrayList {
+ }
+
+ private static class Node {
+ public static final int RETRY_STATE_DISCARD = 1; // exceed retry limits, discard
+
+ public static final int RETRY_STATE_DELAY = 2; // not reach action moment
+
+ public static final int RETRY_STATE_TRIGGER = 3; // reach the action moment, do request immediately
+
+ private final String path;
+
+ private final String marker;
+
+ private final long inode;
+
+ private int retryNum;
+
+ private long previousRetry;
+
+ private long firstProcessTime;
+
+ private String retryMsg;
+
+ public String getRetryMsg() {
+ return retryMsg;
+ }
+
+ public long getInode() {
+ return inode;
+ }
+
+ public int retryState() throws IOException {
+ if (retryNum == 0) {
+ return RETRY_STATE_TRIGGER;
+ }
+ RetryPolicy.RetryAction rc;
+ try {
+ rc = retryPolicy.shouldRetryByMaxTime(firstProcessTime, null, retryNum, 0, true);
+ } catch (Exception e) {
+ throw new IOException("unexpected exception", e);
+ }
+
+ if (rc.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
+ return RETRY_STATE_DISCARD;
+ }
+
+ if (rc.action == RetryPolicy.RetryAction.RetryDecision.RETRY) {
+ long delayMs = rc.delayMillis;
+ long now = System.currentTimeMillis();
+ if (previousRetry + delayMs > now) {
+ return RETRY_STATE_DELAY;
+ }
+ }
+ return RETRY_STATE_TRIGGER;
+ }
+
+ public int getRetryNum() {
+ return retryNum;
+ }
+
+ public String getPath() {
+ return path;
+ }
+
+ public String getMarker() {
+ return marker;
+ }
+
+ public Node(String path, String marker, long inode) {
+ this.path = OBSCommonUtils.maybeAddTrailingSlash(path);
+ this.marker = marker;
+ this.retryNum = 0;
+ this.previousRetry = 0;
+ this.inode = inode;
+ }
+
+ public void increaseRetryNum(String retryMsg) {
+ LOG.debug("node[{}, {}] increase retry, msg[{}]", path, marker, retryMsg);
+ this.retryNum++;
+ this.previousRetry = System.currentTimeMillis();
+ if (retryNum == 1) {
+ firstProcessTime = previousRetry;
}
+ this.retryMsg = retryMsg;
+ }
+
+ }
+
+ private static class Counter {
+ private volatile long dirNum;
+
+ private volatile long fileNum;
+
+ private volatile long size;
+
+ synchronized void increase(long ndir, long nfile, long size) {
+ this.dirNum += ndir;
+ this.fileNum += nfile;
+ this.size += size;
+ }
+
+ ContentSummary getContentSummary() {
+ return new ContentSummary.Builder().length(this.size)
+ .fileCount(this.fileNum)
+ .directoryCount(this.dirNum)
+ .spaceConsumed(this.size)
+ .build();
}
}
+ }
- innerFsTruncate(owner, f, newLength);
+ static void fsSetOwner(final OBSFileSystem owner, final Path f, final String username, final String groupname)
+ throws IOException {
+ Map userMeta = ImmutableMap.of("user", username, "group", groupname);
+ addObjectMetadata(owner, f, userMeta);
}
- private static void innerFsTruncate(final OBSFileSystem owner, final Path f, final long newLength)
+ static void fsSetPermission(final OBSFileSystem owner, final Path f, final FsPermission permission)
throws IOException {
- LOG.debug("truncate {} to newLength {}", f, newLength);
+ Map userMeta = ImmutableMap.of("permission", String.valueOf(permission.toShort()));
+ addObjectMetadata(owner, f, userMeta);
+ }
- try {
- String key = OBSCommonUtils.pathToKey(owner, f);
- owner.getObsClient().truncateObject(owner.getBucket(), key, newLength);
- owner.getSchemeStatistics().incrementWriteOps(1);
- } catch (ObsException e) {
- throw OBSCommonUtils.translateException("truncate", f, e);
- }
+ private static void addObjectMetadata(final OBSFileSystem owner, final Path f, final Map metaToAdd)
+ throws IOException {
+ String key = OBSCommonUtils.pathToKey(owner, f);
+ SetObjectMetadataRequest req = new SetObjectMetadataRequest(owner.getBucket(), key);
+ req.addAllUserMetadata(metaToAdd);
+
+ // query existed objMeta and add it to setMetaReq to avoid existed objMeta been overwritten.
+ Map objMeta = OBSCommonUtils.getObjectMetadata(owner, key);
+ objMeta.forEach((k, v) -> req.addUserMetadata(k, Optional.of(v).map(Objects::toString)
+ .orElse(null)));
+
+ OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.setObjectMetadata, key, () -> {
+ owner.getObsClient().setObjectMetadata(req);
+ return null;
+ }, true);
}
}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/ObsClientFactory.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSQosException.java
similarity index 52%
rename from hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/ObsClientFactory.java
rename to hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSQosException.java
index f0cbc3a..cbc5322 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/ObsClientFactory.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSQosException.java
@@ -18,29 +18,17 @@
package org.apache.hadoop.fs.obs;
-import com.obs.services.ObsClient;
+import com.obs.services.exception.ObsException;
-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}.
+ * IOException equivalent to {@link ObsException}.
*/
-@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;
+public class OBSQosException extends OBSIOException implements WithErrCode {
+ private static final long serialVersionUID = -1582681108285856259L;
+
+ OBSQosException(final String operationMsg, final ObsException cause) {
+ super(operationMsg,cause);
+ }
+
}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RenameFailedException.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSRenameFailedException.java
similarity index 81%
rename from hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RenameFailedException.java
rename to hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSRenameFailedException.java
index a67dbf0..14259dc 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RenameFailedException.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSRenameFailedException.java
@@ -25,13 +25,15 @@
* Exception to indicate a specific rename failure. The exit code defines the
* value returned by {@link OBSFileSystem#rename(Path, Path)}.
*/
-class RenameFailedException extends PathIOException {
+class OBSRenameFailedException extends PathIOException {
+ private static final long serialVersionUID = -1993835850714285642L;
+
/**
* Exit code to be returned.
*/
private boolean exitCode = false;
- RenameFailedException(final Path src, final Path optionalDest, final String error) {
+ OBSRenameFailedException(final Path src, final Path optionalDest, final String error) {
super(src.toString(), error);
setOperation("rename");
if (optionalDest != null) {
@@ -43,13 +45,7 @@ public boolean getExitCode() {
return exitCode;
}
- /**
- * Set the exit code.
- *
- * @param code exit code to raise
- * @return the exception
- */
- public RenameFailedException withExitCode(final boolean code) {
+ public OBSRenameFailedException withExitCode(final boolean code) {
this.exitCode = code;
return this;
}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSRetryPolicy.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSRetryPolicy.java
new file mode 100644
index 0000000..692a519
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSRetryPolicy.java
@@ -0,0 +1,210 @@
+/*
+ * 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.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.net.NoRouteToHostException;
+import java.net.UnknownHostException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.retry.RetryPolicy;
+
+
+public class OBSRetryPolicy implements RetryPolicyWithMaxTime {
+
+ private RetryPolicyWithMaxTime defaultPolicy;
+ private Map, RetryPolicyWithMaxTime> exceptionToPolicyMap;
+
+ private final RetryPolicyWithMaxTime throttleRetryPolicy;
+
+ private final RetryPolicyWithMaxTime idempotencyIoRetryPolicy;
+
+ private final RetryPolicyWithMaxTime failRetryPolicy;
+
+ public OBSRetryPolicy(Configuration conf) {
+ failRetryPolicy = new NoRetryPolicy();
+
+ ExponentialBackoffRetryPolicy baseRetryPolicy = new ExponentialBackoffRetryPolicy(
+ conf.getInt(OBSConstants.RETRY_LIMIT, OBSConstants.DEFAULT_RETRY_LIMIT),
+ conf.getLong(OBSConstants.RETRY_MAXTIME, OBSConstants.DEFAULT_RETRY_MAXTIME),
+ conf.getLong(OBSConstants.RETRY_SLEEP_BASETIME, OBSConstants.DEFAULT_RETRY_SLEEP_BASETIME),
+ conf.getLong(OBSConstants.RETRY_SLEEP_MAXTIME, OBSConstants.DEFAULT_RETRY_SLEEP_MAXTIME));
+ idempotencyIoRetryPolicy = new IdempotencyIoRetryPolicy(baseRetryPolicy);
+
+ throttleRetryPolicy = new ExponentialBackoffRetryPolicy(
+ conf.getInt(OBSConstants.RETRY_QOS_LIMIT, OBSConstants.DEFAULT_RETRY_QOS_LIMIT),
+ conf.getLong(OBSConstants.RETRY_QOS_MAXTIME, OBSConstants.DEFAULT_RETRY_QOS_MAXTIME),
+ conf.getLong(OBSConstants.RETRY_QOS_SLEEP_BASETIME, OBSConstants.DEFAULT_RETRY_QOS_SLEEP_BASETIME),
+ conf.getLong(OBSConstants.RETRY_QOS_SLEEP_MAXTIME, OBSConstants.DEFAULT_RETRY_QOS_SLEEP_MAXTIME));
+
+ exceptionToPolicyMap = createExceptionMap();
+ defaultPolicy = idempotencyIoRetryPolicy;
+ }
+
+ private Map, RetryPolicyWithMaxTime> createExceptionMap() {
+ Map, RetryPolicyWithMaxTime> policyMap = new HashMap<>();
+
+ policyMap.put(UnknownHostException.class, failRetryPolicy);
+ policyMap.put(NoRouteToHostException.class, failRetryPolicy);
+ policyMap.put(InterruptedIOException.class, failRetryPolicy);
+ policyMap.put(InterruptedException.class, failRetryPolicy);
+
+ policyMap.put(AccessControlException.class, failRetryPolicy);
+ policyMap.put(FileNotFoundException.class, failRetryPolicy);
+ policyMap.put(OBSIllegalArgumentException.class, failRetryPolicy);
+ policyMap.put(OBSMethodNotAllowedException.class, failRetryPolicy);
+ policyMap.put(OBSFileConflictException.class, failRetryPolicy);
+ policyMap.put(EOFException.class, failRetryPolicy);
+
+ policyMap.put(OBSQosException.class, throttleRetryPolicy);
+ policyMap.put(OBSIOException.class, idempotencyIoRetryPolicy);
+ return policyMap;
+ }
+
+ @Override
+ public RetryAction shouldRetry(Exception exception, int retries, int failovers, boolean idempotent)
+ throws Exception {
+ RetryPolicy policy = exceptionToPolicyMap.get(exception.getClass());
+ if (policy == null) {
+ policy = defaultPolicy;
+ }
+ return policy.shouldRetry(exception, retries, failovers, idempotent);
+ }
+
+ @Override
+ public RetryAction shouldRetryByMaxTime(long startTime, Exception exception, int retries, int failovers,
+ boolean idempotent) throws Exception {
+ RetryPolicyWithMaxTime policy = exceptionToPolicyMap.get(exception.getClass());
+ if (policy == null) {
+ policy = defaultPolicy;
+ }
+ return policy.shouldRetryByMaxTime(startTime, exception, retries, failovers, idempotent);
+ }
+
+ @Override
+ public RetryAction shouldRetryByMix(long startTime, Exception e, int retries, int failovers, boolean idempotent)
+ throws Exception {
+ RetryPolicyWithMaxTime policy = exceptionToPolicyMap.get(e.getClass());
+ if (policy == null) {
+ policy = defaultPolicy;
+ }
+ return policy.shouldRetryByMaxTime(startTime, e, retries, failovers, idempotent);
+ }
+
+ public static class IdempotencyIoRetryPolicy implements RetryPolicyWithMaxTime {
+
+ private RetryPolicyWithMaxTime next;
+
+ public IdempotencyIoRetryPolicy(RetryPolicyWithMaxTime retryPolicy) {
+ this.next = retryPolicy;
+ }
+
+ @Override
+ public RetryAction shouldRetry(Exception e, int retries, int failovers, boolean idempotent) throws Exception {
+ return next.shouldRetry(e, retries, failovers, idempotent);
+ }
+
+ @Override
+ public RetryAction shouldRetryByMaxTime(long startTime, Exception e, int retries, int failovers, boolean idempotent)
+ throws Exception {
+ if (!(e instanceof IOException) || !idempotent) {
+ return RetryAction.FAIL;
+ }
+ return next.shouldRetryByMaxTime(startTime, e, retries, failovers, idempotent);
+ }
+
+ @Override
+ public RetryAction shouldRetryByMix(long startTime, Exception e, int retries, int failovers, boolean idempotent)
+ throws Exception {
+ return next.shouldRetryByMix(startTime, e, retries, failovers, idempotent);
+ }
+ }
+
+ public static class ExponentialBackoffRetryPolicy implements RetryPolicyWithMaxTime {
+ private final int maxRetries;
+ private final long maxTime;
+ private final long sleepTime;
+ private final long sleepMaxTime;
+
+ public ExponentialBackoffRetryPolicy(int maxRetries, long maxTime, long sleepTime,
+ long sleepMaxTime ) {
+ this.maxRetries = maxRetries;
+ this.maxTime = maxTime;
+ this.sleepTime = sleepTime;
+ this.sleepMaxTime = sleepMaxTime;
+ }
+
+ private long calculateExponentialTime(int retries) {
+ long baseTime = Math.min(sleepTime * ((int) Math.pow(2.0D, (double) retries)), sleepMaxTime);
+ return (long)((double)baseTime * (ThreadLocalRandom.current().nextDouble() + 0.5D));
+ }
+
+ @Override
+ public RetryAction shouldRetry(Exception e, int retries, int failovers, boolean idempotent) {
+ return retries >= maxRetries ?
+ new RetryAction(RetryAction.RetryDecision.FAIL, 0 , "") :
+ new RetryAction(RetryAction.RetryDecision.RETRY,
+ TimeUnit.MILLISECONDS.toMillis(this.calculateExponentialTime(retries)), "");
+ }
+
+ @Override
+ public RetryAction shouldRetryByMaxTime(long startTime, Exception e, int retries, int failovers,
+ boolean idempotent) {
+ return System.currentTimeMillis() - startTime > maxTime ?
+ new RetryAction(RetryAction.RetryDecision.FAIL, 0L, "") :
+ new RetryAction(RetryAction.RetryDecision.RETRY,
+ TimeUnit.MILLISECONDS.toMillis(this.calculateExponentialTime(retries)), "");
+ }
+
+ @Override
+ public RetryAction shouldRetryByMix(long startTime, Exception e, int retries, int failovers,
+ boolean idempotent) {
+ return retries >= maxRetries || (System.currentTimeMillis() - startTime > maxTime) ?
+ new RetryAction(RetryAction.RetryDecision.FAIL, 0 , "") :
+ new RetryAction(RetryAction.RetryDecision.RETRY,
+ TimeUnit.MILLISECONDS.toMillis(this.calculateExponentialTime(retries)), "");
+ }
+ }
+
+ public static class NoRetryPolicy implements RetryPolicyWithMaxTime {
+ @Override
+ public RetryAction shouldRetry(Exception e, int retries, int failovers, boolean idempotent) {
+ return new RetryAction(RetryAction.RetryDecision.FAIL, 0L, "try once and fail.");
+ }
+ @Override
+ public RetryAction shouldRetryByMaxTime(long startTime, Exception e, int retries, int failovers,
+ boolean idempotent) {
+ return new RetryAction(RetryAction.RetryDecision.FAIL, 0L, "try once and fail.");
+ }
+
+ @Override
+ public RetryAction shouldRetryByMix(long startTime, Exception e, int retries, int failovers,
+ boolean idempotent) {
+ return new RetryAction(RetryAction.RetryDecision.FAIL, 0L, "try once and fail.");
+ }
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSSecurityProviderUtil.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSSecurityProviderUtil.java
new file mode 100644
index 0000000..4713910
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSSecurityProviderUtil.java
@@ -0,0 +1,114 @@
+package org.apache.hadoop.fs.obs;
+
+import com.obs.services.BasicObsCredentialsProvider;
+import com.obs.services.IObsCredentialsProvider;
+
+import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.net.URI;
+import java.util.Optional;
+
+public class OBSSecurityProviderUtil {
+
+ /**
+ * Class logger.
+ */
+ private static final Logger LOG = LoggerFactory.getLogger(OBSSecurityProviderUtil.class);
+
+ public static IObsCredentialsProvider createObsSecurityProvider(final Configuration conf, final URI name)
+ throws IOException {
+ Class> credentialsProviderClass;
+ BasicSessionCredential credentialsProvider;
+
+ try {
+ credentialsProviderClass = conf.getClass(OBSConstants.OBS_CREDENTIALS_PROVIDER, null);
+ } catch (RuntimeException e) {
+ Throwable c = e.getCause() != null ? e.getCause() : e;
+ throw new IOException("From option " + OBSConstants.OBS_CREDENTIALS_PROVIDER + ' ' + c, c);
+ }
+
+ if (credentialsProviderClass == null) {
+ return innerCreateObsSecurityProvider(conf, name);
+ }
+
+ try {
+ Constructor> cons = credentialsProviderClass.getDeclaredConstructor(URI.class, Configuration.class);
+ credentialsProvider = (BasicSessionCredential) cons.newInstance(name, conf);
+ } catch (NoSuchMethodException | SecurityException | IllegalAccessException | InstantiationException | InvocationTargetException e) {
+ Throwable c = e.getCause() != null ? e.getCause() : e;
+ throw new IOException("From option " + OBSConstants.OBS_CREDENTIALS_PROVIDER + ' ' + c, c);
+ }
+
+ LOG.info("create ObsClient using credentialsProvider: {}", credentialsProviderClass.getName());
+ String sessionToken = credentialsProvider.getSessionToken();
+ String ak = credentialsProvider.getOBSAccessKeyId();
+ String sk = credentialsProvider.getOBSSecretKey();
+ if (sessionToken != null && sessionToken.length() != 0) {
+ return new BasicObsCredentialsProvider(ak, sk, sessionToken);
+ } else {
+ return new BasicObsCredentialsProvider(ak, sk);
+ }
+ }
+
+ private static IObsCredentialsProvider innerCreateObsSecurityProvider(final Configuration conf,
+ final URI name) throws IOException {
+ OBSLoginHelper.Login creds = OBSCommonUtils.getOBSAccessKeys(name, conf);
+
+ String ak = creds.getUser();
+ String sk = creds.getPassword();
+ String token = creds.getToken();
+
+ if (OBSCommonUtils.isStringNotEmpty(ak) || OBSCommonUtils.isStringNotEmpty(sk)) {
+ LOG.info("create ObsClient using aksk from configuration");
+ return new BasicObsCredentialsProvider(ak, sk, token);
+ }
+
+ Class> securityProviderClass;
+ try {
+ securityProviderClass = conf.getClass(OBSConstants.OBS_SECURITY_PROVIDER, null);
+ LOG.info("From option {} get {}", OBSConstants.OBS_SECURITY_PROVIDER, securityProviderClass);
+ } catch (RuntimeException e) {
+ Throwable c = e.getCause() != null ? e.getCause() : e;
+ throw new IOException("From option " + OBSConstants.OBS_SECURITY_PROVIDER + ' ' + c, c);
+ }
+
+ if (securityProviderClass == null) {
+ LOG.info("create ObsClient when securityProviderClass is null");
+ return new BasicObsCredentialsProvider(ak, sk, token);
+ }
+
+ LOG.info("create ObsClient using securityProvider {}", securityProviderClass.getName());
+ IObsCredentialsProvider securityProvider;
+ try {
+ Optional> cons = tryGetConstructor(securityProviderClass,
+ new Class[] {URI.class, Configuration.class});
+
+ if (cons.isPresent()) {
+ securityProvider = (IObsCredentialsProvider) cons.get().newInstance(name, conf);
+ } else {
+ securityProvider = (IObsCredentialsProvider) securityProviderClass.getDeclaredConstructor()
+ .newInstance();
+ }
+
+ } catch (NoSuchMethodException | IllegalAccessException | InstantiationException | InvocationTargetException | RuntimeException e) {
+ Throwable c = e.getCause() != null ? e.getCause() : e;
+ throw new IOException("From option " + OBSConstants.OBS_SECURITY_PROVIDER + ' ' + c, c);
+ }
+ return securityProvider;
+ }
+
+ private static Optional> tryGetConstructor(final Class> mainClss, final Class>[] args) {
+ try {
+ Constructor> constructor = mainClss.getDeclaredConstructor(args);
+ return Optional.ofNullable(constructor);
+ } catch (NoSuchMethodException e) {
+ // ignore
+ return Optional.empty();
+ }
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSWriteOperationHelper.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSWriteOperationHelper.java
index fefab52..6d6b6eb 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSWriteOperationHelper.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSWriteOperationHelper.java
@@ -35,6 +35,8 @@
import org.slf4j.LoggerFactory;
import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
@@ -62,7 +64,9 @@ class OBSWriteOperationHelper {
/**
* Class logger.
*/
- public static final Logger LOG = LoggerFactory.getLogger(OBSWriteOperationHelper.class);
+ private static final Logger LOG = LoggerFactory.getLogger(OBSWriteOperationHelper.class);
+
+ static final String CONTENT_SHA256 = "x-obs-content-sha256";
/**
* Part number of the multipart task.
@@ -94,13 +98,19 @@ protected OBSWriteOperationHelper(final OBSFileSystem fs) {
* 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 destKey key of object
* @param inputStream source data
- * @param length size, if known. Use -1 for not known
+ * @param length size, if known. Use -1 for not known
+ * @param type checksum type
+ * @param checksum checksum for source data
* @return the request
*/
- PutObjectRequest newPutRequest(final String destKey, final InputStream inputStream, final long length) {
- return OBSCommonUtils.newPutObjectRequest(owner, destKey, newObjectMetadata(length), inputStream);
+ PutObjectRequest newPutRequest(final String destKey, final InputStream inputStream, final long length,
+ OBSDataBlocks.ChecksumType type, final String checksum) {
+ PutObjectRequest request =
+ OBSCommonUtils.newPutObjectRequest(owner, destKey, newObjectMetadata(length), inputStream);
+ setPutObjectRequestChecksumInfo(request, type, checksum);
+ return request;
}
/**
@@ -108,11 +118,31 @@ PutObjectRequest newPutRequest(final String destKey, final InputStream inputStre
*
* @param destKey object key for request
* @param sourceFile source file
+ * @param type checksum type
+ * @param checksum checksum for source data
* @return the request
*/
- PutObjectRequest newPutRequest(final String destKey, final File sourceFile) {
+ PutObjectRequest newPutRequest(final String destKey, final File sourceFile, OBSDataBlocks.ChecksumType type,
+ final String checksum) throws FileNotFoundException {
int length = (int) sourceFile.length();
- return OBSCommonUtils.newPutObjectRequest(owner, destKey, newObjectMetadata(length), sourceFile);
+ PutObjectRequest request =
+ OBSCommonUtils.newPutObjectRequest(owner, destKey, newObjectMetadata(length), sourceFile);
+ setPutObjectRequestChecksumInfo(request, type, checksum);
+ return request;
+ }
+
+ private static void setPutObjectRequestChecksumInfo(PutObjectRequest request, OBSDataBlocks.ChecksumType type,
+ String checksum) {
+ switch (type) {
+ case MD5:
+ request.getMetadata().setContentMd5(checksum);
+ break;
+ case SHA256:
+ request.addUserHeaders(CONTENT_SHA256, checksum);
+ break;
+ default:
+ break;
+ }
}
/**
@@ -152,11 +182,10 @@ String initiateMultiPartUpload(final String destKey) throws IOException {
} else if (owner.getSse().isSseKmsEnable()) {
initiateMPURequest.setSseKmsHeader(owner.getSse().getSseKmsHeader());
}
- try {
+ return OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.initMultiPartUpload,
+ destKey, () -> {
return obs.initiateMultipartUpload(initiateMPURequest).getUploadId();
- } catch (ObsException ace) {
- throw OBSCommonUtils.translateException("Initiate MultiPartUpload", destKey, ace);
- }
+ },true);
}
/**
@@ -169,15 +198,18 @@ String initiateMultiPartUpload(final String destKey) throws IOException {
* @throws ObsException on problems.
*/
CompleteMultipartUploadResult completeMultipartUpload(final String destKey, final String uploadId,
- final List partETags) throws ObsException {
+ final List partETags) throws IOException {
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)));
+ return OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.completeMultipartUpload,
+ destKey, () -> {
+ return obs.completeMultipartUpload(
+ new CompleteMultipartUploadRequest(bucket, destKey, uploadId, new ArrayList<>(partETags)));
+ },true);
}
/**
@@ -195,47 +227,39 @@ void abortMultipartUpload(final String destKey, final String uploadId) throws Ob
/**
* Create request for uploading one part of a multipart task.
*
- * @param destKey destination object key
- * @param uploadId upload id
+ * @param destKey destination object key
+ * @param uploadId upload id
* @param partNumber part number
- * @param size data size
+ * @param size data size
* @param sourceFile source file to be uploaded
+ * @param type checksum type
+ * @param checksum checksum for source data
* @return part upload request
*/
UploadPartRequest newUploadPartRequest(final String destKey, final String uploadId, final int partNumber,
- final int size, final File sourceFile) {
- Preconditions.checkNotNull(uploadId);
-
+ final int size, final File sourceFile, OBSDataBlocks.ChecksumType type, final String checksum)
+ throws FileNotFoundException {
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());
- }
+ UploadPartRequest request =
+ newUploadPartRequest(destKey, uploadId, partNumber, size, new FileInputStream(sourceFile), type, checksum);
+ request.setAutoClose(false);
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 destKey destination object key
+ * @param uploadId upload id
+ * @param partNumber part number
+ * @param size data size
* @param uploadStream upload stream for the part
+ * @param type checksum type
+ * @param checksum checksum for source data
* @return part upload request
*/
UploadPartRequest newUploadPartRequest(final String destKey, final String uploadId, final int partNumber,
- final int size, final InputStream uploadStream) {
+ final int size, final InputStream uploadStream, final OBSDataBlocks.ChecksumType type, final String checksum) {
Preconditions.checkNotNull(uploadId);
Preconditions.checkArgument(uploadStream != null, "Data source");
@@ -253,6 +277,16 @@ UploadPartRequest newUploadPartRequest(final String destKey, final String upload
if (owner.getSse().isSseCEnable()) {
request.setSseCHeader(owner.getSse().getSseCHeader());
}
+ switch (type) {
+ case MD5:
+ request.setContentMd5(checksum);
+ break;
+ case SHA256:
+ request.addUserHeaders(CONTENT_SHA256, checksum);
+ break;
+ default:
+ break;
+ }
return request;
}
@@ -268,10 +302,6 @@ public String toString(final String destKey) {
* @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);
- }
+ return OBSCommonUtils.putObjectDirect(owner, putObjectRequest);
}
}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OpenFileWithJobConf.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OpenFileWithJobConf.java
new file mode 100644
index 0000000..b21551e
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OpenFileWithJobConf.java
@@ -0,0 +1,12 @@
+package org.apache.hadoop.fs.obs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+
+public interface OpenFileWithJobConf {
+ FSDataInputStream open(Path f, Configuration jobConf)
+ throws IOException;
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/Pair.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/Pair.java
index 586a9a6..8024d46 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/Pair.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/Pair.java
@@ -19,73 +19,24 @@
package org.apache.hadoop.fs.obs;
public class Pair {
- /**
- * Key.
- */
private final K key;
-
- /**
- * Value.
- */
private final V value;
-
- /**
- * Create an entry representing a mapping from the specified key to the
- * specified value.
- *
- * @param k Key (first element of the pair).
- * @param v Value (second element of the pair).
- */
public Pair(K k, V v) {
key = k;
value = v;
}
-
- /**
- * Get the key.
- *
- * @return the key (first element of the pair).
- */
public K getKey() {
return key;
}
-
- /**
- * Get the value.
- *
- * @return the value (second element of the pair).
- */
public V getValue() {
return value;
}
-
- /**
- * Get the first element of the pair.
- *
- * @return the first element of the pair.
- * @since 3.1
- */
public K getFirst() {
return key;
}
-
- /**
- * Get the second element of the pair.
- *
- * @return the second element of the pair.
- * @since 3.1
- */
public V getSecond() {
return value;
}
-
- /**
- * Compare the specified object with this entry for equality.
- *
- * @param o Object.
- * @return {@code true} if the given object is also a map entry and
- * the two entries represent the same mapping.
- */
@Override
public boolean equals(Object o) {
if (this == o) {
@@ -95,24 +46,15 @@ public boolean equals(Object o) {
return false;
} else {
Pair, ?> oP = (Pair, ?>) o;
- return (key == null ? oP.key == null : key.equals(oP.key)) && (value == null
- ? oP.value == null
- : value.equals(oP.value));
+ return (key == null ? oP.key == null : key.equals(oP.key))
+ && (value == null ? oP.value == null : value.equals(oP.value));
}
}
-
- /**
- * Compute a hash code.
- *
- * @return the hash code value.
- */
@Override
public int hashCode() {
int result = key == null ? 0 : key.hashCode();
final int h = value == null ? 0 : value.hashCode();
- result = 37 * result + h ^ (h >>> 16);
-
- return result;
+ return 37 * result + h ^ (h >>> 16);
}
}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RetryPolicyWithMaxTime.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RetryPolicyWithMaxTime.java
new file mode 100644
index 0000000..96dc531
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RetryPolicyWithMaxTime.java
@@ -0,0 +1,17 @@
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.io.retry.RetryPolicy;
+
+/**
+ * description
+ *
+ * @since 2022-05-10
+ */
+public interface RetryPolicyWithMaxTime extends RetryPolicy {
+ RetryPolicy.RetryAction shouldRetryByMaxTime(long startTime, Exception e, int retries, int failovers,
+ boolean idempotent)
+ throws Exception;
+ RetryPolicy.RetryAction shouldRetryByMix(long startTime, Exception e, int retries, int failovers,
+ boolean idempotent)
+ throws Exception;
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/SemaphoredDelegatingExecutor.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/SemaphoredDelegatingExecutor.java
index 782d9a5..d745411 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/SemaphoredDelegatingExecutor.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/SemaphoredDelegatingExecutor.java
@@ -86,41 +86,41 @@ protected ListeningExecutorService delegate() {
return executorDelegatee;
}
+ @NotNull
+ @Override
+ public ListenableFuture submit(@NotNull final Callable task) {
+ try {
+ queueingPermits.acquire();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ return Futures.immediateFailedCheckedFuture(e);
+ }
+ return super.submit(new CallableWithPermitRelease<>(task));
+ }
+
@NotNull
@Override
public List> invokeAll(@NotNull final Collection extends Callable> tasks) {
- throw new RuntimeException("Not implemented");
+ throw new UnsupportedOperationException("Not implemented");
}
@NotNull
@Override
public List> invokeAll(@NotNull final Collection extends Callable> tasks, final long timeout,
@NotNull final TimeUnit unit) {
- throw new RuntimeException("Not implemented");
+ throw new UnsupportedOperationException("Not implemented");
}
@NotNull
@Override
public T invokeAny(@NotNull final Collection extends Callable> tasks) {
- throw new RuntimeException("Not implemented");
+ throw new UnsupportedOperationException("Not implemented");
}
@Override
public T invokeAny(@NotNull final Collection extends Callable> tasks, final long timeout,
@NotNull final TimeUnit unit) {
- throw new RuntimeException("Not implemented");
- }
-
- @NotNull
- @Override
- public ListenableFuture submit(@NotNull final Callable task) {
- try {
- queueingPermits.acquire();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- return Futures.immediateFailedCheckedFuture(e);
- }
- return super.submit(new CallableWithPermitRelease<>(task));
+ throw new UnsupportedOperationException("Not implemented");
}
@NotNull
@@ -135,16 +135,8 @@ public ListenableFuture submit(@NotNull final Runnable task, @NotNull fin
return super.submit(new RunnableWithPermitRelease(task), result);
}
- @NotNull
- @Override
- public ListenableFuture> submit(@NotNull final Runnable task) {
- try {
- queueingPermits.acquire();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- return Futures.immediateFailedCheckedFuture(e);
- }
- return super.submit(new RunnableWithPermitRelease(task));
+ public int getAvailablePermits() {
+ return queueingPermits.availablePermits();
}
@Override
@@ -157,34 +149,26 @@ public void execute(@NotNull final Runnable command) {
super.execute(new RunnableWithPermitRelease(command));
}
- /**
- * Get the number of permits available; guaranteed to be {@code 0 <=
- * availablePermits <= size}.
- *
- * @return the number of permits available at the time of invocation.
- */
- public int getAvailablePermits() {
- return queueingPermits.availablePermits();
- }
-
- /**
- * Get the number of threads waiting to acquire a permit.
- *
- * @return snapshot of the length of the queue of blocked threads.
- */
public int getWaitingCount() {
return queueingPermits.getQueueLength();
}
- /**
- * Total number of permits.
- *
- * @return the number of permits as set in the constructor
- */
public int getPermitCount() {
return permitCount;
}
+ @NotNull
+ @Override
+ public ListenableFuture> submit(@NotNull final Runnable task) {
+ try {
+ queueingPermits.acquire();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ return Futures.immediateFailedCheckedFuture(e);
+ }
+ return super.submit(new RunnableWithPermitRelease(task));
+ }
+
@Override
public String toString() {
return "SemaphoredDelegatingExecutor{" + "permitCount=" + getPermitCount() + ", available="
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/TrafficStatistics.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/TrafficStatistics.java
new file mode 100644
index 0000000..2135c03
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/TrafficStatistics.java
@@ -0,0 +1,94 @@
+package org.apache.hadoop.fs.obs;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongUnaryOperator;
+
+public class TrafficStatistics {
+ // originalTraffic: Q
+ private final AtomicLong originalTraffic = new AtomicLong();
+
+ /**
+ * ApplicationTraffic: Q`.
+ * In fact Q` = Q1 + Q2. One should not operate the value of Q` directly.
+ *
The value of Q` shall only be determined as Q1 + Q2.
+ */
+ private final AtomicLong applicationTraffic = new AtomicLong();
+
+ /**
+ * missTraffic: Q1
+ */
+ private final AtomicLong missTraffic = new AtomicLong();
+
+ /**
+ * hitTraffic: Q2
+ */
+ private final AtomicLong hitTraffic = new AtomicLong();
+
+ private static final Logger LOG = LoggerFactory.getLogger(TrafficStatistics.class);
+
+ public enum TrafficType {
+ Q,
+ QDot,
+ Q1,
+ Q2
+ }
+
+ public void increase(long val, TrafficType type) {
+ if (val > 0) {
+ long now;
+ switch (type) {
+ case Q:
+ now = originalTraffic.updateAndGet(addWithinLimit(val));
+ LOG.debug("originalTraffic(Q) added {}, now {}.", val, now);
+ break;
+ case Q1:
+ now = missTraffic.updateAndGet(addWithinLimit(val));
+ LOG.debug("missTraffic(Q1) added {}, now {}.", val, now);
+ break;
+ case Q2:
+ now = hitTraffic.updateAndGet(addWithinLimit(val));
+ LOG.debug("hitTraffic(Q2) added {}, now {}.", val, now);
+ break;
+ default:
+ LOG.error("Wrong type of TrafficType, val {}.", val);
+ }
+ }
+ }
+
+ private LongUnaryOperator addWithinLimit(long val) {
+ return (x) -> {
+ if (x >= Long.MAX_VALUE) {
+ return 0;
+ } else {
+ return x + val;
+ }
+ };
+ }
+
+ public long getStatistics(TrafficType type) {
+ switch (type) {
+ case Q:
+ return originalTraffic.get();
+ case QDot:
+ return missTraffic.get() + hitTraffic.get();
+ case Q1:
+ return missTraffic.get();
+ case Q2:
+ return hitTraffic.get();
+ default:
+ LOG.error("Wrong type of TrafficType.");
+ return -1;
+ }
+ }
+
+ public void clearStatistics() {
+ originalTraffic.getAndSet(0);
+ applicationTraffic.getAndSet(0);
+ missTraffic.getAndSet(0);
+ hitTraffic.getAndSet(0);
+ LOG.debug("Cleared all traffic statistics.");
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/TrafficStatisticsReporter.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/TrafficStatisticsReporter.java
new file mode 100644
index 0000000..5fadeff
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/TrafficStatisticsReporter.java
@@ -0,0 +1,95 @@
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.fs.obs.memartscc.MemArtsCCClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.fs.obs.BlockingThreadPoolExecutorService.newDaemonThreadFactory;
+
+public class TrafficStatisticsReporter {
+ private static final Logger LOG = LoggerFactory.getLogger(TrafficStatisticsReporter.class);
+
+ private final TrafficStatistics trafficStatistics;
+
+ private final MemArtsCCClient memArtsCCClient;
+
+ /**
+ * Time interval of the report
+ */
+ private final long interval;
+
+ /**
+ * MemArtsCC traffic report Thread pool & Schedule
+ */
+ private ScheduledExecutorService reportPool;
+
+ private ScheduledFuture reportSchedule;
+
+ public TrafficStatisticsReporter(TrafficStatistics trafficStatistics,
+ MemArtsCCClient memArtsCCClient, long interval) {
+ this.trafficStatistics = trafficStatistics;
+ this.memArtsCCClient = memArtsCCClient;
+ this.interval = interval;
+ }
+
+ public void startReport() {
+ initThreadPool();
+ initReportSchedule();
+ }
+
+ private void initThreadPool() {
+ reportPool = new ScheduledThreadPoolExecutor(1,
+ newDaemonThreadFactory("obs-traffic-statistics-report"));
+ }
+
+ private void initReportSchedule() {
+ reportSchedule = reportPool.scheduleAtFixedRate(
+ this::reportTraffic, interval, interval, TimeUnit.SECONDS);
+ }
+
+ private void reportTraffic() {
+ if (memArtsCCClient == null || trafficStatistics == null) {
+ if (memArtsCCClient == null) {
+ LOG.debug("memArtsCCClient is null, statistics cannot be reported.");
+ }
+ if (trafficStatistics == null) {
+ LOG.debug("trafficStatistics is null, statistics cannot be reported.");
+ }
+ return;
+ }
+
+ memArtsCCClient.reportReadStatistics(trafficStatistics);
+ LOG.debug("Statistics has been reported: Q:{} Q`:{} Q2:{} Q1:{}",
+ trafficStatistics.getStatistics(TrafficStatistics.TrafficType.Q),
+ trafficStatistics.getStatistics(TrafficStatistics.TrafficType.QDot),
+ trafficStatistics.getStatistics(TrafficStatistics.TrafficType.Q2),
+ trafficStatistics.getStatistics(TrafficStatistics.TrafficType.Q1));
+
+ trafficStatistics.clearStatistics();
+ }
+
+ public void shutdownReport() {
+ if (memArtsCCClient == null || trafficStatistics == null) {
+ return;
+ }
+ reportTraffic();
+ // cancel report schedule
+ try {
+ reportSchedule.cancel(true);
+ if (reportSchedule.isCancelled()) {
+ LOG.debug("TrafficStatistics reportSchedule is canceled.");
+ }
+ reportPool.shutdownNow();
+ if (reportPool.isShutdown()) {
+ LOG.debug("TrafficStatistics reportPool is shutdowned.");
+ }
+ } catch (Exception e) {
+ LOG.debug("Exception occurred when canceling scheduledFuture");
+ }
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/WithErrCode.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/WithErrCode.java
new file mode 100644
index 0000000..e2f43e0
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/WithErrCode.java
@@ -0,0 +1,5 @@
+package org.apache.hadoop.fs.obs;
+
+public interface WithErrCode {
+ String getErrCode();
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/BasicInputPolicyFactory.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/BasicInputPolicyFactory.java
index 408d09c..a895027 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/BasicInputPolicyFactory.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/BasicInputPolicyFactory.java
@@ -6,6 +6,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.obs.OBSCommonUtils;
import org.apache.hadoop.fs.obs.OBSConstants;
+import org.apache.hadoop.fs.obs.OBSFileStatus;
import org.apache.hadoop.fs.obs.OBSFileSystem;
/**
@@ -17,7 +18,7 @@ public class BasicInputPolicyFactory implements InputPolicyFactory {
@Override
public FSInputStream create(final OBSFileSystem obsFileSystem, String bucket, String key, Long contentLength,
- FileSystem.Statistics statistics, ListeningExecutorService boundedThreadPool) {
+ FileSystem.Statistics statistics, ListeningExecutorService boundedThreadPool, OBSFileStatus fileStatus) {
long readAheadRange = OBSCommonUtils.longBytesOption(obsFileSystem.getConf(), OBSConstants.READAHEAD_RANGE,
OBSConstants.DEFAULT_READAHEAD_RANGE, 0);
return new OBSInputStream(bucket, key, contentLength, obsFileSystem.getObsClient(), statistics, readAheadRange,
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ExtendInputPolicyFactory.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ExtendInputPolicyFactory.java
index 0bae43c..71d336e 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ExtendInputPolicyFactory.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ExtendInputPolicyFactory.java
@@ -6,6 +6,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.obs.OBSCommonUtils;
import org.apache.hadoop.fs.obs.OBSConstants;
+import org.apache.hadoop.fs.obs.OBSFileStatus;
import org.apache.hadoop.fs.obs.OBSFileSystem;
import org.apache.hadoop.fs.obs.SemaphoredDelegatingExecutor;
@@ -28,7 +29,7 @@ public class ExtendInputPolicyFactory implements InputPolicyFactory {
*/
@Override
public FSInputStream create(final OBSFileSystem obsFileSystem, String bucket, String key, Long contentLength,
- FileSystem.Statistics statistics, ListeningExecutorService boundedThreadPool) {
+ FileSystem.Statistics statistics, ListeningExecutorService boundedThreadPool, OBSFileStatus fileStatus) {
int maxReadAhead = OBSCommonUtils.intOption(obsFileSystem.getConf(), OBSConstants.READAHEAD_MAX_NUM,
OBSConstants.DEFAULT_READAHEAD_MAX_NUM, 1);
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/InputPolicyFactory.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/InputPolicyFactory.java
index 374930c..ac6eca7 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/InputPolicyFactory.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/InputPolicyFactory.java
@@ -4,6 +4,7 @@
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.obs.OBSFileStatus;
import org.apache.hadoop.fs.obs.OBSFileSystem;
/**
@@ -13,5 +14,5 @@
*/
public interface InputPolicyFactory {
FSInputStream create(final OBSFileSystem obsFileSystem, String bucket, String key, Long contentLength,
- FileSystem.Statistics statistics, ListeningExecutorService boundedThreadPool);
+ FileSystem.Statistics statistics, ListeningExecutorService boundedThreadPool, OBSFileStatus fileStatus);
}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/InputPolicys.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/InputPolicys.java
index 1b5b73d..7fbbc4e 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/InputPolicys.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/InputPolicys.java
@@ -21,6 +21,8 @@ public static InputPolicyFactory createFactory(final String name) {
return new BasicInputPolicyFactory();
case OBSConstants.READAHEAD_POLICY_ADVANCE:
return new ExtendInputPolicyFactory();
+ case OBSConstants.READAHEAD_POLICY_MEMARTSCC:
+ return new MemArtsCCInputPolicyFactory();
default:
throw new IllegalArgumentException("Unsupported block buffer" + " \"" + name + '"');
}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/MemArtsCCInputPolicyFactory.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/MemArtsCCInputPolicyFactory.java
new file mode 100644
index 0000000..736e6f4
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/MemArtsCCInputPolicyFactory.java
@@ -0,0 +1,29 @@
+package org.apache.hadoop.fs.obs.input;
+
+import com.google.common.util.concurrent.ListeningExecutorService;
+
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.obs.OBSCommonUtils;
+import org.apache.hadoop.fs.obs.OBSConstants;
+import org.apache.hadoop.fs.obs.OBSFileStatus;
+import org.apache.hadoop.fs.obs.OBSFileSystem;
+
+/**
+ * 功能描述
+ *
+ * @since 2021-05-19
+ */
+public class MemArtsCCInputPolicyFactory implements InputPolicyFactory {
+
+ @Override
+ public FSInputStream create(final OBSFileSystem obsFileSystem, String bucket, String key, Long contentLength,
+ FileSystem.Statistics statistics, ListeningExecutorService boundedThreadPool, OBSFileStatus fileStatus) {
+ long readAheadRange = OBSCommonUtils.longBytesOption(obsFileSystem.getConf(), OBSConstants.READAHEAD_RANGE,
+ OBSConstants.DEFAULT_READAHEAD_RANGE, 0);
+ long memartsccReadAheadRangeValue = OBSCommonUtils.longBytesOption(obsFileSystem.getConf(),
+ OBSConstants.MEMARTSCC_READAHEAD_RANGE, OBSConstants.DEFAULT_MEMARTSCC_READAHEAD_RANGE, 0);
+ return new OBSMemArtsCCInputStream(bucket, key, contentLength, statistics, readAheadRange,
+ memartsccReadAheadRangeValue, obsFileSystem, fileStatus);
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSExtendInputStream.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSExtendInputStream.java
index 73c4fae..be77f77 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSExtendInputStream.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSExtendInputStream.java
@@ -28,7 +28,7 @@
* stream.
*/
public class OBSExtendInputStream extends FSInputStream implements CanSetReadahead, ByteBufferReadable {
- public static final Logger LOG = LoggerFactory.getLogger(OBSExtendInputStream.class);
+ private static final Logger LOG = LoggerFactory.getLogger(OBSExtendInputStream.class);
private OBSFileSystem fs;
@@ -218,9 +218,7 @@ private void checkNotClosed() throws IOException {
}
private void validateReadArgs(byte[] buf, int off, int len) {
- if (buf == null) {
- throw new NullPointerException();
- } else if (off < 0 || len < 0 || len > buf.length - off) {
+ if (off < 0 || len < 0 || len > buf.length - off) {
throw new IndexOutOfBoundsException();
}
}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSInputStream.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSInputStream.java
index c840653..a036401 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSInputStream.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSInputStream.java
@@ -6,7 +6,6 @@
import com.obs.services.model.GetObjectRequest;
import com.sun.istack.NotNull;
-import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.ByteBufferReadable;
@@ -14,11 +13,10 @@
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.obs.BasicMetricsConsumer;
import org.apache.hadoop.fs.obs.OBSCommonUtils;
import org.apache.hadoop.fs.obs.OBSConstants;
import org.apache.hadoop.fs.obs.OBSFileSystem;
-import org.apache.hadoop.fs.obs.OBSIOException;
+import org.apache.hadoop.fs.obs.OBSOperateAction;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -51,7 +49,7 @@ public class OBSInputStream extends FSInputStream implements CanSetReadahead, By
/**
* Class logger.
*/
- public static final Logger LOG = LoggerFactory.getLogger(OBSInputStream.class);
+ private static final Logger LOG = LoggerFactory.getLogger(OBSInputStream.class);
/**
* The statistics for OBS file system.
@@ -130,8 +128,8 @@ public class OBSInputStream extends FSInputStream implements CanSetReadahead, By
OBSInputStream(final String bucketName, final String bucketKey, final long fileStatusLength,
final ObsClient obsClient, final FileSystem.Statistics stats, final long readAheadRangeValue,
final OBSFileSystem obsFileSystem) {
- Preconditions.checkArgument(StringUtils.isNotEmpty(bucketName), "No Bucket");
- Preconditions.checkArgument(StringUtils.isNotEmpty(bucketKey), "No Key");
+ Preconditions.checkArgument(OBSCommonUtils.isStringNotEmpty(bucketName), "No Bucket");
+ Preconditions.checkArgument(OBSCommonUtils.isStringNotEmpty(bucketKey), "No Key");
Preconditions.checkArgument(fileStatusLength >= 0, "Negative content length");
this.bucket = bucketName;
this.key = bucketKey;
@@ -160,6 +158,11 @@ static long calculateRequestLimit(final long targetPos, final long length, final
return Math.min(contentLength, length < 0 ? contentLength : targetPos + Math.max(readahead, length));
}
+ protected long calculateOBSTraffic(final long targetPos, final long length) {
+ long contentRangeEnd = Math.min(contentLength, length < 0 ? contentLength : targetPos + Math.max(readAheadRange, length));
+ return contentRangeEnd - targetPos;
+ }
+
/**
* Opens up the stream at specified target position and for given length.
*
@@ -168,7 +171,8 @@ static long calculateRequestLimit(final long targetPos, final long length, final
* @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 {
+ protected 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) {
@@ -211,17 +215,21 @@ public synchronized long getPos() throws IOException {
public synchronized void seek(final long targetPos) throws IOException {
fs.checkOpen();
checkStreamOpen();
-
- // Do not allow negative seek
if (targetPos < 0) {
- throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK + " " + targetPos);
+ EOFException eof = new EOFException(String.format("%s %s", FSExceptionMessages.NEGATIVE_SEEK, targetPos));
+ OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.lazySeek, eof);
+ throw eof;
+ }
+
+ if (targetPos > contentLength) {
+ EOFException eof = new EOFException(String.format("%s %s", FSExceptionMessages.CANNOT_SEEK_PAST_EOF, targetPos));
+ OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.lazySeek, eof);
+ throw eof;
}
if (this.contentLength <= 0) {
return;
}
-
- // Lazy seek
nextReadPos = targetPos;
}
@@ -250,22 +258,14 @@ private void seekInStream(final long targetPos) throws IOException {
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) {
@@ -276,22 +276,14 @@ private void seekInStream(final long targetPos) throws IOException {
}
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;
}
@@ -311,58 +303,19 @@ public boolean seekToNewSource(final long targetPos) throws IOException {
* @throws IOException on any failure to lazy seek
*/
private void lazySeek(final long targetPos, final long len) throws IOException {
- int retryTime = 0;
- long delayMs;
- long startTime = System.currentTimeMillis();
- while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
+ OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.lazySeek, key, () -> {
try {
- // For lazy seek
seekInStream(targetPos);
} catch (IOException e) {
if (wrappedStream != null) {
closeStream("lazySeek() seekInStream has exception ", this.contentRangeFinish);
}
-
- LOG.warn("IOException occurred in lazySeek, retry: {}", retryTime, e);
- delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime);
- retryTime++;
- if (System.currentTimeMillis() - startTime + delayMs
- < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
- try {
- Thread.sleep(delayMs);
- } catch (InterruptedException ie) {
- throw e;
- }
- } else {
- throw e;
- }
-
- continue;
}
-
- try {
- // re-open at specific location if needed
- if (wrappedStream == null) {
- reopen("read from new offset", targetPos, len);
- }
-
- return;
- } catch (OBSIOException e) {
- LOG.debug("IOException occurred in lazySeek, retry: {}", retryTime, e);
- delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime);
- retryTime++;
- if (System.currentTimeMillis() - startTime + delayMs
- < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
- try {
- Thread.sleep(delayMs);
- } catch (InterruptedException ie) {
- throw e;
- }
- } else {
- throw e;
- }
+ if (wrappedStream == null) {
+ reopen("read from new offset", targetPos, len);
}
- }
+ return null;
+ },true);
}
/**
@@ -377,15 +330,6 @@ private void incrementBytesRead(final long bytesRead) {
}
}
- private void sleepInLock(long sleepTime) throws InterruptedException {
- long start = System.currentTimeMillis();
- long now = start;
- while (now - start < sleepTime) {
- wait(start + sleepTime - now);
- now = System.currentTimeMillis();
- }
- }
-
@Override
public synchronized int read() throws IOException {
fs.checkOpen();
@@ -393,82 +337,55 @@ public synchronized int read() throws IOException {
long startTime = System.currentTimeMillis();
long threadId = Thread.currentThread().getId();
long endTime;
- boolean isTrue = this.contentLength == 0 || nextReadPos >= contentLength;
+ boolean isTrue = this.contentLength == 0 || nextReadPos >= contentLength;
if (isTrue) {
return -1;
}
- int byteRead = -1;
try {
lazySeek(nextReadPos, 1);
} catch (EOFException e) {
onReadFailure(e, 1);
return -1;
+ } catch (IOException e) {
+ OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.readOneByte,e);
+ throw e;
}
- IOException exception = null;
- int retryTime = 0;
- long delayMs;
- long retryStartTime = System.currentTimeMillis();
- do {
- try {
- byteRead = wrappedStream.read();
- exception = null;
- break;
- } catch (EOFException e) {
- onReadFailure(e, 1);
- return -1;
- } catch (IOException e) {
- exception = e;
- onReadFailure(e, 1);
- LOG.debug("read of [{}] failed, retry time[{}], due to exception[{}]", uri, retryTime, e);
- delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime);
- retryTime++;
- if (System.currentTimeMillis() - startTime + delayMs
- < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
- try {
- sleepInLock(delayMs);
- } catch (InterruptedException ie) {
- LOG.error("read of [{}] failed, retry time[{}], due to " + "exception[{}]", uri, retryTime, e);
- throw e;
- }
+ try {
+ int byteRead = OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.readOneByte, key, () -> {
+ int b;
+ try {
+ b = wrappedStream.read();
+ } catch (EOFException e) {
+ onReadFailure(e, 1);
+ return -1;
+ } catch (IOException e) {
+ onReadFailure(e, 1);
+ throw e;
}
- }
- } while (System.currentTimeMillis() - retryStartTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY);
+ return b;
+ }, true);
- if (exception != null) {
- endTime = System.currentTimeMillis();
- if (fs.getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.BYTEBUF, BasicMetricsConsumer.MetricRecord.READ, false,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(fs, record);
+ if (byteRead >= 0) {
+ streamCurrentPos++;
+ nextReadPos++;
}
- LOG.error("read of [{}] failed, retry time[{}], due to exception[{}]", uri, retryTime, exception);
- throw exception;
- }
-
- if (byteRead >= 0) {
- streamCurrentPos++;
- nextReadPos++;
- }
- if (byteRead >= 0) {
- incrementBytesRead(1);
- }
+ if (byteRead >= 0) {
+ incrementBytesRead(1);
+ }
- endTime = System.currentTimeMillis();
- long position = byteRead >= 0 ? nextReadPos - 1 : nextReadPos;
- if (fs.getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.ONEBYTE, BasicMetricsConsumer.MetricRecord.READ, true,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(fs, record);
+ endTime = System.currentTimeMillis();
+ long position = byteRead >= 0 ? nextReadPos - 1 : nextReadPos;
+ LOG.debug("read-0arg uri:{}, contentLength:{}, position:{}, readValue:{}, " + "thread:{}, timeUsedMilliSec:{}",
+ uri, contentLength, position, byteRead, threadId, endTime - startTime);
+ return byteRead;
+ } catch (IOException e) {
+ OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.readOneByte,e);
+ throw e;
}
- LOG.debug("read-0arg uri:{}, contentLength:{}, position:{}, readValue:{}, " + "thread:{}, timeUsedMilliSec:{}",
- uri, contentLength, position, byteRead, threadId, endTime - startTime);
- return byteRead;
}
/**
@@ -481,27 +398,10 @@ public synchronized int read() throws IOException {
*/
private synchronized 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 retryTime = 0;
- long delayMs;
- long startTime = System.currentTimeMillis();
- while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
- try {
- reopen("failure recovery", streamCurrentPos, length);
- return;
- } catch (OBSIOException e) {
- LOG.debug("OBSIOException occurred in reopen for failure recovery, " + "the {} retry time", retryTime,
- e);
- delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime);
- retryTime++;
- try {
- sleepInLock(delayMs);
- } catch (InterruptedException ie) {
- throw e;
- }
- }
- }
-
- reopen("failure recovery", streamCurrentPos, length);
+ OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.onReadFailure, key, () -> {
+ reopen("failure recovery", streamCurrentPos, length);
+ return null;
+ },true);
}
@Override
@@ -517,7 +417,6 @@ public synchronized int read(final ByteBuffer byteBuffer) throws IOException {
if (len == 0) {
return 0;
}
-
byte[] buf = new byte[len];
boolean isTrue = this.contentLength == 0 || nextReadPos >= contentLength;
if (isTrue) {
@@ -530,76 +429,46 @@ public synchronized int read(final ByteBuffer byteBuffer) throws IOException {
onReadFailure(e, len);
// the end of the file has moved
return -1;
+ } catch (IOException e) {
+ OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.readByteBuff,e);
+ throw e;
}
- int bytesRead = 0;
- IOException exception = null;
- int retryTime = 0;
- long delayMs;
- long startRetryTime = System.currentTimeMillis();
- do {
- 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.debug("read len[{}] of [{}] failed, retry time[{}], " + "due to exception[{}]", len, uri, retryTime,
- exception);
- delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime);
- retryTime++;
- if (System.currentTimeMillis() - startTime + delayMs
- < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
- try {
- sleepInLock(delayMs);
- } catch (InterruptedException ie) {
- LOG.error("read len[{}] of [{}] failed, retry time[{}], " + "due to exception[{}]", len, uri,
- retryTime, exception);
- throw exception;
+ try {
+ int bytesRead = OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.readByteBuff, key, () -> {
+ int count;
+ try {
+ count = tryToReadFromInputStream(wrappedStream, buf, 0, len);
+ if (count == -1) {
+ return -1;
}
+ } catch (EOFException e) {
+ onReadFailure(e, len);
+ return -1;
+ } catch (IOException e) {
+ onReadFailure(e, len);
+ throw e;
}
+ return count;
+ }, true);
+
+ if (bytesRead > 0) {
+ streamCurrentPos += bytesRead;
+ nextReadPos += bytesRead;
+ byteBuffer.put(buf, 0, bytesRead);
}
- } while (System.currentTimeMillis() - startRetryTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY);
- if (exception != null) {
+ incrementBytesRead(bytesRead);
+ long position = bytesRead >= 0 ? nextReadPos - 1 : nextReadPos;
endTime = System.currentTimeMillis();
- if (fs.getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.BYTEBUF, BasicMetricsConsumer.MetricRecord.READ, false,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(fs, record);
- }
-
- LOG.error("read len[{}] of [{}] failed, retry time[{}], " + "due to exception[{}]", len, uri, retryTime,
- exception);
- throw exception;
- }
- if (bytesRead > 0) {
- streamCurrentPos += bytesRead;
- nextReadPos += bytesRead;
- byteBuffer.put(buf, 0, bytesRead);
- }
- incrementBytesRead(bytesRead);
- long position = bytesRead >= 0 ? nextReadPos - 1 : nextReadPos;
- endTime = System.currentTimeMillis();
-
- if (fs.getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.BYTEBUF, BasicMetricsConsumer.MetricRecord.READ, true,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(fs, record);
+ LOG.debug("Read-ByteBuffer uri:{}, contentLength:{}, destLen:{}, readLen:{}, "
+ + "position:{}, thread:{}, timeUsedMilliSec:{}", uri, contentLength, len, bytesRead, position, threadId,
+ endTime - startTime);
+ return bytesRead;
+ } catch (IOException e) {
+ OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.readByteBuff,e);
+ throw e;
}
- LOG.debug("Read-ByteBuffer uri:{}, contentLength:{}, destLen:{}, readLen:{}, "
- + "position:{}, thread:{}, timeUsedMilliSec:{}", uri, contentLength, len, bytesRead, position, threadId,
- endTime - startTime);
- return bytesRead;
}
private int tryToReadFromInputStream(final InputStream in, final byte[] buf, final int off, final int len)
@@ -635,13 +504,10 @@ public synchronized int read(@NotNull final byte[] buf, final int off, final int
checkStreamOpen();
long startTime = System.currentTimeMillis();
long threadId = Thread.currentThread().getId();
- long endTime;
-
validatePositionedReadArgs(nextReadPos, buf, off, len);
if (len == 0) {
return 0;
}
-
boolean isTrue = this.contentLength == 0 || nextReadPos >= contentLength;
if (isTrue) {
return -1;
@@ -653,81 +519,45 @@ public synchronized int read(@NotNull final byte[] buf, final int off, final int
onReadFailure(e, len);
// the end of the file has moved
return -1;
+ } catch (IOException e) {
+ OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.readbytes,e);
+ throw e;
}
- int bytesRead = 0;
- IOException exception = null;
- int retryTime = 0;
- long delayMs;
- long startRetryTime = System.currentTimeMillis();
- do {
- 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.debug("read offset[{}] len[{}] of [{}] failed, retry time[{}], " + "due to exception[{}]", off, len,
- uri, retryTime, exception);
- delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime);
- retryTime++;
- if (System.currentTimeMillis() - startTime + delayMs
- < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
- try {
- sleepInLock(delayMs);
- } catch (InterruptedException ie) {
- LOG.error("read offset[{}] len[{}] of [{}] failed, " + "retry time[{}], due to exception[{}]",
- off, len, uri, retryTime, exception);
- throw exception;
+ try {
+ int bytesRead = OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.readbytes, key, () -> {
+ int count;
+ try {
+ count = tryToReadFromInputStream(wrappedStream, buf, off, len);
+ if (count == -1) {
+ return -1;
}
+ } catch (EOFException e) {
+ onReadFailure(e, len);
+ return -1;
+ } catch (IOException e) {
+ onReadFailure(e, len);
+ throw e;
}
- }
- } while (System.currentTimeMillis() - startRetryTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY);
-
- long costTime;
+ return count;
+ }, true);
- if (exception != null) {
- endTime = System.currentTimeMillis();
- if (fs.getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.SEQ, BasicMetricsConsumer.MetricRecord.READ, false,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(fs, record);
+ if (bytesRead > 0) {
+ streamCurrentPos += bytesRead;
+ nextReadPos += bytesRead;
}
- LOG.error("read offset[{}] len[{}] of [{}] failed, retry time[{}], " + "due to exception[{}]", off, len,
- uri, retryTime, exception);
- throw exception;
- }
-
- if (bytesRead > 0) {
- streamCurrentPos += bytesRead;
- nextReadPos += bytesRead;
- }
- incrementBytesRead(bytesRead);
-
- endTime = System.currentTimeMillis();
- costTime = endTime - startTime;
- readMetric(costTime);
+ incrementBytesRead(bytesRead);
- long position = bytesRead >= 0 ? nextReadPos - 1 : nextReadPos;
- LOG.debug("Read-3args uri:{}, contentLength:{}, destLen:{}, readLen:{}, "
- + "position:{}, thread:{}, timeUsedMilliSec:{}", uri, contentLength, len, bytesRead, position, threadId,
- endTime - startTime);
- return bytesRead;
- }
+ long endTime = System.currentTimeMillis();
- private void readMetric(long costTime) {
- if (fs.getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.SEQ, BasicMetricsConsumer.MetricRecord.READ, true, costTime);
- OBSCommonUtils.setMetricsInfo(fs, record);
+ long position = bytesRead >= 0 ? nextReadPos - 1 : nextReadPos;
+ LOG.debug("Read-3args uri:{}, contentLength:{}, destLen:{}, readLen:{}, "
+ + "position:{}, thread:{}, timeUsedMilliSec:{}", uri, contentLength, len, bytesRead, position, threadId,
+ endTime - startTime);
+ return bytesRead;
+ } catch (IOException e) {
+ OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.readbytes,e);
+ throw e;
}
}
@@ -753,7 +583,6 @@ private void checkStreamOpen() throws IOException {
*/
@Override
public synchronized void close() throws IOException {
- long startTime = System.currentTimeMillis();
if (!closed) {
fs.checkOpen();
// close or abort the stream
@@ -762,14 +591,6 @@ public synchronized void close() throws IOException {
super.close();
closed = true;
}
-
- long endTime = System.currentTimeMillis();
- if (fs.getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.INPUT, BasicMetricsConsumer.MetricRecord.CLOSE, true,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(fs, record);
- }
}
/**
@@ -783,7 +604,7 @@ public synchronized void close() throws IOException {
* @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 {
+ protected synchronized void closeStream(final String reason, final long length) throws IOException {
if (wrappedStream != null) {
try {
wrappedStream.close();
@@ -804,7 +625,7 @@ public synchronized int available() throws IOException {
fs.checkOpen();
checkStreamOpen();
- long remaining = remainingInFile();
+ long remaining = remainingInStream();
if (remaining > Integer.MAX_VALUE) {
return Integer.MAX_VALUE;
}
@@ -818,7 +639,7 @@ public synchronized int available() throws IOException {
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
- private synchronized long remainingInFile() {
+ private synchronized long remainingInStream() {
return this.contentLength - this.streamCurrentPos;
}
@@ -869,40 +690,40 @@ public String toString() {
@Override
public void readFully(final long position, final byte[] buffer, final int offset, final int length)
throws IOException {
- fs.checkOpen();
- checkStreamOpen();
- long startTime = System.currentTimeMillis();
- long threadId = Thread.currentThread().getId();
+ try {
+ fs.checkOpen();
+ checkStreamOpen();
+ long startTime = System.currentTimeMillis();
+ long threadId = Thread.currentThread().getId();
- 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);
+ validatePositionedReadArgs(position, buffer, offset, length);
+ if (length == 0) {
+ return;
+ }
+ int bytesRead = 0;
+ synchronized (this) {
+ long oldPos = getPos();
+ try {
+ seek(position);
+ while (bytesRead < length) {
+ int bRead = read(buffer, offset + bytesRead, length - bytesRead);
+ if (bRead < 0) {
+ throw new EOFException(FSExceptionMessages.EOF_IN_READ_FULLY);
+ }
+ bytesRead += bRead;
}
- nread += nbytes;
+ } finally {
+ seekQuietly(oldPos);
}
- } finally {
- seekQuietly(oldPos);
}
+ long endTime = System.currentTimeMillis();
+ LOG.debug("ReadFully uri:{}, contentLength:{}, destLen:{}, readLen:{}, "
+ + "position:{}, thread:{}, timeUsedMilliSec:{}", uri, contentLength, length,
+ bytesRead, position, threadId, endTime - startTime);
+ } catch (IOException e) {
+ OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.readfully,e);
+ throw e;
}
- long endTime = System.currentTimeMillis();
- if (fs.getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null,
- BasicMetricsConsumer.MetricRecord.READFULLY, true, endTime - startTime);
- OBSCommonUtils.setMetricsInfo(fs, record);
- }
- LOG.debug("ReadFully uri:{}, contentLength:{}, destLen:{}, readLen:{}, "
- + "position:{}, thread:{}, timeUsedMilliSec:{}", uri, contentLength, length, nread, position, threadId,
- endTime - startTime);
}
/**
@@ -920,121 +741,59 @@ public int read(final long position, final byte[] buffer, final int offset, fina
fs.checkOpen();
checkStreamOpen();
int len = length;
- long startTime = System.currentTimeMillis();
- long endTime;
-
int readSize;
- validatePositionedReadArgs(position, buffer, offset, len);
- if (position < 0 || position >= contentLength) {
- return -1;
- }
- if ((position + len) > contentLength) {
- len = (int) (contentLength - position);
- }
+ try {
+ validatePositionedReadArgs(position, buffer, offset, len);
+ if (position < 0 || position >= contentLength) {
+ return -1;
+ }
+ if ((position + len) > contentLength) {
+ len = (int) (contentLength - position);
+ }
- if (fs.isReadTransformEnabled()) {
- readSize = super.read(position, buffer, offset, len);
- endTime = System.currentTimeMillis();
- if (fs.getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.RANDOM, BasicMetricsConsumer.MetricRecord.READ, true,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(fs, record);
+ if (fs.isReadTransformEnabled()) {
+ readSize = super.read(position, buffer, offset, len);
+ return readSize;
}
+ readSize = randomReadWithNewInputStream(position, buffer, offset, len);
return readSize;
+ } catch (IOException e) {
+ OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.readrandom,e);
+ throw e;
}
- readSize = randomReadWithNewInputStream(position, buffer, offset, len);
- endTime = System.currentTimeMillis();
- if (fs.getMetricSwitch()) {
- BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(
- BasicMetricsConsumer.MetricRecord.RANDOM, BasicMetricsConsumer.MetricRecord.READ, true,
- endTime - startTime);
- OBSCommonUtils.setMetricsInfo(fs, record);
- }
- return readSize;
}
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());
- }
- int retryTime = 0;
- long delayMs;
- long startRetryTime = System.currentTimeMillis();
- do {
- exception = null;
+ int bytesRead = OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.readrandom, key, () -> {
+ int count;
+ InputStream inputStream = null;
try {
- inputStream = client.getObject(request).getObjectContent();
- } catch (ObsException e) {
- exception = OBSCommonUtils.translateException("Read at position " + position, uri, e);
-
- LOG.debug(
- "read position[{}] destLen[{}] offset[{}] readLen[{}] " + "of [{}] failed, retry time[{}], due to "
- + "exception[{}]", position, length, offset, bytesRead, uri, retryTime, exception);
-
- if (!(exception instanceof OBSIOException)) {
- throw exception;
+ GetObjectRequest request = new GetObjectRequest(bucket, key);
+ request.setRangeStart(position);
+ request.setRangeEnd(position + length);
+ if (fs.getSse().isSseCEnable()) {
+ request.setSseCHeader(fs.getSse().getSseCHeader());
}
- }
-
- if (exception == null) {
- try {
- bytesRead = tryToReadFromInputStream(inputStream, buffer, offset, length);
- if (bytesRead == -1) {
- return -1;
- }
-
- exception = null;
- break;
- } catch (EOFException e) {
- onReadFailure(e, length);
+ inputStream = client.getObject(request).getObjectContent();
+ count = tryToReadFromInputStream(inputStream, buffer, offset, length);
+ if (count == -1) {
return -1;
- } catch (IOException e) {
- exception = e;
-
- LOG.debug("read position[{}] destLen[{}] offset[{}] readLen[{}] "
- + "of [{}] failed, retry time[{}], due to " + "exception[{}]", position, length, offset,
- bytesRead, uri, retryTime, exception);
- } finally {
- if (inputStream != null) {
- inputStream.close();
- }
}
- }
-
- delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime);
- retryTime++;
- if (System.currentTimeMillis() - startTime + delayMs < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) {
- try {
- Thread.sleep(delayMs);
- } catch (InterruptedException ie) {
- LOG.error(
- "read position[{}] destLen[{}] offset[{}] " + "readLen[{}] of [{}] failed, retry time[{}], "
- + "due to exception[{}]", position, length, offset, bytesRead, uri, retryTime, exception);
- throw exception;
+ } catch (EOFException e) {
+ return -1;
+ } finally {
+ if (inputStream != null) {
+ inputStream.close();
}
}
- } while (System.currentTimeMillis() - startRetryTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY);
-
- if (inputStream == null || exception != null) {
- IOException e = new IOException(
- "read failed of " + uri + ", inputStream is " + (inputStream == null ? "null" : "not null"), exception);
- LOG.error(
- "read position[{}] destLen[{}] offset[{}] len[{}] failed, " + "retry time[{}], due to exception[{}]",
- position, length, offset, bytesRead, retryTime, exception);
- throw e;
- }
+ return count;
+ }, true);
+
long endTime = System.currentTimeMillis();
LOG.debug("Read-4args uri:{}, contentLength:{}, destLen:{}, readLen:{}, "
+ "position:{}, thread:{}, timeUsedMilliSec:{}", uri, contentLength, length, bytesRead, position, threadId,
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSMemArtsCCInputStream.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSMemArtsCCInputStream.java
new file mode 100644
index 0000000..7d637b5
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSMemArtsCCInputStream.java
@@ -0,0 +1,631 @@
+package org.apache.hadoop.fs.obs.input;
+
+import com.google.common.base.Preconditions;
+
+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.apache.hadoop.fs.obs.OBSCommonUtils;
+import org.apache.hadoop.fs.obs.OBSConstants;
+import org.apache.hadoop.fs.obs.OBSFileStatus;
+import org.apache.hadoop.fs.obs.OBSFileSystem;
+import org.apache.hadoop.fs.obs.TrafficStatistics;
+import org.apache.hadoop.fs.obs.memartscc.MemArtsCCClient;
+import org.apache.hadoop.fs.obs.memartscc.MemArtsCCInputStream;
+import org.apache.hadoop.fs.obs.memartscc.MemArtsCCInputStreamBase;
+import org.apache.hadoop.fs.obs.memartscc.MemArtsCCInputStreamWithPool;
+import org.apache.hadoop.fs.obs.memartscc.OBSInputStreamSimulator;
+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 java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * 功能描述
+ *
+ * @since 2021-05-19
+ */
+public class OBSMemArtsCCInputStream extends FSInputStream implements CanSetReadahead, ByteBufferReadable {
+ /**
+ * Class logger.
+ */
+ private static final Logger LOG = LoggerFactory.getLogger(OBSMemArtsCCInputStream.class);
+
+ /**
+ * The State of this inputStream:
+ * New: newly created
+ * ORead: read from OBS directly
+ * MRead: read from MemArtsCC
+ * Close: closed
+ */
+ State state;
+
+ /**
+ * The statistics for OBS file system.
+ */
+ private final FileSystem.Statistics statistics;
+
+ /**
+ * MemArtsCC client.
+ */
+ private final MemArtsCCClient memArtsCCClient;
+
+ /**
+ * Bucket name.
+ */
+ private final String bucket;
+
+ /**
+ * Object key.
+ */
+ private final String key;
+
+ /**
+ * Content length.
+ */
+ private final long contentLength;
+
+ /**
+ * Object uri.
+ */
+ private final String uri;
+
+ /**
+ * Obs file system instance.
+ */
+ private OBSFileSystem fs;
+
+ /**
+ * 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 = false;
+
+ /**
+ * This is the actual position within the object, used by lazy seek to
+ * decide whether to seek on the next read or not.
+ */
+ private long nextReadPos = 0;
+
+ /**
+ * InputStream used for read from MemArtsCC
+ */
+ private MemArtsCCInputStreamBase ccStream;
+
+ /**
+ * MemArtsCC ccRead fail, or return CacheMiss, use this input stream to escape
+ */
+ OBSMemArtsPartnerInputStream partnerInputStream;
+
+ private int bufSize;
+
+ private byte[] tailBuf;
+
+ private Counter oCounter;
+
+ private Counter mCounter;
+
+ private TrafficStatistics trafficStatistics;
+
+ private final OBSInputStreamSimulator oisSimulator;
+
+ public OBSMemArtsCCInputStream(final String bucketName, final String objectKey, final long fileStatusLength,
+ final FileSystem.Statistics stats, final long readAheadRangeValue, final long memartsccReadAheadRangeValue,
+ final OBSFileSystem obsFileSystem, OBSFileStatus fileStatus) {
+ Preconditions.checkArgument(OBSCommonUtils.isStringNotEmpty(bucketName), "No Bucket");
+ Preconditions.checkArgument(OBSCommonUtils.isStringNotEmpty(objectKey), "No Key");
+ Preconditions.checkArgument(fileStatusLength >= 0, "Negative content length");
+ this.bucket = bucketName;
+ this.key = objectKey;
+ this.contentLength = fileStatusLength;
+ this.memArtsCCClient = obsFileSystem.getMemArtsCCClient();
+ this.statistics = stats;
+ this.uri = "obs://" + this.bucket + "/" + this.key;
+ this.fs = obsFileSystem;
+ this.partnerInputStream = new OBSMemArtsPartnerInputStream(bucketName, objectKey, fileStatusLength,
+ obsFileSystem.getObsClient(), stats, readAheadRangeValue, obsFileSystem, this);
+
+ // the initial state of the stream
+ this.state = State.MREAD;
+
+ this.bufSize = obsFileSystem.getConf()
+ .getInt(OBSConstants.MEMARTSCC_BUFFER_SIZE, OBSConstants.DEFAULT_MEMARTSCC_BUFFER_SIZE);
+ int directBufferSize = obsFileSystem.getConf()
+ .getInt(OBSConstants.MEMARTSCC_DIRECTBUFFER_SIZE, OBSConstants.DEFAULT_MEMARTSCC_DIRECTBUFFER_SIZE);
+ String inputSteamType = obsFileSystem.getConf()
+ .get(OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_TYPE, OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_TYPE_POOL);
+ if (inputSteamType.equals(OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_TYPE_BIND)) {
+ this.ccStream = new MemArtsCCInputStream(
+ memArtsCCClient, objectKey, fileStatus, memartsccReadAheadRangeValue, stats, bufSize, directBufferSize);
+ } else if (inputSteamType.equals(OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_TYPE_POOL)) {
+ int borrowTimeout = obsFileSystem.getConf()
+ .getInt(OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_BORROW_TIMEOUT,
+ OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_BORROW_DEFAULT_TIMEOUT);
+ this.ccStream = new MemArtsCCInputStreamWithPool(
+ memArtsCCClient, objectKey, fileStatus, memartsccReadAheadRangeValue, stats, borrowTimeout);
+ } else {
+ throw new IllegalArgumentException("invalid input stream type:" + inputSteamType);
+ }
+ this.oCounter = new Counter();
+ this.mCounter = new Counter();
+
+ oisSimulator = new OBSInputStreamSimulator(fileStatusLength, readAheadRangeValue);
+ initTrafficReport(obsFileSystem);
+
+ LOG.debug("create OBSMemArtsCCInputStream[{}] for file {}", this.hashCode(), objectKey);
+ }
+
+ private void initTrafficReport(OBSFileSystem obsFileSystem) {
+ // traffic statistics report
+ trafficStatistics = obsFileSystem.getTrafficStatistics();
+ partnerInputStream.setTrafficStaticsClass(trafficStatistics);
+ ccStream.setTrafficStaticsClass(trafficStatistics);
+ }
+
+ public synchronized int available() throws IOException {
+ fs.checkOpen();
+ checkStreamOpen();
+
+ long remaining = this.contentLength - this.nextReadPos;
+ if (remaining > Integer.MAX_VALUE) {
+ return Integer.MAX_VALUE;
+ }
+
+ return (int) remaining;
+ }
+
+ /**
+ * {@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(final byte[] buf, final int off, final int len) throws IOException {
+ LOG.debug("read(buf,off,len), offset[{}], len[{}] ", off, len);
+ fs.checkOpen();
+ checkStreamOpen();
+
+ long startTime = System.currentTimeMillis();
+ long threadId = Thread.currentThread().getId();
+ long endTime;
+
+ validatePositionedReadArgs(nextReadPos, buf, off, len);
+ if (len == 0) {
+ return 0;
+ }
+
+ boolean isTrue = this.contentLength == 0 || nextReadPos >= contentLength;
+ if (isTrue) {
+ return -1;
+ }
+
+ // when the position of read is in the tail of the file
+ int bytesRead = 0;
+ int posInTail = posInTailBuf(nextReadPos);
+ if (posInTail != -1 && tailBuf != null) {
+ int copyLen = (int)(contentLength - nextReadPos);
+ copyLen = Math.min(copyLen, Math.min(len, buf.length - off));
+ System.arraycopy(tailBuf, posInTail, buf, off, copyLen);
+ nextReadPos += copyLen;
+ this.partnerInputStream.seek(nextReadPos);
+ this.ccStream.seek(nextReadPos);
+ oisSimulator.seek(nextReadPos);
+ incrementBytesRead(copyLen);
+ return copyLen;
+ }
+
+ bytesRead = readInState(buf, off, len, false);
+
+ if (bytesRead > 0) {
+ nextReadPos += bytesRead;
+ }
+
+ endTime = System.currentTimeMillis();
+ long position = bytesRead >= 0 ? nextReadPos - 1 : nextReadPos;
+ LOG.debug("Read-3args uri:{}, contentLength:{}, destLen:{}, readLen:{}, "
+ + "position:{}, thread:{}, timeUsedMilliSec:{}", uri, contentLength, len, bytesRead, position, threadId,
+ endTime - startTime);
+ return bytesRead;
+ }
+
+ private int readInputStream(final InputStream is, final byte[] buf, final int off,
+ final int len, boolean oneByteRead, final Counter counter, final State state)
+ throws IOException {
+
+ int ret;
+ int readLen;
+ long start;
+ long end;
+ long pos = this.nextReadPos;
+
+ start = System.nanoTime();
+ if (oneByteRead) {
+ ret = is.read();
+ readLen = 1;
+ } else {
+ ret = is.read(buf, off, len);
+ readLen = ret;
+ }
+ end = System.nanoTime();
+ counter.increase(end - start, readLen);
+ LOG.debug("{} {}({},{},{})", this.hashCode(), state, pos, readLen, end - start);
+ return ret;
+ }
+
+ private int readInState(final byte[] buf, final int off, final int len, boolean oneByteRead) throws IOException {
+ switch (this.state) {
+ case NEW:
+ this.partnerInputStream.reopen("open first connection", this.nextReadPos, len);
+
+ // simulate reopen without passing through MemArts
+ long readBytes = oisSimulator.reopen(this.nextReadPos, len);
+ increaseSimTraffic(readBytes);
+
+ stateTransitionToORead();
+ // continue to oread
+ case OREAD:
+ try {
+ // lazy seek
+ this.partnerInputStream.seek(this.nextReadPos);
+ oisSimulator.seek(nextReadPos);
+
+ // read
+ int ret = readInputStream(this.partnerInputStream, buf, off, len, oneByteRead, oCounter, State.OREAD);
+
+ // simulate the read without passing through MemArts
+ long obsReadBytes = oisSimulator.read(oneByteRead ? 1 : len);
+ increaseSimTraffic(obsReadBytes);
+
+ return ret;
+ } catch (OBSMemArtsPartnerInputStream.OReadToMReadTransitionException e) {
+ if (this.state != State.OREAD) {
+ throw new IllegalStateException("state must be oread");
+ }
+ stateTransitionToMRead();
+
+ /**
+ * based on the implementation of OBSInputStream,
+ * when OBSInputStream trigger reopen(),
+ * it will not read any data from stream,
+ * thus, we should not update the {nextReadPos}
+ */
+ // continue to mread
+ }
+ case MREAD:
+ try {
+ int ret = tryToReadFromCCStream(buf, off, len, oneByteRead);
+
+ // simulate the read passing through MemArts
+ long obsReadBytes = oisSimulator.read(oneByteRead ? 1 : len);
+ increaseSimTraffic(obsReadBytes);
+
+ return ret;
+ } catch (EOFException e) {
+ return -1;
+ } catch (IOException e) {
+ LOG.error("tryToReadFromCCStream offset[{}] len[{}] of [{}] failed, due to exception[{}]",
+ off, len, uri, e);
+ throw e;
+ }
+
+ default:
+ throw new IllegalStateException("unreachable code");
+ }
+ }
+
+ @Override
+ public synchronized int read() throws IOException {
+ fs.checkOpen();
+ checkStreamOpen();
+ long startTime = System.currentTimeMillis();
+ long threadId = Thread.currentThread().getId();
+ long endTime;
+ boolean isTrue = this.contentLength == 0 || nextReadPos >= contentLength;
+
+ if (isTrue) {
+ return -1;
+ }
+
+ int byteRead = 0;
+ // tail case
+ int posInTail = posInTailBuf(nextReadPos);
+ if (posInTail != -1 && tailBuf != null) {
+ byteRead = tailBuf[posInTail] & 0xFF;
+ nextReadPos++;
+ this.partnerInputStream.seek(nextReadPos);
+ oisSimulator.seek(nextReadPos);
+ this.ccStream.seek(nextReadPos);
+ incrementBytesRead(1);
+ return byteRead;
+ }
+
+ byteRead = readInState(null, 0, 0, true);
+
+ if (byteRead >= 0) {
+ nextReadPos++;
+ }
+
+ endTime = System.currentTimeMillis();
+ long position = byteRead >= 0 ? nextReadPos - 1 : nextReadPos;
+ LOG.debug("read-0arg uri:{}, contentLength:{}, position:{}, readValue:{}, " + "thread:{}, timeUsedMilliSec:{}",
+ uri, contentLength, position, byteRead, threadId, endTime - startTime);
+
+ return byteRead;
+ }
+
+ @Override
+ public synchronized int read(ByteBuffer byteBuffer) throws IOException {
+ int len = byteBuffer.remaining();
+ if (len == 0) {
+ return 0;
+ }
+
+ byte[] buf = new byte[len];
+ int size = read(buf, 0, len);
+ if (size != -1) {
+ byteBuffer.put(buf, 0, size);
+ }
+
+ return size;
+ }
+
+ private int posInTailBuf(long pos) {
+ if (pos < 0) {
+ return -1;
+ }
+ if (pos < contentLength - bufSize) {
+ return -1;
+ }
+ long tailBufHead = contentLength - bufSize;
+ if (tailBufHead < 0) tailBufHead = 0;
+ long idx = pos - tailBufHead;
+ if (idx < 0 || idx >= bufSize) {
+ LOG.warn("nextReadPos is in invalid state, pos = {}, contentLength = {}, bufSize = {}", pos, contentLength, bufSize);
+ return -1;
+ }
+ return (int)idx;
+ }
+
+ @Override
+ public synchronized void seek(final long targetPos) throws IOException {
+ LOG.debug("seek(targetPos), targetPos [{}] ", targetPos);
+
+ fs.checkOpen();
+ checkStreamOpen();
+
+ // Do not allow negative seek
+ if (targetPos < 0) throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK + " " + targetPos);
+ if (targetPos > contentLength) throw new EOFException("Cannot seek after EOF");
+
+ if (this.contentLength <= 0) {
+ return;
+ }
+
+ // tail case
+ if (posInTailBuf(targetPos) != -1 && tailBuf == null) {
+ byte[] tmpBuf = new byte[bufSize];
+ long tailBufHead = contentLength - bufSize;
+ if (tailBufHead < 0) tailBufHead = 0;
+ // seek to tailBufHead
+ nextReadPos = tailBufHead;
+ this.partnerInputStream.seek(tailBufHead);
+ oisSimulator.seek(tailBufHead);
+ this.ccStream.seek(tailBufHead);
+ int off = 0;
+ int bytesRead = 0;
+ do {
+ bytesRead = read(tmpBuf, off, bufSize - off);
+ if (bytesRead == -1) {
+ break;
+ }
+ off += bytesRead;
+ } while (off < bufSize);
+ tailBuf = tmpBuf;
+ }
+
+ // Lazy seek
+ nextReadPos = targetPos;
+ this.partnerInputStream.seek(targetPos);
+ oisSimulator.seek(targetPos);
+ this.ccStream.seek(targetPos);
+ }
+
+ @Override
+ public synchronized long getPos() throws IOException {
+ fs.checkOpen();
+ checkStreamOpen();
+ return nextReadPos < 0 ? 0 : nextReadPos;
+ }
+
+ @Override
+ public boolean seekToNewSource(long targetPos) throws IOException {
+ return false;
+ }
+
+ @Override
+ public synchronized void setReadahead(Long readahead) throws IOException {
+ fs.checkOpen();
+ checkStreamOpen();
+ this.ccStream.setReadahead(readahead);
+ }
+
+ /**
+ * 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) {
+ fs.checkOpen();
+ // this is actually a no-op
+ super.close();
+ if (partnerInputStream != null) {
+ partnerInputStream.close();
+ partnerInputStream = null;
+ }
+ if (ccStream != null) {
+ ccStream.close();
+ ccStream = null;
+ }
+ closed = true;
+ oisSimulator.close();
+ }
+ LOG.debug("{} SUMMARY: OREAD{}, MREAD{}", this.hashCode(), oCounter, mCounter);
+ }
+
+ /**
+ * 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 checkStreamOpen() throws IOException {
+ if (closed) {
+ throw new IOException(uri + ": " + FSExceptionMessages.STREAM_IS_CLOSED);
+ }
+ }
+
+ /**
+ * 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 int tryToReadFromCCStream(final byte[] buf, final int off, final int len, boolean oneByteRead) throws IOException {
+ int bytesRead;
+ try {
+ bytesRead = readInputStream(this.ccStream, buf, off, len, oneByteRead, mCounter, State.MREAD);
+ return bytesRead;
+ } catch (IOException e) {
+ // escape IOException and CacheMiss
+ LOG.debug("{} mread escape, caused by {}", this.hashCode(), e.getMessage());
+ }
+
+ /**
+ * escape:
+ * 1. partnerInputStream lazy seek to {nextReadPos}
+ * 2. do one read() from partnerInputStream (ORead),
+ * 3. transfer to ORead state
+ */
+ if (this.getState() != State.MREAD) {
+ throw new IllegalStateException("cachemiss reopen must in state mread");
+ }
+
+ // 1. lazy seek to the right position
+ this.partnerInputStream.seek(this.nextReadPos);
+ oisSimulator.seek(nextReadPos);
+
+ // 2. do once OBSInputStream.read
+ // in MRead state, if this read() trigger reopen(),
+ // it will handled by partnerInputStream quietly.
+ try {
+ bytesRead = readInputStream(this.partnerInputStream, buf, off, len, oneByteRead, oCounter, State.OREAD);
+ } catch (OBSMemArtsPartnerInputStream.OReadToMReadTransitionException e) {
+ // we should not cache transfer signal in this place,
+ LOG.error("catch unexpected reopen signal, {}", e.getMessage());
+ throw new IllegalStateException("catch unexpected reopen signal", e);
+ }
+
+ // 3. into ORead state, cache signal
+ this.stateTransitionToORead();
+ return bytesRead;
+ }
+
+ public State getState() {
+ return this.state;
+ }
+
+ private void stateTransitionToMRead() throws IOException {
+ if (this.state != State.OREAD) {
+ throw new IllegalStateException("cannot transit state from " + this.state.toString() + " to mread");
+ }
+ this.state = State.MREAD;
+ this.ccStream.seek(nextReadPos);
+ oisSimulator.seek(nextReadPos);
+ }
+
+ private void stateTransitionToORead() {
+ if (this.state != State.MREAD && this.state != State.NEW) {
+ throw new IllegalStateException("cannot transit state from " + this.state.toString() + " to oread");
+ }
+ this.state = State.OREAD;
+ }
+
+ public enum State {
+ /**
+ * new created input stream
+ */
+ NEW("NEW"),
+
+ /**
+ * using partnerInputStream(OBSInputStream) to do actual read()
+ */
+ OREAD("OREAD"),
+
+ /**
+ * using memarts to do actual read()
+ */
+ MREAD("MREAD"),
+
+ /**
+ * closed state
+ */
+ CLOSED("CLOSED");
+
+ private String state;
+
+ State(String s) {
+ this.state = s;
+ }
+
+ public String toString() {
+ return this.state;
+ }
+ }
+
+ private static class Counter { // because read is synchronized this count need not lock
+
+ private volatile AtomicLong num = new AtomicLong();
+
+ private volatile AtomicLong totalTime = new AtomicLong();
+
+ private volatile AtomicLong totalLen = new AtomicLong();
+
+ public void increase(long time, long len) {
+ totalTime.getAndAdd(time);
+ totalLen.getAndAdd(len);
+ num.getAndIncrement();
+ }
+
+ @Override
+ public String toString() {
+ return String.format("[count=%s,time=%s,size=%s]", num, totalTime, totalLen);
+ }
+ }
+
+ private void increaseSimTraffic(long value) {
+ if (trafficStatistics != null) {
+ trafficStatistics.increase(value, TrafficStatistics.TrafficType.Q);
+ }
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSMemArtsPartnerInputStream.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSMemArtsPartnerInputStream.java
new file mode 100644
index 0000000..347d840
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSMemArtsPartnerInputStream.java
@@ -0,0 +1,95 @@
+package org.apache.hadoop.fs.obs.input;
+
+import com.obs.services.ObsClient;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.obs.OBSFileSystem;
+import org.apache.hadoop.fs.obs.TrafficStatistics;
+
+import java.io.IOException;
+
+public class OBSMemArtsPartnerInputStream extends OBSInputStream {
+ private OBSMemArtsCCInputStream mistream;
+
+ private TrafficStatistics trafficStatistics;
+
+ OBSMemArtsPartnerInputStream(
+ String bucketName,
+ String bucketKey,
+ long fileStatusLength,
+ ObsClient obsClient,
+ FileSystem.Statistics stats,
+ long readAheadRangeValue,
+ OBSFileSystem obsFileSystem,
+ OBSMemArtsCCInputStream mistream) {
+ super(bucketName, bucketKey, fileStatusLength, obsClient, stats, readAheadRangeValue, obsFileSystem);
+ this.mistream = mistream;
+ }
+
+ public void setTrafficStaticsClass(TrafficStatistics instance) {
+ trafficStatistics = instance;
+ }
+
+ private void increaseMissTraffic(long value) {
+ if (trafficStatistics != null) {
+ trafficStatistics.increase(value, TrafficStatistics.TrafficType.Q1);
+ }
+ }
+
+ @Override
+ protected synchronized void reopen(final String reason, final long targetPos, final long length) throws IOException {
+ /**
+ * reopen() in New state should call super reopen directly
+ */
+ if (this.mistream.getState() == OBSMemArtsCCInputStream.State.NEW) {
+ super.reopen(reason, targetPos, length);
+ long readFromOBS = calculateOBSTraffic(targetPos, length);
+ increaseMissTraffic(readFromOBS);
+ }
+
+ /**
+ * oread reopen
+ */
+ if (this.mistream.getState() == OBSMemArtsCCInputStream.State.OREAD) {
+ /**
+ * based on the implementation of OBSInputStream
+ * when reopen() occurred,byteRead must be 0,
+ * thus, the caller need not to consider the
+ * intermediate state of reading a fractional of data.
+ * Then just close the wrapper stream and throw the state transfer signal.
+ */
+ closeStream(reason, length);
+ throw new OReadToMReadTransitionException("oread reopen(), transit to mread, origin reason: " + reason, targetPos, length);
+ }
+
+ /**
+ * escape from MemArtsCC, reopen the OBSInputStream
+ */
+ if (this.mistream.getState() == OBSMemArtsCCInputStream.State.MREAD) {
+ super.reopen(reason, targetPos, length);
+ long readFromOBS = calculateOBSTraffic(targetPos, length);
+ increaseMissTraffic(readFromOBS);
+ }
+ }
+
+ public static class OReadToMReadTransitionException extends RuntimeException {
+ static final long serialVersionUID = 5364319876219655679L;
+
+ public long getTargetPos() {
+ return targetPos;
+ }
+
+ public long getLen() {
+ return len;
+ }
+
+ private long targetPos;
+ private long len;
+
+ public OReadToMReadTransitionException(String msg, long targetPos, long len) {
+ super(msg);
+ this.len = len;
+ this.targetPos = targetPos;
+ }
+
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ObsSelectInputStream.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ObsSelectInputStream.java
new file mode 100644
index 0000000..8c9133a
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ObsSelectInputStream.java
@@ -0,0 +1,299 @@
+package org.apache.hadoop.fs.obs.input;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import javax.annotation.Nullable;
+
+import com.google.common.base.Preconditions;
+import com.obs.services.model.select.SelectEventVisitor;
+import com.obs.services.model.select.SelectInputStream;
+import com.obs.services.model.select.SelectObjectResult;
+
+import org.apache.hadoop.fs.CanSetReadahead;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.obs.OBSCommonUtils;
+import org.apache.hadoop.fs.obs.OBSConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ObsSelectInputStream
+ extends FSInputStream implements CanSetReadahead {
+ private static final Logger LOG = LoggerFactory.getLogger(ObsSelectInputStream.class);
+
+ private AtomicLong pos = new AtomicLong(0);
+ private final AtomicBoolean closed = new AtomicBoolean(false);
+ private final AtomicBoolean completedSuccessfully = new AtomicBoolean(false);
+
+ private final SelectInputStream recordsInputStream;
+
+ private final String bucket;
+ private final String key;
+ private final String uri;
+
+ private long readahead;
+
+ public ObsSelectInputStream(
+ final String bucket,
+ final String key,
+ final SelectObjectResult selectResult) {
+ this.recordsInputStream = selectResult.getInputStream(
+ new SelectEventVisitor() {
+ @Override
+ public void visitEndEvent() {
+ completedSuccessfully.set(true);
+ }
+ });
+
+ this.bucket = bucket;
+ this.key = key;
+ this.uri = String.format("obs://%s/%s", this.bucket, this.key);
+ }
+
+ @Override
+ public void close()
+ throws IOException {
+ if (!closed.getAndSet(true)) {
+ try {
+ // set up for aborts.
+ // if we know the available amount > readahead. Abort.
+ //
+ boolean shouldAbort = recordsInputStream.available() > readahead;
+ if (!shouldAbort) {
+ // read our readahead range worth of data
+ long skipped = recordsInputStream.skip(readahead);
+ shouldAbort = recordsInputStream.read() >= 0;
+ }
+ // now, either there is data left or not.
+ if (shouldAbort) {
+ recordsInputStream.abort();
+ }
+ } catch (IOException e) {
+ LOG.debug("While closing stream", e);
+ } finally {
+ OBSCommonUtils.closeAll(recordsInputStream);
+ super.close();
+ }
+ }
+ }
+
+ /**
+ * Verify that the input stream is open. Non blocking; this gives
+ * the last state of the atomic {@link #closed} field.
+ *
+ * @throws PathIOException if the connection is closed.
+ */
+ private void checkNotClosed()
+ throws IOException {
+ if (closed.get()) {
+ throw new PathIOException(uri, FSExceptionMessages.STREAM_IS_CLOSED);
+ }
+ }
+
+ @Override
+ public int available()
+ throws IOException {
+ checkNotClosed();
+ return recordsInputStream.available();
+ }
+
+ @Override
+ public long skip(long n)
+ throws IOException {
+ checkNotClosed();
+ long skipped = recordsInputStream.skip(n);
+ pos.addAndGet(skipped);
+ return skipped;
+ }
+
+ @Override
+ public long getPos() {
+ return pos.get();
+ }
+
+ /**
+ * from a possibly null Long value, return a valid
+ * readahead.
+ *
+ * @param readahead new readahead
+ * @return a natural number.
+ * @throws IllegalArgumentException if the range is invalid.
+ */
+ public static long validateReadahead(@Nullable Long readahead) {
+ if (readahead == null) {
+ return OBSConstants.DEFAULT_READAHEAD_RANGE;
+ } else {
+ Preconditions.checkArgument(readahead >= 0, "Negative readahead value" /* E_NEGATIVE_READAHEAD_VALUE */);
+ return readahead;
+ }
+ }
+
+ @Override
+ public void setReadahead(Long readahead) {
+ this.readahead = validateReadahead(readahead);
+ }
+
+ public long getReadahead() {
+ return readahead;
+ }
+
+ @Override
+ public int read()
+ throws IOException {
+ checkNotClosed();
+ int byteRead;
+ try {
+ byteRead = recordsInputStream.read();
+ } catch (EOFException e) {
+ // this could be one of: end of file, some IO failure
+ if (completedSuccessfully.get()) {
+ // read was successful
+ return -1;
+ } else {
+ // the stream closed prematurely
+ LOG.info("Reading of OBS Select data from {} failed before all results "
+ + " were generated.", uri);
+ throw new PathIOException(uri,
+ "Read of OBS Select data did not complete");
+ }
+ }
+
+ if (byteRead >= 0) {
+ incrementBytesRead(1);
+ }
+ return byteRead;
+ }
+
+ @Override
+ public int read(byte[] buf, int off, int len)
+ throws IOException {
+ checkNotClosed();
+ validatePositionedReadArgs(pos.get(), buf, off, len);
+ if (len == 0) {
+ return 0;
+ }
+
+ int bytesRead;
+ try {
+ bytesRead = recordsInputStream.read(buf, off, len);
+ } catch (EOFException e) {
+ // the base implementation swallows EOFs.
+ return -1;
+ }
+
+ incrementBytesRead(bytesRead);
+ return bytesRead;
+ }
+
+ @Override
+ public void seek(long newPos)
+ throws IOException {
+ long current = getPos();
+ long distance = newPos - current;
+ if (distance < 0) {
+ throw unsupported("seek() backwards from " + current + " to " + newPos);
+ }
+ if (distance == 0) {
+ LOG.debug("ignoring seek to current position.");
+ } else {
+ // the complicated one: Forward seeking. Useful for split files.
+ LOG.debug("Forward seek by reading {} bytes", distance);
+ long bytesSkipped = 0L;
+ // read byte-by-byte, hoping that buffering will compensate for this.
+ // doing it this way ensures that the seek stops at exactly the right
+ // place. skip(len) can return a smaller value, at which point
+ // it's not clear what to do.
+ while (distance > 0) {
+ int r = read();
+ if (r == -1) {
+ // reached an EOF too early
+ throw new EOFException("Seek to " + newPos
+ + " reached End of File at offset " + getPos());
+ }
+ distance--;
+ bytesSkipped++;
+ }
+ }
+ }
+
+ /**
+ * Build an exception to raise when an operation is not supported here.
+ *
+ * @param action action which is unsupported.
+ * @return an exception to throw.
+ */
+ protected PathIOException unsupported(final String action) {
+ return new PathIOException(this.uri, action + " not supported");
+ }
+
+ @Override
+ public boolean seekToNewSource(long targetPos)
+ throws IOException {
+ return false;
+ }
+
+ @Override
+ public boolean markSupported() {
+ return false;
+ }
+
+ @Override
+ public synchronized void mark(int readlimit) {
+ }
+
+ @Override
+ public synchronized void reset()
+ throws IOException {
+ throw unsupported("Mark");
+ }
+
+ public void abort() {
+ if (!closed.get()) {
+ LOG.debug("Aborting");
+ recordsInputStream.abort();
+ }
+ }
+
+ /**
+ * Read at a specific position.
+ * Reads at a position earlier than the current {@link #getPos()} position
+ * will fail with a {@link PathIOException}. See {@link #seek(long)}.
+ * Unlike the base implementation And the requirements of the filesystem
+ * specification, this updates the stream position as returned in
+ * {@link #getPos()}.
+ *
+ * @param position offset in the stream.
+ * @param buffer buffer to read in to.
+ * @param offset offset within the buffer
+ * @param length amount of data to read.
+ * @return the result.
+ * @throws PathIOException Backwards seek attempted.
+ * @throws EOFException attempt to seek past the end of the stream.
+ * @throws IOException IO failure while seeking in the stream or reading
+ * data.
+ */
+ @Override
+ public int read(long position, byte[] buffer, int offset, int length)
+ throws IOException {
+ // maybe seek forwards to the position.
+ seek(position);
+ return read(buffer, offset, length);
+ }
+
+ /**
+ * Increment the bytes read counter if there is a stats instance
+ * and the number of bytes read is more than zero.
+ * This also updates the {@link #pos} marker by the same value.
+ *
+ * @param bytesRead number of bytes read
+ */
+ private void incrementBytesRead(long bytesRead) {
+ if (bytesRead > 0) {
+ pos.addAndGet(bytesRead);
+ }
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ReadAheadTask.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ReadAheadTask.java
index 46c0106..f4ff1de 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ReadAheadTask.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ReadAheadTask.java
@@ -22,7 +22,7 @@
* we can accelerate the sequential read.
*/
public class ReadAheadTask implements Runnable {
- public final Logger log = LoggerFactory.getLogger(ReadAheadTask.class);
+ private static final Logger log = LoggerFactory.getLogger(ReadAheadTask.class);
private String bucketName;
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/CcGetShardParam.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/CcGetShardParam.java
new file mode 100644
index 0000000..e13cb33
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/CcGetShardParam.java
@@ -0,0 +1,65 @@
+package org.apache.hadoop.fs.obs.memartscc;
+
+/**
+ * 功能描述
+ *
+ * @since 2021-05-24
+ */
+public class CcGetShardParam {
+ long start;
+
+ long end;
+
+ String bucketName;
+
+ boolean enablePosix;
+
+ public String objectKey;
+
+ ObjectShard[] ObjectShard;
+
+ int allocShardNum;
+
+ int validShardNum;
+
+ public CcGetShardParam(long start, long end, String bucketName, boolean enablePosix, String objectKey,
+ ObjectShard[] ObjectShard, int allocShardNum, int validShardNum) {
+ this.start = start;
+ this.end = end;
+ this.bucketName = bucketName;
+ this.enablePosix = enablePosix;
+ this.objectKey = objectKey;
+ this.ObjectShard = ObjectShard;
+ this.allocShardNum = allocShardNum;
+ this.validShardNum = validShardNum;
+ }
+
+ public int getValidShardNum() {
+ return validShardNum;
+ }
+
+ public ObjectShard[] getObjectShard() {
+ return ObjectShard;
+ }
+
+ public long getStart() {
+ return start;
+ }
+
+ public long getEnd() {
+ return end;
+ }
+
+ public String getBucketName() {
+ return bucketName;
+ }
+
+ public boolean isEnablePosix() {
+ return enablePosix;
+ }
+
+ public int getAllocShardNum() {
+ return allocShardNum;
+ }
+
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ICache.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ICache.java
new file mode 100644
index 0000000..19ac2e1
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ICache.java
@@ -0,0 +1,78 @@
+package org.apache.hadoop.fs.obs.memartscc;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public interface ICache {
+ int init(String config, String otherInfo) throws IOException;
+
+ int read(ReadParam readParam, ByteBuffer buf, byte[] dt) throws IOException;
+
+ int getObjectShardInfo(CcGetShardParam ccGetShardParam);
+
+ byte[] getDT();
+
+ // originalTraffic: Q
+ // applicationTraffic: Q'
+ // hitTraffic: Q2
+ // missTraffic: Q1
+ void reportReadStatistics(long originalTraffic, long applicationTraffic, long hitTraffic, long missTraffic);
+
+ void close();
+
+ class ReadParam {
+ public long offset;
+ public long length;
+ public long prefetchStart;
+ public long prefetchEnd;
+ public boolean isPrefetch;
+ public boolean isConsistencyCheck;
+ public boolean isFileLayout; // 特性暂未启用,默认false
+ public BucketContext bucketCtx;
+ public ObjectAttr objectAttr;
+
+ public ReadParam(long offset, long length, long prefetchStart, long prefetchEnd,
+ boolean isPrefetch, boolean isConsistencyCheck, boolean isFileLayout,
+ BucketContext bucketCtx, ObjectAttr objectAttr) {
+ this.offset = offset;
+ this.length = length;
+ this.prefetchStart = prefetchStart;
+ this.prefetchEnd = prefetchEnd;
+ this.isPrefetch = isPrefetch;
+ this.isConsistencyCheck = isConsistencyCheck;
+ this.isFileLayout = isFileLayout;
+ this.bucketCtx = bucketCtx;
+ this.objectAttr = objectAttr;
+ }
+ }
+
+ class BucketContext {
+ public String ak;
+ public String sk;
+ public String token;
+ public String endpoint;
+ public String bucketName;
+ public boolean enablePosix;
+
+ public BucketContext(String ak, String sk, String token, String endpoint, String bucketName, boolean enablePosix) {
+ this.ak = ak;
+ this.sk = sk;
+ this.token = token;
+ this.endpoint = endpoint;
+ this.bucketName = bucketName;
+ this.enablePosix = enablePosix;
+ }
+ }
+
+ class ObjectAttr {
+ public String name;
+ public String etag;
+ public long mtime;
+
+ public ObjectAttr(String name, String etag, long mtime) {
+ this.name = name;
+ this.etag = etag;
+ this.mtime = mtime;
+ }
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCClient.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCClient.java
new file mode 100644
index 0000000..6a3681d
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCClient.java
@@ -0,0 +1,440 @@
+package org.apache.hadoop.fs.obs.memartscc;
+
+import com.obs.services.IObsCredentialsProvider;
+import com.obs.services.internal.security.LimitedTimeSecurityKey;
+import com.obs.services.model.ISecurityKey;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.obs.OBSConstants;
+import org.apache.hadoop.fs.obs.OBSSecurityProviderUtil;
+import org.apache.hadoop.fs.obs.TrafficStatistics;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.codehaus.jettison.json.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
+import java.nio.ByteBuffer;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import sun.nio.ch.DirectBuffer;
+
+/**
+ * 功能描述
+ *
+ * @since 2021-05-18
+ */
+public class MemArtsCCClient {
+
+ private static final String MRS_AZ_ENV_VARNAME = "AZ";
+
+ private static final String SECURITY_ENABLE = "security_enable";
+
+ private static final Logger LOG = LoggerFactory.getLogger(MemArtsCCClient.class);
+
+ private static final boolean IS_FILE_LAYOUT = false;
+
+ private IObsCredentialsProvider securityProvider;
+
+ private String endpoint;
+
+ private final String bucket;
+
+ private final boolean enablePosix;
+
+ private volatile boolean closed;
+
+ private boolean initialized;
+
+ private int akskRefreshInterval;
+
+ AtomicReference iSecurityKey = new AtomicReference<>();
+
+ private ICache cache;
+
+ private Configuration conf;
+
+ private final AtomicReference password;
+
+ public MemArtsCCClient(String bucket, boolean enablePosix) {
+ this.bucket = bucket;
+ this.enablePosix = enablePosix;
+ this.password = new AtomicReference<>();
+ }
+
+ public boolean initialize(URI name, Configuration conf) {
+ if (initialized) {
+ LOG.warn("MemArtsCCClient have been initialized more than once");
+ return false;
+ }
+
+ if (needEscapePyspark(conf)) {
+ return false;
+ }
+
+ this.conf = conf;
+
+ if (!createCacheClass(conf)) {
+ return false;
+ }
+
+ setEndpoint(conf);
+
+ akskRefreshInterval = conf.getInt(
+ OBSConstants.MEMARTSCC_AKSK_REFRESH_INTERVAL,
+ OBSConstants.DEFAULT_MEMARTSCC_AKSK_REFRESH_INTERVAL);
+ try {
+ securityProvider = OBSSecurityProviderUtil.createObsSecurityProvider(conf, name);
+ } catch (IOException e) {
+ LOG.warn("create security provider failed, {}", e.getMessage());
+ return false;
+ }
+ setSecurityKey(securityProvider.getSecurityKey());
+
+ if (!initCacheClass(conf)) {
+ return false;
+ }
+
+ startRefreshJob();
+ initBufferPool(conf);
+ initialized = true;
+ return true;
+ }
+
+ /**
+ * Initialize MemArtsCCClient in the delegation token only scenario,
+ * i.e. where DELEGATION_TOKEN_ONLY is set to true
+ */
+ public boolean initializeDtOnly(final Configuration conf) {
+ if (initialized) {
+ LOG.warn("MemArtsCCClient hav e been initialized more than once");
+ return false;
+ }
+
+ this.conf = conf;
+
+ if (needEscapePyspark(conf)) {
+ return false;
+ }
+
+ if (!createCacheClass(conf)) {
+ return false;
+ }
+
+ if (!initCacheClass(conf)) {
+ return false;
+ }
+
+ initialized = true;
+ return true;
+ }
+
+ private void initBufferPool(Configuration conf) {
+ int maxNum = conf.getInt(OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_POOL_MAX_SIZE,
+ OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_POOL_DEFAULT_MAX_SIZE);
+ int bufferSize = conf.getInt(OBSConstants.MEMARTSCC_DIRECTBUFFER_SIZE,
+ OBSConstants.DEFAULT_MEMARTSCC_DIRECTBUFFER_SIZE);
+ bufferPool.initialize(maxNum, bufferSize);
+ }
+
+ private void setEndpoint(Configuration conf) {
+ String ep = conf.getTrimmed(OBSConstants.ENDPOINT);
+ if (ep.startsWith("http://")) {
+ ep = ep.substring("http://".length());
+ }
+ if (ep.startsWith("https://")) {
+ ep = ep.substring("https://".length());
+ }
+ this.endpoint = ep;
+ }
+
+ private boolean initCacheClass(Configuration conf) {
+ try {
+ String config = filterCCConfig(conf);
+ String otherInfo = collectOtherInfo(conf);
+ int result = init(config, otherInfo);
+ if (result == 0) {
+ LOG.debug("memArtsCCClient.ccInit OK!");
+ } else {
+ LOG.warn("memArtsCC init failed, ccInit ret code = {}, will trying to fallback", result);
+ return false;
+ }
+ } catch (Throwable e) {
+ LOG.warn("memArtsCC init exception, will trying to fallback, caused by {}", e.getMessage());
+ return false;
+ }
+ return true;
+ }
+
+ private boolean createCacheClass(Configuration conf) {
+ try {
+ Class extends ICache> cacheClass = conf.getClass(OBSConstants.MEMARTSCC_CACHE_IMPL,
+ null, ICache.class);
+ if (cacheClass == null) {
+ LOG.warn("get null ICache instance");
+ return false;
+ }
+ this.cache = cacheClass.newInstance();
+ } catch (InstantiationException | IllegalAccessException | RuntimeException e) {
+ LOG.warn("get instance of ICache failed", e);
+ return false;
+ }
+ return true;
+ }
+
+ private boolean needEscapePyspark(Configuration conf) {
+ boolean isPySpark = conf.getBoolean("spark.yarn.isPython", false);
+ if (isPySpark) {
+ boolean pySparkOptimizedConf = conf.getBoolean(OBSConstants.MEMARTSCC_PYSPARK_OPTIMIZED,
+ OBSConstants.DEFAULT_MEMARTSCC_PYSPARK_OPTIMIZED);
+ if (!pySparkOptimizedConf) {
+ LOG.error("disable pyspark optimize from config");
+ }
+ boolean pySparkOptimizedProp = Boolean.parseBoolean(System.getProperty(OBSConstants.MEMARTSCC_PYSPARK_OPTIMIZED,
+ String.valueOf(OBSConstants.DEFAULT_MEMARTSCC_PYSPARK_OPTIMIZED)));
+ if (!pySparkOptimizedProp) {
+ LOG.error("disable pyspark optimize from properties");
+ }
+ boolean pySparkOptimized = pySparkOptimizedConf && pySparkOptimizedProp;
+ if (!pySparkOptimized) {
+ LOG.error("escape in pyspark");
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private String filterCCConfig(Configuration conf) {
+ String prefix = conf.get(OBSConstants.CACHE_CONFIG_PREFIX, OBSConstants.DEFAULT_CACHE_CONFIG_PREFIX);
+ if (!prefix.endsWith(".")) {
+ prefix = prefix + ".";
+ }
+ Map confMap = conf.getPropsWithPrefix(prefix);
+ String az = System.getenv(MRS_AZ_ENV_VARNAME);
+ if (az != null && !az.equals("")) {
+ confMap.put("zk_root_node", "/memartscc/" + az);
+ }
+ confMap.put(SECURITY_ENABLE, Boolean.toString(UserGroupInformation.isSecurityEnabled()));
+ JSONObject jsonObject = new JSONObject(confMap);
+ String jsonStr = jsonObject.toString();
+ LOG.info("memArtsCC config json: {}", jsonStr);
+ return jsonStr;
+ }
+
+ private byte[] getDtFromUgi() throws IOException {
+ byte[] dt = this.password.get();
+ if (dt != null) {
+ return dt;
+ }
+ Credentials credentials = UserGroupInformation.getLoginUser().getCredentials();
+ Text serviceName = new Text(MemArtsCCDelegationTokenProvider.getCanonicalName(conf));
+ Token extends TokenIdentifier> token = credentials.getToken(serviceName);
+ if (token == null) {
+ return null;
+ }
+ dt = token.getPassword();
+ this.password.set(dt);
+ return dt;
+ }
+
+ private String collectOtherInfo(Configuration conf) {
+ Map otherInfo = new HashMap<>();
+ boolean locality = conf.getBoolean(OBSConstants.MEMARTSCC_LOCALITY_ENABLE, OBSConstants.DEFAULT_MEMARTSCC_LOCALITY_ENABLE);
+ otherInfo.put("locality_switch", locality);
+ otherInfo.put("client_type", "obsa");
+ JSONObject jsonObject = new JSONObject(otherInfo);
+ String jsonStr = jsonObject.toString();
+ LOG.info("memArtsCC other info json: {}", jsonStr);
+ return jsonStr;
+ }
+
+ private void startRefreshJob() {
+ Thread refreshThread = new Thread(() -> {
+ while(true) {
+ try {
+ while (true) {
+ if (closed) {
+ this.cache.close();
+ return;
+ }
+ long sleepInSec = akskRefreshInterval;
+ ISecurityKey securityKey = securityProvider.getSecurityKey();
+ if (securityKey instanceof LimitedTimeSecurityKey) {
+ LimitedTimeSecurityKey lsk = (LimitedTimeSecurityKey) securityKey;
+ long expireAt = lsk.getExpiryDate().getTime();
+ long now = LimitedTimeSecurityKey.getUtcTime().getTime();
+ long keyAgeMill = expireAt - now;
+ if (keyAgeMill < akskRefreshInterval * 1000L) {
+ sleepInSec = keyAgeMill / 10000;
+ if (sleepInSec < 1) {
+ sleepInSec = 1;
+ }
+ LOG.warn("Refresh MemArtsCC AK/SK interval reset to {} sec, "
+ + "please check fs.obs.memartscc.aksk.refresh.interval", sleepInSec);
+ }
+ }
+ setSecurityKey(securityKey);
+ try {
+ Thread.sleep(sleepInSec * 1000L);
+ } catch (InterruptedException e) {
+ LOG.warn("Refresh ak sk interrupted", e);
+ }
+ }
+ } catch (Exception e) {
+ LOG.warn("Refresh ak sk job failed, will trying to restart.", e);
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException interruptedException) {
+ LOG.warn("Refresh ak sk interrupted", e);
+ }
+ }
+ }
+ });
+ refreshThread.setDaemon(true);
+ refreshThread.start();
+ }
+
+ private void setSecurityKey(ISecurityKey key) {
+ if (key == null) {
+ LOG.warn("SecurityKey cannot be set, securityKey is null");
+ return;
+ }
+ this.iSecurityKey.set(key);
+ }
+
+ public void close() {
+ this.closed = true;
+ }
+
+ public int init(String config, String otherInfo) throws IOException {
+ return this.cache.init(config, otherInfo);
+ }
+
+ public int read(boolean isPrefetch, long prefetchStart, long prefetchEnd, ByteBuffer buf, long offset,
+ long len, String objectKey, long modifyTime, String etag, boolean isConsistencyCheck) throws IOException {
+ if(!initialized) {
+ throw new IOException("MemArtsCCClient read before initializing.");
+ }
+
+ ICache.BucketContext bucketContext = getBucketContext();
+
+ ICache.ObjectAttr objectAttr = new ICache.ObjectAttr(objectKey, etag, modifyTime);
+
+ ICache.ReadParam readParam = new ICache.ReadParam(offset, len, prefetchStart, prefetchEnd,
+ isPrefetch, isConsistencyCheck, IS_FILE_LAYOUT, bucketContext, objectAttr);
+
+ byte[] dt = getDtFromUgi();
+
+ return this.cache.read(readParam, buf, dt);
+ }
+
+ private ICache.BucketContext getBucketContext() {
+ String ak = iSecurityKey.get() == null ? "" : iSecurityKey.get().getAccessKey();
+ String sk = iSecurityKey.get() == null ? "" : iSecurityKey.get().getSecretKey();
+ String securityToken = iSecurityKey.get() == null ? "" : iSecurityKey.get().getSecurityToken();
+
+ return new ICache.BucketContext(ak, sk, securityToken, endpoint, bucket, enablePosix);
+ }
+
+ public int getObjectShardInfo(CcGetShardParam ccGetShardParam) {
+ return this.cache.getObjectShardInfo(ccGetShardParam);
+ }
+
+ public void reportReadStatistics(TrafficStatistics trafficStatistics) {
+ if (!initialized || this.cache == null) {
+ LOG.debug("MemArtsCCClient is not initialized, statistics cannot be reported.");
+ return;
+ }
+
+ this.cache.reportReadStatistics(trafficStatistics.getStatistics(TrafficStatistics.TrafficType.Q),
+ trafficStatistics.getStatistics(TrafficStatistics.TrafficType.QDot),
+ trafficStatistics.getStatistics(TrafficStatistics.TrafficType.Q2),
+ trafficStatistics.getStatistics(TrafficStatistics.TrafficType.Q1));
+ }
+
+ public byte[] getDT() {
+ return this.cache.getDT();
+ }
+
+ public static final int CCREAD_RETCODE_CACHEMISS = -100;
+
+ public static class ByteBufferPool {
+ private final AtomicBoolean initialized = new AtomicBoolean(false);
+
+ private final LinkedBlockingQueue pool = new LinkedBlockingQueue<>();
+
+ private int maxNum;
+
+ private final AtomicInteger createdNum = new AtomicInteger(0);
+
+ private int bufferSize;
+
+ public void initialize(int maxNum, int bufferSize) {
+ if (!initialized.compareAndSet(false, true)) {
+ return;
+ }
+ this.maxNum = maxNum;
+ this.bufferSize = bufferSize;
+ }
+
+ public ByteBuffer borrowBuffer(int timeout) throws InterruptedException, IOException {
+ // 1.从队列中非阻塞获取
+ ByteBuffer buffer = pool.poll();
+ // 2. 若队列为空,尝试创建
+ if (buffer == null) {
+ buffer = creatBuffer();
+ }
+ // 3. 创建失败则等待
+ if (buffer == null) {
+ if (timeout < 0) {
+ buffer = pool.take();
+ } else if (timeout > 0) {
+ buffer = pool.poll(timeout, TimeUnit.MILLISECONDS);
+ }
+ // 等于0不等待,直接报错
+ }
+ // 4. 等待不到则报错
+ if (buffer == null) {
+ throw new IOException("ByteBuffer pool exhausted");
+ }
+ return buffer;
+ }
+
+ public void returnBuffer(ByteBuffer buffer) {
+ if (buffer == null) {
+ return;
+ }
+ if (!pool.offer(buffer)) {
+ destroyBuffer(buffer);
+ }
+ }
+
+ private ByteBuffer creatBuffer() {
+ final long newCreateCount = createdNum.incrementAndGet();
+ if (newCreateCount > maxNum) {
+ createdNum.decrementAndGet();
+ return null;
+ }
+ return ByteBuffer.allocateDirect(bufferSize);
+ }
+
+ private void destroyBuffer(ByteBuffer buffer) {
+ if (buffer instanceof DirectBuffer) {
+ ((DirectBuffer)buffer).cleaner().clean();
+ }
+ }
+ }
+
+ public static final ByteBufferPool bufferPool = new ByteBufferPool();
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCDelegationTokenIdentifier.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCDelegationTokenIdentifier.java
new file mode 100644
index 0000000..13f285e
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCDelegationTokenIdentifier.java
@@ -0,0 +1,109 @@
+package org.apache.hadoop.fs.obs.memartscc;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.TokenIdentifier;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class MemArtsCCDelegationTokenIdentifier extends TokenIdentifier {
+ public static final Text MEMARTSCC_DELEGATION_KIND =
+ new Text("MEMARTSCC_DELEGATION_TOKEN");
+
+ private Text owner;
+ private Text renewer;
+ private Text realUser;
+
+ public MemArtsCCDelegationTokenIdentifier() {
+ owner = new Text();
+ renewer = new Text();
+ realUser = new Text();
+ }
+
+
+ public MemArtsCCDelegationTokenIdentifier(Text owner, Text renewer, Text realUser) {
+ setOwner(owner);
+ setRenewer(renewer);
+ setRealUser(realUser);
+ }
+
+ public MemArtsCCDelegationTokenIdentifier(Text renewer) {
+ this(null, renewer, null);
+ }
+
+ @Override
+ public Text getKind() {
+ return MEMARTSCC_DELEGATION_KIND;
+ }
+
+ @Override
+ public UserGroupInformation getUser() {
+ if(owner == null) {
+ return null;
+ }
+ if(owner.toString().isEmpty()) {
+ return null;
+ }
+ final UserGroupInformation realUgi;
+ final UserGroupInformation ugi;
+ if ((realUser == null) || (realUser.toString().isEmpty())
+ || realUser.equals(owner)) {
+ realUgi = UserGroupInformation.createRemoteUser(owner.toString());
+ ugi = realUgi;
+ } else {
+ realUgi = UserGroupInformation.createRemoteUser(realUser.toString());
+ ugi = UserGroupInformation.createProxyUser(owner.toString(), realUgi);
+ }
+ realUgi.setAuthenticationMethod(UserGroupInformation.AuthenticationMethod.TOKEN);
+ return ugi;
+ }
+
+ @Override
+ public void write(DataOutput dataOutput) throws IOException {
+ owner.write(dataOutput);
+ renewer.write(dataOutput);
+ realUser.write(dataOutput);
+ }
+
+ @Override
+ public void readFields(DataInput dataInput) throws IOException {
+ owner.readFields(dataInput, Text.DEFAULT_MAX_LEN);
+ renewer.readFields(dataInput, Text.DEFAULT_MAX_LEN);
+ realUser.readFields(dataInput, Text.DEFAULT_MAX_LEN);
+ }
+
+ public Text getOwner() {
+ return owner;
+ }
+
+ private void setOwner(Text owner) {
+ if (owner == null) {
+ this.owner = new Text();
+ } else {
+ this.owner = owner;
+ }
+ }
+
+ public Text getRenewer() {
+ return renewer;
+ }
+
+ private void setRenewer(Text renewer) {
+ this.renewer = renewer;
+ }
+
+ public Text getRealUser() {
+ return realUser;
+ }
+
+ private void setRealUser(Text realUser) {
+ if (realUser == null) {
+ this.realUser = new Text();
+ } else {
+ this.realUser = realUser;
+ }
+ }
+
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCDelegationTokenProvider.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCDelegationTokenProvider.java
new file mode 100644
index 0000000..009ac46
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCDelegationTokenProvider.java
@@ -0,0 +1,64 @@
+package org.apache.hadoop.fs.obs.memartscc;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.obs.OBSFileSystem;
+import org.apache.hadoop.fs.obs.security.DelegationTokenProvider;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+
+public class MemArtsCCDelegationTokenProvider implements DelegationTokenProvider {
+ private static final Logger logger = LoggerFactory.getLogger(MemArtsCCDelegationTokenProvider.class);
+
+ public static final String ServiceNameKey = "fs.obs.memartscc.service.name";
+
+ public static final String DefaultServiceName = "memartscc";
+
+ public static String getCanonicalName(Configuration conf) {
+ return conf.get(ServiceNameKey, DefaultServiceName);
+ }
+
+ private String canonicalServiceName;
+
+ private Configuration conf;
+
+ private URI uri;
+
+ private OBSFileSystem fs;
+
+ @Override
+ public void initialize(FileSystem fs, URI uri, Configuration conf) {
+ this.canonicalServiceName = getCanonicalName(conf);
+ this.uri = uri;
+ this.conf = conf;
+ if (fs instanceof OBSFileSystem) {
+ this.fs = (OBSFileSystem) fs;
+ } else {
+ throw new IllegalArgumentException("fs only support OBSFileSystem");
+ }
+ }
+
+ @Override
+ public Token> getDelegationToken(String renewer) {
+ logger.info("get delegation token for renewer {}", renewer);
+ try {
+ MemArtsCCDelegationTokenIdentifier tokenIdentifier = new MemArtsCCDelegationTokenIdentifier(
+ new Text(renewer));
+ byte[] password = fs.getMemArtsCCClient().getDT();
+ return new Token(tokenIdentifier.getBytes(), password,
+ MemArtsCCDelegationTokenIdentifier.MEMARTSCC_DELEGATION_KIND, new Text(this.canonicalServiceName));
+ } catch (Exception e) {
+ logger.warn("get dt from memartscc failed", e);
+ }
+ return null;
+ }
+
+ @Override
+ public String getCanonicalServiceName() {
+ return canonicalServiceName;
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStream.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStream.java
new file mode 100644
index 0000000..ccd95c6
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStream.java
@@ -0,0 +1,219 @@
+package org.apache.hadoop.fs.obs.memartscc;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.obs.OBSFileStatus;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import sun.nio.ch.DirectBuffer;
+
+public class MemArtsCCInputStream extends MemArtsCCInputStreamBase {
+ // private static final Logger LOG = LoggerFactory.getLogger(MemArtsCCInputStream.class);
+
+ private final FileSystem.Statistics statistics;
+
+ private boolean closed;
+
+ private MemArtsCCClient ccClient;
+
+ private final String objKey;
+
+ private long prefetchRange;
+
+ private final int bufSize;
+
+ private long nextReadPos;
+
+ private final long contentLength;
+
+ private final String etag;
+
+ private final long mtime;
+
+ private ByteBuffer buffer;
+
+ private int directBufferSize;
+
+ private long bufferStartPos;
+
+ public MemArtsCCInputStream(MemArtsCCClient ccClient, String objKey, OBSFileStatus fileStatus,
+ long prefetchRange, final FileSystem.Statistics stats, int bufSize, int directBufferSize) {
+ this.directBufferSize = directBufferSize;
+ this.buffer = ByteBuffer.allocateDirect(directBufferSize);
+ this.ccClient = ccClient;
+ this.objKey = objKey;
+ this.prefetchRange = prefetchRange;
+ this.statistics = stats;
+ this.bufSize = bufSize;
+ this.contentLength = fileStatus.getLen();
+ this.etag = fileStatus.getEtag();
+ this.mtime = fileStatus.getModificationTime();
+ this.closed = false;
+ buffer.position(0);
+ buffer.limit(0);
+ }
+
+ private long getPrefetchEnd(long pos, int len) {
+ long prefetchEnd = pos + Math.max(prefetchRange, len);
+ return Math.min(prefetchEnd, contentLength);
+ }
+
+ private int readInBuffer(byte[] buf, int off, int len, boolean oneByteRead) throws IOException {
+ if (ensureData(oneByteRead ? 1 : len) < 0) {
+ return -1;
+ }
+ // buffer must have data
+ if (oneByteRead) {
+ return buffer.get() & 0xFF;
+ }
+ int readLen = Math.min(len, buffer.remaining());
+ buffer.get(buf, off, readLen);
+ return readLen;
+ }
+
+ private void ensurePos() {
+ int bufPos = (int)(nextReadPos - bufferStartPos);
+ if (buffer.remaining() == 0 || bufPos < 0 || bufPos >= buffer.limit()) {
+ buffer.position(0);
+ buffer.limit(0);
+ } else {
+ buffer.position(bufPos);
+ }
+ }
+
+ private int ensureData(int len) throws IOException {
+ ensurePos();
+ int remaining = buffer.remaining();
+ if (remaining > 0) {
+ return remaining;
+ }
+ // try to fill
+ int readLen = len < bufSize ? bufSize : Math.min(len, directBufferSize);
+ readLen = nextReadPos + readLen > contentLength ? (int) (contentLength - nextReadPos) : readLen;
+ buffer.position(0);
+ buffer.limit(0);
+ int bytesRead = ccClient.read(
+ true,
+ nextReadPos,
+ getPrefetchEnd(nextReadPos, len),
+ buffer,
+ nextReadPos,
+ readLen,
+ objKey,
+ mtime,
+ etag,
+ true
+ );
+ if (bytesRead == MemArtsCCClient.CCREAD_RETCODE_CACHEMISS) {
+ throw new IOException("cache miss");
+ }
+ if (bytesRead > 0) {
+ buffer.position(0);
+ buffer.limit(bytesRead);
+ bufferStartPos = nextReadPos;
+ }
+ increaseHitTrafficTraffic(readLen);
+ incrementBytesRead(readLen);
+ return bytesRead;
+ }
+
+ @Override
+ public int read() throws IOException {
+ checkClosed();
+ if (nextReadPos >= contentLength) {
+ return -1;
+ }
+ int ret = readInBuffer(null, 0, 0, true);
+ if (ret >= 0) {
+ nextReadPos++;
+ }
+ return ret;
+ }
+
+ @Override
+ public int read(byte[] buf, int off, int len) throws IOException {
+ checkClosed();
+ if (nextReadPos >= contentLength) {
+ return -1;
+ }
+ if (nextReadPos + len > contentLength) {
+ len = (int) (contentLength - nextReadPos);
+ }
+ int bytesRead = readInBuffer(buf, off, len, false);
+ if (bytesRead > 0) {
+ nextReadPos += bytesRead;
+ }
+ return bytesRead;
+ }
+
+ @Override
+ public void seek(long l) throws IOException {
+ checkClosed();
+ if (l < 0) {
+ throw new EOFException("Cannot seek to negative offset");
+ }
+ if (l > contentLength) {
+ throw new EOFException("seek pos " + l + " is larger than contentLength" + contentLength);
+ }
+ this.nextReadPos = l;
+ }
+
+ @Override
+ public long skip(long n) throws IOException {
+ checkClosed();
+ ensurePos();
+ if (nextReadPos + n > contentLength) {
+ n = contentLength - nextReadPos;
+ }
+ if (n > buffer.remaining()) {
+ // clear the buffer
+ buffer.limit(0);
+ buffer.position(0);
+ }
+ nextReadPos += n;
+ return n;
+ }
+
+ @Override
+ public long getPos() throws IOException {
+ checkClosed();
+ return nextReadPos;
+ }
+
+ @Override
+ public boolean seekToNewSource(long l) throws IOException {
+ checkClosed();
+ return false;
+ }
+
+ private void checkClosed() throws IOException {
+ if (closed) {
+ throw new IOException("MemArtsCCInputStream already closed");
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ closed = true;
+ ccClient = null;
+ nextReadPos = 0;
+ ((DirectBuffer)buffer).cleaner().clean();
+ buffer = null;
+ }
+
+ @Override
+ public void setReadahead(Long aLong) throws IOException, UnsupportedOperationException {
+ if (aLong <= 0) {
+ return;
+ }
+ this.prefetchRange = aLong;
+ }
+
+ private void incrementBytesRead(final long bytesRead) {
+ if (statistics != null && bytesRead > 0) {
+ statistics.incrementBytesRead(bytesRead);
+ }
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStreamBase.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStreamBase.java
new file mode 100644
index 0000000..bb7f7df
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStreamBase.java
@@ -0,0 +1,21 @@
+package org.apache.hadoop.fs.obs.memartscc;
+
+import org.apache.hadoop.fs.CanSetReadahead;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.fs.obs.TrafficStatistics;
+
+import java.io.InputStream;
+
+public abstract class MemArtsCCInputStreamBase extends InputStream implements Seekable, CanSetReadahead {
+ protected TrafficStatistics trafficStatistics;
+
+ public void setTrafficStaticsClass(TrafficStatistics instance) {
+ trafficStatistics = instance;
+ }
+
+ protected void increaseHitTrafficTraffic(long value) {
+ if (trafficStatistics != null) {
+ trafficStatistics.increase(value, TrafficStatistics.TrafficType.Q2);
+ }
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStreamWithPool.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStreamWithPool.java
new file mode 100644
index 0000000..0b068d8
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStreamWithPool.java
@@ -0,0 +1,201 @@
+package org.apache.hadoop.fs.obs.memartscc;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.obs.OBSFileStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * 为了规避CBG spark组件inputstream 泄露,引入规避措施:bufferPool, 每次进行ccRead之前申请buffer,从ccRead读取数据后再还回pool
+ * 代价:性能变差
+ * 1. 频繁借还buffer
+ * 2. 原先inputStream中的buffer具有缓冲数据的作用,当前因为要read完后立即还回去,所以缓冲失效。
+ */
+public class MemArtsCCInputStreamWithPool extends MemArtsCCInputStreamBase {
+ private static final Logger LOG = LoggerFactory.getLogger(MemArtsCCInputStreamWithPool.class);
+
+ private final FileSystem.Statistics statistics;
+
+ private boolean closed;
+
+ private MemArtsCCClient ccClient;
+
+ private final String objKey;
+
+ private long prefetchRange;
+
+ // private final int bufSize;
+
+ private long nextReadPos;
+
+ private final long contentLength;
+
+ private final String etag;
+
+ private final long mtime;
+
+ private final int borrowTimeout;
+
+ public MemArtsCCInputStreamWithPool(MemArtsCCClient ccClient, String objKey, OBSFileStatus fileStatus,
+ long prefetchRange, final FileSystem.Statistics stats, int borrowTimeout) {
+ this.ccClient = ccClient;
+ this.objKey = objKey;
+ this.prefetchRange = prefetchRange;
+ this.statistics = stats;
+ this.contentLength = fileStatus.getLen();
+ this.etag = fileStatus.getEtag();
+ this.mtime = fileStatus.getModificationTime();
+ this.closed = false;
+ this.borrowTimeout = borrowTimeout;
+ }
+
+ private long getPrefetchEnd(long pos, int len) {
+ long prefetchEnd = pos + Math.max(prefetchRange, len);
+ return Math.min(prefetchEnd, contentLength);
+ }
+
+ private int readInBuffer(byte[] buf, int off, int len, boolean oneByteRead) throws IOException {
+ ByteBuffer buffer = null;
+ int readLen = oneByteRead ? 1 : Math.min(len, buf.length - off);
+ try {
+ buffer = MemArtsCCClient.bufferPool.borrowBuffer(borrowTimeout);
+ int bytesRead = fillData(buffer, readLen);
+ if (bytesRead < 0) {
+ return -1;
+ }
+ if (oneByteRead) {
+ return buffer.get() & 0xFF;
+ }
+ readLen = Math.min(len, buffer.remaining());
+ buffer.get(buf, off, readLen);
+ return readLen;
+ } catch (InterruptedException e) {
+ throw new IOException("borrow buffer interrupted", e);
+ } finally {
+ MemArtsCCClient.bufferPool.returnBuffer(buffer);
+ }
+ }
+
+ private int fillData(ByteBuffer buffer, int len) throws IOException {
+ buffer.position(0);
+ buffer.limit(0);
+ int readLen = Math.min(len, buffer.capacity());
+ readLen = nextReadPos + readLen > contentLength ? (int) (contentLength - nextReadPos) : readLen;
+ int bytesRead = ccClient.read(
+ true,
+ nextReadPos,
+ getPrefetchEnd(nextReadPos, len),
+ buffer,
+ nextReadPos,
+ readLen,
+ objKey,
+ mtime,
+ etag,
+ true
+ );
+ if (bytesRead == MemArtsCCClient.CCREAD_RETCODE_CACHEMISS) {
+ throw new IOException("cache miss");
+ }
+ if (bytesRead > 0) {
+ buffer.position(0);
+ buffer.limit(bytesRead);
+ }
+ increaseHitTrafficTraffic(readLen);
+ incrementBytesRead(readLen);
+ return bytesRead;
+ }
+
+ @Override
+ public int read() throws IOException {
+ checkClosed();
+ if (nextReadPos >= contentLength) {
+ return -1;
+ }
+ int ret = readInBuffer(null, 0, 0, true);
+ if (ret >= 0) {
+ nextReadPos++;
+ }
+ return ret;
+ }
+
+ @Override
+ public int read(byte[] buf, int off, int len) throws IOException {
+ checkClosed();
+ if (nextReadPos >= contentLength) {
+ return -1;
+ }
+ if (nextReadPos + len > contentLength) {
+ len = (int) (contentLength - nextReadPos);
+ }
+ int bytesRead = readInBuffer(buf, off, len, false);
+ if (bytesRead > 0) {
+ nextReadPos += bytesRead;
+ }
+ return bytesRead;
+ }
+
+ @Override
+ public void seek(long l) throws IOException {
+ checkClosed();
+ if (l < 0) {
+ throw new EOFException("Cannot seek to negative offset");
+ }
+ if (l > contentLength) {
+ throw new EOFException("seek pos " + l + " is larger than contentLength" + contentLength);
+ }
+ this.nextReadPos = l;
+ }
+
+ @Override
+ public long skip(long n) throws IOException {
+ checkClosed();
+ if (nextReadPos + n > contentLength) {
+ n = contentLength - nextReadPos;
+ }
+ nextReadPos += n;
+ return n;
+ }
+
+ @Override
+ public long getPos() throws IOException {
+ checkClosed();
+ return nextReadPos;
+ }
+
+ @Override
+ public boolean seekToNewSource(long l) throws IOException {
+ checkClosed();
+ return false;
+ }
+
+ private void checkClosed() throws IOException {
+ if (closed) {
+ throw new IOException("MemArtsCCInputStream already closed");
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ closed = true;
+ ccClient = null;
+ nextReadPos = 0;
+ }
+
+ @Override
+ public void setReadahead(Long aLong) throws IOException, UnsupportedOperationException {
+ if (aLong <= 0) {
+ return;
+ }
+ this.prefetchRange = aLong;
+ }
+
+ private void incrementBytesRead(final long bytesRead) {
+ if (statistics != null && bytesRead > 0) {
+ statistics.incrementBytesRead(bytesRead);
+ }
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/OBSInputStreamSimulator.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/OBSInputStreamSimulator.java
new file mode 100644
index 0000000..977076f
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/OBSInputStreamSimulator.java
@@ -0,0 +1,210 @@
+package org.apache.hadoop.fs.obs.memartscc;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.Locale;
+
+import static java.lang.Math.min;
+
+public class OBSInputStreamSimulator {
+ private final long contentLength;
+
+ private final long readAheadRange;
+
+ private long nextReadPos;
+
+ private InputStreamSimulator mockInputStream;
+
+ private long streamCurrentPos;
+
+ private long contentRangeFinish;
+
+ private long actualReadFromObs;
+
+ private static final Logger LOG = LoggerFactory.getLogger(OBSInputStreamSimulator.class);
+
+ public OBSInputStreamSimulator(final long fileStatusLength, final long readAheadRangeValue) {
+ Preconditions.checkArgument(fileStatusLength >= 0, "Negative content length");
+ this.contentLength = fileStatusLength;
+ this.readAheadRange = readAheadRangeValue;
+ }
+
+ private class InputStreamSimulator {
+
+ private final int size;
+
+ private int pos;
+
+ public InputStreamSimulator(int bytes) {
+ size = bytes;
+ pos = 0;
+ }
+
+ public int available() {
+ return size - pos;
+ }
+
+ public long skip(long n) {
+ if (n <= 0) {
+ return 0;
+ }
+
+ if (pos + n >= size) {
+ long skipped = size - pos;
+ pos = size;
+ return skipped;
+ } else {
+ pos += n;
+ return n;
+ }
+ }
+ }
+
+ /**
+ * 模拟读取OBS的流量
+ * @param len 需要读多少字节的数据
+ * @return 实际上读取了多少OBS的数据
+ */
+ public synchronized long read(final int len) throws IOException {
+ LOG.debug("Simulator.read len: {}", len);
+
+ actualReadFromObs = 0;
+
+ if (len == 0) {
+ return 0;
+ }
+ boolean isTrue = this.contentLength == 0 || nextReadPos >= contentLength;
+ if (isTrue) {
+ LOG.debug("Simulator.read: no need to read");
+ return -1;
+ }
+
+ lazySeek(nextReadPos, len);
+
+ // 这里模拟读取操作,假设读取len长度的数据全部成功,那么直接+len
+ if (mockInputStream == null) {
+ throw new IOException("mockInputStream closed, cannot read.");
+ }
+ int ioRead = Math.min(len, mockInputStream.available());
+ streamCurrentPos += ioRead;
+ nextReadPos += ioRead;
+
+ return actualReadFromObs;
+ }
+
+ private void lazySeek(final long targetPos, final long len) throws IOException {
+ seekInStream(targetPos);
+
+ if (isStreamClosed()) {
+ reopen(targetPos, len);
+ }
+ }
+
+ private boolean isStreamClosed() {
+ return mockInputStream == null;
+ }
+
+ private void seekInStream(final long targetPos) {
+ LOG.debug("Simulator.seekInStream targetPos: {}", targetPos);
+
+ if (isStreamClosed()) {
+ LOG.debug("Simulator.seekInStream: the stream is not opened, seekInStream not operated");
+ return;
+ }
+ long diff = targetPos - streamCurrentPos;
+
+ if (diff == 0 && this.contentRangeFinish - this.streamCurrentPos > 0) {
+ LOG.debug("Simulator.seekInStream: the seek position does not require seek");
+ return;
+ }
+
+ if (diff > 0) {
+ int available = mockInputStream.available();
+ long forwardSeekRange = Math.max(readAheadRange, available);
+ long remainingInCurrentRequest = this.contentRangeFinish - this.streamCurrentPos;
+ long forwardSeekLimit = min(remainingInCurrentRequest, forwardSeekRange);
+ boolean skipForward = remainingInCurrentRequest > 0 && diff <= forwardSeekLimit;
+ if (skipForward) {
+ long skippedOnce = mockInputStream.skip(diff);
+ while (diff > 0 && skippedOnce > 0) {
+ streamCurrentPos += skippedOnce;
+ diff -= skippedOnce;
+ skippedOnce = mockInputStream.skip(diff);
+ }
+
+ if (streamCurrentPos == targetPos) {
+ return;
+ }
+ }
+ }
+
+ closeStream();
+ streamCurrentPos = targetPos;
+ }
+
+ /**
+ * Simulate reopen operation in OBSInputStream
+ * @param targetPos the position which the stream should open from
+ * @param length the length of the stream
+ * @return the actual traffic when reading from OBS, unit: bytes
+ * @throws IOException
+ */
+ public long reopen(final long targetPos, final long length) {
+ if (!isStreamClosed()) {
+ closeStream();
+ }
+
+ contentRangeFinish = calculateRequestLimit(targetPos, length, contentLength, readAheadRange);
+ long actualLength = contentRangeFinish - targetPos;
+ mockInputStream = new InputStreamSimulator((int)actualLength);
+ LOG.debug("Opened a simulated stream, param length: {}, length {}.", length, actualLength);
+
+ actualReadFromObs += actualLength;
+ this.streamCurrentPos = targetPos;
+ return actualLength;
+ }
+
+ /**
+ * 注意seek和seekInStream
+ * @param targetPos seek目标位置
+ * @throws IOException
+ */
+ public void seek(final long targetPos) throws IOException {
+ LOG.debug("Simulator.seek targetPos {} ", targetPos);
+
+ if (targetPos > contentLength) {
+ LOG.error("The position of seek is beyond content length, targetPos:{}, contentLength:{}",
+ targetPos, contentLength);
+ throw new EOFException(String.format(Locale.ENGLISH, "%s %d", FSExceptionMessages.CANNOT_SEEK_PAST_EOF, targetPos));
+ }
+
+ if (targetPos < 0) {
+ LOG.error("The position of seek negative, targetPos:{}", targetPos);
+ throw new EOFException(String.format(Locale.ENGLISH, "%s %d", FSExceptionMessages.NEGATIVE_SEEK, targetPos));
+ }
+
+ if (this.contentLength <= 0) {
+ return;
+ }
+ nextReadPos = targetPos;
+ }
+
+ protected void closeStream() {
+ LOG.debug("Closed a simulator stream");
+ mockInputStream = null;
+ }
+
+ public void close() throws IOException {
+ closeStream();
+ }
+
+ protected long calculateRequestLimit(final long targetPos, final long length, final long contentLength,
+ final long readAhead) {
+ return min(contentLength, length < 0 ? contentLength : targetPos + Math.max(readAhead, length));
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ObjectInfo.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ObjectInfo.java
new file mode 100644
index 0000000..072429e
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ObjectInfo.java
@@ -0,0 +1,20 @@
+package org.apache.hadoop.fs.obs.memartscc;
+
+/**
+ * 功能描述
+ *
+ * @since 2021-05-20
+ */
+public class ObjectInfo {
+ public String objectKey;
+
+ public long modifyTime;
+
+ public boolean isConsistencyCheck;
+
+ public ObjectInfo(String objectKey, long modifyTime, boolean isConsistencyCheck) {
+ this.objectKey = objectKey;
+ this.modifyTime = modifyTime;
+ this.isConsistencyCheck = isConsistencyCheck;
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ObjectShard.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ObjectShard.java
new file mode 100644
index 0000000..2d8a512
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ObjectShard.java
@@ -0,0 +1,43 @@
+package org.apache.hadoop.fs.obs.memartscc;
+
+/**
+ * 功能描述
+ *
+ * @since 2021-05-24
+ */
+public class ObjectShard {
+ long start;
+
+ long end;
+
+ String[] hosts;
+
+ int validHostNum;
+
+ public ObjectShard(long start, long end, String[] hosts, int validHostNum) {
+ this.start = start;
+ this.end = end;
+ this.hosts = hosts;
+ this.validHostNum = validHostNum;
+ }
+
+ public ObjectShard(String[] hosts) {
+ this.hosts = hosts;
+ }
+
+ public long getStart() {
+ return start;
+ }
+
+ public long getEnd() {
+ return end;
+ }
+
+ public String[] getHosts() {
+ return hosts;
+ }
+
+ public int getValidHostNum() {
+ return validHostNum;
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/DelegationTokenCapability.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/DelegationTokenCapability.java
index e525c6f..f944901 100644
--- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/DelegationTokenCapability.java
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/DelegationTokenCapability.java
@@ -9,6 +9,7 @@
*
* @since 2021-09-15
*/
+@Deprecated
public interface DelegationTokenCapability {
String getCanonicalServiceName();
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/DelegationTokenProvider.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/DelegationTokenProvider.java
new file mode 100644
index 0000000..4e94250
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/DelegationTokenProvider.java
@@ -0,0 +1,21 @@
+package org.apache.hadoop.fs.obs.security;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.security.token.Token;
+
+import java.net.URI;
+
+/**
+ * description
+ *
+ * @since 2022-05-24
+ */
+public interface DelegationTokenProvider {
+
+ void initialize(FileSystem fs, URI uri, Configuration conf);
+
+ Token> getDelegationToken(String renewer);
+
+ String getCanonicalServiceName();
+}
diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/ObsDelegationTokenManger.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/ObsDelegationTokenManger.java
new file mode 100644
index 0000000..b87e000
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/ObsDelegationTokenManger.java
@@ -0,0 +1,82 @@
+package org.apache.hadoop.fs.obs.security;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.obs.OBSCommonUtils;
+import org.apache.hadoop.fs.obs.OBSConstants;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * description
+ *
+ * @since 2022-05-24
+ */
+public class ObsDelegationTokenManger {
+ private static final Logger logger = LoggerFactory.getLogger(ObsDelegationTokenManger.class);
+ private List providers = new ArrayList<>();
+ private DelegationTokenProvider defaultProvider;
+
+ public static boolean hasDelegationTokenProviders(Configuration conf) {
+ return UserGroupInformation.isSecurityEnabled() && OBSCommonUtils.isStringNotEmpty(
+ conf.getTrimmed(OBSConstants.DELEGATION_TOKEN_PROVIDERS, OBSConstants.DEFAULT_DELEGATION_TOKEN_PROVIDERS));
+ }
+
+ public void initialize(FileSystem fs, URI uri, Configuration conf) throws IOException {
+ List> providerClasses = Arrays.asList(
+ conf.getClasses(OBSConstants.DELEGATION_TOKEN_PROVIDERS, new Class[] {}));
+
+ for (int i = 0; i < providerClasses.size(); i++) {
+ DelegationTokenProvider provider = null;
+ try {
+ provider = (DelegationTokenProvider)providerClasses.get(i).newInstance();
+ } catch (InstantiationException | IllegalAccessException e) {
+ throw new IOException("Instantiation DelegationTokenProvider", e);
+ }
+ provider.initialize(fs, uri, conf);
+ if (i == 0) {
+ this.defaultProvider = provider;
+ }
+ this.providers.add(provider);
+ }
+ }
+
+ public Token>[] addDelegationTokens(String renewer, Credentials credentials) {
+ List> tokens = new ArrayList<>();
+ for (DelegationTokenProvider provider : providers) {
+ final String serviceName = provider.getCanonicalServiceName();
+ logger.info("get delegation token from provider {} with serviceName {}",
+ provider.getClass().getName(), serviceName);
+ if (serviceName != null) {
+ final Text service = new Text(serviceName);
+ Token> token = credentials.getToken(service);
+ if (token == null) {
+ token = provider.getDelegationToken(renewer);
+ if (token != null) {
+ tokens.add(token);
+ credentials.addToken(service, token);
+ }
+ }
+ }
+ }
+ return tokens.toArray(new Token>[tokens.size()]);
+ }
+
+ public Token> getDelegationToken(String renewer) {
+ return defaultProvider.getDelegationToken(renewer);
+ }
+
+ public String getCanonicalServiceName() {
+ return defaultProvider.getCanonicalServiceName();
+ }
+}
diff --git a/hadoop-huaweicloud/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier b/hadoop-huaweicloud/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier
new file mode 100644
index 0000000..252e51e
--- /dev/null
+++ b/hadoop-huaweicloud/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier
@@ -0,0 +1 @@
+org.apache.hadoop.fs.obs.memartscc.MemArtsCCDelegationTokenIdentifier
\ No newline at end of file
diff --git a/hadoop-huaweicloud/src/site/markdown/index.md b/hadoop-huaweicloud/src/site/markdown/index.md
new file mode 100644
index 0000000..83cc403
--- /dev/null
+++ b/hadoop-huaweicloud/src/site/markdown/index.md
@@ -0,0 +1,379 @@
+
+
+# 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.0.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-huaweicloud/src/site/resources/css/site.css b/hadoop-huaweicloud/src/site/resources/css/site.css
new file mode 100644
index 0000000..f830baa
--- /dev/null
+++ b/hadoop-huaweicloud/src/site/resources/css/site.css
@@ -0,0 +1,30 @@
+/*
+* 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-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/AbstractOBSTestBase.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/AbstractOBSTestBase.java
new file mode 100644
index 0000000..18d1a02
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/AbstractOBSTestBase.java
@@ -0,0 +1,126 @@
+/*
+ * 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.AbstractFSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+
+/**
+ * An extension of the contract test base set up for OBS tests.
+ */
+@Deprecated
+public abstract class AbstractOBSTestBase extends AbstractFSContractTestBase {
+
+ protected static final Logger LOG =
+ LoggerFactory.getLogger(AbstractOBSTestBase.class);
+
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new OBSContract(conf);
+ }
+
+ @Override
+ public void teardown() throws Exception {
+ super.teardown();
+ describe("closing file system");
+ IOUtils.closeStream(getFileSystem());
+ }
+
+ @Before
+ public void nameThread() {
+ Thread.currentThread().setName("JUnit-" + methodName.getMethodName());
+ }
+
+ @Override
+ protected int getTestTimeoutMillis() {
+ return OBSTestConstants.OBS_TEST_TIMEOUT;
+ }
+
+ protected Configuration getConfiguration() {
+ return getContract().getConf();
+ }
+
+ /**
+ * Get the filesystem as an OBS filesystem.
+ *
+ * @return the typecast FS
+ */
+ @Override
+ public OBSFileSystem getFileSystem() {
+ return (OBSFileSystem) super.getFileSystem();
+ }
+
+ /**
+ * Describe a test in the logs.
+ *
+ * @param text text to print
+ * @param args arguments to format in the printing
+ */
+ protected void describe(String text, Object... args) {
+ LOG.info("\n\n{}: {}\n",
+ methodName.getMethodName(),
+ String.format(text, args));
+ }
+
+ /**
+ * Write a file, read it back, validate the dataset. Overwrites the file if
+ * it is present
+ *
+ * @param name filename (will have the test path prepended to it)
+ * @param len length of file
+ * @return the full path to the file
+ * @throws IOException any IO problem
+ */
+ protected Path writeThenReadFile(String name, int len) throws IOException {
+ Path path = path(name);
+ byte[] data = dataset(len, 'a', 'z');
+ writeDataset(getFileSystem(), path, data, data.length, 1024 * 1024,
+ true);
+ ContractTestUtils.verifyFileContents(getFileSystem(), path, data);
+ return path;
+ }
+
+ /**
+ * Assert that an exception failed with a specific status code.
+ *
+ * @param e exception
+ * @param code expected status code
+ * @throws OBSIOException rethrown if the status code does not match.
+ */
+ protected void assertStatusCode(OBSIOException e, int code)
+ throws OBSIOException {
+ if (e.getCause().getResponseCode() != code) {
+ throw e;
+ }
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestListFiles.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestListFiles.java
new file mode 100644
index 0000000..8f9611d
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestListFiles.java
@@ -0,0 +1,248 @@
+/**
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * This class tests the FileStatus API.
+ */
+public class ITestListFiles {
+ static final long seed = 0xDEADBEEFL;
+
+ private OBSFileSystem fs;
+
+ private Configuration conf;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ private static Path TEST_DIR;
+
+ final private static int FILE_LEN = 10;
+
+ private static Path FILE1;
+
+ private static Path DIR1;
+
+ private static Path FILE2;
+
+ private static Path FILE3;
+
+ static {
+ setTestPaths(new Path(testRootPath, "main_"));
+ }
+
+ /**
+ * Sets the root testing directory and reinitializes any additional test paths
+ * that are under the root. This method is intended to be called from a
+ * subclass's @BeforeClass method if there is a need to override the testing
+ * directory.
+ *
+ * @param testDir Path root testing directory
+ */
+ protected static void setTestPaths(Path testDir) {
+ TEST_DIR = testDir;
+ FILE1 = new Path(TEST_DIR, "file1");
+ DIR1 = new Path(TEST_DIR, "dir1");
+ FILE2 = new Path(DIR1, "file2");
+ FILE3 = new Path(DIR1, "file3");
+ }
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ conf = OBSContract.getConfiguration(null);
+ conf.setClass(OBSConstants.OBS_METRICS_CONSUMER,
+ MockMetricsConsumer.class, BasicMetricsConsumer.class);
+ conf.setBoolean(OBSConstants.METRICS_SWITCH, true);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ fs.delete(TEST_DIR, true);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(new Path(testRootPath), true);
+ fs.close();
+ fs = null;
+ }
+ }
+
+ private Path getTestPath(String testPath) {
+ return new Path(testRootPath + "/" + testPath);
+ }
+
+ private static void writeFile(FileSystem fileSys, Path name, int fileSize)
+ throws IOException {
+ // Create and write a file that contains three blocks of data
+ FSDataOutputStream stm = fileSys.create(name);
+ byte[] buffer = new byte[fileSize];
+ Random rand = new Random(seed);
+ rand.nextBytes(buffer);
+ stm.write(buffer);
+ stm.close();
+ }
+
+ /**
+ * Test when input path is a file
+ */
+ @Test
+ public void testFile() throws IOException {
+ fs.mkdirs(TEST_DIR);
+ writeFile(fs, FILE1, FILE_LEN);
+
+ RemoteIterator itor = fs.listFiles(
+ FILE1, true);
+ LocatedFileStatus stat = itor.next();
+ assertFalse(itor.hasNext());
+ assertTrue(stat.isFile());
+ assertEquals(FILE_LEN, stat.getLen());
+ assertEquals(fs.makeQualified(FILE1), stat.getPath());
+ assertEquals(1, stat.getBlockLocations().length);
+
+ itor = fs.listFiles(FILE1, false);
+ stat = itor.next();
+ assertFalse(itor.hasNext());
+ assertTrue(stat.isFile());
+ assertEquals(FILE_LEN, stat.getLen());
+ assertEquals(fs.makeQualified(FILE1), stat.getPath());
+ assertEquals(1, stat.getBlockLocations().length);
+
+ fs.delete(FILE1, true);
+ }
+
+ /**
+ * Test when input path is a directory
+ */
+ @Test
+ public void testDirectory() throws IOException {
+ fs.mkdirs(DIR1);
+
+ // test empty directory
+ RemoteIterator itor = fs.listFiles(
+ DIR1, true);
+ assertFalse(itor.hasNext());
+ itor = fs.listFiles(DIR1, false);
+ assertFalse(itor.hasNext());
+
+ // testing directory with 1 file
+ writeFile(fs, FILE2, FILE_LEN);
+ itor = fs.listFiles(DIR1, true);
+ LocatedFileStatus stat = itor.next();
+ assertFalse(itor.hasNext());
+ assertTrue(stat.isFile());
+ assertEquals(FILE_LEN, stat.getLen());
+ assertEquals(fs.makeQualified(FILE2), stat.getPath());
+ assertEquals(1, stat.getBlockLocations().length);
+
+ itor = fs.listFiles(DIR1, false);
+ stat = itor.next();
+ assertFalse(itor.hasNext());
+ assertTrue(stat.isFile());
+ assertEquals(FILE_LEN, stat.getLen());
+ assertEquals(fs.makeQualified(FILE2), stat.getPath());
+ assertEquals(1, stat.getBlockLocations().length);
+
+ // test more complicated directory
+ writeFile(fs, FILE1, FILE_LEN);
+ writeFile(fs, FILE3, FILE_LEN);
+
+ Set filesToFind = new HashSet();
+ filesToFind.add(fs.makeQualified(FILE1));
+ filesToFind.add(fs.makeQualified(FILE2));
+ filesToFind.add(fs.makeQualified(FILE3));
+
+ itor = fs.listFiles(TEST_DIR, true);
+ stat = itor.next();
+ assertTrue(stat.isFile());
+ assertTrue("Path " + stat.getPath() + " unexpected",
+ filesToFind.remove(stat.getPath()));
+
+ stat = itor.next();
+ assertTrue(stat.isFile());
+ assertTrue("Path " + stat.getPath() + " unexpected",
+ filesToFind.remove(stat.getPath()));
+
+ stat = itor.next();
+ assertTrue(stat.isFile());
+ assertTrue("Path " + stat.getPath() + " unexpected",
+ filesToFind.remove(stat.getPath()));
+ assertFalse(itor.hasNext());
+ assertTrue(filesToFind.isEmpty());
+
+ itor = fs.listFiles(TEST_DIR, false);
+ stat = itor.next();
+ assertTrue(stat.isFile());
+ assertEquals(fs.makeQualified(FILE1), stat.getPath());
+ assertFalse(itor.hasNext());
+
+ fs.delete(TEST_DIR, true);
+ }
+
+ @Test
+ // 调用listStatus递归列举接口,recursive为true,返回目录及子目录下所有对象;recursive为false,只返回当前目录下对象
+ public void testListStatusRecursive() throws Exception {
+ Path testDir = getTestPath("test_dir/");
+ Path subDir = getTestPath("test_dir/sub_dir/");
+ Path file1 = getTestPath("test_dir/file1");
+ Path file2 = getTestPath("test_dir/sub_dir/file2");
+ fs.mkdirs(testDir);
+ fs.mkdirs(subDir);
+ FSDataOutputStream outputStream = fs.create(file1, false);
+ outputStream.close();
+ outputStream = fs.create(file2, false);
+ outputStream.close();
+
+ FileStatus[] objects = fs.listStatus(testDir, false);
+ assertEquals(2, objects.length);
+
+ objects = fs.listStatus(testDir, true);
+ assertEquals(3, objects.length);
+ fs.delete(testDir, true);
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSAppend.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSAppend.java
new file mode 100644
index 0000000..ce35008
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSAppend.java
@@ -0,0 +1,467 @@
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.AccessControlException;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.EnumSet;
+import java.util.Random;
+import java.util.UUID;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.NanoTimer;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.bandwidth;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyReceivedData;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class ITestOBSAppend {
+ private OBSFileSystem fs;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ private int testBufferSize;
+
+ private int modulus;
+
+ private byte[] testBuffer;
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.setClass(OBSConstants.OBS_METRICS_CONSUMER,
+ MockMetricsConsumer.class, BasicMetricsConsumer.class);
+ conf.setBoolean(OBSConstants.METRICS_SWITCH, true);
+ conf.set(OBSConstants.MULTIPART_SIZE, String.valueOf(5 * 1024 * 1024));
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ testBufferSize = fs.getConf().getInt(ContractTestUtils.IO_CHUNK_BUFFER_SIZE, 128);
+ modulus = fs.getConf().getInt(ContractTestUtils.IO_CHUNK_MODULUS_SIZE, 128);
+ testBuffer = new byte[testBufferSize];
+
+ for (int i = 0; i < testBuffer.length; ++i) {
+ testBuffer[i] = (byte) (i % modulus);
+ }
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(new Path(testRootPath), true);
+ fs.close();
+ fs = null;
+ }
+ }
+
+ private Path getTestPath(String relativePath) {
+ return new Path(testRootPath + "/" + relativePath);
+ }
+
+ /**
+ * 文件桶,对一个已经存在的文件做追加(每次小于multipart size),追加3次
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testAppendPosixNormal001() throws Exception {
+ verifyAppend(1024 * 1024 * 10, 1024, 3, 4096);
+ }
+
+ /**
+ * 文件桶,对一个已经存在的文件做追加(每次小于multipart size),追加10次
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testAppendPosixNormal002() throws Exception {
+ verifyAppend(1024 * 1024 * 10, 1024, 10, 4096);
+ }
+
+ /**
+ * 文件桶,对一个已经存在的文件做追加(每次大于multipart size),追加3次
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testAppendPosixNormal003() throws Exception {
+ verifyAppend(1024 * 1024 * 10, 1024 * 1024 * 10, 3, 4096);
+ }
+
+ /**
+ * 文件桶,对一个已经存在的文件做追加(每次大于multipart size),追加10次
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testAppendPosixNormal004() throws Exception {
+ verifyAppend(1024 * 1024 * 10, 1024 * 1024 * 10, 10, 4096);
+ }
+
+ /**
+ * 文件桶,追加多次,小于和大于100M交替
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testAppendPosixNormal005() throws Exception {
+ verifyCrossSizeAppend(1024 * 1024 * 10, 1024, 1024 * 1024 * 200,
+ 1024 * 1024 * 100, 3, 4096);
+ }
+
+ /**
+ * 文件桶,追加多次,每次追加随机大小
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testAppendPosixNormal006() throws Exception {
+ verifyMultiSizeAppend(1024 * 1024 * 10, 1024, 1024 * 1024 * 200, 3,
+ 4096);
+ }
+
+ /**
+ * 文件桶,追加多次,每次追加1字节
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testAppendPosixNormal007() throws Exception {
+ verifyAppend(1024 * 1024 * 10, 1, 30, 4096);
+ }
+
+ /**
+ * 文件桶,追加写大于100M,中间中断后继续追加
+ * @throws Exception
+ */
+ // @Test
+ // public void testAppendPosixNormal008() throws Exception {
+ //
+ // }
+
+ /**
+ * 对象桶创建append stream,抛UnsupportedOperationException
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testAppendPosixAbnormal001() throws Exception {
+ if (fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("a001/test_file");
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ boolean hasException = false;
+ FSDataOutputStream outputStream = fs.create(testFile, false);
+ outputStream.close();
+ try {
+ outputStream = fs.append(testFile, 4096, null);
+ } catch (UnsupportedOperationException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ assertTrue(
+ "create append stream in object bucket should throw UnsupportedOperationException",
+ hasException);
+
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ /**
+ * 路径为一个不存在的文件,抛出FileNotFoundException
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testAppendPosixAbnormal002() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("a001/test_file");
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ boolean hasException = false;
+ FSDataOutputStream outputStream = null;
+ try {
+ outputStream = fs.append(testFile, 4096, null);
+ } catch (FileNotFoundException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ assertTrue(
+ "append on non exist file should throw FileNotFoundException",
+ hasException);
+ }
+
+ /**
+ * 路径为一个目录,抛出FileAlreadyExistsException
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testAppendPosixAbnormal003() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testPath = getTestPath("test-dir");
+ fs.mkdirs(testPath);
+ boolean hasException = false;
+ FSDataOutputStream outputStream = null;
+ try {
+ outputStream = fs.append(testPath, 4096, null);
+ } catch (FileAlreadyExistsException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ assertTrue(
+ "append on an exist directory should throw FileAlreadyExistsException",
+ hasException);
+ OBSFSTestUtil.deletePathRecursive(fs, testPath);
+ }
+
+ /**
+ * 文件的父目录及上级目录不存在,抛FileNotFoundException
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testAppendPosixAbnormal004() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testPath = getTestPath("a001/b001/test_file");
+ fs.delete(testPath.getParent(), true);
+ boolean hasException = false;
+ FSDataOutputStream outputStream = null;
+ try {
+ outputStream = fs.append(testPath, 4096, null);
+ } catch (FileNotFoundException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ assertTrue(
+ "append on file whose parent not exists should throw FileNotFoundException",
+ hasException);
+ }
+
+ /**
+ * 文件的父目录及上级目录非目录,抛AccessControlException
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testAppendPosixAbnormal005() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testPath = getTestPath("a001/b001/test_file");
+ fs.delete(testPath.getParent(), true);
+ FSDataOutputStream outputStream = fs.create(testPath.getParent(),
+ false);
+ outputStream.close();
+
+ boolean hasException = false;
+ try {
+ outputStream = fs.append(testPath, 4096, null);
+ } catch (AccessControlException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ assertTrue(
+ "append on file whose parent is not directory should throw AccessControlException",
+ hasException);
+ }
+
+ private void verifyAppend(long fileSize, long appendSize, int appendTimes,
+ int bufferSize)
+ throws IOException {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ long total = fileSize;
+ Path objectPath = createAppendFile(fileSize);
+ for (int i = 0; i < appendTimes; i++) {
+ appendFileByBuffer(objectPath, appendSize, bufferSize, total);
+ total = total + appendSize;
+ }
+ verifyReceivedData(fs, objectPath, total, testBufferSize, modulus);
+ OBSFSTestUtil.deletePathRecursive(fs, objectPath);
+ }
+
+ private void verifyCrossSizeAppend(long fileSize, int appendMinSize,
+ int appendMaxSize,
+ int mediumSize, int appendTimes, int bufferSize)
+ throws IOException {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ long totoal = fileSize;
+ Random random = new Random();
+ Path objectPath = createAppendFile(fileSize);
+ for (int i = 0; i < appendTimes; i++) {
+ int appendSize =
+ random.nextInt(appendMaxSize) % (appendMaxSize - appendMinSize
+ + 1) + appendMinSize;
+ if (i % 2 == 0) {
+ appendSize = appendSize > mediumSize
+ ? appendSize - mediumSize
+ : appendSize;
+ } else {
+ appendSize = appendSize < mediumSize
+ ? appendSize + mediumSize
+ : appendSize;
+ }
+ appendFileByBuffer(objectPath, appendSize, bufferSize, totoal);
+ totoal = totoal + appendSize;
+ }
+ verifyReceivedData(fs, objectPath, totoal, testBufferSize, modulus);
+ }
+
+ private void verifyMultiSizeAppend(long fileSize, int appendMinSize,
+ int appendMaxSize,
+ int appendTimes, int bufferSize)
+ throws IOException {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ long totoal = fileSize;
+ Random random = new Random();
+ Path objectPath = createAppendFile(fileSize);
+ for (int i = 0; i < appendTimes; i++) {
+ int appendSize =
+ random.nextInt(appendMaxSize) % (appendMaxSize - appendMinSize
+ + 1) + appendMinSize;
+ appendFileByBuffer(objectPath, appendSize, bufferSize, totoal);
+ totoal = totoal + appendSize;
+ }
+ verifyReceivedData(fs, objectPath, totoal, testBufferSize, modulus);
+ }
+
+ private Path createAppendFile(long fileSize) throws IOException {
+ String objectName = UUID.randomUUID().toString();
+ Path objectPath = getTestPath(objectName);
+ NanoTimer timer = new NanoTimer();
+
+ OutputStream outputStream = createAppendStream(objectPath);
+ writStream(outputStream, fileSize, 0);
+ bandwidth(timer, fileSize);
+ assertPathExists(fs, "not created successful", objectPath);
+ return objectPath;
+ }
+
+ private void appendFileByBuffer(Path objectPath, long appendSize,
+ int bufferSize, long offset) throws IOException {
+ OutputStream outputStream = fs.append(objectPath, bufferSize, null);
+ writStream(outputStream, appendSize, offset);
+ assertPathExists(fs, "not created successful", objectPath);
+ }
+
+ private void writStream(OutputStream outputStream, long fileSize,
+ long offset) throws IOException {
+ long bytesWritten = 0L;
+ Throwable var10 = null;
+ long diff;
+ try {
+ int off = (int) (offset % testBuffer.length);
+ while (bytesWritten < fileSize) {
+ diff = fileSize - bytesWritten;
+ if (diff + off <= (long) testBuffer.length) {
+ outputStream.write(testBuffer, off, (int) diff);
+ bytesWritten += diff;
+ break;
+ } else {
+ outputStream.write(testBuffer, off,
+ (testBuffer.length - off));
+ bytesWritten += (long) testBuffer.length - off;
+ }
+ off = 0;
+ }
+ } catch (Throwable var21) {
+ var10 = var21;
+ throw var21;
+ } finally {
+ if (outputStream != null) {
+ if (var10 != null) {
+ try {
+ outputStream.close();
+ } catch (Throwable var20) {
+ var10.addSuppressed(var20);
+ }
+ } else {
+ outputStream.close();
+ }
+ }
+
+ }
+ assertEquals(fileSize, bytesWritten);
+ }
+
+ private FSDataOutputStream createAppendStream(Path objectPath)
+ throws IOException {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.APPEND);
+ FsPermission permission = new FsPermission((short) 00644);
+ return fs.create(objectPath, permission, flags,
+ fs.getConf().getInt("io.file.buffer.size", 4096),
+ fs.getDefaultReplication(objectPath),
+ fs.getDefaultBlockSize(objectPath),
+ null, null);
+ }
+
+ @Test
+ public void testAppendNormal() throws IOException {
+ if (fs.getMetricSwitch()) {
+ long fileSize = 10;
+ long appendSize = 10;
+ int bufferSize = 4096;
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ long total = fileSize + appendSize;
+
+ Path objectPath = createAppendFile(fileSize);
+ // appendFileByBuffer(objectPath, appendSize, bufferSize, total);
+ OutputStream outputStream = fs.append(objectPath, bufferSize, null);
+ writStream(outputStream, appendSize, total);
+
+ OBSFSTestUtil.deletePathRecursive(fs, objectPath);
+ }
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSAppendOutputStream.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSAppendOutputStream.java
new file mode 100644
index 0000000..d965bda
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSAppendOutputStream.java
@@ -0,0 +1,271 @@
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.EnumSet;
+import java.util.UUID;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.junit.Assert.assertEquals;
+
+public class ITestOBSAppendOutputStream {
+
+ private OBSFileSystem fs;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ private int testBufferSize;
+
+ private int modulus;
+
+ private byte[] testBuffer;
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.setBoolean(OBSConstants.FAST_UPLOAD, true);
+ conf.set(OBSConstants.MULTIPART_SIZE, String.valueOf(5 * 1024 * 1024));
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ testBufferSize = fs.getConf().getInt(
+ ContractTestUtils.IO_CHUNK_BUFFER_SIZE, 128);
+ modulus = fs.getConf().getInt(ContractTestUtils.IO_CHUNK_MODULUS_SIZE
+ + ".size", 128);
+ testBuffer = new byte[testBufferSize];
+
+ for (int i = 0; i < testBuffer.length; ++i) {
+ testBuffer[i] = (byte) (i % modulus);
+ }
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(new Path(testRootPath), true);
+ }
+ }
+
+ private Path getTestPath() {
+ return new Path(testRootPath + "/test-obs");
+ }
+
+ private Path getRenameTestPath() {
+ return new Path(testRootPath + "/test-obs-rename");
+ }
+
+ @Test
+ // append空文件,校验append后大小
+ public void testZeroSizeFileAppend() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ verifyAppend(0, 1024, 3, 4096);
+ }
+
+ @Test
+ // 单次write小于缓存块大小,多次append,写满本地缓存块,触发上传到服务端,校验大小
+ public void testBellowMultipartSizeAppend() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ // 在128字节 buffer写(单次write小于设置的5MB缓存的append),即会走OBSBlockOutputStream的275行逻辑
+ verifyAppend(1024 * 1024, 1024 * 1024 * 80, 3, 4096);
+ verifyAppend(1024 * 1024 * 10, 1024 * 1024 * 4, 3, 4096);
+ verifyAppend(1024 * 1024, 1024, 10, 4096);
+ verifyAppend(1024 * 1024, 1024 * 1024, 3, 4096);
+ }
+
+ @Test
+ // 单次write大于缓存块大小,多次append,写满本地缓存块,触发上传到服务端,校验大小
+ public void testAboveMultipartSizeAppend() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ // 在6MB buffer写(单次write大于设置的5MB缓存的append),即会走OBSBlockOutputStream的265行逻辑
+ testBufferSize = fs.getConf()
+ .getInt(ContractTestUtils.IO_CHUNK_BUFFER_SIZE, 6 * 1024 * 1024);
+ modulus = fs.getConf().getInt(ContractTestUtils.IO_CHUNK_MODULUS_SIZE, 128);
+ testBuffer = new byte[testBufferSize];
+
+ for (int i = 0; i < testBuffer.length; ++i) {
+ testBuffer[i] = (byte) (i % modulus);
+ }
+ verifyAppend(1024 * 1024, 1024 * 1024 * 11, 3, 6 * 1024 * 1024);
+ verifyAppend(1024 * 1024 * 10, 1024 * 1024 * 10, 3, 6 * 1024 * 1024);
+ verifyAppend(1024 * 1024, 1024, 10, 6 * 1024 * 1024);
+ verifyAppend(1024 * 1024, 1024 * 1024, 3, 6 * 1024 * 1024);
+ }
+
+ @Test
+ // 文件rename后,append目标文件,校验大小正常
+ public void testAppendAfterRename() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ long fileSize = 1024 * 1024;
+ long total = fileSize;
+ int appendTimes = 3;
+ long appendSize = 1024;
+ Path objectPath = createAppendFile(fileSize);
+ for (int i = 0; i < appendTimes; i++) {
+ appendFile(objectPath, appendSize, total);
+ total = total + appendSize;
+ }
+
+ verifyReceivedData(fs, objectPath, total, testBufferSize, modulus);
+
+ String objectName = objectPath.getName();
+ Path renamePath = new Path(getRenameTestPath(), objectName);
+ fs.mkdirs(getRenameTestPath());
+ fs.rename(objectPath, renamePath);
+
+ for (int i = 0; i < appendTimes; i++) {
+ appendFile(renamePath, appendSize, total);
+ total = total + appendSize;
+ }
+
+ verifyReceivedData(fs, renamePath, total, testBufferSize, modulus);
+ }
+
+ @Test
+ // append流被关闭后,再写数据,抛出IOException
+ public void testAppendAfterClose() throws IOException {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+
+ Path dest = createAppendFile(1024);
+ OutputStream stream = creatAppendStream(dest);
+ byte[] data = ContractTestUtils.dataset(16, 'a', 26);
+ try {
+ stream.write(data);
+ stream.close();
+ boolean hasException = false;
+ try {
+ stream.write(data);
+ } catch (IOException e) {
+ hasException = true;
+ }
+ assertTrue(hasException);
+
+ } finally {
+ fs.delete(dest, false);
+ if (stream != null) {
+ stream.close();
+ }
+ }
+ }
+
+ private void verifyAppend(long fileSize, long appendSize, int appendTimes,
+ int bufferSize) throws IOException {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ long total = fileSize;
+ Path objectPath = createAppendFile(fileSize);
+ for (int i = 0; i < appendTimes; i++) {
+ appendFileByBuffer(objectPath, appendSize, bufferSize, total);
+ total = total + appendSize;
+ }
+ verifyReceivedData(fs, objectPath, total, testBufferSize, modulus);
+ }
+
+ private Path createAppendFile(long fileSize) throws IOException {
+
+ String objectName = UUID.randomUUID().toString();
+ Path objectPath = new Path(getTestPath(), objectName);
+ ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+
+ OutputStream outputStream = creatAppendStream(objectPath);
+ writStream(outputStream, fileSize, 0);
+ bandwidth(timer, fileSize);
+ assertPathExists(fs, "not created successful", objectPath);
+ return objectPath;
+ }
+
+ private void appendFile(Path objectPath, long appendSize, long offset)
+ throws IOException {
+ OutputStream outputStream = fs.append(objectPath, 4096, null);
+ writStream(outputStream, appendSize, offset);
+ assertPathExists(fs, "not created successful", objectPath);
+ }
+
+ private void appendFileByBuffer(Path objectPath, long appendSize,
+ int bufferSize, long offset) throws IOException {
+ OutputStream outputStream = fs.append(objectPath, bufferSize, null);
+ writStream(outputStream, appendSize, offset);
+ assertPathExists(fs, "not created successful", objectPath);
+ }
+
+ private void writStream(OutputStream outputStream, long fileSize,
+ long offset) throws IOException {
+ long bytesWritten = 0L;
+ Throwable var10 = null;
+ long diff;
+ try {
+ int off = (int) (offset % testBuffer.length);
+ while (bytesWritten < fileSize) {
+ diff = fileSize - bytesWritten;
+ if (diff + off <= (long) testBuffer.length) {
+ outputStream.write(testBuffer, off, (int) diff);
+ bytesWritten += diff;
+ break;
+ } else {
+ outputStream.write(testBuffer, off,
+ (testBuffer.length - off));
+ bytesWritten += (long) testBuffer.length - off;
+ }
+ off = 0;
+ }
+ } catch (Throwable var21) {
+ var10 = var21;
+ throw var21;
+ } finally {
+ if (outputStream != null) {
+ if (var10 != null) {
+ try {
+ outputStream.close();
+ } catch (Throwable var20) {
+ var10.addSuppressed(var20);
+ }
+ } else {
+ outputStream.close();
+ }
+ }
+
+ }
+ assertEquals(fileSize, bytesWritten);
+ }
+
+ private OutputStream creatAppendStream(Path objectPath) throws IOException {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.APPEND);
+ FsPermission permission = new FsPermission((short) 00644);
+ return fs.create(objectPath, permission, flags,
+ fs.getConf().getInt("io.file.buffer.size", 4096),
+ fs.getDefaultReplication(objectPath),
+ fs.getDefaultBlockSize(objectPath), null);
+ }
+}
+
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSArrayBufferOutputStream.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSArrayBufferOutputStream.java
new file mode 100644
index 0000000..76ff757
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSArrayBufferOutputStream.java
@@ -0,0 +1,220 @@
+/*
+ * 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.FAST_UPLOAD_BUFFER;
+import static org.apache.hadoop.fs.obs.OBSConstants.FAST_UPLOAD_BUFFER_ARRAY;
+import static org.apache.hadoop.fs.obs.OBSConstants.FAST_UPLOAD_BUFFER_ARRAY_FIRST_BLOCK_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.obs.OBSConstants.MULTIPART_MIN_SIZE;
+import static org.apache.hadoop.fs.obs.OBSConstants.MULTIPART_SIZE;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collection;
+
+/**
+ * Tests small file upload functionality for {@link OBSBlockOutputStream} with
+ * the blocks buffered in byte arrays.
+ *
+ * File sizes are kept small to reduce test duration on slow connections;
+ * multipart tests are kept in scale tests.
+ */
+@Deprecated
+@RunWith(Parameterized.class)
+public class ITestOBSArrayBufferOutputStream extends AbstractOBSTestBase {
+ private static final int BLOCK_SIZE = 256 * 1024;
+
+ private static byte[] dataset;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ private boolean calcMd5;
+
+ @Parameterized.Parameters
+ public static Collection calcMd5() {
+ return Arrays.asList(false, true);
+ }
+
+ public ITestOBSArrayBufferOutputStream(boolean calcMd5) {
+ this.calcMd5 = calcMd5;
+ }
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Override
+ protected Configuration createConfiguration() {
+ Configuration conf = super.createConfiguration();
+ OBSTestUtils.disableFilesystemCaching(conf);
+ conf.setInt(MULTIPART_SIZE, MULTIPART_MIN_SIZE);
+ conf.setBoolean(OBSConstants.FAST_UPLOAD, true);
+ conf.set(FAST_UPLOAD_BUFFER, getBlockOutputBufferName());
+ conf.setBoolean(OBSConstants.OUTPUT_STREAM_ATTACH_MD5, calcMd5);
+ return conf;
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (getFileSystem() != null) {
+ getFileSystem().delete(new Path(testRootPath), true);
+ }
+ }
+
+ private Path getTestPath(String relativePath) {
+ return new Path(testRootPath + "/" + relativePath);
+ }
+
+ protected String getBlockOutputBufferName() {
+ return FAST_UPLOAD_BUFFER_ARRAY;
+ }
+
+ @Test
+ // array内存缓存,首个缓存块大小为1M
+ public void testArrayFirstBufferSize() throws Exception {
+ Path dest = getTestPath("testMockRetryError");
+ FSDataOutputStream stream = getFileSystem().create(dest, true);
+ OBSBlockOutputStream obs
+ = (OBSBlockOutputStream) stream.getWrappedStream();
+ OBSDataBlocks.ByteArrayBlock block
+ = (OBSDataBlocks.ByteArrayBlock) obs.getActiveBlock();
+ assertTrue(String.format("Array first buffer must 1MB. real (%d)",
+ block.firstBlockSize()),
+ block.firstBlockSize()
+ == FAST_UPLOAD_BUFFER_ARRAY_FIRST_BLOCK_SIZE_DEFAULT);
+ if (stream != null) {
+ stream.close();
+ }
+ }
+
+ @Test
+ // 上传0字节的文件
+ public void testZeroByteUpload() throws IOException {
+ verifyUpload("0", 0);
+ }
+
+ @Test
+ // 上传1K大小文件
+ public void testRegularUpload() throws IOException {
+ verifyUpload("regular", 1024);
+ }
+
+ @Test
+ // 流关闭后再写数据,抛IOException
+ public void testArrayWriteAfterStreamClose() throws IOException {
+ Path dest = getTestPath("testWriteAfterStreamClose");
+ describe(" testWriteAfterStreamClose");
+ FSDataOutputStream stream = getFileSystem().create(dest, true);
+ byte[] data = ContractTestUtils.dataset(16, 'a', 26);
+ try {
+ stream.write(data);
+ stream.close();
+ boolean hasException = false;
+ try {
+ stream.write(data);
+ } catch (IOException e) {
+ hasException = true;
+ }
+ assertTrue(hasException);
+ } finally {
+ IOUtils.closeStream(stream);
+ }
+ }
+
+ @Test
+ // 流关闭后,activeBlock置为空
+ public void testBlocksClosed() throws Throwable {
+ Path dest = getTestPath("testBlocksClosed");
+ describe(" testBlocksClosed");
+ FSDataOutputStream stream = getFileSystem().create(dest, true);
+ byte[] data = ContractTestUtils.dataset(16, 'a', 26);
+ stream.write(data);
+ stream.close();
+ OBSBlockOutputStream obsStream =
+ (OBSBlockOutputStream) stream.getWrappedStream();
+ assertEquals(null, obsStream.getActiveBlock());
+ }
+
+ private void verifyUpload(String name, int fileSize) throws IOException {
+ Path dest = getTestPath(name);
+ describe(name + " upload to " + dest);
+ ContractTestUtils.createAndVerifyFile(
+ getFileSystem(),
+ dest,
+ fileSize);
+ }
+
+ /**
+ * Create a factory for used in mark/reset tests.
+ *
+ * @param fileSystem source FS
+ * @return the factory
+ */
+ protected OBSDataBlocks.BlockFactory createFactory(
+ OBSFileSystem fileSystem) {
+ return new OBSDataBlocks.ByteArrayBlockFactory(fileSystem);
+ }
+
+ private void markAndResetDatablock(OBSDataBlocks.BlockFactory factory)
+ throws Exception {
+ OBSDataBlocks.DataBlock block = factory.create(1, BLOCK_SIZE);
+ dataset = ContractTestUtils.dataset(BLOCK_SIZE, 0, 256);
+ block.write(dataset, 0, dataset.length);
+ InputStream stream = (InputStream) block.startUpload();
+ assertNotNull(stream);
+ assertTrue("Mark not supported in " + stream, stream.markSupported());
+ assertEquals(0, stream.read());
+ stream.mark(BLOCK_SIZE);
+ // read a lot
+ long l = 0;
+ while (stream.read() != -1) {
+ // do nothing
+ l++;
+ }
+ stream.reset();
+ assertEquals(1, stream.read());
+ }
+
+ @Test
+ // byte array流支持mark和reset操作
+ public void testMarkReset() throws Throwable {
+ markAndResetDatablock(createFactory(getFileSystem()));
+ }
+
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBlockSize.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBlockSize.java
new file mode 100644
index 0000000..a1a83f9
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBlockSize.java
@@ -0,0 +1,126 @@
+/*
+ * 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.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static junit.framework.TestCase.assertTrue;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.fileStatsToString;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * OBS tests for configuring block size.
+ */
+public class ITestOBSBlockSize {
+ private FileSystem fs;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ITestOBSBlockSize.class);
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(new Path(testRootPath), true);
+ }
+ }
+
+ private Path getTestPath(String testPath) {
+ return new Path(fs.getUri() + testRootPath + testPath);
+ }
+
+ @Test
+ @SuppressWarnings("deprecation")
+ // getDefaultBlockSize返回128M,并且在已存在的fs实例修改conf配置,不会生效
+ public void testBlockSize() throws Exception {
+ long defaultBlockSize = fs.getDefaultBlockSize();
+ assertEquals("incorrect blocksize",
+ OBSConstants.DEFAULT_FS_OBS_BLOCK_SIZE, defaultBlockSize);
+ long newBlockSize = defaultBlockSize * 2;
+ fs.getConf().setLong(OBSConstants.FS_OBS_BLOCK_SIZE, newBlockSize);
+
+ Path dir = getTestPath("/testBlockSize");
+ Path file = new Path(dir, "file");
+ createFile(fs, file, true, dataset(1024, 'a', 'z' - 'a'));
+ FileStatus fileStatus = fs.getFileStatus(file);
+ assertEquals(
+ "modify configuration in exist fs won't have effect: " + fileStatus,
+ defaultBlockSize,
+ fileStatus.getBlockSize());
+
+ // check the listing & assert that the block size is picked up by
+ // this route too.
+ boolean found = false;
+ FileStatus[] listing = fs.listStatus(dir);
+ fs.delete(file, false);
+ for (FileStatus stat : listing) {
+ LOG.info("entry: {}", stat);
+ if (file.equals(stat.getPath())) {
+ found = true;
+ assertEquals(
+ "modify configuration in exist fs won't have effect: "
+ + stat,
+ defaultBlockSize,
+ stat.getBlockSize());
+ }
+ }
+ assertTrue("Did not find " + fileStatsToString(listing, ", "), found);
+
+ }
+
+ @Test
+ // 根目录blocksize大小为0
+ public void testRootFileStatusBlockSize() throws Throwable {
+ FileStatus status = fs.getFileStatus(new Path("/"));
+ assertTrue("Invalid root blocksize",
+ status.getBlockSize() == 0);
+ }
+
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBlockingThreadPoolExecutorService.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBlockingThreadPoolExecutorService.java
new file mode 100644
index 0000000..1d6646a
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBlockingThreadPoolExecutorService.java
@@ -0,0 +1,223 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+import org.apache.hadoop.util.StopWatch;
+import org.junit.AfterClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Basic test for OBS's blocking executor service.
+ */
+public class ITestOBSBlockingThreadPoolExecutorService {
+
+ private static final Logger LOG = LoggerFactory.getLogger(
+ BlockingThreadPoolExecutorService.class);
+
+ private static final int NUM_ACTIVE_TASKS = 4;
+
+ private static final int NUM_WAITING_TASKS = 2;
+
+ private static final int TASK_SLEEP_MSEC = 100;
+
+ private static final int SHUTDOWN_WAIT_MSEC = 200;
+
+ private static final int SHUTDOWN_WAIT_TRIES = 5;
+
+ private static final int BLOCKING_THRESHOLD_MSEC = 50;
+
+ private static final Integer SOME_VALUE = 1337;
+
+ private static BlockingThreadPoolExecutorService tpe;
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @Rule
+ public Timeout testTimeout = new Timeout(60 * 1000);
+
+ @AfterClass
+ public static void afterClass() throws Exception {
+ ensureDestroyed();
+ }
+
+ /**
+ * Basic test of running one trivial task.
+ */
+ @Test
+ // 线程池可正常提交Callable任务
+ public void testSubmitCallable() throws Exception {
+ ensureCreated();
+ ListenableFuture f = tpe.submit(callableSleeper);
+ Integer v = f.get();
+ assertEquals(SOME_VALUE, v);
+ }
+
+ /**
+ * More involved test, including detecting blocking when at capacity.
+ */
+ @Test
+ // 线程池可正常提交Runnable任务
+ public void testSubmitRunnable() throws Exception {
+ ensureCreated();
+ verifyQueueSize(tpe, NUM_ACTIVE_TASKS + NUM_WAITING_TASKS);
+ }
+
+ /**
+ * Verify the size of the executor's queue, by verifying that the first
+ * submission to block is {@code expectedQueueSize + 1}.
+ *
+ * @param executorService executor service to test
+ * @param expectedQueueSize size of queue
+ */
+ protected void verifyQueueSize(ExecutorService executorService,
+ int expectedQueueSize) {
+ StopWatch stopWatch = new StopWatch().start();
+ for (int i = 0; i < expectedQueueSize; i++) {
+ executorService.submit(sleeper);
+ assertDidntBlock(stopWatch);
+ }
+ executorService.submit(sleeper);
+ assertDidBlock(stopWatch);
+ }
+
+ @Test
+ // 线程池可正常shutdown
+ public void testShutdown() throws Exception {
+ // Cover create / destroy, regardless of when this test case runs
+ ensureCreated();
+ ensureDestroyed();
+
+ // Cover create, execute, destroy, regardless of when test case runs
+ ensureCreated();
+ testSubmitRunnable();
+ ensureDestroyed();
+ }
+
+ @Test
+ // 线程池任务可排队
+ public void testChainedQueue() throws Throwable {
+ ensureCreated();
+ int size = 2;
+ ExecutorService wrapper = new SemaphoredDelegatingExecutor(tpe,
+ size, true);
+ verifyQueueSize(wrapper, size);
+ }
+
+ // Helper functions, etc.
+
+ private void assertDidntBlock(StopWatch sw) {
+ try {
+ assertFalse("Non-blocking call took too long.",
+ sw.now(TimeUnit.MILLISECONDS) > BLOCKING_THRESHOLD_MSEC);
+ } finally {
+ sw.reset().start();
+ }
+ }
+
+ private void assertDidBlock(StopWatch sw) {
+ try {
+ if (sw.now(TimeUnit.MILLISECONDS) < BLOCKING_THRESHOLD_MSEC) {
+ throw new RuntimeException("Blocking call returned too fast.");
+ }
+ } finally {
+ sw.reset().start();
+ }
+ }
+
+ private Runnable sleeper = new Runnable() {
+ @Override
+ public void run() {
+ String name = Thread.currentThread().getName();
+ try {
+ Thread.sleep(TASK_SLEEP_MSEC);
+ } catch (InterruptedException e) {
+ LOG.info("Thread {} interrupted.", name);
+ Thread.currentThread().interrupt();
+ }
+ }
+ };
+
+ private Callable callableSleeper = new Callable() {
+ @Override
+ public Integer call() throws Exception {
+ sleeper.run();
+ return SOME_VALUE;
+ }
+ };
+
+ /**
+ * Helper function to create thread pool under test.
+ */
+ private synchronized static void ensureCreated() throws Exception {
+ if (tpe == null) {
+ LOG.debug("Creating thread pool");
+ tpe = BlockingThreadPoolExecutorService.newInstance(
+ NUM_ACTIVE_TASKS, NUM_WAITING_TASKS,
+ 1, "btpetest");
+ }
+ }
+
+ /**
+ * Helper function to terminate thread pool under test, asserting that
+ * shutdown -> terminate works as expected.
+ */
+ private static void ensureDestroyed() throws Exception {
+ if (tpe == null) {
+ return;
+ }
+ int shutdownTries = SHUTDOWN_WAIT_TRIES;
+
+ tpe.shutdown();
+ if (!tpe.isShutdown()) {
+ throw new RuntimeException("Shutdown had no effect.");
+ }
+
+ while (!tpe.awaitTermination(SHUTDOWN_WAIT_MSEC,
+ TimeUnit.MILLISECONDS)) {
+ LOG.info("Waiting for thread pool shutdown.");
+ if (shutdownTries-- <= 0) {
+ LOG.error("Failed to terminate thread pool gracefully.");
+ break;
+ }
+ }
+ if (!tpe.isTerminated()) {
+ tpe.shutdownNow();
+ if (!tpe.awaitTermination(SHUTDOWN_WAIT_MSEC,
+ TimeUnit.MILLISECONDS)) {
+ throw new RuntimeException(
+ "Failed to terminate thread pool in timely manner.");
+ }
+ }
+ tpe = null;
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBucketAcl.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBucketAcl.java
new file mode 100644
index 0000000..8da6a3b
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBucketAcl.java
@@ -0,0 +1,217 @@
+package org.apache.hadoop.fs.obs;
+
+import com.obs.services.ObsClient;
+import com.obs.services.model.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Progressable;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.EnumSet;
+import java.util.Set;
+import java.util.UUID;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+
+public class ITestOBSBucketAcl {
+ private OBSFileSystem fs;
+
+ private int testBufferSize;
+
+ private int modulus;
+
+ private byte[] testBuffer;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ String vAccountId = null;
+
+ String permissionStr = null;
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ testBufferSize = fs.getConf().getInt("io.chunk.buffer.size", 128);
+ modulus = fs.getConf().getInt("io.chunk.modulus.size", 128);
+ testBuffer = new byte[testBufferSize];
+
+ for (int i = 0; i < testBuffer.length; ++i) {
+ testBuffer[i] = (byte) (i % modulus);
+ }
+ vAccountId = "domainiddomainiddomainiddo000001";
+ permissionStr = String.valueOf(Permission.PERMISSION_FULL_CONTROL);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(new Path(testRootPath), true);
+ }
+ }
+
+ private Path getTestPath() {
+ return new Path(testRootPath + "/test-obs");
+ }
+
+ @Test
+ // 上传文件,桶owner有FULL_CONTROL权限
+ public void testUploadFileBucketOwnerPermission() throws IOException {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ verifyAppendAndAcl(1024 * 1024, 1024, 3);
+ }
+
+ @Test
+ // 创建目录,桶owner有FULL_CONTROL权限
+ public void testMkDirBucketOwnerPermission() throws IOException {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path dirPath = new Path(getTestPath(), "mkdir-zh");
+ fs.mkdirs(dirPath);
+ String key = dirPath.toString().substring(1);
+ verifyAcl(key, vAccountId, permissionStr);
+ }
+
+ @Test
+ // 从本地copy文件,桶owner有FULL_CONTROL权限
+ public void testCopyFromLocalBucketOwnerPermission() throws IOException {
+ String localFile = "local_file";
+ new File(testRootPath, "." + localFile + ".crc").delete();
+ OBSFSTestUtil.createLocalTestFile(localFile);
+ Path localPath = new Path(localFile);
+ Path dstPath = new Path(testRootPath + "/copylocal-huge");
+
+ fs.copyFromLocalFile(false, true, localPath, dstPath);
+ String key = dstPath.toString().substring(1);
+ verifyAcl(key, vAccountId, permissionStr);
+ OBSFSTestUtil.deleteLocalFile(localFile);
+ }
+
+ private void verifyAppendAndAcl(long fileSize, long appendSize,
+ int appendTimes) throws IOException {
+ long total = fileSize;
+ Path objectPath = createAppendFile(fileSize);
+
+ for (int i = 0; i < appendTimes; i++) {
+ appendFile(objectPath, appendSize);
+ total = total + appendSize;
+ }
+ verifyReceivedData(fs, objectPath, total, testBufferSize, modulus);
+ String key = objectPath.toString().substring(1);
+ verifyAcl(key, vAccountId, permissionStr);
+ }
+
+ private void verifyAcl(String key, String vAccountId,
+ String permissionStr) {
+ ObsClient client = fs.getObsClient();
+ AccessControlList controlList = client.getObjectAcl(fs.getBucket(),
+ key);
+ Set grants = controlList.getGrants();
+ boolean aclOk = false;
+ for (GrantAndPermission grant : grants) {
+ String accountId = grant.getGrantee().getIdentifier();
+ String permission = grant.getPermission().getPermissionString();
+ if (accountId.equals(vAccountId) && permission.equals(
+ permissionStr)) {
+ aclOk = true;
+ break;
+ }
+ }
+ assertTrue(aclOk);
+ }
+
+ private void appendFile(Path objectPath, long appendSize)
+ throws IOException {
+ OutputStream outputStream = fs.append(objectPath, 4096, null);
+ writStream(outputStream, appendSize);
+ assertPathExists(fs, "not created successful", objectPath);
+ }
+
+ private void writStream(OutputStream outputStream, long fileSize)
+ throws IOException {
+ long bytesWritten = 0L;
+ Throwable var10 = null;
+ long diff;
+ try {
+ while (bytesWritten < fileSize) {
+ diff = fileSize - bytesWritten;
+ if (diff < (long) testBuffer.length) {
+ outputStream.write(testBuffer, 0, (int) diff);
+ bytesWritten += diff;
+ } else {
+ outputStream.write(testBuffer);
+ bytesWritten += (long) testBuffer.length;
+ }
+ }
+
+ diff = bytesWritten;
+ } catch (Throwable var21) {
+ var10 = var21;
+ throw var21;
+ } finally {
+ if (outputStream != null) {
+ if (var10 != null) {
+ try {
+ outputStream.close();
+ } catch (Throwable var20) {
+ var10.addSuppressed(var20);
+ }
+ } else {
+ outputStream.close();
+ }
+ }
+
+ }
+ assertEquals(fileSize, diff);
+ }
+
+ private Path createAppendFile(long fileSize) throws IOException {
+
+ String objectName = UUID.randomUUID().toString();
+ Path objectPath = new Path(getTestPath(), objectName);
+ ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+
+ OutputStream outputStream = creatAppendStream(objectPath);
+ writStream(outputStream, fileSize);
+ bandwidth(timer, fileSize);
+ assertPathExists(fs, "not created successful", objectPath);
+ return objectPath;
+ }
+
+ private OutputStream creatAppendStream(Path objectPath) throws IOException {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.APPEND);
+ FsPermission permission = new FsPermission((short) 00644);
+ return fs.create(objectPath, permission, flags,
+ fs.getConf().getInt("io.file.buffer.size", 4096),
+ fs.getDefaultReplication(objectPath),
+ fs.getDefaultBlockSize(objectPath), (Progressable) null);
+
+ }
+
+}
\ No newline at end of file
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBucketPolicy.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBucketPolicy.java
new file mode 100644
index 0000000..cd7d226
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBucketPolicy.java
@@ -0,0 +1,402 @@
+package org.apache.hadoop.fs.obs;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+public class ITestOBSBucketPolicy {
+ /**
+ * Resources_s3 前缀
+ */
+ private static final String RES_S3_PREFIX = "arn:aws:s3:::";
+
+ /**
+ * Resources_IAM 前缀
+ */
+ private static final String RES_IAM_PREFIX = "arn:aws:iam::";
+
+ private static final String NEW_USER_AK = "UDSIAMSTUBTEST000999";
+
+ private static final String NEW_USER_SK
+ = "Udsiamstubtest000000UDSIAMSTUBTEST000999";
+
+ private static final String NEW_USER_DOMAIN_ID
+ = "domainiddomainiddomainiddo000999";
+
+ private OBSFileSystem fs;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ fs.delete(new Path(testRootPath), true);
+ fs.mkdirs(new Path(testRootPath), new FsPermission((short) 00644));
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(new Path(testRootPath), true);
+ fs.close();
+ fs = null;
+ }
+ }
+
+ private Path getTestPath(String relativePath) {
+ return new Path(testRootPath + "/" + relativePath);
+ }
+
+ private JSONObject getOtherTenantAccessPolicy(Path testDir) throws JSONException {
+ return new JSONObject().put("Statement", new JSONArray().put(new JSONObject().put("Effect", "Allow")
+ .put("Sid", "1")
+ .put("Principal",
+ new JSONObject().put("AWS", new JSONArray().put(RES_IAM_PREFIX + NEW_USER_DOMAIN_ID + ":root")))
+ .put("Action", new JSONArray().put("*"))
+ .put("Resource", new JSONArray()
+ .put(RES_S3_PREFIX + fs.getBucket())
+ .put(RES_S3_PREFIX + fs.getBucket() + testRootPath + "/")
+ .put(RES_S3_PREFIX + fs.getBucket() + "/" + OBSCommonUtils.pathToKey(fs, testDir) + "/*")
+ .put(RES_S3_PREFIX + fs.getBucket() + "/" + OBSCommonUtils.pathToKey(fs, testDir))
+ )));
+ }
+
+ @Test
+ // 配置policy后,创建目录权限校验
+ public void testMkdir() throws IOException, JSONException, InterruptedException {
+ if(!fs.isFsBucket()) {
+ return;
+ }
+
+ Path testDir = getTestPath("test_dir");
+ fs.delete(testDir, true);
+
+ setPolicy(getOtherTenantAccessPolicy(testDir));
+
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK);
+ conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK);
+ OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf);
+
+ assertTrue(newFs.mkdirs(testDir, new FsPermission((short) 00644)));
+
+ newFs.delete(testDir, true);
+ }
+
+ private void setPolicy(JSONObject testDir) throws InterruptedException {
+ fs.setBucketPolicy(testDir.toString());
+ // policy 有缓存,需要3秒才能生效
+ TimeUnit.SECONDS.sleep(5);
+ }
+
+ @Test
+ // 配置policy后,listStatus权限校验
+ public void testListStatus() throws IOException, JSONException, InterruptedException {
+ if(!fs.isFsBucket()) {
+ return;
+ }
+
+ Path testDir = getTestPath("test_dir");
+ fs.delete(testDir, true);
+
+ assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1"),
+ new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1/subsubdir1"),
+ new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir2"),
+ new FsPermission((short) 00644)));
+
+ setPolicy(getOtherTenantAccessPolicy(testDir));
+
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK);
+ conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK);
+ OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf);
+
+ assertEquals(2, newFs.listStatus(testDir).length);
+ assertEquals(3, newFs.listStatus(testDir, true).length);
+
+ fs.delete(testDir, true);
+ }
+
+ @Test
+ // 配置policy后,listFiles权限校验
+ public void testListFiles() throws IOException, JSONException, InterruptedException {
+ if(!fs.isFsBucket()) {
+ return;
+ }
+
+ Path testDir = getTestPath("test_dir");
+ fs.delete(testDir, true);
+
+ assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1"),
+ new FsPermission((short) 00644)));
+ FSDataOutputStream outputStream = fs.create(getTestPath("test_dir"
+ + "/subdir1/file"), false);
+ outputStream.close();
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1/subsubdir1"),
+ new FsPermission((short) 00644)));
+ outputStream = fs.create(getTestPath("test_dir"
+ + "/subdir1/subsubdir1/file"), false);
+ outputStream.close();
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir2"),
+ new FsPermission((short) 00644)));
+ outputStream = fs.create(getTestPath("test_dir"
+ + "/subdir2/file"), false);
+ outputStream.close();
+
+ setPolicy(getOtherTenantAccessPolicy(testDir));
+
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK);
+ conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK);
+ OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf);
+
+ RemoteIterator iterator = newFs.listFiles(testDir,
+ true);
+ int fileNum = 0;
+ while (iterator.hasNext()) {
+ iterator.next();
+ fileNum++;
+ }
+ assertEquals(3, fileNum);
+
+ fs.delete(testDir, true);
+ }
+
+ @Test
+ // 配置policy后,ListLocatedStatus权限校验
+ public void testListLocatedStatus() throws IOException, JSONException, InterruptedException {
+ if(!fs.isFsBucket()) {
+ return;
+ }
+
+ Path testDir = getTestPath("test_dir");
+ fs.delete(testDir, true);
+
+ assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1"),
+ new FsPermission((short) 00644)));
+ FSDataOutputStream outputStream = fs.create(getTestPath("test_dir"
+ + "/subdir1/file"), false);
+ outputStream.close();
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1/subsubdir1"),
+ new FsPermission((short) 00644)));
+ outputStream = fs.create(getTestPath("test_dir"
+ + "/subdir1/subsubdir1/file"), false);
+ outputStream.close();
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir2"),
+ new FsPermission((short) 00644)));
+ outputStream = fs.create(getTestPath("test_dir"
+ + "/subdir2/file"), false);
+ outputStream.close();
+
+ setPolicy(getOtherTenantAccessPolicy(testDir));
+
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK);
+ conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK);
+ OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf);
+
+ RemoteIterator iterator = newFs.listLocatedStatus(
+ testDir);
+ int fileNum = 0;
+ while (iterator.hasNext()) {
+ iterator.next();
+ fileNum++;
+ }
+ assertEquals(2, fileNum);
+
+ fs.delete(testDir, true);
+ }
+
+ @Test
+ // 配置policy后,GetContentSummary权限校验
+ public void testGetContentSummary() throws IOException, JSONException, InterruptedException {
+ if(!fs.isFsBucket()) {
+ return;
+ }
+
+ Path testDir = getTestPath("test_dir");
+ fs.delete(testDir, true);
+
+ assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1"),
+ new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1/subsubdir1"),
+ new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir2"),
+ new FsPermission((short) 00644)));
+
+ setPolicy(getOtherTenantAccessPolicy(testDir));
+
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK);
+ conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK);
+ OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf);
+
+ assertEquals(4, newFs.getContentSummary(testDir).getDirectoryCount());
+
+ fs.delete(testDir, true);
+ }
+
+ @Test
+ // 配置policy后,GetFileStatus权限校验
+ public void testGetFileStatus() throws IOException, JSONException, InterruptedException {
+ if(!fs.isFsBucket()) {
+ return;
+ }
+
+ Path testDir = getTestPath("test_dir");
+ fs.delete(testDir, true);
+
+ assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1"),
+ new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1/subsubdir1"),
+ new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir2"),
+ new FsPermission((short) 00644)));
+
+ setPolicy(getOtherTenantAccessPolicy(testDir));
+
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK);
+ conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK);
+ OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf);
+
+ assertTrue(newFs.getFileStatus(testDir).isDirectory());
+
+ fs.delete(testDir, true);
+ }
+
+ @Test
+ // 配置policy后,Exist权限校验
+ public void testExist() throws IOException, JSONException, InterruptedException {
+ if(!fs.isFsBucket()) {
+ return;
+ }
+
+ Path testDir = getTestPath("test_dir");
+ fs.delete(testDir, true);
+
+ assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1"),
+ new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1/subsubdir1"),
+ new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir2"),
+ new FsPermission((short) 00644)));
+
+ setPolicy(getOtherTenantAccessPolicy(testDir));
+
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK);
+ conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK);
+ OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf);
+
+ assertTrue(newFs.exists(testDir));
+
+ fs.delete(testDir, true);
+ }
+
+ @Test
+ // 配置policy后,Delete权限校验
+ public void testDelete() throws IOException, JSONException, InterruptedException {
+ if(!fs.isFsBucket()) {
+ return;
+ }
+
+ Path testDir = getTestPath("test_dir");
+ fs.delete(testDir, true);
+
+ assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1"),
+ new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1/subsubdir1"),
+ new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir2"),
+ new FsPermission((short) 00644)));
+
+ setPolicy(getOtherTenantAccessPolicy(testDir));
+
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK);
+ conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK);
+ OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf);
+
+ assertTrue(newFs.delete(testDir, true));
+ }
+
+ @Test
+ // 配置policy后,Rename权限校验
+ public void testRename() throws IOException, JSONException, InterruptedException {
+ if(!fs.isFsBucket()) {
+ return;
+ }
+
+ Path testDir = getTestPath("test_dir");
+ Path renameToDir = getTestPath("test_dir_to");
+ fs.delete(testDir, true);
+
+ assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1"),
+ new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1/subsubdir1"),
+ new FsPermission((short) 00644)));
+ assertTrue(fs.mkdirs(getTestPath("test_dir/subdir2"),
+ new FsPermission((short) 00644)));
+
+ setPolicy(new JSONObject().put("Statement", new JSONArray().put(new JSONObject().put("Effect", "Allow")
+ .put("Sid", "1")
+ .put("Principal",
+ new JSONObject().put("AWS", new JSONArray().put(RES_IAM_PREFIX + NEW_USER_DOMAIN_ID + ":root")))
+ .put("Action", new JSONArray().put("*"))
+ .put("Resource", new JSONArray().put(RES_S3_PREFIX + fs.getBucket())
+ .put(RES_S3_PREFIX + fs.getBucket() + testRootPath + "/")
+ .put(RES_S3_PREFIX + fs.getBucket() + "/" + OBSCommonUtils.pathToKey(fs, testDir) + "/*")
+ .put(RES_S3_PREFIX + fs.getBucket() + "/" + OBSCommonUtils.pathToKey(fs, testDir))
+ .put(RES_S3_PREFIX + fs.getBucket() + "/" + OBSCommonUtils.pathToKey(fs, renameToDir) + "/*")
+ .put(RES_S3_PREFIX + fs.getBucket() + "/" + OBSCommonUtils.pathToKey(fs, renameToDir))))));
+
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK);
+ conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK);
+ OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf);
+
+ assertTrue(newFs.rename(testDir, renameToDir));
+
+ fs.delete(renameToDir, true);
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCloseFunction.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCloseFunction.java
new file mode 100644
index 0000000..3153eae
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCloseFunction.java
@@ -0,0 +1,114 @@
+package org.apache.hadoop.fs.obs;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.util.concurrent.ThreadPoolExecutor;
+
+public class ITestOBSCloseFunction {
+ private OBSFileSystem fs;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ OBSFSTestUtil.deletePathRecursive(fs, new Path(testRootPath));
+ }
+ }
+
+ private Path getTestPath(String relativePath) {
+ return new Path(testRootPath + "/" + relativePath);
+ }
+
+ @Test
+ // close时应关闭所有的OutputStream
+ public void testOutputStreamClose() throws Exception {
+ Path testFile = getTestPath("test_file");
+ fs.delete(testFile);
+ FSDataOutputStream outputStream = fs.create(testFile, false);
+ OBSBlockOutputStream obsBlockOutputStream
+ = (OBSBlockOutputStream) outputStream.getWrappedStream();
+ Field field = obsBlockOutputStream.getClass()
+ .getDeclaredField("closed");
+ field.setAccessible(true);
+ boolean closed = (boolean) field.get(obsBlockOutputStream);
+ assertFalse(closed);
+ fs.close();
+ closed = (boolean) field.get(obsBlockOutputStream);
+ fs = null;
+ assertTrue(closed);
+ }
+
+ @Test
+ // finalize接口释放实例申请的各类资源
+ public void testThreadPoolClose() throws Throwable {
+
+ Field boundedCopyThreadPoolField = fs.getClass()
+ .getDeclaredField("boundedCopyThreadPool");
+ boundedCopyThreadPoolField.setAccessible(true);
+ ThreadPoolExecutor boundedCopyThreadPool
+ = (ThreadPoolExecutor) boundedCopyThreadPoolField.get(fs);
+
+ Field boundedDeleteThreadPoolField = fs.getClass()
+ .getDeclaredField("boundedDeleteThreadPool");
+ boundedDeleteThreadPoolField.setAccessible(true);
+ ThreadPoolExecutor boundedDeleteThreadPool
+ = (ThreadPoolExecutor) boundedDeleteThreadPoolField.get(fs);
+
+ Field boundedCopyPartThreadPoolField = fs.getClass()
+ .getDeclaredField("boundedCopyPartThreadPool");
+ boundedCopyPartThreadPoolField.setAccessible(true);
+ ThreadPoolExecutor boundedCopyPartThreadPool
+ = (ThreadPoolExecutor) boundedCopyPartThreadPoolField.get(fs);
+
+ Field boundedListThreadPoolField = fs.getClass()
+ .getDeclaredField("boundedListThreadPool");
+ boundedListThreadPoolField.setAccessible(true);
+ ThreadPoolExecutor boundedListThreadPool
+ = (ThreadPoolExecutor) boundedListThreadPoolField.get(fs);
+
+ fs.close();
+ fs = null;
+
+ if (boundedCopyThreadPool != null) {
+ assertTrue(boundedCopyThreadPool.isShutdown());
+ }
+ if (boundedDeleteThreadPool != null) {
+ assertTrue(boundedDeleteThreadPool.isShutdown());
+ }
+ if (boundedCopyPartThreadPool != null) {
+ assertTrue(boundedCopyPartThreadPool.isShutdown());
+ }
+ if (boundedListThreadPool != null) {
+ assertTrue(boundedListThreadPool.isShutdown());
+ }
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCloseProtect.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCloseProtect.java
new file mode 100644
index 0000000..8d415c6
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCloseProtect.java
@@ -0,0 +1,947 @@
+/*
+ * 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.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.*;
+
+public class ITestOBSCloseProtect {
+ private static final Logger LOG = LoggerFactory.getLogger(
+ ITestOBSRename.class);
+ private OBSFileSystem fs;
+
+ private static String testRootPath = OBSTestUtils.generateUniqueTestPath();
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ final int partSize = 5 * 1024 * 1024;
+ conf.set(OBSConstants.MULTIPART_SIZE, String.valueOf(partSize));
+ conf.setClass(OBSConstants.OBS_METRICS_CONSUMER,
+ MockMetricsConsumer.class, BasicMetricsConsumer.class);
+ conf.setBoolean(OBSConstants.METRICS_SWITCH, true);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ if (fs.exists(new Path(testRootPath))) {
+ fs.delete(new Path(testRootPath), true);
+ }
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ }
+
+ @Test
+ // fs关闭后再执行open操作,抛出IOException
+ public void testOpenAfterFSClose() throws IOException {
+ String fileName = "file";
+ Path filePath = new Path(testRootPath + "/" + fileName);
+
+ fs.close();
+
+ boolean closeException = false;
+ try {
+ fs.open(filePath);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ fs.open(filePath, 100);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // fs关闭后再执行create操作,抛出IOException
+ public void testCreateAfterFSClose() throws IOException {
+ String fileName = "file";
+ Path filePath = new Path(testRootPath + "/" + fileName);
+
+ fs.close();
+
+ boolean closeException = false;
+ try {
+ fs.create(filePath, new FsPermission((short) 00644), false, 4096,
+ (short) 3, 128 * 1024 * 1024, null);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ fs.create(filePath, new FsPermission((short) 00644), null, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ fs.create(filePath);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ fs.create(filePath, null);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ fs.create(filePath, true);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ fs.create(filePath, (short) 3);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ fs.create(filePath, (short) 3, null);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ fs.create(filePath, true, 4096);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ fs.create(filePath, true, 4096, null);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ fs.create(filePath, true, 4096, (short) 3,
+ 128 * 1024 * 1024);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ fs.create(filePath, true, 4096, (short) 3,
+ 128 * 1024 * 1024, null);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // fs关闭后再执行CreateNonRecursive操作,抛出IOException
+ public void testCreateNonRecursiveAfterFSClose() throws IOException {
+ String fileName = "file";
+ Path filePath = new Path(testRootPath + "/" + fileName);
+
+ fs.close();
+
+ boolean closeException = false;
+ try {
+ fs.createNonRecursive(filePath, new FsPermission((short) 00644),
+ true, 4096, (short) 3, 128 * 1024 * 1024, null);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ fs.createNonRecursive(filePath, new FsPermission((short) 00644),
+ null, 4096, (short) 3, 128 * 1024 * 1024, null);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // fs关闭后再执行append操作,抛出IOException
+ public void testAppendAfterFSClose() throws IOException {
+ String fileName = "file";
+ Path filePath = new Path(testRootPath + "/" + fileName);
+
+ fs.close();
+
+ boolean closeException = false;
+ try {
+ fs.append(filePath, 4096, null);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ fs.append(filePath);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // fs关闭后再执行truncate操作,抛出IOException
+ public void testTruncateAfterFSClose() throws IOException {
+ String fileName = "file";
+ Path filePath = new Path(testRootPath + "/" + fileName);
+
+ fs.close();
+
+ boolean closeException = false;
+ try {
+ fs.truncate(filePath, 4096);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // fs关闭后再执行exists操作,抛出IOException
+ public void testExistsAfterFSClose() throws IOException {
+ String fileName = "file";
+ Path filePath = new Path(testRootPath + "/" + fileName);
+
+ fs.close();
+
+ boolean closeException = false;
+ try {
+ fs.exists(filePath);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // fs关闭后再执行rename操作,抛出IOException
+ public void testRenameAfterFSClose() throws IOException {
+ String fileNameFrom = "fileFrom";
+ Path filePathFrom = new Path(
+ testRootPath + "/" + fileNameFrom);
+ String fileNameTo = "fileTo";
+ Path filePathTo = new Path(
+ testRootPath + "/" + fileNameTo);
+
+ fs.close();
+
+ boolean closeException = false;
+ try {
+ fs.rename(filePathFrom, filePathTo);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // fs关闭后再执行delete操作,抛出IOException
+ public void testDeleteAfterFSClose() throws IOException {
+ String dirName = "dir";
+ Path dirPath = new Path(testRootPath + "/" + dirName);
+
+ fs.close();
+
+ boolean closeException = false;
+ try {
+ fs.delete(dirPath, true);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // fs关闭后再执行ListStatus操作,抛出IOException
+ public void testListStatusAfterFSClose() throws IOException {
+ String dirName = "dir";
+ Path dirPath = new Path(testRootPath + "/" + dirName);
+
+ fs.close();
+
+ boolean closeException = false;
+ try {
+ fs.listStatus(dirPath, true);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ fs.listStatus(dirPath);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // fs关闭后再执行Mkdirs操作,抛出IOException
+ public void testMkdirsAfterFSClose() throws IOException {
+ String dirName = "dir";
+ Path dirPath = new Path(testRootPath + "/" + dirName);
+
+ fs.close();
+
+ boolean closeException = false;
+ try {
+ fs.mkdirs(dirPath, new FsPermission((short) 00644));
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // fs关闭后再执行GetFileStatus操作,抛出IOException
+ public void testGetFileStatusAfterFSClose() throws IOException {
+ String fileName = "file";
+ Path filePath = new Path(testRootPath + "/" + fileName);
+
+ fs.close();
+
+ boolean closeException = false;
+ try {
+ fs.getFileStatus(filePath);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // fs关闭后再执行GetContentSummary操作,抛出IOException
+ public void testGetContentSummaryAfterFSClose() throws IOException {
+ String dirName = "dir";
+ Path dirPath = new Path(testRootPath + "/" + dirName);
+
+ fs.close();
+
+ boolean closeException = false;
+ try {
+ fs.getContentSummary(dirPath);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // fs关闭后再执行CopyFromLocalFile操作,抛出IOException
+ public void testCopyFromLocalFileAfterFSClose() throws IOException {
+ String fileNameLocal = "fileLocal";
+ Path filePathLocal = new Path("/usr/" + fileNameLocal);
+ String fileNameRemote = "fileRemote";
+ Path filePathRemote = new Path(
+ testRootPath + "/" + fileNameRemote);
+
+ fs.close();
+
+ boolean closeException = false;
+ try {
+ fs.copyFromLocalFile(true, true, filePathLocal, filePathRemote);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // fs关闭后再执行ListFiles操作,抛出IOException
+ public void testListFilesAfterFSClose() throws IOException {
+ String dirName = "dir";
+ Path dirPath = new Path(testRootPath + "/" + dirName);
+
+ fs.close();
+
+ boolean closeException = false;
+ try {
+ fs.listFiles(dirPath, true);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // fs关闭后再执行ListLocatedStatus操作,抛出IOException
+ public void testListLocatedStatusAfterFSClose() throws IOException {
+ String dirName = "dir";
+ Path dirPath = new Path(testRootPath + "/" + dirName);
+
+ fs.close();
+
+ boolean closeException = false;
+ try {
+ fs.listLocatedStatus(dirPath);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // fs关闭后再执行InputStream相关操作,抛出IOException
+ public void testInputStreamAfterFSClose() throws Exception {
+ final int bufLen = 256 * 1024;
+ final int sizeMB = 10;
+ String fileName = "readTestFile_" + sizeMB + ".txt";
+ Path readTestFile = new Path(
+ testRootPath + "/" + fileName);
+ long size = sizeMB * 1024 * 1024;
+
+ ContractTestUtils.generateTestFile(this.fs, readTestFile, size, 256,
+ 255);
+
+ FSDataInputStream instream = this.fs.open(readTestFile);
+
+ fs.close();
+
+ boolean closeException = false;
+ try {
+ instream.getPos();
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ instream.seek(1024);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ instream.seekToNewSource(1024);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ instream.read();
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ ByteBuffer byteBuffer = ByteBuffer.allocateDirect(bufLen);
+ instream.read(byteBuffer);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ byte[] buf = new byte[1024];
+ instream.read(buf, 0, 1024);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ instream.close();
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ instream.available();
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ byte[] buf = new byte[1024];
+ instream.readFully(1024, buf, 0, 1024);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ if (fs.getMetricSwitch()) {
+ MockMetricsConsumer mmc
+ = (MockMetricsConsumer) fs.getMetricsConsumer();
+ assertEquals(BasicMetricsConsumer.MetricKind.abnormal, mmc.getMr().getKind());
+ LOG.warn("metricInfo:"+mmc.getMr().getExceptionIns());
+ }
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ byte[] buf = new byte[1024];
+ instream.read(1024, buf, 0, 1024);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ instream.setReadahead(1024L);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // fs关闭后再执行OutputStream相关操作,抛出IOException
+ public void testOutStreamAfterFSClose() throws Exception {
+ String fileName = "file";
+ Path filePath = new Path(
+ testRootPath + "/" + fileName);
+ FSDataOutputStream outputStream = fs.create(filePath);
+ outputStream.write(10);
+
+ fs.close();
+
+ boolean closeException = false;
+ try {
+ outputStream.close();
+ } catch (IOException e) {
+ closeException = true;
+ }
+ assertFalse(closeException);
+
+ closeException = false;
+ try {
+ outputStream.flush();
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ outputStream.write(10);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ byte[] testBuffer = new byte[1024];
+
+ for (int i = 0; i < testBuffer.length; ++i) {
+ testBuffer[i] = (byte) (i % 255);
+ }
+ outputStream.write(testBuffer, 0 , 100);
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ outputStream.hsync();
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ outputStream.hflush();
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ outputStream.hsync();
+ } catch (IOException e) {
+ if (e.getMessage().equals("OBSFilesystem closed")) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // InputStream关闭后再执行InputStream相关操作,抛出IOException
+ public void testInputStreamClose() throws Exception {
+ final int bufLen = 256 * 1024;
+ final int sizeMB = 10;
+ String fileName = "readTestFile_" + sizeMB + ".txt";
+ Path readTestFile = new Path(
+ testRootPath + "/" + fileName);
+ long size = sizeMB * 1024 * 1024;
+
+ ContractTestUtils.generateTestFile(this.fs, readTestFile, size, 256,
+ 255);
+
+ FSDataInputStream instream = this.fs.open(readTestFile);
+
+ instream.close();
+
+ boolean closeException = false;
+ try {
+ instream.getPos();
+ } catch (IOException e) {
+ if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ instream.seek(1024);
+ } catch (IOException e) {
+ if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ instream.seekToNewSource(1024);
+ } catch (IOException e) {
+ if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ instream.read();
+ } catch (IOException e) {
+ if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ ByteBuffer byteBuffer = ByteBuffer.allocateDirect(bufLen);
+ instream.read(byteBuffer);
+ } catch (IOException e) {
+ if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ byte[] buf = new byte[1024];
+ instream.read(buf, 0, 1024);
+ } catch (IOException e) {
+ if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ instream.available();
+ } catch (IOException e) {
+ if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ byte[] buf = new byte[1024];
+ instream.readFully(1024, buf, 0, 1024);
+ } catch (IOException e) {
+ if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) {
+ closeException = true;
+ if (fs.getMetricSwitch()) {
+ MockMetricsConsumer mmc
+ = (MockMetricsConsumer) fs.getMetricsConsumer();
+ assertEquals(BasicMetricsConsumer.MetricKind.abnormal, mmc.getMr().getKind());
+ LOG.warn("metricInfo:"+mmc.getMr().getExceptionIns());
+ }
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ byte[] buf = new byte[1024];
+ instream.read(1024, buf, 0, 1024);
+ } catch (IOException e) {
+ if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) {
+ closeException = true;
+ if (fs.getMetricSwitch()) {
+ MockMetricsConsumer mmc
+ = (MockMetricsConsumer) fs.getMetricsConsumer();
+ assertEquals(BasicMetricsConsumer.MetricKind.abnormal, mmc.getMr().getKind());
+ LOG.warn("metricInfo:"+mmc.getMr().getExceptionIns());
+ }
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ instream.setReadahead(1024L);
+ } catch (IOException e) {
+ if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+
+ @Test
+ // OutputStream关闭后再执行OutputStream相关操作,抛出IOException
+ public void testOutStreamClose() throws Exception {
+ String fileName = "file";
+ Path filePath = new Path(
+ testRootPath + "/" + fileName);
+ FSDataOutputStream outputStream = fs.create(filePath);
+ outputStream.write(10);
+
+ outputStream.close();
+
+ boolean closeException = false;
+ try {
+ outputStream.flush();
+ } catch (IOException e) {
+ if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ outputStream.write(10);
+ } catch (IOException e) {
+ if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ byte[] testBuffer = new byte[1024];
+
+ for (int i = 0; i < testBuffer.length; ++i) {
+ testBuffer[i] = (byte) (i % 255);
+ }
+ outputStream.write(testBuffer, 0 , 100);
+ } catch (IOException e) {
+ if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ outputStream.hsync();
+ } catch (IOException e) {
+ if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ outputStream.hflush();
+ } catch (IOException e) {
+ if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+
+ closeException = false;
+ try {
+ outputStream.hsync();
+ } catch (IOException e) {
+ if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) {
+ closeException = true;
+ }
+ }
+ assertTrue(closeException);
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSConfiguration.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSConfiguration.java
new file mode 100644
index 0000000..7b398b0
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSConfiguration.java
@@ -0,0 +1,518 @@
+/**
+ * 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 com.obs.services.internal.ObsProperties;
+import com.obs.services.internal.RestConnectionService;
+import com.obs.services.internal.RestStorageService;
+
+import org.apache.commons.lang.reflect.FieldUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.security.ProviderUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.lang.reflect.Field;
+import java.net.URI;
+import java.security.PrivilegedExceptionAction;
+import java.util.Collection;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.apache.hadoop.fs.obs.OBSConstants.OBS_SECURITY_CREDENTIAL_PROVIDER_PATH;
+import static org.apache.hadoop.fs.obs.OBSConstants.USER_AGENT_PREFIX;
+import static org.apache.hadoop.fs.obs.OBSTestConstants.TEST_FS_OBS_NAME;
+import static org.apache.hadoop.fs.obs.OBSTestUtils.assertOptionEquals;
+import static org.apache.hadoop.fs.obs.OBSTestUtils.setBucketOption;
+import static org.apache.hadoop.fs.obs.OBSCommonUtils.CREDENTIAL_PROVIDER_PATH;
+import static org.apache.hadoop.fs.obs.OBSCommonUtils.patchSecurityCredentialProviders;
+import static org.apache.hadoop.fs.obs.OBSCommonUtils.propagateBucketOptions;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * OBS tests for configuration.
+ */
+public class ITestOBSConfiguration {
+ private static final String EXAMPLE_ID = "AKASOMEACCESSKEY";
+
+ private static final String EXAMPLE_KEY =
+ "RGV0cm9pdCBSZ/WQgY2xl/YW5lZCB1cAEXAMPLE";
+
+ private Configuration conf;
+
+ private OBSFileSystem fs;
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ITestOBSConfiguration.class);
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @Rule
+ public Timeout testTimeout = new Timeout(
+ OBSTestConstants.OBS_TEST_TIMEOUT
+ );
+
+ @Rule
+ public final TemporaryFolder tempDir = new TemporaryFolder();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ conf = OBSContract.getConfiguration(null);
+ fs = OBSTestUtils.createTestFileSystem(conf);;
+ }
+
+ @Test
+ // 使用无效的proxy host和port,创建fs会超时失败
+ public void testProxyConnection() throws Exception {
+ conf.setInt(OBSConstants.MAX_ERROR_RETRIES, 2);
+ conf.set(OBSConstants.PROXY_HOST, "127.0.0.1");
+ conf.setInt(OBSConstants.PROXY_PORT, 1);
+ String proxy =
+ conf.get(OBSConstants.PROXY_HOST) + ":" + conf.get(
+ OBSConstants.PROXY_PORT);
+ try {
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ fail("Expected a connection error for proxy server at " + proxy);
+ } catch (OBSIOException e) {
+ // expected
+ }
+ }
+
+ @Test
+ // proxy未指定host时,创建fs会超时失败
+ public void testProxyPortWithoutHost() throws Exception {
+ conf.setInt(OBSConstants.MAX_ERROR_RETRIES, 2);
+ conf.setInt(OBSConstants.PROXY_PORT, 1);
+ try {
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ fail("Expected a proxy configuration error");
+ } catch (OBSIOException e) {
+ String msg = e.toString();
+ if (!msg.contains(OBSConstants.PROXY_HOST) &&
+ !msg.contains(OBSConstants.PROXY_PORT)) {
+ //expected
+ }
+ }
+ }
+
+ void provisionAccessKeys(final Configuration conf) throws Exception {
+ // add our creds to the provider
+ final CredentialProvider provider =
+ CredentialProviderFactory.getProviders(conf).get(0);
+ provider.createCredentialEntry(OBSConstants.ACCESS_KEY,
+ EXAMPLE_ID.toCharArray());
+ provider.createCredentialEntry(OBSConstants.SECRET_KEY,
+ EXAMPLE_KEY.toCharArray());
+ provider.flush();
+ }
+
+ // 从userInfo中解析credentials,能正确解析出ak、sk
+ public void testCredsFromUserInfo() throws Exception {
+ final File file = tempDir.newFile("test.jks");
+ final URI jks = ProviderUtils.nestURIForLocalJavaKeyStoreProvider(
+ file.toURI());
+ conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
+ jks.toString());
+
+ LOG.warn("file URI: {}, jks: {}", file.toURI(), jks.toString());
+ System.out.println("file URI: " + file.toURI() + ", jks: " + "jceks://obs/foobar," + jks.toString());
+
+ LOG.warn("provider: {}", conf.get(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH));
+ System.out.println("provider: " + conf.get(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH));
+
+ try {
+ provisionAccessKeys(conf);
+ } catch (Exception e) {
+ e.printStackTrace();
+ LOG.error(e.getMessage());
+ System.exit(-1);
+ }
+
+ conf.set(OBSConstants.ACCESS_KEY, EXAMPLE_ID + "LJM");
+ URI uriWithUserInfo = new URI("obs://123:456@foobar");
+ OBSLoginHelper.Login creds =
+ OBSCommonUtils.getOBSAccessKeys(uriWithUserInfo, conf);
+ assertEquals("AccessKey incorrect.", "123", creds.getUser());
+ assertEquals("SecretKey incorrect.", "456", creds.getPassword());
+ }
+
+ // 测试可正常排除不兼容的S3ACredentialProvider
+ public void testExcludingS3ACredentialProvider() throws Exception {
+ // set up conf to have a cred provider
+ final File file = tempDir.newFile("test.jks");
+ final URI jks = ProviderUtils.nestURIForLocalJavaKeyStoreProvider(
+ file.toURI());
+ conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
+ "jceks://obs/foobar," + jks.toString());
+
+ LOG.warn("file URI: {}, jks: {}", file.toURI(), jks.toString());
+ System.out.println("file URI: " + file.toURI() + ", jks: " + "jceks://obs/foobar," + jks.toString());
+
+ LOG.warn("provider: {}", conf.get(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH));
+ System.out.println("provider: " + conf.get(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH));
+
+ // first make sure that the obs based provider is removed
+ Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders(
+ conf, OBSFileSystem.class);
+ String newPath = conf.get(
+ CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH);
+ assertFalse("Provider Path incorrect", newPath.contains("obs://"));
+
+ LOG.warn("after exclude, newPath: {}", newPath);
+ System.out.println("after exclude, newPath: " + newPath);
+
+ // now let's make sure the new path is created by the OBSFileSystem
+ // and the integration still works. Let's provision the keys through
+ // the altered configuration instance and then try and access them
+ // using the original config with the obs provider in the path.
+ try {
+ provisionAccessKeys(c);
+ } catch (Exception e) {
+ e.printStackTrace();
+ System.exit(-1);
+ }
+
+ conf.set(OBSConstants.ACCESS_KEY, EXAMPLE_ID + "LJM");
+ URI uriWithUserInfo = new URI("obs://123:456@foobar");
+ OBSLoginHelper.Login creds =
+ OBSCommonUtils.getOBSAccessKeys(uriWithUserInfo, conf);
+ assertEquals("AccessKey incorrect.", "123", creds.getUser());
+ assertEquals("SecretKey incorrect.", "456", creds.getPassword());
+
+ }
+
+ // @Test
+ // public void testDirectoryAllocatorRR() throws Throwable {
+ // String localDir1 = "local_dir1";
+ // OBSFSTestUtil.createLocalTestDir(localDir1);
+ // String localDir2 = "local_dir2";
+ // OBSFSTestUtil.createLocalTestDir(localDir2);
+ // File dir1 = new File(localDir1);
+ // File dir2 = new File(localDir2);
+ // dir1.mkdirs();
+ // dir2.mkdirs();
+ // conf.set(OBSConstants.BUFFER_DIR, dir1 + ", " + dir2);
+ // fs = OBSTestUtils.createTestFileSystem(conf);
+ // File tmp1 = createTmpFileForWrite("out-", 1024, conf);
+ // tmp1.delete();
+ // File tmp2 = createTmpFileForWrite("out-", 1024, conf);
+ // tmp2.delete();
+ // assertNotEquals("round robin not working",
+ // tmp1.getParent(), tmp2.getParent());
+ //
+ // File tmp = createTmpFileForWrite("out-", 1024, conf);
+ // assertTrue("not found: " + tmp, tmp.exists());
+ // tmp.delete();
+ // }
+
+ @Test
+ // 测试readahead.range可正常配置
+ public void testReadAheadRange() throws Exception {
+ conf.set(OBSConstants.READAHEAD_RANGE, "300K");
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ assertNotNull(fs);
+ long readAheadRange = fs.getReadAheadRange();
+ assertNotNull(readAheadRange);
+ assertEquals("Read Ahead Range Incorrect.", 300 * 1024, readAheadRange);
+ }
+
+ @Test
+ // 测试可正常获取UGI信息
+ public void testUsernameFromUGI() throws Throwable {
+ final String alice = "alice";
+ UserGroupInformation fakeUser =
+ UserGroupInformation.createUserForTesting(alice,
+ new String[] {"users", "administrators"});
+ fs = fakeUser.doAs(new PrivilegedExceptionAction() {
+ @Override
+ public OBSFileSystem run() throws Exception {
+ return OBSTestUtils.createTestFileSystem(conf);
+ }
+ });
+ assertEquals("username", alice, fs.getShortUserName());
+ FileStatus status = fs.getFileStatus(new Path("/"));
+ assertEquals("owner in " + status, alice, status.getOwner());
+ assertEquals("group in " + status, alice, status.getGroup());
+ }
+
+ /**
+ * Reads and returns a field from an object using reflection. If the field
+ * cannot be found, is null, or is not the expected type, then this method
+ * fails the test.
+ *
+ * @param target object to read
+ * @param fieldType type of field to read, which will also be the return type
+ * @param fieldName name of field to read
+ * @return field that was read
+ * @throws IllegalAccessException if access not allowed
+ */
+ private static T getField(Object target, Class fieldType,
+ String fieldName) throws IllegalAccessException {
+ Object obj = FieldUtils.readField(target, fieldName, true);
+ assertNotNull(String.format(
+ "Could not read field named %s in object with class %s.", fieldName,
+ target.getClass().getName()), obj);
+ assertTrue(String.format(
+ "Unexpected type found for field named %s, expected %s, actual %s.",
+ fieldName, fieldType.getName(), obj.getClass().getName()),
+ fieldType.isAssignableFrom(obj.getClass()));
+ return fieldType.cast(obj);
+ }
+
+ @Test
+ // 测试可正常将桶特定配置转化成通用obs配置
+ public void testBucketConfigurationPropagation() throws Throwable {
+ Configuration config = new Configuration(false);
+ setBucketOption(config, "b", "base", "1024");
+ String basekey = "fs.obs.base";
+ assertOptionEquals(config, basekey, null);
+ String bucketKey = "fs.obs.bucket.b.base";
+ assertOptionEquals(config, bucketKey, "1024");
+ Configuration updated = propagateBucketOptions(config, "b");
+ assertOptionEquals(updated, basekey, "1024");
+ // original conf is not updated
+ assertOptionEquals(config, basekey, null);
+
+ String[] sources = updated.getPropertySources(basekey);
+ assertEquals(1, sources.length);
+ String sourceInfo = sources[0];
+ assertTrue("Wrong source " + sourceInfo,
+ sourceInfo.contains(bucketKey));
+ }
+
+ @Test
+ // 测试可正常解析配置文件中定义的变量字段
+ public void testBucketConfigurationPropagationResolution()
+ throws Throwable {
+ Configuration config = new Configuration(false);
+ String basekey = "fs.obs.base";
+ String baseref = "fs.obs.baseref";
+ String baseref2 = "fs.obs.baseref2";
+ config.set(basekey, "orig");
+ config.set(baseref2, "${fs.obs.base}");
+ setBucketOption(config, "b", basekey, "1024");
+ setBucketOption(config, "b", baseref, "${fs.obs.base}");
+ Configuration updated = propagateBucketOptions(config, "b");
+ assertOptionEquals(updated, basekey, "1024");
+ assertOptionEquals(updated, baseref, "1024");
+ assertOptionEquals(updated, baseref2, "1024");
+ }
+
+ @Test
+ // key重复的桶配置项,解析到的值为最后一个配置项的值
+ public void testMultipleBucketConfigurations() throws Throwable {
+ Configuration config = new Configuration(false);
+ setBucketOption(config, "b", USER_AGENT_PREFIX, "UA-b");
+ setBucketOption(config, "c", USER_AGENT_PREFIX, "UA-c");
+ config.set(USER_AGENT_PREFIX, "UA-orig");
+ Configuration updated = propagateBucketOptions(config, "c");
+ assertOptionEquals(updated, USER_AGENT_PREFIX, "UA-c");
+ }
+
+ @Test
+ // 传递桶特定配置到OBS通用配置时,不会覆盖OBS通用配置中相同配置项
+ public void testBucketConfigurationSkipsUnmodifiable() throws Throwable {
+ Configuration config = new Configuration(false);
+ String impl = "fs.obs.impl";
+ config.set(impl, "orig");
+ setBucketOption(config, "b", impl, "b");
+ String metastoreImpl = "fs.obs.metadatastore.impl";
+ String ddb = "org.apache.hadoop.fs.obs.s3guard.DynamoDBMetadataStore";
+ setBucketOption(config, "b", metastoreImpl, ddb);
+ setBucketOption(config, "b", "impl2", "b2");
+ setBucketOption(config, "b", "bucket.b.loop", "b3");
+ assertOptionEquals(config, "fs.obs.bucket.b.impl", "b");
+
+ Configuration updated = propagateBucketOptions(config, "b");
+ assertOptionEquals(updated, impl, "orig");
+ assertOptionEquals(updated, "fs.obs.impl2", "b2");
+ assertOptionEquals(updated, metastoreImpl, ddb);
+ assertOptionEquals(updated, "fs.obs.bucket.b.loop", null);
+ }
+
+ @Test
+ // 桶特定配置传递到OBS通用配置中,fs能正确识别到
+ public void testConfOptionPropagationToFS() throws Exception {
+ String testFSName = conf.getTrimmed(TEST_FS_OBS_NAME, "");
+ String bucket = new URI(testFSName).getHost();
+ setBucketOption(conf, bucket, "propagation", "propagated");
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ Configuration updated = fs.getConf();
+ assertOptionEquals(updated, "fs.obs.propagation", "propagated");
+ }
+
+ @Test
+ // 添加的obs CredentialProviders不会覆盖hadoop CredentialProviders
+ public void testSecurityCredentialPropagationNoOverride() {
+ conf.set(CREDENTIAL_PROVIDER_PATH, "base");
+ patchSecurityCredentialProviders(conf);
+ assertOptionEquals(conf, CREDENTIAL_PROVIDER_PATH,
+ "base");
+ }
+
+ @Test
+ // 未设置hadoop CredentialProviders时,添加的obs CredentialProviders为全集
+ public void testSecurityCredentialPropagationOverrideNoBase() {
+ conf.unset(CREDENTIAL_PROVIDER_PATH);
+ conf.set(OBS_SECURITY_CREDENTIAL_PROVIDER_PATH, "override");
+ patchSecurityCredentialProviders(conf);
+ assertOptionEquals(conf, CREDENTIAL_PROVIDER_PATH,
+ "override");
+ }
+
+ @Test
+ // 同时设置hadoop CredentialProviders和obs CredentialProviders时,全集为两者集合
+ public void testSecurityCredentialPropagationOverride() {
+ conf.set(CREDENTIAL_PROVIDER_PATH, "base");
+ conf.set(OBS_SECURITY_CREDENTIAL_PROVIDER_PATH, "override");
+ patchSecurityCredentialProviders(conf);
+ assertOptionEquals(conf, CREDENTIAL_PROVIDER_PATH,
+ "override,base");
+ Collection all = conf.getStringCollection(
+ CREDENTIAL_PROVIDER_PATH);
+ assertTrue(all.contains("override"));
+ assertTrue(all.contains("base"));
+ }
+
+ @Test
+ // 可将桶特定CredentialProviders配置传递到OBS通用CredentialProviders配置中
+ public void testSecurityCredentialPropagationEndToEnd() {
+ conf.set(CREDENTIAL_PROVIDER_PATH, "base");
+ setBucketOption(conf, "b", OBS_SECURITY_CREDENTIAL_PROVIDER_PATH,
+ "override");
+ Configuration updated = propagateBucketOptions(conf, "b");
+ patchSecurityCredentialProviders(updated);
+ assertOptionEquals(updated, CREDENTIAL_PROVIDER_PATH,
+ "override,base");
+ }
+
+ @Test
+ // 测试可正确配置鉴权协商开关
+ public void testSetAuthTypeNegotiation()
+ throws Exception, NoSuchFieldException, SecurityException {
+ Configuration conf = OBSContract.getConfiguration(null);
+
+ Pattern pattern = Pattern.compile(
+ "^((2[0-4]\\d|25[0-5]|[01]?\\d\\d?)\\.){3}(2[0-4]\\d|25[0-5]|[01]?\\d\\d?)$");
+ Matcher m = pattern.matcher(conf.get(OBSConstants.ENDPOINT));
+
+ // 1、set to true, when endpoint is IP Address, authTypeNegotiationEnabled should be false;
+ // otherwise should be true
+ conf.set(OBSConstants.SDK_AUTH_TYPE_NEGOTIATION_ENABLE, "true");
+ OBSFileSystem obsFs = OBSTestUtils.createTestFileSystem(conf);
+ RestStorageService client = obsFs.getObsClient();
+ Field propertiesField = RestConnectionService.class.getDeclaredField(
+ "obsProperties");
+
+ propertiesField.setAccessible(true);
+ ObsProperties properties = (ObsProperties) propertiesField.get(client);
+ String authTypeNegotiationEnabled = properties
+ .getStringProperty("httpclient.auth-type-negotiation", "false");
+ if (m.matches()) {
+ assertTrue(authTypeNegotiationEnabled.equalsIgnoreCase("false"));
+ } else {
+ assertTrue(authTypeNegotiationEnabled.equalsIgnoreCase("true"));
+ }
+ obsFs.close();
+
+ // 2、set to false, authTypeNegotiationEnabled should be false
+ conf.set(OBSConstants.SDK_AUTH_TYPE_NEGOTIATION_ENABLE, "false");
+ obsFs = OBSTestUtils.createTestFileSystem(conf);
+ client = (RestStorageService) obsFs.getObsClient();
+ properties = (ObsProperties) propertiesField.get(client);
+ authTypeNegotiationEnabled = properties
+ .getStringProperty("httpclient.auth-type-negotiation", "false");
+ assertTrue(authTypeNegotiationEnabled.equalsIgnoreCase("false"));
+ obsFs.close();
+
+ // 3、set to other value, authTypeNegotiationEnabled should be default value false
+ conf.set(OBSConstants.SDK_AUTH_TYPE_NEGOTIATION_ENABLE, "invalid");
+ obsFs = OBSTestUtils.createTestFileSystem(conf);
+ client = (RestStorageService) obsFs.getObsClient();
+ properties = (ObsProperties) propertiesField.get(client);
+ authTypeNegotiationEnabled = properties
+ .getStringProperty("httpclient.auth-type-negotiation", "false");
+ assertTrue(authTypeNegotiationEnabled.equalsIgnoreCase("false"));
+ obsFs.close();
+
+ // 4、unset, authTypeNegotiationEnabled should be false
+ conf.unset(OBSConstants.SDK_AUTH_TYPE_NEGOTIATION_ENABLE);
+ obsFs = OBSTestUtils.createTestFileSystem(conf);
+ client = (RestStorageService) obsFs.getObsClient();
+ properties = (ObsProperties) propertiesField.get(client);
+ authTypeNegotiationEnabled = properties
+ .getStringProperty("httpclient.auth-type-negotiation", "false");
+ assertTrue(authTypeNegotiationEnabled.equalsIgnoreCase("false"));
+ obsFs.close();
+ }
+
+ // @Test
+ // // 测试有无CanonicalServiceName配置项时,getCanonicalServiceName能返回正确的值
+ // public void testGetCanonicalServiceName() throws Exception {
+ // // 1、未配置CanonicalServiceName开关时,接口默认返回null
+ // Configuration conf = OBSContract.getConfiguration(null);
+ // OBSFileSystem obsFs = OBSTestUtils.createTestFileSystem(conf);
+ // String canonicalServiceName = obsFs.getCanonicalServiceName();
+ // assertEquals("expected null, but get " + canonicalServiceName,
+ // null, canonicalServiceName);
+ // obsFs.close();
+ //
+ // // 2、配置CanonicalServiceName开关为开启时,接口返回字符串obs://{bucketName}
+ // conf.setBoolean(OBSConstants.GET_CANONICAL_SERVICE_NAME_ENABLE, true);
+ // obsFs = OBSTestUtils.createTestFileSystem(conf);
+ // canonicalServiceName = obsFs.getCanonicalServiceName();
+ // String expected = obsFs.getScheme() + "://" + obsFs.getBucket();
+ // assertEquals("expected " + expected + " but get " + canonicalServiceName,
+ // expected, canonicalServiceName);
+ // obsFs.close();
+ //
+ // // 3、配置CanonicalServiceName开关为关闭时,接口返回null
+ // conf.setBoolean(OBSConstants.GET_CANONICAL_SERVICE_NAME_ENABLE, false);
+ // obsFs = OBSTestUtils.createTestFileSystem(conf);
+ // canonicalServiceName = obsFs.getCanonicalServiceName();
+ // assertEquals("expected null, but get " + canonicalServiceName,
+ // null, canonicalServiceName);
+ // obsFs.close();
+ // }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCopyFromLocalFile.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCopyFromLocalFile.java
new file mode 100644
index 0000000..02abc3a
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCopyFromLocalFile.java
@@ -0,0 +1,497 @@
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class ITestOBSCopyFromLocalFile {
+ private OBSFileSystem fs;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.setClass(OBSConstants.OBS_METRICS_CONSUMER,
+ MockMetricsConsumer.class, BasicMetricsConsumer.class);
+ conf.setBoolean(OBSConstants.METRICS_SWITCH, true);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(new Path(testRootPath), true);
+ }
+ }
+
+ private Path getTestPath(String relativePath) {
+ return new Path(testRootPath + "/" + relativePath);
+ }
+
+ @Test
+ // src是文件,dst是文件,delSrc为true,overwrite为true, dst已存在;src文件大小为0,copy后桶内文件大小也为0
+ public void testCopyFromLocalFileNormal001() throws Exception {
+ String localFile = "local_file";
+ String destFile = "test_file";
+ copyFromLocalFile(localFile, destFile, 0);
+ assertTrue(fs.getFileStatus(getTestPath(destFile)).getLen() == 0);
+ OBSFSTestUtil.deleteLocalFile("local_file");
+ }
+
+ @Test
+ // src是文件,dst是文件,delSrc为true,overwrite为true, dst已存在;src文件长度小于段大小,为100B,copy后桶内文件大小也为100B
+ public void testCopyFromLocalFileNormal002() throws Exception {
+ String localFile = "local_file";
+ String destFile = "test_file";
+ copyFromLocalFile(localFile, destFile, 100);
+ assertTrue(fs.getFileStatus(getTestPath(destFile)).getLen() == 100);
+ OBSFSTestUtil.deleteLocalFile("local_file");
+ }
+
+ @Test
+ // src是文件,dst是文件,delSrc为true,overwrite为true, dst已存在;src文件长度小于段大小,为5M,copy后桶内文件大小也为5M
+ public void testCopyFromLocalFileNormal003() throws Exception {
+ String localFile = "local_file";
+ String destFile = "test_file";
+ copyFromLocalFile(localFile, destFile, 5 * 1024 * 1024);
+ assertEquals(5 * 1024 * 1024,
+ fs.getFileStatus(getTestPath(destFile)).getLen());
+ OBSFSTestUtil.deleteLocalFile("local_file");
+ }
+
+ @Test
+ // src是文件,dst是文件,delSrc为true,overwrite为true, dst已存在;src文件长度等于段大小,为100M,copy后桶内文件大小也为100M
+ public void testCopyFromLocalFileNormal004() throws Exception {
+ String localFile = "local_file";
+ String destFile = "test_file";
+ copyFromLocalFile(localFile, destFile, 100 * 1024 * 1024);
+ assertEquals(100 * 1024 * 1024,
+ fs.getFileStatus(getTestPath(destFile)).getLen());
+ OBSFSTestUtil.deleteLocalFile("local_file");
+ }
+
+ @Test
+ // src是文件,dst是文件,delSrc为true,overwrite为true, dst已存在;src文件长度大于段大小,为120M,copy后桶内文件大小也为120M
+ public void testCopyFromLocalFileNormal005() throws Exception {
+ String localFile = "local_file";
+ String destFile = "test_file";
+ copyFromLocalFile(localFile, destFile, 120 * 1024 * 1024);
+ assertEquals(120 * 1024 * 1024,
+ fs.getFileStatus(getTestPath(destFile)).getLen());
+ OBSFSTestUtil.deleteLocalFile("local_file");
+ }
+
+ @Test
+ // src是文件,dst是文件,delSrc为true,overwrite为false, dst不存在
+ public void testCopyFromLocalFileNormal006() throws Exception {
+ String localFile = "local_file";
+ new File(testRootPath, "." + localFile + ".crc").delete();
+ OBSFSTestUtil.createLocalTestFile(localFile);
+
+ Path dstPath = getTestPath("test_file");
+ OBSFSTestUtil.deletePathRecursive(fs, dstPath);
+
+ fs.copyFromLocalFile(true, false, new Path(localFile), dstPath);
+ OBSFSTestUtil.assertPathExistence(fs, dstPath, true);
+ OBSFSTestUtil.deleteLocalFile(localFile);
+ }
+
+ @Test
+ // src是文件,dst是文件,delSrc为true,overwrite为true, dst不存在
+ public void testCopyFromLocalFileNormal007() throws Exception {
+ String localFile = "local_file";
+ new File(testRootPath, "." + localFile + ".crc").delete();
+ OBSFSTestUtil.createLocalTestFile(localFile);
+
+ Path dstPath = getTestPath("test_file");
+ OBSFSTestUtil.deletePathRecursive(fs, dstPath);
+
+ fs.copyFromLocalFile(false, true, new Path(localFile), dstPath);
+ OBSFSTestUtil.assertPathExistence(fs, dstPath, true);
+ OBSFSTestUtil.deleteLocalFile(localFile);
+ }
+
+ @Test
+ // src是文件,dst是根目录,文件拷贝到根目录下
+ public void testCopyFromLocalFileNormal008() throws Exception {
+ String localFile = "local_file";
+ new File(testRootPath, "." + localFile + ".crc").delete();
+ OBSFSTestUtil.createLocalTestFile(localFile);
+
+ Path dstPath = new Path("/");
+ fs.copyFromLocalFile(false, true, new Path(localFile), dstPath);
+ OBSFSTestUtil.assertPathExistence(fs, new Path("/" + localFile), true);
+
+ OBSFSTestUtil.deletePathRecursive(fs, new Path("/" + localFile));
+ OBSFSTestUtil.deleteLocalFile(localFile);
+ }
+
+ @Test
+ // src是文件,dst是多级目录,文件拷贝到多级目录下
+ public void testCopyFromLocalFileNormal009() throws Exception {
+ String localFile = "local_file";
+ new File(testRootPath, "." + localFile + ".crc").delete();
+ OBSFSTestUtil.createLocalTestFile(localFile);
+
+ Path dstPath = getTestPath("a001/b001");
+ fs.mkdirs(dstPath);
+ fs.copyFromLocalFile(false, true, new Path(localFile), dstPath);
+ OBSFSTestUtil.assertPathExistence(fs,
+ getTestPath("a001/b001/" + localFile), true);
+
+ OBSFSTestUtil.deleteLocalFile(localFile);
+ }
+
+ @Test
+ // src是空目录,dst是目录,dst不存在,创建dst,将源目录child列表拷贝到dst下
+ public void testCopyFromLocalFileNormal010() throws Exception {
+ String localDir = "local_dir";
+ String localFile = "local_dir/local_file";
+ OBSFSTestUtil.createLocalTestDir(localDir);
+ new File(testRootPath, "." + localFile + ".crc").delete();
+ OBSFSTestUtil.createLocalTestFile(localFile);
+
+ Path dstPath = getTestPath("a001/b001");
+ fs.mkdirs(dstPath.getParent());
+ fs.copyFromLocalFile(false, true, new Path(localDir), dstPath);
+ assertTrue(fs.listStatus(dstPath).length == 1);
+ OBSFSTestUtil.deleteLocalFile(localDir);
+ fs.delete(dstPath.getParent(), true);
+ }
+
+ @Test
+ // src是非空目录,dst是目录,dst不存在,创建dst,将源目录child列表拷贝到dst下
+ public void testCopyFromLocalFileNormal011() throws Exception {
+ String localDir = "local_dir";
+ OBSFSTestUtil.createLocalTestDir(localDir);
+
+ Path dstPath = getTestPath("a001/b001");
+ fs.mkdirs(dstPath.getParent());
+ fs.copyFromLocalFile(false, true, new Path(localDir), dstPath);
+ assertTrue(fs.listStatus(dstPath).length == 0);
+ OBSFSTestUtil.deleteLocalFile(localDir);
+ }
+
+ @Test
+ // src是目录,dst是根目录,根目录下不存在与src同名的子目录,正常
+ public void testCopyFromLocalFileNormal012() throws Exception {
+ String localDir = "local_dir";
+ OBSFSTestUtil.createLocalTestDir(localDir);
+
+ Path dstPath = new Path("/");
+ fs.mkdirs(dstPath);
+ boolean hasException = false;
+ try {
+ fs.copyFromLocalFile(false, true, new Path(localDir), dstPath);
+ } catch (IOException e) {
+ hasException = true;
+ }
+ assertFalse(hasException);
+
+ OBSFSTestUtil.deleteLocalFile(localDir);
+ fs.delete(new Path("/local_dir"), true);
+ }
+
+ @Test
+ // src是目录,dst是多级空目录,dst已存在,正常
+ public void testCopyFromLocalFileNormal013() throws Exception {
+ String localDir = "local_dir";
+ OBSFSTestUtil.createLocalTestDir(localDir);
+
+ Path dstPath = getTestPath("a001/b001");
+ fs.mkdirs(dstPath);
+ boolean hasException = false;
+ try {
+ fs.copyFromLocalFile(false, true, new Path(localDir), dstPath);
+ } catch (IOException e) {
+ hasException = true;
+ }
+ assertFalse(hasException);
+ OBSFSTestUtil.deleteLocalFile(localDir);
+ }
+
+ @Test
+ // src是文件,dst是文件,overwrite为false,dst已存在,抛FileAlreadyExistsException
+ public void testCopyFromLocalFileAbnormal001() throws Exception {
+ String localFile = "local_file";
+ new File(testRootPath, "." + localFile + ".crc").delete();
+ OBSFSTestUtil.createLocalTestFile(localFile);
+
+ Path dstPath = getTestPath("test_file");
+ OBSFSTestUtil.deletePathRecursive(fs, dstPath);
+ FSDataOutputStream outputStream = fs.create(dstPath, false);
+ outputStream.close();
+ OBSFSTestUtil.assertPathExistence(fs, dstPath, true);
+
+ boolean hasException = false;
+ try {
+ fs.copyFromLocalFile(false, false, new Path(localFile), dstPath);
+ } catch (IOException e) {
+ hasException = true;
+ }
+ assertTrue(hasException);
+
+ fs.delete(dstPath, true);
+ OBSFSTestUtil.deleteLocalFile(localFile);
+ }
+
+ @Test
+ // src不存在,抛FileNotFoundException
+ public void testCopyFromLocalFileAbnormal002() throws Exception {
+ String localFile = "local_file";
+ OBSFSTestUtil.deleteLocalFile(localFile);
+
+ Path dstPath = getTestPath("test_file");
+
+ boolean hasException = false;
+ try {
+ fs.copyFromLocalFile(false, true, new Path(localFile), dstPath);
+ } catch (FileNotFoundException e) {
+ hasException = true;
+ }
+ assertTrue(hasException);
+
+ fs.delete(dstPath, true);
+ OBSFSTestUtil.deleteLocalFile(localFile);
+ }
+
+ @Test
+ // 测试基类copyFromLocalFile(boolean delSrc, boolean overwrite, Path[] srcs, Path dst)方法
+ public void testCplParentMethod01() throws Exception {
+ String localFile1 = "local_file1";
+ String localFile2 = "local_file2";
+ new File(testRootPath, "." + localFile1 + ".crc").delete();
+ new File(testRootPath, "." + localFile2 + ".crc").delete();
+ OBSFSTestUtil.createLocalTestFile(localFile1);
+ OBSFSTestUtil.createLocalTestFile(localFile2);
+ Path[] srcFiles = {new Path(localFile1), new Path(localFile2)};
+
+ Path dstDir = getTestPath("test_dir");
+ fs.mkdirs(dstDir);
+ fs.copyFromLocalFile(true, false, srcFiles, dstDir);
+ OBSFSTestUtil.assertPathExistence(fs,
+ getTestPath("test_dir/local_file1"), true);
+ OBSFSTestUtil.assertPathExistence(fs,
+ getTestPath("test_dir/local_file2"), true);
+
+ fs.delete(getTestPath("test_dir"), true);
+ OBSFSTestUtil.deleteLocalFile(localFile1);
+ OBSFSTestUtil.deleteLocalFile(localFile2);
+ }
+
+ @Test
+ // 测试基类copyFromLocalFile(boolean delSrc, Path src, Path dst)方法,overwrite始终为true
+ public void testCplParentMethod02() throws Exception {
+ String localFile = "local_file";
+ new File(testRootPath, "." + localFile + ".crc").delete();
+ File file = OBSFSTestUtil.createLocalTestFile(localFile);
+ OBSFSTestUtil.writeLocalFile(file, 16);
+
+ String dstFile = "test_file";
+ FSDataOutputStream outputStream = fs.create(getTestPath(dstFile),
+ false);
+ outputStream.close();
+
+ fs.copyFromLocalFile(true, new Path(localFile), getTestPath(dstFile));
+ assertEquals(16, fs.getFileStatus(getTestPath(dstFile)).getLen());
+ assertFalse(file.exists());
+
+ fs.delete(getTestPath(dstFile), true);
+ }
+
+ @Test
+ // 测试基类copyFromLocalFile(Path src, Path dst)方法, overwrite为true, desSrc为false
+ public void testCplParentMethod03() throws Exception {
+ String localFile = "local_file";
+ new File(testRootPath, "." + localFile + ".crc").delete();
+ File file = OBSFSTestUtil.createLocalTestFile(localFile);
+ OBSFSTestUtil.writeLocalFile(file, 16);
+
+ String dstFile = "test_file";
+ FSDataOutputStream outputStream = fs.create(getTestPath(dstFile),
+ false);
+ outputStream.close();
+
+ fs.copyFromLocalFile(new Path(localFile), getTestPath(dstFile));
+ assertEquals(16, fs.getFileStatus(getTestPath(dstFile)).getLen());
+ assertTrue(file.exists());
+
+ fs.delete(getTestPath(dstFile), true);
+ OBSFSTestUtil.deleteLocalFile(localFile);
+ }
+
+ @Test
+ // 测试基类moveFromLocalFile(Path[] srcs, Path dst)方法,执行copyFromLocalFile(true, true, srcs, dst)
+ public void testCplParentMethod04() throws Exception {
+ String localFile1 = "local_file1";
+ String localFile2 = "local_file2";
+ new File(testRootPath, "." + localFile1 + ".crc").delete();
+ new File(testRootPath, "." + localFile2 + ".crc").delete();
+ File file1 = OBSFSTestUtil.createLocalTestFile(localFile1);
+ File file2 = OBSFSTestUtil.createLocalTestFile(localFile2);
+ OBSFSTestUtil.writeLocalFile(file1, 16);
+ OBSFSTestUtil.writeLocalFile(file2, 16);
+ Path[] srcFiles = {new Path(localFile1), new Path(localFile2)};
+
+ String dstDir = "dest_dir";
+ fs.mkdirs(getTestPath(dstDir));
+
+ fs.moveFromLocalFile(srcFiles, getTestPath(dstDir));
+ OBSFSTestUtil.assertPathExistence(fs,
+ getTestPath("dest_dir/local_file1"), true);
+ OBSFSTestUtil.assertPathExistence(fs,
+ getTestPath("dest_dir/local_file2"), true);
+ assertFalse(file1.exists());
+ assertFalse(file2.exists());
+
+ fs.delete(getTestPath(dstDir), true);
+ }
+
+ @Test
+ // 测试基类moveFromLocalFile(Path src, Path dst)方法,执行copyFromLocalFile(delSrc = true, src, dst)
+ public void testCplParentMethod05() throws Exception {
+ String localFile = "local_file";
+ new File(testRootPath, "." + localFile + ".crc").delete();
+ File file = OBSFSTestUtil.createLocalTestFile(localFile);
+ OBSFSTestUtil.writeLocalFile(file, 16);
+
+ String dstFile = "test_file";
+ FSDataOutputStream outputStream = fs.create(getTestPath(dstFile),
+ false);
+ outputStream.close();
+
+ fs.moveFromLocalFile(new Path(localFile), getTestPath(dstFile));
+ assertEquals(16, fs.getFileStatus(getTestPath(dstFile)).getLen());
+ assertFalse(file.exists());
+
+ fs.delete(getTestPath(dstFile), true);
+ }
+
+ @Test
+ // 测试基类copyToLocalFile(boolean delSrc, Path src, Path dst, boolean useRawLocalFileSystem)方法
+ // overwrite始终为true
+ public void testCplParentMethod06() throws Exception {
+ String localFile = "local_file";
+ new File(testRootPath, "." + localFile + ".crc").delete();
+ File file = OBSFSTestUtil.createLocalTestFile(localFile);
+ OBSFSTestUtil.writeLocalFile(file, 16);
+
+ String srcFile = "test_file";
+ FSDataOutputStream outputStream = fs.create(getTestPath(srcFile),
+ false);
+ outputStream.close();
+
+ fs.copyToLocalFile(true, getTestPath(srcFile), new Path(localFile),
+ true);
+ assertEquals(0, file.length());
+ OBSFSTestUtil.assertPathExistence(fs, getTestPath(srcFile), false);
+
+ OBSFSTestUtil.deleteLocalFile(localFile);
+ fs.delete(getTestPath(srcFile), true);
+ }
+
+ @Test
+ // 测试基类copyToLocalFile(boolean delSrc, Path src, Path dst)方法,useRawLocalFileSystem为false
+ public void testCplParentMethod07() throws Exception {
+ String localFile = "local_file";
+ new File(testRootPath, "." + localFile + ".crc").delete();
+ File file = OBSFSTestUtil.createLocalTestFile(localFile);
+ OBSFSTestUtil.writeLocalFile(file, 16);
+
+ String srcFile = "test_file";
+ FSDataOutputStream outputStream = fs.create(getTestPath(srcFile),
+ false);
+ outputStream.close();
+
+ fs.copyToLocalFile(true, getTestPath(srcFile), new Path(localFile));
+ assertEquals(0, file.length());
+ OBSFSTestUtil.assertPathExistence(fs, getTestPath(srcFile), false);
+
+ OBSFSTestUtil.deleteLocalFile(localFile);
+ fs.delete(getTestPath(srcFile), true);
+ }
+
+ @Test
+ // 测试基类copyToLocalFile(Path src, Path dst)方法, delSrc为false,useRawLocalFileSystem为false
+ public void testCplParentMethod08() throws Exception {
+ String localFile = "local_file";
+ new File(testRootPath, "." + localFile + ".crc").delete();
+ File file = OBSFSTestUtil.createLocalTestFile(localFile);
+ OBSFSTestUtil.writeLocalFile(file, 16);
+
+ String srcFile = "test_file";
+ FSDataOutputStream outputStream = fs.create(getTestPath(srcFile),
+ false);
+ outputStream.close();
+
+ fs.copyToLocalFile(getTestPath(srcFile), new Path(localFile));
+ assertEquals(0, file.length());
+ OBSFSTestUtil.assertPathExistence(fs, getTestPath(srcFile), true);
+
+ OBSFSTestUtil.deleteLocalFile(localFile);
+ fs.delete(getTestPath(srcFile), true);
+ }
+
+ @Test
+ // 测试基类moveToLocalFile(Path src, Path dst)方法, delSrc为true, useRawLocalFileSystem为false
+ public void testCplParentMethod09() throws Exception {
+ String localFile = "local_file";
+ new File(testRootPath, "." + localFile + ".crc").delete();
+ File file = OBSFSTestUtil.createLocalTestFile(localFile);
+ OBSFSTestUtil.writeLocalFile(file, 16);
+
+ String srcFile = "test_file";
+ FSDataOutputStream outputStream = fs.create(getTestPath(srcFile),
+ false);
+ outputStream.close();
+
+ fs.moveToLocalFile(getTestPath(srcFile), new Path(localFile));
+ assertEquals(0, file.length());
+ OBSFSTestUtil.assertPathExistence(fs, getTestPath(srcFile), false);
+
+ OBSFSTestUtil.deleteLocalFile(localFile);
+ fs.delete(getTestPath(srcFile), true);
+ }
+
+ private void copyFromLocalFile(String srcFile, String destFile,
+ long srcFileSize) throws IOException {
+ Path localPath = new Path(srcFile);
+ new File(testRootPath, "." + srcFile + ".crc").delete();
+ File localFile = OBSFSTestUtil.createLocalTestFile(srcFile);
+ OBSFSTestUtil.writeLocalFile(localFile, srcFileSize);
+
+ Path dstPath = getTestPath(destFile);
+ fs.delete(dstPath, true);
+ FSDataOutputStream outputStream = OBSFSTestUtil.createStream(fs,
+ dstPath);
+ outputStream.close();
+
+ fs.copyFromLocalFile(true, true, localPath, dstPath);
+ OBSFSTestUtil.assertPathExistence(fs, dstPath, true);
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCreate.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCreate.java
new file mode 100644
index 0000000..0e71dd0
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCreate.java
@@ -0,0 +1,987 @@
+package org.apache.hadoop.fs.obs;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.util.EnumSet;
+
+public class ITestOBSCreate {
+ private OBSFileSystem fs;
+
+ private static long writeBufferSize = 100 * 1024 * 1024;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ writeBufferSize = OBSCommonUtils
+ .getMultipartSizeProperty(conf, OBSConstants.MULTIPART_SIZE,
+ OBSConstants.DEFAULT_MULTIPART_SIZE);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ OBSFSTestUtil.deletePathRecursive(fs, new Path(testRootPath));
+ }
+ }
+
+ private Path getTestPath(String relativePath) {
+ return new Path(testRootPath + "/" + relativePath);
+ }
+
+ @Test
+ // 创建一个stream,overwrite为false
+ public void testCreateNormal001() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("test_file");
+
+ FSDataOutputStream outputStream = null;
+ // create with CreateFlag argument
+ try {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+
+ // create with overwrite argument
+ try {
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ false, 4096,
+ (short) 3, 128 * 1024 * 1024, null);
+ //normal create test case
+ String position = "0";
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // 创建一个stream,路径为一个存在的文件,overwrite为true
+ public void testCreateNormal002() throws Exception {
+ Path testFile = getTestPath("test_file");
+
+ FSDataOutputStream outputStream = null;
+ try {
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ null, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+
+ // overwrite when exist with overwrite argument
+ try {
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ true, 4096,
+ (short) 3, 128 * 1024 * 1024, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+
+ // overwrite when exist with createflag argument
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.OVERWRITE);
+ try {
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // createNonRecursive stream,父目录已存在
+ public void testCreateNormal003() throws Exception {
+ Path testFile = getTestPath("a001/b001/test_file");
+ fs.mkdirs(testFile.getParent());
+
+ byte[] testBuffer = new byte[128];
+ for (int i = 0; i < testBuffer.length; ++i) {
+ testBuffer[i] = (byte) (i % 128);
+ }
+ FSDataOutputStream outputStream = null;
+ try {
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ false, 4096,
+ (short) 3, 128 * 1024 * 1024, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ outputStream.write(testBuffer);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ OBSFSTestUtil.assertFileHasLength(fs, testFile, testBuffer.length);
+
+ try {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.OVERWRITE);
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+ outputStream.write(testBuffer);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ OBSFSTestUtil.assertFileHasLength(fs, testFile, testBuffer.length);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile.getParent());
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // 创建write stream,调用write(),写入100字节
+ public void testCreateNormal004() throws Exception {
+ Path testFile = getTestPath("a001/b001/test_file");
+
+ byte[] testBuffer = new byte[100];
+ for (int i = 0; i < testBuffer.length; ++i) {
+ testBuffer[i] = (byte) (i % 128);
+ }
+ FSDataOutputStream outputStream = null;
+ try {
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ false, 4096,
+ (short) 3, 128 * 1024 * 1024, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ outputStream.write(testBuffer);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ OBSFSTestUtil.assertFileHasLength(fs, testFile, testBuffer.length);
+
+ try {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.OVERWRITE);
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+ outputStream.write(testBuffer);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ OBSFSTestUtil.assertFileHasLength(fs, testFile, testBuffer.length);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // 创建write stream,调用write(byte[] source, int offset, int len),写入100MB
+ public void testCreateNormal005() throws Exception {
+ Path testFile = getTestPath("a001/b001/test_file");
+ FSDataOutputStream outputStream = null;
+
+ try {
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ false, 4096,
+ (short) 3, 128 * 1024 * 1024, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ OBSFSTestUtil.writeData(outputStream, 100 * 1024 * 1024);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ OBSFSTestUtil.assertFileHasLength(fs, testFile, 100 * 1024 * 1024);
+
+ try {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.OVERWRITE);
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+ OBSFSTestUtil.writeData(outputStream, 100 * 1024 * 1024);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ OBSFSTestUtil.assertFileHasLength(fs, testFile, 100 * 1024 * 1024);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // 创建write stream,调用write(byte[] source, int offset, int len),写入110MB
+ public void testCreateNormal006() throws Exception {
+ Path testFile = getTestPath("a001/b001/test_file");
+ FSDataOutputStream outputStream = null;
+
+ try {
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ false, 4096,
+ (short) 3, 128 * 1024 * 1024, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ OBSFSTestUtil.writeData(outputStream, 110 * 1024 * 1024);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ OBSFSTestUtil.assertFileHasLength(fs, testFile, 110 * 1024 * 1024);
+
+ try {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.OVERWRITE);
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+ OBSFSTestUtil.writeData(outputStream, 110 * 1024 * 1024);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ OBSFSTestUtil.assertFileHasLength(fs, testFile, 110 * 1024 * 1024);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // 创建write stream,调用write(byte[] source, int offset, int len),写入110MB,调用hflush刷数据上云,数据正常。
+ // 然后调用write()写100字节,再调用write(byte[] source, int offset, int len)写150MB
+ public void testCreateNormal007() throws Exception {
+ Path testFile = getTestPath("a001/b001/test_file");
+ FSDataOutputStream outputStream = null;
+
+ try {
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ false, 4096,
+ (short) 3, 128 * 1024 * 1024, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ OBSFSTestUtil.writeData(outputStream, 10 * 1024 * 1024);
+ outputStream.hflush();
+ if (fs.isFsBucket()) {
+ OBSFSTestUtil.verifyReceivedData(fs, testFile,
+ 10 * 1024 * 1024);
+ }
+
+ OBSFSTestUtil.writeData(outputStream, 100);
+ OBSFSTestUtil.writeData(outputStream, 15 * 1024 * 1024);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ OBSFSTestUtil.assertFileHasLength(fs, testFile,
+ 10 * 1024 * 1024 + 100 + 15 * 1024 * 1024);
+
+ try {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.OVERWRITE);
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+ OBSFSTestUtil.writeData(outputStream, 10 * 1024 * 1024);
+ outputStream.hflush();
+ if (fs.isFsBucket()) {
+ OBSFSTestUtil.verifyReceivedData(fs, testFile,
+ 10 * 1024 * 1024);
+ }
+
+ System.out.println(fs.getFileStatus(testFile).getLen());
+ OBSFSTestUtil.writeData(outputStream, 100);
+ System.out.println(fs.getFileStatus(testFile).getLen());
+ OBSFSTestUtil.writeData(outputStream, 15 * 1024 * 1024);
+ System.out.println(fs.getFileStatus(testFile).getLen());
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ OBSFSTestUtil.assertFileHasLength(fs, testFile,
+ 10 * 1024 * 1024 + 100 + 15 * 1024 * 1024);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // create的文件不存在,overwrite参数为true,文件创建成功
+ public void testCreateNormal009() throws Exception {
+ Path testFile = getTestPath("a001/b001/test_file");
+
+ // create with overwrite argument
+ boolean hasException = false;
+ FSDataOutputStream outputStream = null;
+ try {
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ true, 4096,
+ (short) 3, 128 * 1024 * 1024, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ } catch (FileNotFoundException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ // assertEquals("create non exist file with overwrite flag should throw exception.",
+ // true, hasException);
+
+ // create with create flag
+ hasException = false;
+ try {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.OVERWRITE);
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+ } catch (FileNotFoundException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ // assertEquals("create non exist file with overwrite flag should throw exception.",
+ // true, hasException);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // 创建write stream,调用write(byte[] source, int offset, int len),写入110MB,调用hsync刷数据上云,数据正常。
+ // 然后调用write()写100字节,再调用write(byte[] source, int offset, int len)写150MB
+ public void testCreateNormal008() throws Exception {
+ Path testFile = getTestPath("a001/b001/test_file");
+ FSDataOutputStream outputStream = null;
+
+ try {
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ false, 4096,
+ (short) 3, 128 * 1024 * 1024, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ OBSFSTestUtil.writeData(outputStream, 110 * 1024 * 1024);
+ outputStream.hsync();
+ if (fs.isFsBucket()) {
+ OBSFSTestUtil.verifyReceivedData(fs, testFile,
+ 110 * 1024 * 1024);
+ }
+
+ OBSFSTestUtil.writeData(outputStream, 100);
+ OBSFSTestUtil.writeData(outputStream, 150 * 1024 * 1024);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ OBSFSTestUtil.assertFileHasLength(fs, testFile,
+ 110 * 1024 * 1024 + 100 + 150 * 1024 * 1024);
+
+ try {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.OVERWRITE);
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+ OBSFSTestUtil.writeData(outputStream, 110 * 1024 * 1024);
+ outputStream.hsync();
+ if (fs.isFsBucket()) {
+ OBSFSTestUtil.verifyReceivedData(fs, testFile,
+ 110 * 1024 * 1024);
+ }
+
+ OBSFSTestUtil.writeData(outputStream, 100);
+ OBSFSTestUtil.writeData(outputStream, 150 * 1024 * 1024);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ OBSFSTestUtil.assertFileHasLength(fs, testFile,
+ 110 * 1024 * 1024 + 100 + 150 * 1024 * 1024);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // 测试fs.obs.file.visibility.enable开关有效性
+ public void testCreateNormal010() throws Exception {
+ // 1、默认不开启fs.obs.file.visibility.enable开关
+ Configuration conf = OBSContract.getConfiguration(null);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ Path testFile = getTestPath("test_file");
+ fs.delete(testFile, true);
+
+ FSDataOutputStream outputStream = fs.create(testFile);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ outputStream.close();
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+ fs.delete(testFile, true);
+ fs.close();
+
+ // 2、打开fs.obs.file.visibility.enable开关后,create接口创建空文件
+ conf.setBoolean("fs.obs.file.visibility.enable", true);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ outputStream = fs.create(testFile);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+ outputStream.close();
+ fs.delete(testFile, true);
+ fs.close();
+
+ // 3、关闭fs.obs.file.visibility.enable开关后,create接口不会创建空文件
+ conf.setBoolean("fs.obs.file.visibility.enable", false);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ outputStream = fs.create(testFile);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ outputStream.close();
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+ fs.delete(testFile, true);
+ }
+
+ @Test
+ // 文件桶创建一个append stream,路径为一个文件
+ public void testCreatePosixNormal001() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("a001/b001/test_file");
+
+ FSDataOutputStream outputStream = null;
+ try {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.APPEND);
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+
+ OBSFSTestUtil.assertPathExistence(fs, testFile, true);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // 文件桶createNonRecursive append stream,写入128字节
+ public void testCreatePosixNormal002() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("a001/b001/test_file");
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ fs.mkdirs(testFile.getParent());
+
+ byte[] testBuffer = new byte[128];
+ for (int i = 0; i < testBuffer.length; ++i) {
+ testBuffer[i] = (byte) (i % 128);
+ }
+ FSDataOutputStream outputStream = null;
+ try {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.APPEND);
+ outputStream = fs.createNonRecursive(testFile,
+ new FsPermission((short) 00644), flags, 4096,
+ (short) 3, 128 * 1024 * 1024, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ outputStream.write(testBuffer);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+
+ OBSFSTestUtil.assertFileHasLength(fs, testFile, testBuffer.length);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile.getParent());
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // 文件桶create append stream, 写入100字节
+ public void testCreatePosixNormal003() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("a001/b001/test_file");
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+
+ byte[] testBuffer = new byte[128];
+ for (int i = 0; i < testBuffer.length; ++i) {
+ testBuffer[i] = (byte) (i % 128);
+ }
+ FSDataOutputStream outputStream = null;
+ try {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.APPEND);
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ outputStream.write(testBuffer);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+
+ OBSFSTestUtil.assertFileHasLength(fs, testFile, testBuffer.length);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // 文件桶create append stream, 写入100M
+ public void testCreatePosixNormal004() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("a001/b001/test_file");
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ FSDataOutputStream outputStream = null;
+
+ try {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.APPEND);
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ OBSFSTestUtil.writeData(outputStream, 100 * 1024 * 1024);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+
+ OBSFSTestUtil.assertFileHasLength(fs, testFile, 100 * 1024 * 1024);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // 文件桶create append stream, 写入110M
+ public void testCreatePosixNormal005() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("a001/b001/test_file");
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ FSDataOutputStream outputStream = null;
+
+ try {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.APPEND);
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ OBSFSTestUtil.writeData(outputStream, 110 * 1024 * 1024);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+
+ OBSFSTestUtil.assertFileHasLength(fs, testFile, 110 * 1024 * 1024);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // 文件桶create append stream,调用write(byte[] source, int offset, int len),写入110MB,调用hflush刷数据上云,数据正常。
+ // 然后调用write()写100字节,再调用write(byte[] source, int offset, int len)写150MB
+ public void testCreatePosixNormal006() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("a001/b001/test_file");
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ FSDataOutputStream outputStream = null;
+
+ try {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.APPEND);
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ OBSFSTestUtil.writeData(outputStream, 110 * 1024 * 1024);
+ outputStream.hflush();
+ OBSFSTestUtil.verifyReceivedData(fs, testFile, 110 * 1024 * 1024);
+
+ OBSFSTestUtil.writeData(outputStream, 100);
+ OBSFSTestUtil.writeData(outputStream, 150 * 1024 * 1024);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+
+ OBSFSTestUtil.assertFileHasLength(fs, testFile,
+ 110 * 1024 * 1024 + 100 + 150 * 1024 * 1024);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // 文件桶create append stream,调用write(byte[] source, int offset, int len),写入110MB,调用hsync刷数据上云,数据正常。
+ // 然后调用write()写100字节,再调用write(byte[] source, int offset, int len)写150MB
+ public void testCreatePosixNormal007() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("a001/b001/test_file");
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ FSDataOutputStream outputStream = null;
+
+ try {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.APPEND);
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ OBSFSTestUtil.assertPathExistence(fs, testFile, false);
+ OBSFSTestUtil.writeData(outputStream, 110 * 1024 * 1024);
+ outputStream.hsync();
+ OBSFSTestUtil.verifyReceivedData(fs, testFile, 110 * 1024 * 1024);
+
+ OBSFSTestUtil.writeData(outputStream, 100);
+ OBSFSTestUtil.writeData(outputStream, 150 * 1024 * 1024);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+
+ OBSFSTestUtil.assertFileHasLength(fs, testFile,
+ 110 * 1024 * 1024 + 100 + 150 * 1024 * 1024);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // create的文件已存在,overwrite参数为false,抛出FileAlreadyExistsException
+ public void testCreateAbnormal001() throws Exception {
+ Path testFile = getTestPath("a001/b001/test_file");
+
+ FSDataOutputStream outputStream = null;
+ try {
+ outputStream = OBSFSTestUtil.createStream(fs, testFile, false);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+
+ // create with overwrite argument
+ boolean hasException = false;
+ try {
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ false, 4096,
+ (short) 3, 128 * 1024 * 1024, null);
+ } catch (FileAlreadyExistsException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ assertEquals(
+ "create exist file with no overwrite flag should throw exception.",
+ true, hasException);
+
+ // create with createflag argument
+ hasException = false;
+ try {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ } catch (FileAlreadyExistsException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ assertEquals(
+ "create exist file with no overwrite flag should throw exception.",
+ true, hasException);
+
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // create的路径是目录,抛出FileAlreadyExistsException
+ public void testCreateAbnormal002() throws Exception {
+ Path testDir = getTestPath("a001/b001/test_file");
+ fs.mkdirs(testDir);
+
+ FSDataOutputStream outputStream = null;
+ boolean hasException = false;
+ try {
+ outputStream = fs.create(testDir, new FsPermission((short) 00644),
+ false, 4096,
+ (short) 3, 128 * 1024 * 1024, null);
+ } catch (FileAlreadyExistsException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ hasException = false;
+ try {
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ flags.add(CreateFlag.OVERWRITE);
+ outputStream = fs.create(testDir, new FsPermission((short) 00644),
+ flags, 4096, (short) 3,
+ 128 * 1024 * 1024, null, null);
+ } catch (FileAlreadyExistsException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ assertEquals("create stream for folder should throw exception.",
+ true, hasException);
+ OBSFSTestUtil.deletePathRecursive(fs, testDir);
+ }
+
+ @Test
+ // createNonRecursive 路径的父目录是文件,抛出ParentNotDirectoryException
+ public void testCreateAbnormal003() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("a001/b001/test_file");
+ Path parentPath = testFile.getParent();
+
+ FSDataOutputStream outputStream = null;
+ boolean hasException = false;
+ try {
+ outputStream = OBSFSTestUtil.createStream(fs, parentPath);
+ outputStream.close();
+
+ outputStream = OBSFSTestUtil.createNonRecursiveStream(fs, testFile);
+ } catch (ParentNotDirectoryException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ assertEquals(
+ "create non recursive stream for path whose parent path "
+ + "with same file should throw exception.",
+ true,
+ hasException);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile.getParent());
+ }
+
+ @Test
+ // createNonRecursive 路径的父目录不存在,抛出FileNotFoundException
+ public void testCreateAbnormal004() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("a001/b001/test_file");
+
+ FSDataOutputStream outputStream = null;
+ boolean hasException = false;
+ try {
+ outputStream = fs.createNonRecursive(testFile,
+ new FsPermission((short) 00644),
+ EnumSet.noneOf(CreateFlag.class),
+ 4096, (short) 3, 128 * 1024 * 1024, null);
+ } catch (FileNotFoundException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ assertEquals(
+ "create non recursive stream for non exist path should throw exception.",
+ true,
+ hasException);
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+
+ @Test
+ // create 携带LAZY_PERSIST、NEW_BLOCK、NO_LOCAL_WRITE、SHOULD_REPLICATE
+ // 、IGNORE_CLIENT_LOCALITY,
+ // 抛出UnsupportedOperationException
+ // hadoop2.8.3 无SHOULD_REPLICATE和IGNORE_CLIENT_LOCALITY flag
+ public void testCreateAbnormal005() throws Exception {
+ Path testFile = getTestPath("a001/b001/test_file");
+ fs.mkdirs(testFile.getParent());
+
+ FSDataOutputStream outputStream = null;
+ EnumSet flags = EnumSet.noneOf(CreateFlag.class);
+ boolean hasException;
+
+ // LAZY_PERSIST
+ flags.add(CreateFlag.LAZY_PERSIST);
+ hasException = false;
+ try {
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096,
+ (short) 3, 128 * 1024 * 1024, null, null);
+ } catch (UnsupportedOperationException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ assertEquals(
+ "create stream with LAZY_PERSIST flag throw UnsupportedOperationException.",
+ true,
+ hasException);
+
+ hasException = false;
+ try {
+ outputStream = fs.createNonRecursive(testFile,
+ new FsPermission((short) 00644), flags,
+ 4096, (short) 3, 128 * 1024 * 1024, null);
+ } catch (UnsupportedOperationException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ assertEquals(
+ "create stream with LAZY_PERSIST flag throw UnsupportedOperationException.",
+ true,
+ hasException);
+ flags.clear();
+
+ // NEW_BLOCK
+ flags.add(CreateFlag.NEW_BLOCK);
+ hasException = false;
+ try {
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096,
+ (short) 3, 128 * 1024 * 1024, null, null);
+ } catch (UnsupportedOperationException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ assertEquals(
+ "create stream with NEW_BLOCK flag throw UnsupportedOperationException.",
+ true,
+ hasException);
+
+ hasException = false;
+ try {
+ outputStream = fs.createNonRecursive(testFile,
+ new FsPermission((short) 00644), flags,
+ 4096, (short) 3, 128 * 1024 * 1024, null);
+ } catch (UnsupportedOperationException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ assertEquals(
+ "create stream with NEW_BLOCK flag throw UnsupportedOperationException.",
+ true,
+ hasException);
+ flags.clear();
+
+ // NO_LOCAL_WRITE
+ flags.add(CreateFlag.NO_LOCAL_WRITE);
+ hasException = false;
+ try {
+ outputStream = fs.create(testFile, new FsPermission((short) 00644),
+ flags, 4096,
+ (short) 3, 128 * 1024 * 1024, null, null);
+ } catch (UnsupportedOperationException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ assertEquals(
+ "create stream with NO_LOCAL_WRITE flag throw UnsupportedOperationException.",
+ true,
+ hasException);
+
+ hasException = false;
+ try {
+ outputStream = fs.createNonRecursive(testFile,
+ new FsPermission((short) 00644), flags,
+ 4096, (short) 3, 128 * 1024 * 1024, null);
+ } catch (UnsupportedOperationException e) {
+ hasException = true;
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ assertEquals(
+ "create stream with NO_LOCAL_WRITE flag throw UnsupportedOperationException.",
+ true,
+ hasException);
+ flags.clear();
+ OBSFSTestUtil.deletePathRecursive(fs, testFile);
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCredentialsInURL.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCredentialsInURL.java
new file mode 100644
index 0000000..dcdffda
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCredentialsInURL.java
@@ -0,0 +1,214 @@
+/**
+ * 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.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URI;
+import java.net.URLEncoder;
+
+import static org.apache.hadoop.fs.obs.OBSTestConstants.TEST_FS_OBS_NAME;
+
+/**
+ * Tests that credentials can go into the URL. This includes a valid
+ * set, and a check that an invalid set do at least get stripped out
+ * of the final URI
+ */
+public class ITestOBSCredentialsInURL extends Assert {
+ private OBSFileSystem fs;
+ private Configuration conf;
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ITestOBSCredentialsInURL.class);
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @Rule
+ public Timeout testTimeout = new Timeout(30 * 60 * 1000);
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ conf = OBSContract.getConfiguration(null);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ }
+
+ @After
+ public void teardown() {
+ IOUtils.closeStream(fs);
+ }
+
+ /**
+ * Test instantiation.
+ *
+ * @throws Throwable
+ */
+ @Test
+ // uri中携带credential信息,编码后ak、sk不包含/和+符号
+ public void testInstantiateFromURL() throws Throwable {
+
+ String accessKey = conf.get(OBSConstants.ACCESS_KEY);
+ String secretKey = conf.get(OBSConstants.SECRET_KEY);
+ String fsname = conf.getTrimmed(TEST_FS_OBS_NAME, "");
+ Assume.assumeNotNull(fsname, accessKey, secretKey);
+ URI original = new URI(fsname);
+ URI secretsURI = createUriWithEmbeddedSecrets(original,
+ accessKey, secretKey);
+ if (secretKey.contains("/")) {
+ assertTrue("test URI encodes the / symbol", secretsURI.toString().
+ contains("%252F"));
+ }
+ if (secretKey.contains("+")) {
+ assertTrue("test URI encodes the + symbol", secretsURI.toString().
+ contains("%252B"));
+ }
+ assertFalse("Does not contain secrets", original.equals(secretsURI));
+
+ conf.set(TEST_FS_OBS_NAME, secretsURI.toString());
+ conf.unset(OBSConstants.ACCESS_KEY);
+ conf.unset(OBSConstants.SECRET_KEY);
+ String fsURI = fs.getUri().toString();
+ assertFalse("FS URI contains a @ symbol", fsURI.contains("@"));
+ assertFalse("FS URI contains a % symbol", fsURI.contains("%"));
+ if (!original.toString().startsWith(fsURI)) {
+ fail("Filesystem URI does not match original");
+ }
+ validate("original path", new Path(original));
+ validate("bare path", new Path("/"));
+ validate("secrets path", new Path(secretsURI));
+ if (fs != null) {
+ fs.close();
+ fs = null;
+ }
+ }
+
+ @Test
+ // 测试标准化uri功能正常
+ public void testCanonicalizeUri() throws Exception {
+ assertNotNull("OBSFileSystem is null.", fs);
+ URI testURI = new URI(OBSTestConstants.TEST_FS_OBS_NAME);
+ URI canonicalizedURI = fs.canonicalizeUri(testURI);
+ assertEquals(testURI, canonicalizedURI);
+ if (fs != null) {
+ fs.close();
+ fs = null;
+ }
+ }
+
+ @Test
+ // 校验Path/URL符合OBSFS的规范
+ public void testCheckPath() throws IOException {
+ assertNotNull("OBSFileSystem is null.", fs);
+
+ // 1. no schema
+ Path testPath = new Path("/test/abc");
+ fs.checkPath(testPath);
+
+ // 2. schema not match
+ testPath = new Path("sss://testfs/abc");
+ boolean hasException = false;
+ try {
+ fs.checkPath(testPath);
+ } catch (IllegalArgumentException e) {
+ hasException = true;
+ }
+ assertTrue(hasException);
+
+ // 3.1. host is null, fs be same with defaultFs
+ conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
+ OBSTestConstants.TEST_FS_OBS_NAME);
+ testPath = new Path("obs:///test");
+ hasException = false;
+ try {
+ fs.checkPath(testPath);
+ } catch (IllegalArgumentException e) {
+ hasException = true;
+ }
+ assertTrue(hasException);
+
+ // 3.2. host is null, fs not same with defaultFs
+ conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
+ "obs://NOT-SAME-bucket-4521/");
+ testPath = new Path("obs:///test");
+ hasException = false;
+ try {
+ fs.checkPath(testPath);
+ } catch (IllegalArgumentException e) {
+ hasException = true;
+ }
+ assertTrue(hasException);
+ if (fs != null) {
+ fs.close();
+ fs = null;
+ }
+ }
+
+ private void validate(String text, Path path) throws IOException {
+ try {
+ fs.canonicalizeUri(path.toUri());
+ fs.checkPath(path);
+ assertTrue(text + " Not a directory",
+ fs.getFileStatus(new Path("/")).isDirectory());
+ fs.globStatus(path);
+ } catch (Exception e) {
+ LOG.debug("{} failure: {}", text, e, e);
+ fail(text + " Test failed");
+ }
+ }
+
+ private URI createUriWithEmbeddedSecrets(URI original,
+ String accessKey,
+ String secretKey) throws UnsupportedEncodingException {
+ String encodedSecretKey = URLEncoder.encode(secretKey, "UTF-8");
+ String formattedString = String.format("%s://%s:%s@%s/%s/",
+ original.getScheme(),
+ accessKey,
+ encodedSecretKey,
+ original.getHost(),
+ original.getPath());
+ URI testURI;
+ try {
+ testURI = new Path(formattedString).toUri();
+ } catch (IllegalArgumentException e) {
+ // inner cause is stripped to keep any secrets out of stack traces
+ throw new IllegalArgumentException(
+ "Could not encode Path: " + formattedString);
+ }
+ return testURI;
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDataBlocks.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDataBlocks.java
new file mode 100644
index 0000000..9c7286c
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDataBlocks.java
@@ -0,0 +1,203 @@
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Unit tests for {@link OBSDataBlocks}.
+ */
+public class ITestOBSDataBlocks extends Assert {
+ private OBSFileSystem fs;
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @Rule
+ public Timeout testTimeout = new Timeout(30 * 1000);
+
+ private List factoryPolicies = Arrays.asList(OBSConstants.FAST_UPLOAD_BUFFER_DISK,
+ OBSConstants.FAST_UPLOAD_BUFFER_ARRAY, OBSConstants.FAST_UPLOAD_BYTEBUFFER);
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void nameThread() throws IOException {
+ Thread.currentThread().setName("JUnit");
+ Configuration conf = OBSContract.getConfiguration(null);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ }
+
+ /**
+ * Test the {@link OBSDataBlocks.ByteBufferBlockFactory}. That code
+ * implements an input stream over a ByteBuffer, and has to return the
+ * buffer to the pool after the read complete.
+ *
+ * This test verifies the basic contract of the process.
+ */
+ @Test
+ // 测试基于byte buffer写缓存的读写基本功能
+ public void testByteBufferIO() throws Throwable {
+ OBSDataBlocks.ByteBufferBlockFactory factory = new OBSDataBlocks.ByteBufferBlockFactory(fs);
+ int limit = 128;
+ OBSDataBlocks.ByteBufferBlock block = factory.create(1, limit);
+ assertOutstandingBufferCount(factory, 1);
+
+ byte[] buffer = ContractTestUtils.toAsciiByteArray("test data");
+ int bufferLen = buffer.length;
+ block.write(buffer, 0, bufferLen);
+ assertEquals(bufferLen, block.dataSize());
+ assertEquals(limit - bufferLen, block.remainingCapacity());
+ assertTrue(block.hasCapacity(64));
+ assertTrue(block.hasCapacity(limit - bufferLen));
+
+ // now start the write
+ OBSDataBlocks.ByteBufferBlock.ByteBufferInputStream stream =
+ (OBSDataBlocks.ByteBufferBlock.ByteBufferInputStream) block.startUpload();
+ assertTrue(stream.markSupported());
+ assertTrue(stream.hasRemaining());
+ int expected = bufferLen;
+ assertEquals(expected, stream.available());
+
+ assertEquals('t', stream.read());
+ stream.mark(limit);
+ expected--;
+ assertEquals(expected, stream.available());
+
+ // read into a byte array with an offset
+ int offset = 5;
+ byte[] data = new byte[limit];
+ int read = stream.read(data, offset, 2);
+ assertEquals(2, read);
+ assertEquals('e', data[offset]);
+ assertEquals('s', data[offset + 1]);
+ expected -= 2;
+ assertEquals(expected, stream.available());
+
+ // read to end
+ byte[] remains = new byte[limit];
+ int value;
+ int index = 0;
+ while ((value = stream.read()) >= 0) {
+ remains[index++] = (byte) value;
+ }
+ assertEquals(expected, index);
+ assertEquals('a', remains[--index]);
+
+ assertEquals(0, stream.available());
+ assertTrue(!stream.hasRemaining());
+
+ // go the mark point
+ stream.reset();
+ assertEquals('e', stream.read());
+
+ // when the stream is closed, the data should be returned
+ stream.close();
+ assertOutstandingBufferCount(factory, 1);
+ block.close();
+ assertOutstandingBufferCount(factory, 0);
+ stream.close();
+ assertOutstandingBufferCount(factory, 0);
+ }
+
+ @Test
+ public void testNotCalcChecksum() throws IOException {
+ for (String factory : factoryPolicies) {
+ Configuration conf = OBSContract.getConfiguration(null);
+ OBSDataBlocks.DataBlock dataBlock =
+ startUploadDataBlock(factory, conf, ContractTestUtils.toAsciiByteArray("test data"));
+ assertNull("factory is " + factory, dataBlock.getChecksum());
+ assertEquals("factory is " + factory, OBSDataBlocks.ChecksumType.NONE, dataBlock.getChecksumType());
+ dataBlock.close();
+ }
+ }
+
+ @Test
+ public void testCalcMD5() throws IOException {
+ for (String factory : factoryPolicies) {
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.setBoolean(OBSConstants.OUTPUT_STREAM_ATTACH_MD5, true);
+ OBSDataBlocks.DataBlock dataBlock =
+ startUploadDataBlock(factory, conf, ContractTestUtils.toAsciiByteArray("test data"));
+ assertEquals("factory is " + factory, "63M6AMDJ0zbmVpGjerVCkw==", dataBlock.getChecksum());
+ assertEquals("factory is " + factory, OBSDataBlocks.ChecksumType.MD5, dataBlock.getChecksumType());
+ dataBlock.close();
+ }
+ }
+
+ @Test
+ public void testCalcMD5UseNewConfig() throws IOException {
+ for (String factory : factoryPolicies) {
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.set(OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE, OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE_MD5);
+ OBSDataBlocks.DataBlock dataBlock =
+ startUploadDataBlock(factory, conf, ContractTestUtils.toAsciiByteArray("test data"));
+ assertEquals("factory is " + factory, "63M6AMDJ0zbmVpGjerVCkw==", dataBlock.getChecksum());
+ assertEquals("factory is " + factory, OBSDataBlocks.ChecksumType.MD5, dataBlock.getChecksumType());
+ dataBlock.close();
+ }
+ }
+
+ @Test
+ public void testCalcSha256() throws IOException {
+ for (String factory : factoryPolicies) {
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.set(OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE, OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE_SHA256);
+ OBSDataBlocks.DataBlock dataBlock =
+ startUploadDataBlock(factory, conf, ContractTestUtils.toAsciiByteArray("test data"));
+ assertEquals("factory is " + factory, "916f0027a575074ce72a331777c3478d6513f786a591bd892da1a577bf2335f9",
+ dataBlock.getChecksum());
+ assertEquals("factory is " + factory, OBSDataBlocks.ChecksumType.SHA256, dataBlock.getChecksumType());
+ dataBlock.close();
+ }
+ }
+
+ @Test
+ public void testWrongChecksumType() {
+ for (String factory : factoryPolicies) {
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.set(OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE, "wrongType");
+ try {
+ startUploadDataBlock(factory, conf, ContractTestUtils.toAsciiByteArray("test data"));
+ fail("should be throw exception when type is wrong. factoryPolicy: " + factory);
+ } catch (IOException e) {
+ assertTrue(e.getMessage(), e.getMessage().startsWith("Unsupported fast upload checksum type"));
+ }
+ }
+ }
+
+ private OBSDataBlocks.DataBlock startUploadDataBlock(String factoryName, Configuration conf, byte[] data)
+ throws IOException {
+ OBSFileSystem fileSystem = OBSTestUtils.createTestFileSystem(conf);
+ OBSDataBlocks.BlockFactory factory = OBSDataBlocks.createFactory(fileSystem, factoryName);
+ OBSDataBlocks.DataBlock dataBlock = factory.create(1, 128);
+ dataBlock.write(data, 0, data.length);
+ dataBlock.startUpload();
+ return dataBlock;
+ }
+
+ /**
+ * Assert the number of buffers active for a block factory.
+ *
+ * @param factory factory
+ * @param expectedCount expected count.
+ */
+ private static void assertOutstandingBufferCount(OBSDataBlocks.ByteBufferBlockFactory factory, int expectedCount) {
+ assertEquals("outstanding buffers in " + factory, expectedCount, factory.getOutstandingBufferCount());
+ }
+
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDefaultInformation.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDefaultInformation.java
new file mode 100644
index 0000000..9baca07
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDefaultInformation.java
@@ -0,0 +1,86 @@
+package org.apache.hadoop.fs.obs;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class ITestOBSDefaultInformation {
+ private OBSFileSystem fs;
+ private Configuration conf;
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ conf = OBSContract.getConfiguration(null);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ OBSFSTestUtil.deletePathRecursive(fs, new Path(testRootPath));
+ }
+ }
+
+ @Test
+ // 返回客户端可配的默认的BlockSize
+ public void testGetDefaultBlockSize01() throws Exception {
+ long defaultBlockSize = 64 * 1024 * 1024;
+ conf.setLong(OBSConstants.FS_OBS_BLOCK_SIZE, defaultBlockSize);
+
+ OBSFileSystem obsFs = OBSTestUtils.createTestFileSystem(conf);
+ assertEquals(defaultBlockSize, obsFs.getDefaultBlockSize());
+ }
+
+ @Test
+ // 校验BlockSize默认值为128MB
+ public void testGetDefaultBlockSize02() {
+ assertEquals(128 * 1024 * 1024, fs.getDefaultBlockSize());
+ }
+
+ @Test
+ // BlockSize由fs.obs.block.size配置决定,修改dfs.blocksize应无效
+ public void testGetDefaultBlockSize03() throws Exception {
+ conf.setLong(OBSConstants.FS_OBS_BLOCK_SIZE, 128 * 1024 * 1024);
+ conf.setLong("dfs.blocksize", 64 * 1024 * 1024);
+
+ OBSFileSystem obsFs = OBSTestUtils.createTestFileSystem(conf);
+ assertEquals(128 * 1024 * 1024, obsFs.getDefaultBlockSize());
+ }
+
+ @Test
+ // getHomeDirectory()要返回用户的Home路径:"/user/" + ugi.getShortUserName()
+ public void testGetHomeDirectory() {
+ String homeDirectory = fs.getHomeDirectory().toString();
+
+ String username;
+ try {
+ username = UserGroupInformation.getCurrentUser().getShortUserName();
+ } catch (IOException ex) {
+ username = System.getProperty("user.name");
+ }
+
+ assertTrue(homeDirectory.endsWith("/user/" + username));
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDeleteAndRenameManyFiles.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDeleteAndRenameManyFiles.java
new file mode 100644
index 0000000..2bac2a5
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDeleteAndRenameManyFiles.java
@@ -0,0 +1,430 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.bandwidth;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyReceivedData;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test some scalable operations related to file renaming and deletion.
+ */
+public class ITestOBSDeleteAndRenameManyFiles {
+ private OBSFileSystem fs;
+
+ Configuration conf;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ITestOBSDeleteAndRenameManyFiles.class);
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ conf = OBSContract.getConfiguration(null);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ OBSFSTestUtil.deletePathRecursive(fs, new Path(testRootPath));
+ }
+ }
+
+ private Path getTestPath() {
+ return new Path(testRootPath + "/test-obs");
+ }
+
+ @Test
+ // 批量rename多级目录结构路径,rename后,源不存在(对象桶先拷贝后删除,文件桶直接rename)
+ public void testBulkRenameAndDelete() throws Throwable {
+ final Path scaleTestDir = new Path(getTestPath(),
+ "testBulkRenameAndDelete");
+ final Path srcDir = new Path(scaleTestDir, "1/2/3/src");
+ final Path distDir = new Path(scaleTestDir, "1/2/src1");
+ final Path finalDir = new Path(scaleTestDir, "1/2/src1/src");
+ final long count =
+ conf.getLong(OBSTestConstants.KEY_OPERATION_COUNT,
+ OBSTestConstants.DEFAULT_OPERATION_COUNT);
+
+ boolean needCreat = true;
+ ContractTestUtils.rm(fs, new Path("/1"), true, false);
+ ContractTestUtils.rm(fs, scaleTestDir, true, false);
+ fs.mkdirs(srcDir);
+ fs.mkdirs(distDir);
+
+ int testBufferSize = fs.getConf()
+ .getInt(ContractTestUtils.IO_CHUNK_BUFFER_SIZE,
+ ContractTestUtils.DEFAULT_IO_CHUNK_BUFFER_SIZE);
+ // use Executor to speed up file creation
+ ExecutorService exec = Executors.newFixedThreadPool(16);
+ final ExecutorCompletionService completionService =
+ new ExecutorCompletionService<>(exec);
+ try {
+ final byte[] data = ContractTestUtils.dataset(testBufferSize, 'a',
+ 'z');
+
+ if (needCreat) {
+ for (int i = 0; i < count; ++i) {
+ final String fileName = "foo-" + i;
+ completionService.submit(new Callable() {
+ @Override
+ public Boolean call() throws IOException {
+ ContractTestUtils.createFile(fs,
+ new Path(srcDir, fileName), false, data);
+ return fs.exists(new Path(srcDir, fileName));
+ }
+ });
+ }
+ }
+
+ for (int i = 0; i < 999; ++i) {
+ final Future future = completionService.take();
+ try {
+ if (!future.get()) {
+ LOG.warn("cannot create file");
+ }
+ } catch (ExecutionException e) {
+ LOG.warn("Error while uploading file", e.getCause());
+ throw e;
+ }
+ }
+ } finally {
+ exec.shutdown();
+ }
+
+ int nSrcFiles = fs.listStatus(srcDir).length;
+
+ fs.rename(srcDir, distDir);
+ assertEquals(nSrcFiles, fs.listStatus(finalDir).length);
+ ContractTestUtils.assertPathDoesNotExist(fs, "not deleted after rename",
+ new Path(srcDir, "foo-" + 0));
+ ContractTestUtils.assertPathDoesNotExist(fs, "not deleted after rename",
+ new Path(srcDir, "foo-" + count / 2));
+ ContractTestUtils.assertPathDoesNotExist(fs, "not deleted after rename",
+ new Path(srcDir, "foo-" + (count - 1)));
+ assertPathExists(fs, "not renamed to dest dir",
+ new Path(finalDir, "foo-" + 0));
+ assertPathExists(fs, "not renamed to dest dir",
+ new Path(finalDir, "foo-" + count / 2));
+ assertPathExists(fs, "not renamed to dest dir",
+ new Path(finalDir, "foo-" + (count - 1)));
+
+ fs.delete(scaleTestDir, true);
+ }
+
+ @Test
+ // 不带delimiter list多级目录,测试功能正确性
+ public void testListWithoutDelimiter() throws Throwable {
+ final Path scaleTestDir = new Path(getTestPath(),"testListDelimiter");
+ final Path srcDir = new Path(scaleTestDir, "1/2/3/src");
+ final Path finalDir = new Path(scaleTestDir, "1/2/src1");
+ final long count = conf.getLong(OBSTestConstants.KEY_OPERATION_COUNT,
+ 10);
+
+ boolean needCreat = true;
+ ContractTestUtils.rm(fs, new Path("/1"), true, false);
+ ContractTestUtils.rm(fs, scaleTestDir, true, false);
+ fs.mkdirs(srcDir);
+ fs.mkdirs(finalDir);
+
+ int testBufferSize = fs.getConf()
+ .getInt(ContractTestUtils.IO_CHUNK_BUFFER_SIZE,
+ ContractTestUtils.DEFAULT_IO_CHUNK_BUFFER_SIZE);
+ // use Executor to speed up file creation
+ ExecutorService exec = Executors.newFixedThreadPool(16);
+ final ExecutorCompletionService completionService =
+ new ExecutorCompletionService<>(exec);
+ try {
+ final byte[] data = ContractTestUtils.dataset(testBufferSize, 'a',
+ 'z');
+
+ if (needCreat) {
+ for (int i = 0; i < count; ++i) {
+ final String fileName = "foo-" + i;
+ completionService.submit(new Callable() {
+ @Override
+ public Boolean call() throws IOException {
+ ContractTestUtils.createFile(fs,
+ new Path(srcDir, fileName), false, data);
+ return fs.exists(new Path(srcDir, fileName));
+ }
+ });
+ }
+ final Path srcSub1Dir = new Path(scaleTestDir,
+ "1/2/3/src/sub1");
+ for (int i = 0; i < 5; ++i) {
+ final String fileName = "foo1-" + i;
+ ContractTestUtils.createFile(fs,
+ new Path(srcSub1Dir, fileName), false, data);
+ }
+ final Path srcSub2Dir = new Path(scaleTestDir,
+ "1/2/3/src/sub1/sub2");
+ for (int i = 0; i < 5; ++i) {
+ final String fileName = "foo2-" + i;
+ ContractTestUtils.createFile(fs,
+ new Path(srcSub2Dir, fileName), false, data);
+ }
+ }
+
+ for (int i = 0; i < count; ++i) {
+ final Future future = completionService.take();
+ try {
+ if (!future.get()) {
+ LOG.warn("cannot create file");
+ }
+ } catch (ExecutionException e) {
+ LOG.warn("Error while uploading file", e.getCause());
+ throw e;
+ }
+ }
+ } finally {
+ exec.shutdown();
+ }
+ String sreKey = pathToKey(srcDir);
+ if (!sreKey.endsWith("/")) {
+ sreKey = sreKey + "/";
+ }
+ ListObjectsRequest request = new ListObjectsRequest();
+ request.setBucketName(fs.getBucket());
+ request.setPrefix(sreKey);
+ request.setMaxKeys(1000);
+
+ fs.getObsClient().listObjects(request);
+ fs.delete(scaleTestDir, true);
+ }
+
+ @Test
+ // 测试大文件的rename,rename后校验目标文件内容和源文件相同
+ public void testHugeFileRename() throws Throwable {
+ int testBufferSize = fs.getConf()
+ .getInt(
+ ContractTestUtils.IO_CHUNK_BUFFER_SIZE, 1024 * 1024);
+ int modulus = fs.getConf().getInt(
+ ContractTestUtils.IO_CHUNK_MODULUS_SIZE, 128);
+ long fileSize = 1024 * 1024 * 1024L;
+ final Path srcDir = new Path(getTestPath(), "src");
+ final Path distDir = new Path(getTestPath(), "final");
+ final Path finalDir = new Path(distDir, "src");
+
+ ContractTestUtils.rm(fs, getTestPath(), true, false);
+ fs.mkdirs(srcDir);
+ fs.mkdirs(distDir);
+ Path objectPath = new Path(srcDir, "copy-test-file");
+ Path renamePath = new Path(finalDir, "copy-test-file");
+ boolean needUpload = true;
+ ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+
+ byte[] testBuffer = new byte[testBufferSize];
+
+ for (int i = 0; i < testBuffer.length; ++i) {
+ testBuffer[i] = (byte) (i % modulus);
+ }
+
+ long bytesWritten = 0L;
+
+ Throwable var10 = null;
+
+ long diff;
+ if (needUpload) {
+ OutputStream outputStream = fs.create(objectPath, false);
+ try {
+ while (bytesWritten < fileSize) {
+ diff = fileSize - bytesWritten;
+ if (diff < (long) testBuffer.length) {
+ outputStream.write(testBuffer, 0, (int) diff);
+ bytesWritten += diff;
+ } else {
+ outputStream.write(testBuffer);
+ bytesWritten += (long) testBuffer.length;
+ }
+ }
+
+ diff = bytesWritten;
+ } catch (Throwable var21) {
+ var10 = var21;
+ throw var21;
+ } finally {
+ if (outputStream != null) {
+ if (var10 != null) {
+ try {
+ outputStream.close();
+ } catch (Throwable var20) {
+ var10.addSuppressed(var20);
+ }
+ } else {
+ outputStream.close();
+ }
+ }
+
+ }
+ }
+
+ fs.rename(srcDir, distDir);
+ // assertEquals(fileSize, diff);
+ assertPathExists(fs, "not created successful", renamePath);
+ timer.end("Time to write %d bytes", fileSize);
+ bandwidth(timer, fileSize);
+
+ try {
+ verifyReceivedData(fs, renamePath, fileSize, testBufferSize,
+ modulus);
+ } finally {
+ fs.delete(renamePath, false);
+ }
+ }
+
+ @Test
+ // 带delimiter list大目录,测试功能正确性
+ public void testListingWithDelimiter() throws Throwable {
+ final Path scaleTestDir = new Path(getTestPath(), "testListing");
+ final Path srcDir = new Path(scaleTestDir, "1/src");
+ final Path finalDir = new Path(scaleTestDir, "1/2/src2");
+ final long count = conf.getLong(OBSTestConstants.KEY_OPERATION_COUNT,
+ 1200);
+
+ boolean needCreat = true;
+ ContractTestUtils.rm(fs, new Path("/1"), true, false);
+ ContractTestUtils.rm(fs, scaleTestDir, true, false);
+ fs.mkdirs(srcDir);
+ fs.mkdirs(finalDir);
+
+ int testBufferSize = fs.getConf()
+ .getInt(ContractTestUtils.IO_CHUNK_BUFFER_SIZE,
+ ContractTestUtils.DEFAULT_IO_CHUNK_BUFFER_SIZE);
+ // use Executor to speed up file creation
+ ExecutorService exec = Executors.newFixedThreadPool(16);
+ final ExecutorCompletionService completionService =
+ new ExecutorCompletionService<>(exec);
+ try {
+ final byte[] data = ContractTestUtils.dataset(testBufferSize, 'a',
+ 'z');
+
+ if (needCreat) {
+ for (int i = 0; i < count; ++i) {
+ final String fileName = "foo-" + i;
+ completionService.submit(new Callable() {
+ @Override
+ public Boolean call() throws IOException {
+ ContractTestUtils.createFile(fs,
+ new Path(srcDir, fileName), false, data);
+ return fs.exists(new Path(srcDir, fileName));
+ }
+ });
+ }
+ final Path srcSub1Dir = new Path(scaleTestDir, "1/src/sub1");
+ for (int i = 0; i < 5; ++i) {
+ final String fileName = "foo1-" + i;
+ ContractTestUtils.createFile(fs,
+ new Path(srcSub1Dir, fileName), false, data);
+ }
+ final Path srcSub2Dir = new Path(scaleTestDir,
+ "1/src/sub1/sub2");
+ for (int i = 0; i < 5; ++i) {
+ final String fileName = "foo2-" + i;
+ ContractTestUtils.createFile(fs,
+ new Path(srcSub2Dir, fileName), false, data);
+ }
+ }
+
+ for (int i = 0; i < count; ++i) {
+ final Future future = completionService.take();
+ try {
+ if (!future.get()) {
+ LOG.warn("cannot create file");
+ }
+ } catch (ExecutionException e) {
+ LOG.warn("Error while uploading file", e.getCause());
+ throw e;
+ }
+ }
+ } finally {
+ exec.shutdown();
+ }
+ String sreKey = pathToKey(srcDir);
+ if (!sreKey.endsWith("/")) {
+ sreKey = sreKey + "/";
+ }
+ ListObjectsRequest request = new ListObjectsRequest();
+ request.setBucketName(fs.getBucket());
+ request.setPrefix(sreKey);
+ request.setDelimiter("/");
+ request.setMaxKeys(1000);
+
+ ObjectListing objects = fs.getObsClient().listObjects(request);
+
+ while (true) {
+ if (!objects.isTruncated()) {
+ break;
+ }
+
+ objects = OBSCommonUtils.continueListObjects(fs, objects);
+ }
+ fs.delete(scaleTestDir, true);
+ }
+
+ private String pathToKey(Path path) {
+ if (!path.isAbsolute()) {
+ path = new Path(fs.getWorkingDirectory(), path);
+ }
+
+ if (path.toUri().getScheme() != null && path.toUri()
+ .getPath()
+ .isEmpty()) {
+ return "";
+ }
+
+ return path.toUri().getPath().substring(1);
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDisguisePermissionSupport.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDisguisePermissionSupport.java
new file mode 100644
index 0000000..e686e40
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDisguisePermissionSupport.java
@@ -0,0 +1,154 @@
+package org.apache.hadoop.fs.obs;
+
+import com.obs.services.model.SetObjectMetadataRequest;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.Rule;
+
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class ITestOBSDisguisePermissionSupport {
+ private OBSFileSystem fs;
+
+ private static String testRootPath = OBSTestUtils.generateUniqueTestPath();
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.set(OBSConstants.PERMISSIONS_MODE, OBSConstants.PERMISSIONS_MODE_DISGUISE);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(new Path(testRootPath), true);
+ }
+ }
+
+ private Path getTestPath(String relativePath) {
+ return new Path(testRootPath, relativePath);
+ }
+
+ @Test
+ public void testOBSSetOwner() throws Exception {
+ if (!fs.supportDisguisePermissionsMode()) {
+ return;
+ }
+ Path testFile = getTestPath("file_for_permission_test");
+ FSDataOutputStream outputStream = fs.create(testFile, false);
+ outputStream.close();
+ fs.setOwner(testFile, "hadoop", "root");
+ Map objMap = fs.getObsClient().getObjectMetadata(fs.getBucket(),
+ OBSCommonUtils.pathToKey(fs, testFile)).getAllMetadata();
+ assertEquals("hadoop", objMap.get("user").toString());
+ assertEquals("root", objMap.get("group").toString());
+ }
+
+ @Test
+ public void testOBSSetPermission() throws Exception {
+ if (!fs.supportDisguisePermissionsMode()) {
+ return;
+ }
+ Path testFile = getTestPath("file_for_owner_and_group_test");
+ FSDataOutputStream outputStream = fs.create(testFile, false);
+ outputStream.close();
+ fs.setPermission(testFile, new FsPermission((short)00400));
+ Map objMap = fs.getObsClient().getObjectMetadata(fs.getBucket(),
+ OBSCommonUtils.pathToKey(fs, testFile)).getAllMetadata();
+ assertEquals("256", objMap.get("permission").toString());
+ }
+
+ @Test
+ public void testOBSGetFileStatusForFile() throws Exception {
+ if (!fs.supportDisguisePermissionsMode()) {
+ return;
+ }
+ Path testFile= getTestPath("file_for_get_file_status_test");
+ FSDataOutputStream outputStream = fs.create(testFile);
+ outputStream.close();
+ testOBDGetFileStatus(testFile);
+ }
+
+ @Test
+ public void testOBSGetFileStatusForFolder() throws Exception {
+ if (!fs.supportDisguisePermissionsMode()) {
+ return;
+ }
+ Path testFolder = getTestPath("folder_for_get_file_status_test");
+ fs.mkdirs(testFolder);
+ testOBDGetFileStatus(testFolder);
+ }
+
+ private void testOBDGetFileStatus(Path p) throws Exception {
+ fs.setPermission(p, new FsPermission((short) 00640));
+ fs.setOwner(p, "root", "supergroup");
+ FileStatus status = fs.getFileStatus(p);
+ assertEquals("rw-r-----", status.getPermission().toString());
+ assertEquals("root", status.getOwner());
+ assertEquals("supergroup", status.getGroup());
+ }
+
+ @Test
+ public void testOBSGetFileStatusPermissionInvalid() throws Exception {
+ if (!fs.supportDisguisePermissionsMode()) {
+ return;
+ }
+ Path testFile= getTestPath("file_for_get_file_status_test");
+ FSDataOutputStream outputStream = fs.create(testFile);
+ outputStream.close();
+ SetObjectMetadataRequest req = new SetObjectMetadataRequest(fs.getBucket(),
+ OBSCommonUtils.pathToKey(fs, testFile));
+ req.addUserMetadata("permission", "invalid_permission");
+ fs.getObsClient().setObjectMetadata(req);
+ FileStatus status = fs.getFileStatus(testFile);
+ assertEquals("rw-rw-rw-", status.getPermission().toString());
+ }
+
+ @Test
+ public void testOBSListFiles() throws Exception {
+ if (!fs.supportDisguisePermissionsMode()) {
+ return;
+ }
+ Path testFolder = getTestPath("folder_for_list_files_test");
+ fs.mkdirs(testFolder);
+ fs.setPermission(testFolder, new FsPermission((short) 00644));
+ fs.setOwner(testFolder, "root", "supergroup");
+
+ Path testFile = getTestPath("file_for_list_files_test");
+ FSDataOutputStream outputStream = fs.create(testFile);
+ outputStream.close();
+ fs.setPermission(testFile, new FsPermission((short) 00644));
+ fs.setOwner(testFile, "root", "supergroup");
+
+ RemoteIterator fileStatus = fs.listFiles(new Path(testRootPath), true);
+ while(fileStatus.hasNext()) {
+ LocatedFileStatus status = fileStatus.next();
+ assertEquals("rw-r--r--", status.getPermission().toString());
+ assertEquals("root", status.getOwner());
+ assertEquals("supergroup", status.getGroup());
+ }
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDiskBufferOutputStream.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDiskBufferOutputStream.java
new file mode 100644
index 0000000..645d71a
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDiskBufferOutputStream.java
@@ -0,0 +1,283 @@
+package org.apache.hadoop.fs.obs;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertEquals;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createAndVerifyFile;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.anyObject;
+
+import com.obs.services.ObsClient;
+import com.obs.services.model.AbortMultipartUploadRequest;
+import com.obs.services.model.ListMultipartUploadsRequest;
+import com.obs.services.model.MultipartUpload;
+import com.obs.services.model.MultipartUploadListing;
+import com.obs.services.model.fs.ObsFSAttribute;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.mockito.Mockito;
+import org.powermock.reflect.Whitebox;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Date;
+
+@RunWith(Parameterized.class)
+public class ITestOBSDiskBufferOutputStream {
+ private static final Logger LOG = LoggerFactory.getLogger(ITestOBSDiskBufferOutputStream.class);
+ private OBSFileSystem fs;
+ private boolean calcMd5;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ @Parameterized.Parameters
+ public static Collection calcMd5() {
+ return Arrays.asList(false, true);
+ }
+
+ public ITestOBSDiskBufferOutputStream(boolean calcMd5) {
+ this.calcMd5 = calcMd5;
+ }
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.setBoolean(OBSConstants.FAST_UPLOAD, true);
+ conf.setLong(OBSConstants.MULTIPART_SIZE, 5 * 1024 * 1024);
+ conf.setBoolean(OBSConstants.OUTPUT_STREAM_ATTACH_MD5, calcMd5);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ deleteTmpDir(conf);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(new Path(testRootPath), true);
+ }
+ }
+
+ private Path getTestPath(String relativePath) {
+ return new Path(testRootPath + "/" + relativePath);
+ }
+
+ @Test
+ // disk写缓存,上传对象时构造mock异常,触发异常后,清理上传的碎片
+ public void testMockUploadPartError() throws Exception {
+ Path dest = getTestPath("testMockUploadPartError");
+ IOException exception = null;
+ IOException closeException = null;
+ FSDataOutputStream stream = null;
+ try {
+ stream = fs.create(dest, true);
+ ((OBSBlockOutputStream) stream.getWrappedStream()).mockPutPartError(
+ true);
+ byte[] data = ContractTestUtils.dataset(10 * 1204 * 1024, 'a', 26);
+ stream.write(data);
+ stream.close();
+ stream.write(data);
+ } catch (IOException e) {
+ exception = e;
+ } finally {
+ try {
+ if (stream != null) {
+ stream.close();
+ }
+ } catch (IOException e) {
+ closeException = e;
+ }
+ fs.delete(dest, false);
+ }
+ assertTrue(exception != null && exception.getMessage()
+ .contains("Multi-part upload"));
+ assertTrue(closeException != null && closeException.getMessage()
+ .contains("Multi-part upload"));
+
+ // 删除多段碎片
+ final Date purgeBefore = new Date();
+ ListMultipartUploadsRequest request
+ = new ListMultipartUploadsRequest(fs.getBucket());
+ while (true) {
+ // List + purge
+ MultipartUploadListing uploadListing = fs.getObsClient()
+ .listMultipartUploads(request);
+ for (MultipartUpload upload : uploadListing.getMultipartTaskList()) {
+ LOG.info("MultipartTask:create time {},purge time {}",upload.getInitiatedDate(),purgeBefore);
+ if (upload.getInitiatedDate().compareTo(purgeBefore) < 0) {
+ LOG.info("abort MultipartTask");
+ fs.getObsClient().abortMultipartUpload(
+ new AbortMultipartUploadRequest(fs.getBucket(), upload.getObjectKey(), upload.getUploadId()));
+ }
+ }
+ if (!uploadListing.isTruncated()) {
+ break;
+ }
+ request.setUploadIdMarker(
+ uploadListing.getNextUploadIdMarker());
+ request.setKeyMarker(uploadListing.getNextKeyMarker());
+ }
+
+ // 校验缓存目录为空
+ assertTrue(verifyTmpDirEmpty(fs.getConf()));
+ }
+
+ @Test
+ // disk写缓存,上传对象并校验数据通过
+ public void testBlockUpload() throws IOException {
+ // ContractTestUtils.createAndVerifyFile(fs, testPath, 0);
+ // verifyUpload(100*1024-1);
+ verifyUpload(10 * 1024 * 1024 + 1);
+ // 校验缓存目录为空
+ assertTrue(verifyTmpDirEmpty(fs.getConf()));
+ // verifyUpload(10*1024*1024);
+ // verifyUpload(10*1024*1024-1);
+ }
+
+ @Test
+ // disk写缓存,测试0字节大小文件上传并校验文件大小
+ public void testDiskZeroUpload() throws IOException {
+ // ContractTestUtils.createAndVerifyFile(fs, testPath, 0);
+ // verifyUpload(100*1024-1);
+ verifyUpload(0);
+ // 校验缓存目录为空
+ assertTrue(verifyTmpDirEmpty(fs.getConf()));
+ }
+
+ @Test
+ // disk写缓存,上传对象关闭流后,再写数据抛出IOException
+ public void testDiskWriteAfterStreamClose() throws IOException {
+ Path dest = getTestPath("testWriteAfterStreamClose");
+ FSDataOutputStream stream = fs.create(dest, true);
+ byte[] data = ContractTestUtils.dataset(16, 'a', 26);
+ try {
+ stream.write(data);
+ stream.close();
+ boolean hasException = false;
+ try {
+ stream.write(data);
+ } catch (IOException e) {
+ hasException = true;
+ }
+ assertTrue(hasException);
+
+ } finally {
+ fs.delete(dest, false);
+ IOUtils.closeStream(stream);
+ }
+ // 校验缓存目录为空
+ assertTrue(verifyTmpDirEmpty(fs.getConf()));
+ }
+
+ @Test
+ // disk写缓存,上传对象流关闭后缓存块被清空
+ public void testBlocksClosed() throws Throwable {
+ Path dest = getTestPath("testBlocksClosed");
+
+ FSDataOutputStream stream = fs.create(dest, true);
+ OBSBlockOutputStream obsStream =
+ (OBSBlockOutputStream)stream.getWrappedStream();
+ byte[] data = ContractTestUtils.dataset(16, 'a', 26);
+ stream.write(data);
+ stream.close();
+ assertTrue(null == obsStream.getActiveBlock());
+ fs.delete(dest, false);
+ // 校验缓存目录为空
+ assertTrue(verifyTmpDirEmpty(fs.getConf()));
+ }
+
+ private void verifyUpload(long fileSize) throws IOException {
+ createAndVerifyFile(fs, getTestPath("test_file"), fileSize);
+ }
+
+ @Test
+ // append写,当head返回文件长度与客户端记录长度不一致时,以大的长度为准
+ public void testAppendWithIncorrectContentLen() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ OBSFileSystem mockFs = Mockito.spy(fs);
+ ObsClient client = fs.getObsClient();
+ ObsClient mockClient = Mockito.spy(client);
+ Whitebox.setInternalState(mockFs, client, mockClient);
+
+ Path testFile = getTestPath("test_file");
+ FSDataOutputStream outputStream = fs.create(testFile, false);
+ byte[] data = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9};
+ outputStream.write(data);
+ outputStream.close();
+
+ // 第一次append调用实际head接口,第二次append使用mock的ObsFSAttribute
+ ObsFSAttribute attribute = new ObsFSAttribute();
+ attribute.setContentLength(5L);
+ Mockito.doCallRealMethod()
+ .doReturn(attribute)
+ .when(mockClient)
+ .getAttribute(anyObject());
+
+ // 第一次写正常,写完后文件长度为20
+ outputStream = mockFs.append(testFile, 4096, null);
+ outputStream.write(data);
+ outputStream.hflush();
+ assertEquals(20, fs.getFileStatus(testFile).getLen());
+
+ // 第二次getAttribute得到的文件大小不正确,以大的为准,从20追加写,写完后文件长度为30
+ outputStream.write(data);
+ outputStream.close();
+ assertEquals(30, fs.getFileStatus(testFile).getLen());
+
+ fs.delete(testFile, false);
+ // 校验缓存目录为空
+ assertTrue(verifyTmpDirEmpty(fs.getConf()));
+ }
+
+ private boolean verifyTmpDirEmpty(Configuration conf) throws IOException {
+ String bufferDir = conf.get(OBSConstants.BUFFER_DIR) != null
+ ? OBSConstants.BUFFER_DIR
+ : "hadoop.tmp.dir";
+ LocalDirAllocator allocator = new LocalDirAllocator(bufferDir);
+ String tmpBuff = "obs-block-0001";
+ Path path = allocator.getLocalPathForWrite(tmpBuff, conf);
+ File parentDir = new File(path.getParent().toUri().toString());
+ assertTrue(parentDir.isDirectory());
+ return parentDir.list().length == 0;
+ }
+
+ private void deleteTmpDir(Configuration conf) throws IOException {
+ String bufferDir = conf.get(OBSConstants.BUFFER_DIR) != null
+ ? OBSConstants.BUFFER_DIR
+ : "hadoop.tmp.dir";
+ LocalDirAllocator allocator = new LocalDirAllocator(bufferDir);
+ String tmpBuff = "obs-block-0001";
+ Path path = allocator.getLocalPathForWrite(tmpBuff, conf);
+ File parentDir = new File(path.getParent().toUri().toString());
+
+ File[] children = parentDir.listFiles();
+ for (int i = 0; i < children.length; i++) {
+ children[i].delete();
+ }
+ }
+}
\ No newline at end of file
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFSDataOutputStream.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFSDataOutputStream.java
new file mode 100644
index 0000000..225823c
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFSDataOutputStream.java
@@ -0,0 +1,242 @@
+package org.apache.hadoop.fs.obs;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+@RunWith(Parameterized.class)
+public class ITestOBSFSDataOutputStream {
+ private OBSFileSystem fs;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ private final int fileSize4 = 4 * 1024 * 1024;
+
+ private final int fileSize5 = 5 * 1024 * 1024;
+
+ private final int fileSize6 = 6 * 1024 * 1024;
+
+ private final boolean caclMd5;
+
+ private final String blockPolicy;
+
+ @Parameterized.Parameters
+ public static Collection digestPolicy() {
+ return Arrays.asList(
+ new PolicyParam(false, OBSConstants.FAST_UPLOAD_BUFFER_DISK),
+ new PolicyParam(false, OBSConstants.FAST_UPLOAD_BUFFER_ARRAY),
+ new PolicyParam(false, OBSConstants.FAST_UPLOAD_BYTEBUFFER),
+ new PolicyParam(true, OBSConstants.FAST_UPLOAD_BUFFER_DISK),
+ new PolicyParam(true, OBSConstants.FAST_UPLOAD_BUFFER_ARRAY),
+ new PolicyParam(true, OBSConstants.FAST_UPLOAD_BYTEBUFFER)); // 是否开启md5校验
+ }
+
+ public ITestOBSFSDataOutputStream(PolicyParam param) {
+ this.caclMd5 = param.calcMd5;
+ this.blockPolicy = param.blockPolicy;
+ }
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.set(OBSConstants.MULTIPART_SIZE, String.valueOf(5 * 1024 * 1024));
+ conf.setBoolean(OBSConstants.OUTPUT_STREAM_ATTACH_MD5, caclMd5);
+ conf.set(OBSConstants.FAST_UPLOAD_BUFFER, blockPolicy);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(new Path(testRootPath), true);
+ }
+ }
+
+ private Path getTestPath(String relativePath) {
+ return new Path(testRootPath + "/" + relativePath);
+ }
+
+ @Test
+ // 执行write操作写入5字节,调用close前,未将数据刷到服务端;调用close后,强制将客户端数据刷到服务端;
+ public void testClose() throws Exception {
+ if(!fs.isFsBucket()) {
+ return;
+ }
+
+ Path testFile = getTestPath("test_file");
+ FSDataOutputStream outputStream = fs.create(testFile, false);
+ assertFalse(fs.exists(testFile));
+ byte[] data = {0, 1, 2, 3, 4};
+ outputStream.write(data);
+
+ outputStream.hflush();
+ assertEquals(5, fs.getFileStatus(testFile).getLen());
+
+ outputStream.write(data);
+ assertEquals(5, fs.getFileStatus(testFile).getLen());
+
+ outputStream.close();
+ assertEquals(10, fs.getFileStatus(testFile).getLen());
+ fs.delete(testFile, false);
+ }
+
+ @Test
+ // create file write 5M ,write4M,write6M hflush ,write5M hsync,write4M
+ // 共24M
+ public void testCombination001() throws Exception {
+ // if (fs.getMetricSwitch()) {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("test_file");
+ if (fs.exists(testFile)) {
+ fs.delete(testFile);
+ }
+ FSDataOutputStream outputStream = fs.create(testFile);
+ byte[] data5 = ContractTestUtils.dataset(fileSize5, 'a',
+ 26);
+ outputStream.write(data5);
+ byte[] data4 = ContractTestUtils.dataset(fileSize4, 'a',
+ 26);
+ outputStream.write(data4);
+ byte[] data6 = ContractTestUtils.dataset(fileSize6, 'a',
+ 26);
+ outputStream.write(data6);
+ outputStream.hflush();
+ outputStream.write(data5);
+ assertEquals(fileSize5 * 4, fs.getFileStatus(testFile).getLen());
+ outputStream.hsync();
+ assertEquals(fileSize5 * 4, fs.getFileStatus(testFile).getLen());
+
+ outputStream.write(data4);
+ outputStream.hsync();
+ System.out.println(fs.getFileStatus(testFile).getLen());
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ fs.delete(testFile, false);
+ // }
+ }
+
+ @Test
+ // create file write 5M close ,apend10M, write6M ,write5M,write4M close,
+ // 共30M
+ public void testCombination002() throws Exception {
+ // if (fs.getMetricSwitch()) {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("test_file");
+ if (fs.exists(testFile)) {
+ fs.delete(testFile);
+ }
+
+ FSDataOutputStream outputStream = fs.create(testFile);
+ byte[] data5 = ContractTestUtils.dataset(fileSize5, 'a',
+ 26);
+ outputStream.write(data5);
+ outputStream.close();
+
+ outputStream = fs.append(testFile);
+ byte[] data10 = ContractTestUtils.dataset(fileSize5 * 2, 'a',
+ 26);
+ outputStream.write(data10);
+
+
+ byte[] data6 = ContractTestUtils.dataset(fileSize6, 'a',
+ 26);
+ outputStream.write(data6);
+
+ outputStream.write(data5);
+
+
+ byte[] data4 = ContractTestUtils.dataset(fileSize4, 'a',
+ 26);
+ outputStream.write(data4);
+
+ outputStream.close();
+
+ assertEquals(fileSize5 * 6, fs.getFileStatus(testFile).getLen());
+
+ fs.delete(testFile, false);
+ // }
+ }
+
+ @Test
+ // create file write 5M close,apend4M, write6M,hflush,write10M,hflush,close,
+ // 共25M
+ public void testCombination003() throws Exception {
+ // if (fs.getMetricSwitch()) {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("test_file");
+ if (fs.exists(testFile)) {
+ fs.delete(testFile);
+ }
+
+ FSDataOutputStream outputStream = fs.create(testFile);
+ byte[] data5 = ContractTestUtils.dataset(fileSize5, 'a',
+ 26);
+ outputStream.write(data5);
+ outputStream.close();
+
+ outputStream = fs.append(testFile);
+ byte[] data4 = ContractTestUtils.dataset(fileSize4, 'a',
+ 26);
+ outputStream.write(data4);
+
+ byte[] data6 = ContractTestUtils.dataset(fileSize6, 'a',
+ 26);
+ outputStream.write(data6);
+
+ outputStream.hflush();
+ assertEquals(fileSize5 * 3, fs.getFileStatus(testFile).getLen());
+
+ byte[] data10 = ContractTestUtils.dataset(fileSize5 * 2, 'a',
+ 26);
+ outputStream.write(data10);
+ outputStream.hflush();
+ outputStream.close();
+
+ assertEquals(fileSize5 * 5, fs.getFileStatus(testFile).getLen());
+
+ fs.delete(testFile, false);
+ // }
+ }
+
+ public static class PolicyParam {
+ boolean calcMd5;
+ String blockPolicy;
+
+ public PolicyParam(boolean calcMd5, String blockPolicy) {
+ this.calcMd5 = calcMd5;
+ this.blockPolicy = blockPolicy;
+ }
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFastDelete.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFastDelete.java
new file mode 100644
index 0000000..182b8da
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFastDelete.java
@@ -0,0 +1,270 @@
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mockito;
+import org.mockito.runners.MockitoJUnitRunner;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+import org.powermock.reflect.Whitebox;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Matchers.anyString;
+import static org.powermock.api.mockito.PowerMockito.mockStatic;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+import com.obs.services.ObsClient;
+import com.obs.services.exception.ObsException;
+
+@RunWith(MockitoJUnitRunner.class)
+public class ITestOBSFastDelete {
+ OBSFileSystem fs;
+
+ OBSFileSystem mockFs;
+
+ ObsClient obsClient;
+
+ ObsClient mockObsClient;
+
+ static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ private static String trashPathStr = "trash";
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.setBoolean(OBSConstants.FAST_DELETE_ENABLE, true);
+ conf.set(OBSConstants.FAST_DELETE_DIR, trashPathStr);
+ conf.setLong(OBSConstants.RETRY_SLEEP_BASETIME, 2);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ obsClient = fs.getObsClient();
+ initMock();
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null && fs.isFsBucket()) {
+ fs.delete(new Path(testRootPath), true);
+ fs.delete(new Path(fs.getFastDeleteDir()), true);
+ }
+ Mockito.reset(mockFs, mockObsClient);
+ }
+
+ private void initMock() {
+ mockFs = Mockito.spy(fs);
+ mockObsClient = Mockito.spy(obsClient);
+ Whitebox.setInternalState(mockFs, obsClient, mockObsClient);
+ }
+
+ private Path getTestPath(String relativePath) {
+ return new Path(testRootPath + "/ITestOBSTrash/" + relativePath);
+ }
+
+ @Test
+ // 删除目录,校验trash路径下包含被删除的目录
+ public void delFolderToTrash() throws IOException {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ final byte[] data = ContractTestUtils.dataset(1024, 'a', 'z');
+ List pathList = new ArrayList();
+ for (int i = 0; i < 3; i++) {
+ String objectName = "objectINfolder-" + i;
+ Path objectPath = getTestPath(objectName);
+ ContractTestUtils.createFile(fs, objectPath, false, data);
+ pathList.add(objectPath);
+ }
+ fs.delete(new Path(testRootPath), true);
+
+ Path trashPath =
+ new Path(OBSCommonUtils.maybeAddBeginningSlash(trashPathStr));
+ for (Path path : pathList) {
+ assertTrue(fs.exists(new Path(trashPath,
+ OBSCommonUtils.pathToKey(fs, path))));
+ }
+ }
+
+ @Test
+ // 删除文件,校验trash路径下包含被删除的目录
+ public void delObjectToTrash() throws IOException {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ final byte[] data = ContractTestUtils.dataset(1024, 'a', 'z');
+ Path objectPath = getTestPath("test_delete_object");
+ ContractTestUtils.createFile(fs, objectPath, false, data);
+ fs.delete(objectPath, true);
+
+ Path trashPath =
+ new Path(OBSCommonUtils.maybeAddBeginningSlash(trashPathStr));
+ assertTrue(fs.exists(new Path(trashPath,
+ OBSCommonUtils.pathToKey(fs, objectPath))));
+ }
+
+ @Test
+ // 开启trash机制,先后删除两个同名文件,验证第二个文件trash后的文件名中不包含冒号
+ public void testTrashSameNameObject01() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("test_file");
+ FSDataOutputStream outputStream = fs.create(testFile, false);
+ outputStream.close();
+ fs.delete(testFile, false);
+
+ outputStream = fs.create(testFile, false);
+ outputStream.close();
+ fs.delete(testFile, false);
+
+ Path trashPath =
+ new Path(OBSCommonUtils.maybeAddBeginningSlash(trashPathStr));
+ Path trashDir =
+ new Path(trashPath, OBSCommonUtils.pathToKey(fs,
+ testFile.getParent()));
+ FileStatus[] files = fs.listStatus(trashDir);
+ assertTrue(files.length >= 2);
+ for (FileStatus file : files) {
+ String path = file.getPath().toString();
+ assertFalse(path.substring(path.indexOf("://") + 1).contains(":"));
+ }
+
+ fs.delete(trashDir, true);
+ }
+
+ @Test
+ // 开启trash机制,先后删除两个同名非空目录,验证第二个目录trash后的目录名中不包含冒号
+ public void testTrashSameNameObject02() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testDir = getTestPath("test_dir/");
+ Path testFile = getTestPath("test_dir/test_file");
+ fs.mkdirs(testDir);
+ FSDataOutputStream outputStream = fs.create(testFile, false);
+ outputStream.close();
+ fs.delete(testDir, true);
+
+ fs.mkdirs(testDir);
+ outputStream = fs.create(testFile, false);
+ outputStream.close();
+ fs.delete(testDir, true);
+
+ Path trashPath =
+ new Path(OBSCommonUtils.maybeAddBeginningSlash(trashPathStr));
+ Path trashDir =
+ new Path(trashPath, OBSCommonUtils.pathToKey(fs,
+ testDir.getParent()));
+ FileStatus[] files = fs.listStatus(trashDir);
+ assertTrue(files.length >= 2);
+ for (FileStatus file : files) {
+ String path = file.getPath().toString();
+ assertFalse(path.substring(path.indexOf("://") + 1).contains(":"));
+ }
+
+ fs.delete(trashDir, true);
+ }
+
+ @Test
+ // 开启trash机制,先后删除两个同名非空目录和文件,验证第二个文件trash后的文件名中不包含冒号
+ public void testTrashSameNameObject03() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testDir = getTestPath("test_dir/");
+ Path testFile = getTestPath("test_dir/test_file");
+ fs.mkdirs(testDir);
+ FSDataOutputStream outputStream = fs.create(testFile, false);
+ outputStream.close();
+ fs.delete(testDir, true);
+
+ outputStream = fs.create(testDir, false);
+ outputStream.close();
+ fs.delete(testDir, true);
+
+ Path trashPath =
+ new Path(OBSCommonUtils.maybeAddBeginningSlash(trashPathStr));
+ Path trashDir =
+ new Path(trashPath, OBSCommonUtils.pathToKey(fs,
+ testDir.getParent()));
+ FileStatus[] files = fs.listStatus(trashDir);
+ assertTrue(files.length >= 2);
+ for (FileStatus file : files) {
+ String path = file.getPath().toString();
+ assertFalse(path.substring(path.indexOf("://") + 1).contains(":"));
+ }
+
+ fs.delete(trashDir, true);
+ }
+
+ @Test
+ // 开启trash机制,先后删除两个同名非空目录和文件,验证第二个文件trash后的文件名中不包含冒号
+ public void testTrashSameNameObject04() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testDir = getTestPath("test_dir/");
+ Path testFile = getTestPath("test_dir/test_file");
+ fs.mkdirs(testDir);
+ FSDataOutputStream outputStream = fs.create(testFile, false);
+ outputStream.close();
+ fs.delete(testDir, true);
+
+ outputStream = fs.create(testDir, false);
+ outputStream.close();
+ fs.delete(testDir, true);
+
+ Path trashPath =
+ new Path(OBSCommonUtils.maybeAddBeginningSlash(trashPathStr));
+ Path trashDir =
+ new Path(trashPath, OBSCommonUtils.pathToKey(fs,
+ testDir.getParent()));
+ FileStatus[] files = fs.listStatus(trashDir);
+ assertTrue(files.length >= 2);
+ for (FileStatus file : files) {
+ String path = file.getPath().toString();
+ assertFalse(path.substring(path.indexOf("://") + 1).contains(":"));
+ }
+
+ fs.delete(trashDir, true);
+ }
+
+ @Test
+ @Ignore
+ // 测试多客户端执行trash到同一目录时,服务端返409,客户端添加时间戳重试
+ public void testTrashWithConflict() throws Exception {
+
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFastDeleteV2.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFastDeleteV2.java
new file mode 100644
index 0000000..a219e64
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFastDeleteV2.java
@@ -0,0 +1,210 @@
+package org.apache.hadoop.fs.obs;
+
+import static org.apache.hadoop.fs.obs.OBSConstants.FAST_DELETE_VERSION_V2;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mockito;
+import org.mockito.runners.MockitoJUnitRunner;
+import org.powermock.reflect.Whitebox;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.Locale;
+
+@RunWith(MockitoJUnitRunner.class)
+public class ITestOBSFastDeleteV2 extends ITestOBSFastDelete {
+
+ private static String trashPathStr = "trashV2";
+
+ @Before
+ @Override
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.setBoolean(OBSConstants.FAST_DELETE_ENABLE, true);
+ conf.set(OBSConstants.FAST_DELETE_DIR, trashPathStr);
+ conf.set(OBSConstants.FAST_DELETE_VERSION, FAST_DELETE_VERSION_V2);
+ conf.setLong(OBSConstants.RETRY_SLEEP_BASETIME, 2);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ obsClient = fs.getObsClient();
+ initMock();
+ }
+
+ private void initMock() {
+ mockFs = Mockito.spy(fs);
+ mockObsClient = Mockito.spy(obsClient);
+ Whitebox.setInternalState(mockFs, obsClient, mockObsClient);
+ }
+
+ private Path getTestPath(String relativePath) {
+ return new Path(testRootPath + "/ITestOBSTrash/" + relativePath);
+ }
+
+ private Path formatTrashPathV2(String trashPathStr) {
+ SimpleDateFormat dateFmt = new SimpleDateFormat(OBSConstants.FAST_DELETE_VERSION_V2_CHECKPOINT_FORMAT);
+ String checkpointStr = dateFmt.format(new Date());
+ String checkpointDir = String.format(Locale.ROOT, "%s%s/",
+ OBSCommonUtils.maybeAddTrailingSlash(trashPathStr), checkpointStr);
+ return new Path(OBSCommonUtils.maybeAddBeginningSlash(checkpointDir));
+ }
+
+ @Test
+ @Override
+ // 删除目录,校验trash路径下包含被删除的目录
+ public void delFolderToTrash() throws IOException {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ final byte[] data = ContractTestUtils.dataset(1024, 'a', 'z');
+ List pathList = new ArrayList();
+ for (int i = 0; i < 3; i++) {
+ String objectName = "objectINfolder-" + i;
+ Path objectPath = getTestPath(objectName);
+ ContractTestUtils.createFile(fs, objectPath, false, data);
+ pathList.add(objectPath);
+ }
+ fs.delete(new Path(testRootPath), true);
+
+ Path trashPath = formatTrashPathV2(trashPathStr);
+
+ for (Path path : pathList) {
+ assertTrue(fs.exists(new Path(trashPath,
+ OBSCommonUtils.pathToKey(fs, path))));
+ }
+ }
+
+ @Test
+ @Override
+ // 删除文件,校验trash路径下包含被删除的目录
+ public void delObjectToTrash() throws IOException {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ final byte[] data = ContractTestUtils.dataset(1024, 'a', 'z');
+ Path objectPath = getTestPath("test_delete_object");
+ ContractTestUtils.createFile(fs, objectPath, false, data);
+ fs.delete(objectPath, true);
+
+ Path trashPath = formatTrashPathV2(trashPathStr);
+
+ assertTrue(fs.exists(new Path(trashPath,
+ OBSCommonUtils.pathToKey(fs, objectPath))));
+ }
+
+
+ @Test
+ @Override
+ // 开启trash机制,先后删除两个同名文件,验证第二个文件trash后的文件名中不包含冒号
+ public void testTrashSameNameObject01() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("test_file");
+ FSDataOutputStream outputStream = fs.create(testFile, false);
+ outputStream.close();
+ fs.delete(testFile, false);
+
+ outputStream = fs.create(testFile, false);
+ outputStream.close();
+ fs.delete(testFile, false);
+
+ Path trashPath = formatTrashPathV2(trashPathStr);
+ Path trashDir =
+ new Path(trashPath, OBSCommonUtils.pathToKey(fs,
+ testFile.getParent()));
+ FileStatus[] files = fs.listStatus(trashDir);
+ assertTrue(files.length >= 2);
+ for (FileStatus file : files) {
+ String path = file.getPath().toString();
+ assertFalse(path.substring(path.indexOf("://") + 1).contains(":"));
+ }
+
+ fs.delete(trashDir, true);
+ }
+
+ @Test
+ @Override
+ // 开启trash机制,先后删除两个同名非空目录,验证第二个目录trash后的目录名中不包含冒号
+ public void testTrashSameNameObject02() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testDir = getTestPath("test_dir/");
+ Path testFile = getTestPath("test_dir/test_file");
+ fs.mkdirs(testDir);
+ FSDataOutputStream outputStream = fs.create(testFile, false);
+ outputStream.close();
+ fs.delete(testDir, true);
+
+ fs.mkdirs(testDir);
+ outputStream = fs.create(testFile, false);
+ outputStream.close();
+ fs.delete(testDir, true);
+
+ Path trashPath = formatTrashPathV2(trashPathStr);
+ Path trashDir =
+ new Path(trashPath, OBSCommonUtils.pathToKey(fs,
+ testDir.getParent()));
+ FileStatus[] files = fs.listStatus(trashDir);
+ assertTrue(files.length >= 2);
+ for (FileStatus file : files) {
+ String path = file.getPath().toString();
+ assertFalse(path.substring(path.indexOf("://") + 1).contains(":"));
+ }
+
+ fs.delete(trashDir, true);
+ }
+
+ @Test
+ @Override
+ // 开启trash机制,先后删除两个同名非空目录和文件,验证第二个文件trash后的文件名中不包含冒号
+ public void testTrashSameNameObject03() throws Exception {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testDir = getTestPath("test_dir/");
+ Path testFile = getTestPath("test_dir/test_file");
+ fs.mkdirs(testDir);
+ FSDataOutputStream outputStream = fs.create(testFile, false);
+ outputStream.close();
+ fs.delete(testDir, true);
+
+ outputStream = fs.create(testDir, false);
+ outputStream.close();
+ fs.delete(testDir, true);
+
+ Path trashPath = formatTrashPathV2(trashPathStr);
+ Path trashDir =
+ new Path(trashPath, OBSCommonUtils.pathToKey(fs,
+ testDir.getParent()));
+ FileStatus[] files = fs.listStatus(trashDir);
+ assertTrue(files.length >= 2);
+ for (FileStatus file : files) {
+ String path = file.getPath().toString();
+ assertFalse(path.substring(path.indexOf("://") + 1).contains(":"));
+ }
+
+ fs.delete(trashDir, true);
+ }
+
+ @Test
+ @Ignore
+ @Override
+ // 重复用例,不需要跑
+ public void testTrashSameNameObject04() throws Exception {
+ }
+
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFileSystem.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFileSystem.java
new file mode 100644
index 0000000..5cae078
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFileSystem.java
@@ -0,0 +1,260 @@
+/**
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import com.obs.services.ObsClient;
+import com.obs.services.model.ListBucketsRequest;
+import com.obs.services.model.ObsBucket;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Collection;
+import java.util.List;
+
+public class ITestOBSFileSystem {
+
+ private static final int EXPECTED_PORT = -1;
+
+ private static final String EXPECTED_SCHEMA = "obs";
+
+ private Configuration conf;
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ conf = OBSContract.getConfiguration(null);
+ }
+
+ @Test
+ // 通过configuration初始化FS,功能正常
+ public void testInitialization() throws IOException {
+ String bucketName = conf.get(OBSTestConstants.TEST_FS_OBS_NAME);
+ initializationTest(bucketName);
+ }
+
+ @Test
+ public void testInitializeNotExistsBucket() throws URISyntaxException, IOException {
+ String initializationUri = conf.get(OBSTestConstants.TEST_FS_OBS_NAME);
+ URI uri = new URI(initializationUri);
+ String scheme = uri.getScheme();
+ String bucketName = uri.getAuthority();
+ while (true) {
+ bucketName = String.format("%s-not-exists", bucketName);
+ OBSFileSystem fs = new OBSFileSystem();
+ try {
+ initializationUri = String.format("%s://%s", scheme, bucketName);
+ fs.initialize(URI.create(initializationUri), conf);
+ } catch (FileNotFoundException e) {
+ assertEquals("Bucket " + bucketName + " does not exist", e.getMessage());
+ return;
+ }
+ List buckets = getBucketList(fs.getObsClient());
+ boolean isBucketInList = false;
+ for (ObsBucket bucket : buckets) {
+ if (bucket.getBucketName().equals(fs.getBucket())) {
+ isBucketInList = true;
+ break;
+ }
+ }
+ if (isBucketInList) {
+ continue;
+ }
+ assertTrue("not in bucket list, but initalize success", false);
+ }
+ }
+
+ @Test
+ public void testGetTrashRootV1() throws IOException {
+ Path path = new Path("test");
+ try (OBSFileSystem fs = OBSTestUtils.createTestFileSystem(conf)) {
+ Path obsTrashRoot = fs.getTrashRoot(path);
+ Path oldTrashRoot = fs.makeQualified(new Path(fs.getHomeDirectory().toUri().getPath(), ".Trash"));
+ assertEquals(oldTrashRoot, obsTrashRoot);
+ }
+ }
+
+ @Test
+ public void testGetTrashRootV2() throws IOException {
+ Path path = new Path("test");
+ Configuration newConf = new Configuration(conf);
+ newConf.set(OBSConstants.HDFS_TRASH_VERSION, OBSConstants.HDFS_TRASH_VERSION_V2);
+ try (OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(newConf)) {
+ Path obsTrashRoot2 = newFs.getTrashRoot(path);
+ Path ourTrashRoot =
+ newFs.makeQualified(new Path(OBSConstants.DEFAULT_HDFS_TRASH_PREFIX, OBSFileSystem.getUsername()));
+ assertEquals(ourTrashRoot, obsTrashRoot2);
+ }
+ }
+
+ @Test
+ public void testCustomTrashPrefix() throws IOException {
+ Path path = new Path("test");
+ Configuration newConf = new Configuration(conf);
+ newConf.set(OBSConstants.HDFS_TRASH_VERSION, OBSConstants.HDFS_TRASH_VERSION_V2);
+ String prefix = "/tmp/.trash";
+ newConf.set(OBSConstants.HDFS_TRASH_PREFIX, prefix);
+ try (OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(newConf)) {
+ Path obsTrashRoot2 = newFs.getTrashRoot(path);
+ Path ourTrashRoot =
+ newFs.makeQualified(new Path(prefix, OBSFileSystem.getUsername()));
+ assertEquals(ourTrashRoot, obsTrashRoot2);
+ }
+ }
+
+ @Test
+ public void testGetAllTrashRootsV1() throws IOException {
+ testGetTrashRootsV1(true);
+ }
+
+ @Test
+ public void testGetCurUserTrashRootsV1() throws IOException {
+ testGetTrashRootsV1(false);
+ }
+
+ @Test
+ public void testGetAllTrashRootsV2() throws IOException {
+ testGetTrashRootsV2(true, null);
+ }
+
+ @Test
+ public void testCustomPrefixGetAllTrashRootsV2() throws IOException {
+ testGetTrashRootsV2(true, "/tmp/.Trash");
+ }
+
+ @Test
+ public void testGetCurUserTrashRootsV2() throws IOException {
+ testGetTrashRootsV2(false, null);
+ }
+
+ @Test
+ public void testCustomPrefixGetCurUserTrashRootsV2() throws IOException {
+ testGetTrashRootsV2(false, "/tmp/.Trash");
+ }
+
+ public void testGetTrashRootsV1(boolean allUsers) throws IOException {
+ OBSFileSystem fs = OBSTestUtils.createTestFileSystem(conf);
+ Path user66TrashRoot = new Path(fs.getHomeDirectory().getParent(), "user66/.Trash");
+ Path curUserTrashRoot = fs.makeQualified(new Path(fs.getHomeDirectory().toUri().getPath(), ".Trash"));
+ try {
+ fs.mkdirs(user66TrashRoot);
+ fs.mkdirs(curUserTrashRoot);
+ Collection trashRoots = fs.getTrashRoots(allUsers);
+ if (allUsers) {
+ assertTrue(containsPathInFileStatuses(trashRoots, user66TrashRoot));
+ assertTrue(containsPathInFileStatuses(trashRoots, curUserTrashRoot));
+ } else {
+ assertEquals(1, trashRoots.size());
+ assertEquals(curUserTrashRoot, trashRoots.toArray(new FileStatus[0])[0].getPath());
+ }
+ } finally {
+ fs.delete(user66TrashRoot.getParent(), true);
+ fs.delete(curUserTrashRoot.getParent(), true);
+ fs.close();
+ }
+ }
+
+ public void testGetTrashRootsV2(boolean allUsers, String hdfsTrashPrefix) throws IOException {
+ Configuration newConf = new Configuration(conf);
+ newConf.set(OBSConstants.HDFS_TRASH_VERSION, OBSConstants.HDFS_TRASH_VERSION_V2);
+ if (hdfsTrashPrefix == null) {
+ hdfsTrashPrefix = OBSConstants.DEFAULT_HDFS_TRASH_PREFIX;
+ } else {
+ newConf.set(OBSConstants.HDFS_TRASH_PREFIX, hdfsTrashPrefix);
+ }
+
+ OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(newConf);
+ Path ourCurUserTrashRoot = newFs.makeQualified(new Path(hdfsTrashPrefix, OBSFileSystem.getUsername()));
+ Path user88TrashDir = new Path(ourCurUserTrashRoot.getParent(), "user88");
+ try {
+ newFs.mkdirs(user88TrashDir);
+ newFs.mkdirs(ourCurUserTrashRoot);
+ Collection trashRoots = newFs.getTrashRoots(allUsers);
+ if (allUsers) {
+ assertTrue(containsPathInFileStatuses(trashRoots, user88TrashDir));
+ assertTrue(containsPathInFileStatuses(trashRoots, ourCurUserTrashRoot));
+ } else {
+ assertEquals(1, trashRoots.size());
+ assertEquals(ourCurUserTrashRoot, trashRoots.toArray(new FileStatus[0])[0].getPath());
+ }
+ } finally {
+ newFs.delete(user88TrashDir, true);
+ newFs.delete(ourCurUserTrashRoot, true);
+ newFs.close();
+ }
+ }
+
+ private boolean containsPathInFileStatuses(Collection fileStatuses, Path path) {
+ for (FileStatus status : fileStatuses) {
+ if (status.getPath().equals(path)) {
+ return true;
+ }
+ }
+
+ return false;
+ }
+
+ private void initializationTest(String initializationUri) throws IOException {
+ OBSFileSystem fs = new OBSFileSystem();
+ fs.initialize(URI.create(initializationUri), conf);
+ ObsClient obsTest = fs.getObsClient();
+ List buckets = getBucketList(obsTest);
+
+ boolean isBucketInList = false;
+ for (ObsBucket bucket : buckets) {
+ if (bucket.getBucketName().equals(fs.getBucket())) {
+ isBucketInList = true;
+ break;
+ }
+ }
+
+ URI EXPECTED_URI = URI.create(conf.get(OBSTestConstants.TEST_FS_OBS_NAME));
+ assertTrue(isBucketInList);
+ assertEquals(EXPECTED_SCHEMA, fs.getScheme());
+ assertEquals(EXPECTED_URI, fs.getUri());
+ assertEquals(EXPECTED_PORT, fs.getDefaultPort());
+ }
+
+ private List getBucketList(ObsClient obsTest) {
+ ListBucketsRequest request = new ListBucketsRequest();
+ request.setQueryLocation(true);
+ List buckets = obsTest.listBuckets(request);
+ return buckets;
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFileSystemContract.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFileSystemContract.java
new file mode 100644
index 0000000..b84ea62
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFileSystemContract.java
@@ -0,0 +1,722 @@
+/*
+ * 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.FileStatus;
+import org.apache.hadoop.fs.FileSystemContractBaseTest;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static junit.framework.TestCase.assertTrue;
+import static junit.framework.TestCase.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * 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 ITestOBSFileSystemContract extends FileSystemContractBaseTest {
+
+ protected static final Logger LOG =
+ LoggerFactory.getLogger(ITestOBSFileSystemContract.class);
+
+ private Path basePath;
+
+ private OBSFileSystem obsFS;
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @Rule
+ public TestName methodName = new TestName();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ obsFS = OBSTestUtils.createTestFileSystem(conf);
+ fs = obsFS;
+ basePath = fs.makeQualified(
+ OBSTestUtils.createTestPath(new Path("/obsfilesystemcontract")));
+ }
+
+ @Override
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(basePath, true);
+ }
+ super.tearDown();
+ }
+
+ @Override
+ public void testMkdirsWithUmask() throws Exception {
+ // not supported
+ }
+
+ @Override
+ // rename源目录到已存在目标目录,源目录下对象被rename到目标目录下
+ public void testRenameDirectoryAsExistingDirectory() throws Exception {
+ if (!renameSupported()) {
+ return;
+ }
+
+ // Prepare the source folder with some nested files or sub folders.
+ Path src = path("/test/hadoop/dir");
+ fs.mkdirs(src);
+
+ createFile(path("/test/hadoop/dir/file1"));
+ createFile(path("/test/hadoop/dir/subdir1/file11"));
+ createFile(path("/test/hadoop/dir/subdir2/file21"));
+ createFile(path("/test/hadoop/dir/subdir1/subdir11/file111"));
+ createFile(path("/test/hadoop/dir/file2"));
+ fs.mkdirs(path("/test/hadoop/dir/subdir3/subdir31"));
+
+ // Prepare the destination folder.
+ Path dst = path("/test/new/newdir");
+ fs.mkdirs(dst);
+
+ // Execute the rename operation.
+ rename(src, dst, true, false, true);
+
+ // Assert that all nested files or sub folders under the source
+ // folder should be not existed after rename.
+ assertFalse("Nested file1 exists",
+ fs.exists(path("/test/hadoop/dir/file1")));
+ assertFalse("Nested file2 exists",
+ fs.exists(path("/test/hadoop/dir/file2")));
+ assertFalse("Nested file11 exists",
+ fs.exists(path("/test/hadoop/dir/subdir1/file11")));
+ assertFalse("Nested file111 exists",
+ fs.exists(path("/test/hadoop/dir/subdir1/subdir11/file111")));
+ assertFalse("Nested file111 exists",
+ fs.exists(path("/test/hadoop/dir/subdir1/subdir11")));
+ assertFalse("Nested file111 exists",
+ fs.exists(path("/test/hadoop/dir/subdir1")));
+ assertFalse("Nested file21 exists",
+ fs.exists(path("/test/hadoop/dir/subdir2/file21")));
+ assertFalse("Nested file21 exists",
+ fs.exists(path("/test/hadoop/dir/subdir2")));
+ assertFalse("Nested subdir31 exists",
+ fs.exists(path("/test/hadoop/dir/subdir3/subdir31")));
+ assertFalse("Nested subdir31 exists",
+ fs.exists(path("/test/hadoop/dir/subdir3")));
+ assertFalse("Nested subdir31 exists",
+ fs.exists(path("/test/hadoop/dir")));
+
+ // Assert that all nested files or sub folders should be under the destination folder after rename.
+ assertTrue("Renamed nested file1 exists",
+ fs.exists(path("/test/new/newdir/dir/file1")));
+ assertTrue("Renamed nested file2 exists",
+ fs.exists(path("/test/new/newdir/dir/file2")));
+ assertTrue("Renamed nested file11 exists",
+ fs.exists(path("/test/new/newdir/dir/subdir1")));
+ assertTrue("Renamed nested file11 exists",
+ fs.exists(path("/test/new/newdir/dir/subdir1/file11")));
+ assertTrue("Renamed nested file111 exists",
+ fs.exists(path("/test/new/newdir/dir/subdir1/subdir11")));
+ assertTrue("Renamed nested file111 exists",
+ fs.exists(path("/test/new/newdir/dir/subdir1/subdir11/file111")));
+ assertTrue("Renamed nested file21 exists",
+ fs.exists(path("/test/new/newdir/dir/subdir2")));
+ assertTrue("Renamed nested file21 exists",
+ fs.exists(path("/test/new/newdir/dir/subdir2/file21")));
+ assertTrue("Renamed nested file21 exists",
+ fs.exists(path("/test/new/newdir/dir/subdir3")));
+ assertTrue("Renamed nested subdir31 exists",
+ fs.exists(path("/test/new/newdir/dir/subdir3/subdir31")));
+ }
+
+ @Override
+ //
+ public void testRenameDirectoryMoveToExistingDirectory() throws Exception {
+ if (this.renameSupported()) {
+ Path src = this.path("/test/hadoop/dir");
+ this.fs.mkdirs(src);
+ this.createFile(this.path("/test/hadoop/dir/file1"));
+ this.createFile(this.path("/test/hadoop/dir/subdir/file2"));
+ Path dst = this.path("/test/new/newdir");
+ this.fs.mkdirs(dst.getParent());
+ this.rename(src, dst, true, false, true);
+
+ assertFalse("Nested file2 exists",
+ this.fs.exists(this.path("/test/hadoop/dir")));
+ assertTrue("Renamed nested newdir exists",
+ this.fs.exists(this.path("/test/new/newdir")));
+ assertTrue("Renamed nested file1 exists",
+ this.fs.exists(this.path("/test/new/newdir/file1")));
+ assertTrue("Renamed nested subdir exists",
+ this.fs.exists(this.path("/test/new/newdir/subdir")));
+ assertTrue("Renamed nested file2 exists",
+ this.fs.exists(this.path("/test/new/newdir/subdir/file2")));
+ }
+ }
+
+ // @Override
+ public void testMoveDirUnderParent() throws Throwable {
+ // not support because
+ // Fails if dst is a directory that is not empty.
+ }
+
+ public void testRecursivelyDeleteDirectory() throws Exception {
+
+ // Prepare the source folder with some nested files or sub folders.
+ Path p_dir = path("/test/hadoop/dir");
+ fs.mkdirs(p_dir);
+
+ createFile(path("/test/hadoop/dir/FILE_0"));
+ createFile(path("/test/hadoop/dir/SDIR_0/FILE_00"));
+ createFile(path("/test/hadoop/dir/SDIR_0/SDIR_00/FILE_000"));
+ fs.mkdirs(path("/test/hadoop/dir/SDIR_0/SDIR_00/SDIR_000"));
+ createFile(path("/test/hadoop/dir/SDIR_0/SDIR_00/file_001"));
+ fs.mkdirs(path("/test/hadoop/dir/SDIR_0/SDIR_00/sdir_001"));
+ createFile(path("/test/hadoop/dir/SDIR_0/file_01"));
+ createFile(path("/test/hadoop/dir/SDIR_0/sdir_01/FILE_010"));
+ fs.mkdirs(path("/test/hadoop/dir/SDIR_0/sdir_01/SDIR_010"));
+ createFile(path("/test/hadoop/dir/SDIR_0/sdir_01/file_011"));
+ fs.mkdirs(path("/test/hadoop/dir/SDIR_0/sdir_01/sdir_011"));
+ createFile(path("/test/hadoop/dir/file_1"));
+ createFile(path("/test/hadoop/dir/sdir_1/FILE_10"));
+ createFile(path("/test/hadoop/dir/sdir_1/SDIR_10/FILE_100"));
+ fs.mkdirs(path("/test/hadoop/dir/sdir_1/SDIR_10/SDIR_100"));
+ createFile(path("/test/hadoop/dir/sdir_1/SDIR_10/file_101"));
+ fs.mkdirs(path("/test/hadoop/dir/sdir_1/SDIR_10/sdir_101"));
+ createFile(path("/test/hadoop/dir/sdir_1/file_11"));
+ createFile(path("/test/hadoop/dir/sdir_1/sdir_11/FILE_110"));
+ fs.mkdirs(path("/test/hadoop/dir/sdir_1/sdir_11/SDIR_110"));
+ createFile(path("/test/hadoop/dir/sdir_1/sdir_11/file_111"));
+ fs.mkdirs(path("/test/hadoop/dir/sdir_1/sdir_11/sdir_111"));
+
+ // Assert that all nested files or sub folders under the source
+ // folder should be existed before delete.
+ assertTrue("created", fs.exists(path("/test")));
+ assertTrue("created", fs.exists(path("/test/hadoop")));
+ assertTrue("created", fs.exists(path("/test/hadoop/dir")));
+ assertTrue("created", fs.exists(path("/test/hadoop/dir/FILE_0")));
+ assertTrue("created", fs.exists(path("/test/hadoop/dir/SDIR_0")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/FILE_00")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00/FILE_000")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00/SDIR_000")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00/file_001")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00/sdir_001")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/file_01")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01/FILE_010")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01/SDIR_010")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01/file_011")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01/sdir_011")));
+ assertTrue("created", fs.exists(path("/test/hadoop/dir/file_1")));
+ assertTrue("created", fs.exists(path("/test/hadoop/dir/sdir_1")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/FILE_10")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10/FILE_100")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10/SDIR_100")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10/file_101")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10/sdir_101")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/file_11")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11/FILE_110")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11/SDIR_110")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11/file_111")));
+ assertTrue("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11/sdir_111")));
+
+ assertTrue(fs.delete(p_dir, true));
+
+ // Assert that all nested files or sub folders under the source
+ // folder should be not existed after delete.
+ assertFalse("created", fs.exists(path("/test/hadoop/dir")));
+ assertFalse("created", fs.exists(path("/test/hadoop/dir/FILE_0")));
+ assertFalse("created", fs.exists(path("/test/hadoop/dir/SDIR_0")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/FILE_00")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00/FILE_000")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00/SDIR_000")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00/file_001")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00/sdir_001")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/file_01")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01/FILE_010")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01/SDIR_010")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01/file_011")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01/sdir_011")));
+ assertFalse("created", fs.exists(path("/test/hadoop/dir/file_1")));
+ assertFalse("created", fs.exists(path("/test/hadoop/dir/sdir_1")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/FILE_10")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10/FILE_100")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10/SDIR_100")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10/file_101")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10/sdir_101")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/file_11")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11/FILE_110")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11/SDIR_110")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11/file_111")));
+ assertFalse("created",
+ fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11/sdir_111")));
+ assertTrue("created", fs.exists(path("/test")));
+ assertTrue("created", fs.exists(path("/test/hadoop")));
+
+ return;
+ }
+
+ private long getDigitsNumber(long num) {
+ long num1 = num - 1;
+ long len = 0;
+ while (num1 > 0) {
+ len++;
+ num1 = num1 / 10;
+ }
+ return (len == 0 ? 1 : len);
+ }
+
+ private long prepareObjectsTree(String parent, int[] arWidth,
+ String[] arNamePrefix, boolean isFile)
+ throws Exception {
+ // Check the input parameters.
+ int depth = arWidth.length;
+ if (arNamePrefix.length != depth) {
+ throw new Exception("Unmatched depth.");
+ }
+
+ String digitFormats[] = new String[depth];
+ for (int dep = 0; dep < depth; dep++) {
+ if (arWidth[dep] <= 0) {
+ throw new Exception("The width of any level must be positive.");
+ }
+ digitFormats[dep] = String.format("%%0%dd",
+ getDigitsNumber(arWidth[dep]));
+ }
+
+ String p_key = (!parent.endsWith("/")
+ ? parent
+ : parent.substring(0, parent.length() - 1));
+
+ long totalNum = 0;
+ long leavesNum = 1;
+ long factors[] = new long[depth];
+ for (int dep = 0; dep < depth; dep++) {
+ if (dep == 0) {
+ factors[depth - 1] = 1;
+ } else {
+ factors[depth - dep - 1] = factors[depth - dep] * arWidth[depth
+ - dep];
+ }
+ leavesNum = leavesNum * arWidth[dep];
+ totalNum += leavesNum;
+ }
+
+ for (long seq = 0; seq < leavesNum; seq++) {
+ String seqStr = p_key;
+ String digitStr = "";
+ long seqTmp = seq;
+ for (int dep = 0; dep < depth; dep++) {
+ long fac = factors[dep];
+ digitStr = digitStr + "_" + String.format(digitFormats[dep],
+ seqTmp / fac);
+ seqTmp = seqTmp % fac;
+ seqStr = seqStr + "/" + arNamePrefix[dep] + digitStr;
+ }
+ if (isFile) {
+ createFile(path(seqStr));
+ } else {
+ fs.mkdirs(path(seqStr));
+ }
+ }
+
+ return totalNum;
+ }
+
+ private long prepareFilesTree(String parent, int[] arWidth,
+ String[] arNamePrefix)
+ throws Exception {
+ return prepareObjectsTree(parent, arWidth, arNamePrefix, true);
+ }
+
+ private long prepareFoldersTree(String parent, int[] arWidth,
+ String[] arNamePrefix)
+ throws Exception {
+ return prepareObjectsTree(parent, arWidth, arNamePrefix, false);
+ }
+
+ private long prepareObjectsTree(String parent, int objNumOfEachPart)
+ throws Exception {
+ Path parentPath = path(parent);
+ fs.mkdirs(parentPath);
+
+ long partsNum = 0;
+ long totalNum = 0;
+
+ // Prepare files tree.
+ int[] arWidth1 = {objNumOfEachPart};
+ String[] arPrefix1 = {"file"};
+ partsNum = prepareFilesTree(parent, arWidth1, arPrefix1);
+ assertEquals(objNumOfEachPart, partsNum);
+ totalNum += partsNum;
+
+ int[] arWidth2 = {objNumOfEachPart};
+ String[] arPrefix2 = {"FILE"};
+ partsNum = prepareFilesTree(parent, arWidth2, arPrefix2);
+ assertEquals(objNumOfEachPart, partsNum);
+ totalNum += partsNum;
+
+ int[] arWidth3 = {objNumOfEachPart, 2};
+ String[] arPrefix3 = {"sdir", "FILE"};
+ partsNum = prepareFilesTree(parent, arWidth3, arPrefix3);
+ assertEquals(objNumOfEachPart * 3, partsNum);
+ totalNum += partsNum;
+
+ int[] arWidth4 = {objNumOfEachPart, 2};
+ String[] arPrefix4 = {"SDIR", "file"};
+ partsNum = prepareFilesTree(parent, arWidth4, arPrefix4);
+ assertEquals(objNumOfEachPart * 3, partsNum);
+ totalNum += partsNum;
+
+ return totalNum;
+ }
+
+ public class ObjectClusterDescription {
+ boolean isFile;
+
+ String prefix;
+
+ int width;
+
+ String seqFmt;
+
+ public ObjectClusterDescription(boolean isFile, String prefix,
+ int width) {
+ this.isFile = isFile;
+ this.prefix = prefix;
+ this.width = width;
+ if (this.width > 0) {
+ this.seqFmt = String.format("%%0%dd",
+ getDigitsNumber(this.width));
+ } else {
+ this.seqFmt = null;
+ }
+ }
+
+ public void statistic(ObjectTreeInfo info) {
+ if (this.isFile) {
+ info.filesNum += this.width;
+ } else {
+ info.foldersNum += this.width;
+ }
+ }
+ }
+
+ public class ObjectTreeInfo {
+ public int filesNum = 0;
+
+ public int foldersNum = 0;
+
+ public int leafFoldersNum = 0;
+
+ public ObjectTreeInfo() {
+ reset();
+ }
+
+ public ObjectTreeInfo(int filesNum, int foldersNum,
+ int leafFoldersNum) {
+ this.filesNum = filesNum;
+ this.foldersNum = foldersNum;
+ this.leafFoldersNum = leafFoldersNum;
+ }
+
+ public boolean isEmtpy() {
+ return ((this.filesNum == 0) && (this.foldersNum == 0));
+ }
+
+ public void reset() {
+ filesNum = 0;
+ foldersNum = 0;
+ leafFoldersNum = 0;
+ }
+ }
+
+ private ObjectTreeInfo getObjectTreeInfoOfOneDepth(
+ ObjectClusterDescription[] arDesc) {
+ ObjectTreeInfo info = new ObjectTreeInfo();
+ for (int i = 0; i < arDesc.length; i++) {
+ if (arDesc[i] == null) {
+ continue;
+ }
+ arDesc[i].statistic(info);
+ }
+ return info;
+ }
+
+ // prepare sub objects of one cluster that described by one object
+ // cluster description.
+ private void prepareObjectsOfOneCluster(ObjectClusterDescription desc,
+ String parent,
+ boolean atDeepestDepth,
+ List leaves,
+ List currFolders) {
+ if (desc == null) {
+ return;
+ }
+ for (int seq = 0; seq < desc.width; seq++) {
+ String key = parent + desc.prefix + "_" + String.format(desc.seqFmt,
+ seq);
+ if (desc.isFile) {
+ // file must be leaf.
+ leaves.add(key);
+ continue;
+ }
+ // folder
+ key += "/";
+ if (atDeepestDepth) {
+ // folder at deepest depth is leaf.
+ leaves.add(key);
+ }
+ currFolders.add(key);
+ }
+ }
+
+ // prepare sub objects of clusters that described by an array of object
+ // clusters description.
+ private void prepareSonObjectsOfOneFolder(ObjectClusterDescription[] arDesc,
+ String parentFolder,
+ boolean deepest,
+ List leaves,
+ List currFolders) {
+ for (int clusterIdx = 0; clusterIdx < arDesc.length; clusterIdx++) {
+ ObjectClusterDescription desc = arDesc[clusterIdx];
+ prepareObjectsOfOneCluster(desc, parentFolder, deepest, leaves,
+ currFolders);
+ }
+ }
+
+ private void createLeafObjects(List leaves) throws IOException {
+ for (int i = 0; i < leaves.size(); i++) {
+ String key = leaves.get(i);
+ if (key.endsWith("/")) {
+ // Create leaf folder.
+ fs.mkdirs(path(key.substring(0, key.length() - 1)));
+ } else {
+ // Create file.
+ createFile(path(key));
+ }
+ }
+ }
+
+ private ObjectTreeInfo getObjectTreeInfo(
+ ObjectClusterDescription[][] arDescription) {
+ int filesNum = 0;
+ int foldersNum = 0;
+ int foldersNumAtPreviousDepth = 1;
+ int depth = arDescription.length;
+ assertTrue(depth > 0);
+
+ for (int currDepth = 0; currDepth < depth; currDepth++) {
+ ObjectClusterDescription[] desc = arDescription[currDepth];
+
+ // Statistic the number of files and folders at current depth of
+ // a parent folder at previous depth.
+ ObjectTreeInfo info = getObjectTreeInfoOfOneDepth(desc);
+ assertTrue(!info.isEmtpy());
+
+ // Statistic the number of files at current depth of all parent
+ // folders at previous depth.
+ info.filesNum *= foldersNumAtPreviousDepth;
+ filesNum += info.filesNum;
+
+ // Statistic the number of folders at current depth of all parent
+ // folders at previous depth.
+ info.foldersNum *= foldersNumAtPreviousDepth;
+ foldersNum += info.foldersNum;
+
+ // Assert that the depth that has no folder must be the deepest one.
+ assertTrue((info.foldersNum > 0) || ((currDepth + 1) == depth));
+
+ // Save the number of folders at current depth and and the
+ // deepest folders are leaves.
+ foldersNumAtPreviousDepth = info.foldersNum;
+ }
+
+ return new ObjectTreeInfo(filesNum, foldersNum,
+ foldersNumAtPreviousDepth);
+ }
+
+ public long prepareObjectsTree(String parent,
+ ObjectClusterDescription[][] arDescription,
+ List leaves)
+ throws Exception {
+ Path parentPath = path(parent);
+ fs.mkdirs(parentPath);
+ parent = (parent.endsWith("/") ? parent : (parent + "/"));
+
+ // Statistic the number of sub objects of parent.
+ ObjectTreeInfo info = getObjectTreeInfo(arDescription);
+
+ // allocate two lists to save folders names at two adjacent depth.
+ int leavesNum = info.filesNum + info.leafFoldersNum;
+ List prevFolders = null;
+ List currFolders = new ArrayList(1);
+ currFolders.add(parent);
+
+ // prepare sub objects of each depth one by one.
+ int depth = arDescription.length;
+ for (int currDepth = 0; currDepth < depth; currDepth++) {
+ boolean deepest = ((currDepth + 1) == depth);
+
+ // Swap the two lists.
+ prevFolders = currFolders;
+ currFolders = new ArrayList(prevFolders.size());
+
+ // prepare sub objects at current depth.
+ ObjectClusterDescription[] arDesc = arDescription[currDepth];
+ for (int i = 0; i < prevFolders.size(); i++) {
+ // prepare sub objects of current parent folder.
+ prepareSonObjectsOfOneFolder(arDesc, prevFolders.get(i),
+ deepest, leaves, currFolders);
+ }
+
+ if (currFolders.size() == 0) {
+ // Current depth has no folder, so it must be deepest one.
+ break;
+ }
+ }
+ assertEquals(leavesNum, leaves.size());
+
+ // Create all leaf objects.
+ createLeafObjects(leaves);
+
+ // Check that each node in each path of leaves should be existed
+ // after preparation.
+ assertPathStatus(parent, leaves, true);
+
+ return (info.filesNum + info.foldersNum);
+ }
+
+ private void assertDepth(FileStatus[] arFileStatus) {
+ int currDepth = Integer.MAX_VALUE;
+ for (int i = arFileStatus.length - 1; i >= 0; i--) {
+ int prevDepth = currDepth;
+ currDepth = arFileStatus[i].getPath().depth();
+ if (currDepth > prevDepth) {
+ assertTrue(false);
+ }
+ }
+ }
+
+ private void assertObjectStatus(Path f, boolean isExisted) {
+ try {
+ fs.getFileStatus(f);
+ assertTrue(isExisted);
+ } catch (FileNotFoundException e) {
+ assertFalse(isExisted);
+ } catch (IOException e) {
+ fail(e.toString());
+ }
+ }
+
+ private void assertPathStatus(String parent, String key,
+ boolean isExisted) {
+ assertTrue(key.startsWith(parent));
+ Path parentPath = path(parent);
+ Path currPath = path(key);
+ while (!currPath.equals(parentPath)) {
+ assertObjectStatus(currPath, isExisted);
+ currPath = currPath.getParent();
+ assertFalse(currPath.isRoot());
+ }
+ }
+
+ private void assertPathStatus(String parent, List leaves,
+ boolean isExisted) {
+ for (int i = 0; i < leaves.size(); i++) {
+ String key = leaves.get(i);
+ assertPathStatus(parent, key, isExisted);
+ }
+ assertObjectStatus(path(parent), isExisted);
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetAndSetWorkingDirectory.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetAndSetWorkingDirectory.java
new file mode 100644
index 0000000..59afdd3
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetAndSetWorkingDirectory.java
@@ -0,0 +1,93 @@
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class ITestOBSGetAndSetWorkingDirectory {
+ private OBSFileSystem fs;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(new Path(testRootPath), true);
+ }
+ }
+
+ @Test
+ // 设置WorkingDirectory时相对路径转化为绝对路径赋值给当前工作目录workingDir
+ public void testWorkingDirectory01() throws Exception {
+ String path = "a001/b001";
+ Path workPath = new Path(path);
+ fs.setWorkingDirectory(workPath);
+ assertTrue(fs.getWorkingDirectory()
+ .toString()
+ .startsWith(fs.getUri().toString()));
+ }
+
+ @Test
+ // 设置WorkingDirectory时绝对路径赋值给当前工作目录workingDir
+ public void testWorkingDirectory02() throws Exception {
+ String path = fs.getUri().toString() + "/" + "a001/b001";
+ Path workPath = new Path(path);
+ fs.setWorkingDirectory(workPath);
+ assertEquals(workPath, fs.getWorkingDirectory());
+ }
+
+ @Test
+ // 设置WorkingDirectory时,需检查路径有效性,不包含"."、":"、"//"等不合法的字符
+ public void testWorkingDirectory03() throws Exception {
+ // 实际不满足,需整改
+ Path workPath = new Path("a001/./b001");
+ boolean hasException = false;
+ try {
+ fs.setWorkingDirectory(workPath);
+ } catch (IllegalArgumentException e) {
+ hasException = true;
+ }
+ assertFalse(hasException);
+
+ try {
+ workPath = new Path("a:a001");
+ hasException = false;
+ fs.setWorkingDirectory(workPath);
+ } catch (IllegalArgumentException e) {
+ hasException = true;
+ }
+ assertTrue(hasException);
+
+ workPath = new Path("//a001");
+ hasException = false;
+ try {
+ fs.setWorkingDirectory(workPath);
+ } catch (IllegalArgumentException e) {
+ hasException = true;
+ }
+ assertTrue(hasException);
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetContentSummary.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetContentSummary.java
new file mode 100644
index 0000000..ef087c3
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetContentSummary.java
@@ -0,0 +1,367 @@
+/*
+ * 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.FAST_UPLOAD_BYTEBUFFER;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.security.AccessControlException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * OBS tests for configuring block size.
+ */
+public class ITestOBSGetContentSummary {
+ private OBSFileSystem fs;
+
+ private Configuration conf;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ private static final String TEST_PREFIX = "test_du/";
+
+ private static final byte[] dataSet = ContractTestUtils.dataset(16, 0, 10);
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ITestOBSGetContentSummary.class);
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ conf = OBSContract.getConfiguration(null);
+ }
+
+ private void init(boolean contentEnable, boolean fsDFSListEnable)
+ throws IOException {
+ conf.setClass(OBSConstants.OBS_METRICS_CONSUMER,
+ MockMetricsConsumer.class, BasicMetricsConsumer.class);
+ conf.setBoolean(OBSConstants.METRICS_SWITCH, true);
+ conf.setBoolean(OBSConstants.OBS_CONTENT_SUMMARY_ENABLE, contentEnable);
+ conf.setBoolean(OBSConstants.OBS_CLIENT_DFS_LIST_ENABLE, fsDFSListEnable);
+ conf.set(OBSConstants.FAST_UPLOAD_BUFFER, FAST_UPLOAD_BYTEBUFFER);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(new Path(testRootPath), true);
+ }
+ }
+
+ private Path getTestPath(String testPath) {
+ return new Path(testRootPath + "/" + testPath);
+ }
+
+ @Test
+ // 计算对象桶一级目录的ContentSummary,校验正确性
+ public void testObjectBucketOnePath() throws Exception {
+ testGetContentSummaryOnePath(TEST_PREFIX, false);
+ }
+
+ @Test
+ // 计算对象桶子目录的ContentSummary,校验正确性
+ public void testObjectBucketSubPath() throws Exception {
+ testGetContentSummarySubPath(TEST_PREFIX, false);
+ }
+
+ @Test
+ // 计算文件桶一级目录的ContentSummary,校验正确性
+ public void testPosixBucketOnePath() throws Exception {
+ testGetContentSummaryOnePath(TEST_PREFIX, true);
+ }
+
+ @Test
+ // 计算文件桶子目录的ContentSummary,校验正确性
+ public void testPosixBucketSubPath() throws Exception {
+ testGetContentSummarySubPath(TEST_PREFIX, true);
+ }
+
+ @Test
+ // 测试空文件的ContentSummary,DirectoryCount为0,FileCount为1,Length为0
+ public void testGetContentSummaryOfFile01() throws Exception {
+ init(true, true);
+ Path testFile = getTestPath("test_file");
+ fs.delete(testFile, true);
+
+ FSDataOutputStream outputStream = fs.create(testFile, false);
+ outputStream.close();
+
+ ContentSummary summary = fs.getContentSummary(testFile);
+ assertTrue(summary.getDirectoryCount() == 0);
+ assertTrue(summary.getFileCount() == 1);
+ assertTrue(summary.getLength() == 0);
+ fs.delete(testFile, true);
+ }
+
+ @Test
+ // 测试非空文件的ContentSummary,DirectoryCount为0,FileCount为1,Length为文件长度
+ public void testGetContentSummaryOfFile02() throws Exception {
+ init(true, true);
+ Path testFile = getTestPath("test_file");
+ fs.delete(testFile, true);
+
+ FSDataOutputStream outputStream = fs.create(testFile, false);
+ byte[] data = {1, 2, 3, 4, 5};
+ outputStream.write(data);
+ outputStream.close();
+
+ ContentSummary summary = fs.getContentSummary(testFile);
+ assertTrue(summary.getDirectoryCount() == 0);
+ assertTrue(summary.getFileCount() == 1);
+ assertTrue(summary.getLength() == data.length);
+ fs.delete(testFile, true);
+ }
+
+ @Test
+ // 路径不存在,抛FileNotFoundException
+ public void testGetContentSummaryAbnormal01() throws Exception {
+ init(true, true);
+ Path testFile = getTestPath("test_file");
+ fs.delete(testFile, true);
+
+ boolean hasException = false;
+ try {
+ fs.getContentSummary(testFile);
+ } catch (FileNotFoundException e) {
+ hasException = true;
+ }
+ assertTrue(hasException);
+ }
+
+ @Test
+ // 路径的父目录及上级目录不存在,抛FileNotFoundException
+ public void testGetContentSummaryAbnormal02() throws Exception {
+ init(true, true);
+ Path testFile = getTestPath("a001/b001/test_file");
+ fs.delete(testFile.getParent().getParent(), true);
+
+ assertFalse(fs.exists(testFile.getParent()));
+
+ boolean hasException = false;
+ try {
+ fs.getContentSummary(testFile);
+ } catch (FileNotFoundException e) {
+ hasException = true;
+ }
+ assertTrue(hasException);
+ }
+
+ @Test
+ // 路径的父目录及上级目录非目录,抛AccessControlException
+ public void testGetContentSummaryAbnormal03() throws Exception {
+ init(true, true);
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ Path testFile = getTestPath("a001/b001/test_file");
+ fs.delete(testFile.getParent().getParent(), true);
+
+ FSDataOutputStream outputStream = fs.create(testFile.getParent(),
+ false);
+ outputStream.close();
+
+ boolean hasException = false;
+ try {
+ fs.getContentSummary(testFile);
+ } catch (AccessControlException e) {
+ hasException = true;
+ }
+ assertTrue(hasException);
+ }
+
+ private void constructTestPaths(String testDir, int dirDeep, int subDirNum,
+ int subFileNum) throws Exception {
+ if (dirDeep <= 0) {
+ return;
+ }
+ if (!testDir.endsWith("/")) {
+ testDir = testDir + "/";
+ }
+ boolean rslt = fs.mkdirs(getTestPath(testDir));
+ Assert.assertTrue(rslt);
+
+ for (int i = 0; i < subFileNum; i++) {
+ FSDataOutputStream outputStream = fs.create(
+ getTestPath(testDir + "-subFile-" + i), true);
+ outputStream.write(dataSet);
+ outputStream.close();
+ }
+
+ for (int i = 0; i < subDirNum; i++) {
+ constructTestPaths(testDir + "-" + dirDeep + "-" + i + "/",
+ dirDeep - 1, subDirNum, subFileNum);
+ }
+ }
+
+ private void testGetContentSummaryOnePath(String prefix,
+ boolean isPosixTest) throws Exception {
+ // test primitive
+ ContentSummary primitive, optimization, optimizationWithDFS = null;
+ long delay1, delay2, delay3;
+ try {
+ init(false, false);
+ if (isPosixTest != fs.isFsBucket()) {
+ return;
+ }
+ fs.delete(getTestPath(prefix), true);
+ constructTestPaths(prefix, 3, 5, 5);
+ long startTime = System.nanoTime();
+ primitive = fs.getContentSummary(getTestPath(prefix));
+ delay1 = System.nanoTime() - startTime;
+ } finally {
+ fs.close();
+ fs = null;
+ }
+
+ try {
+ init(true, false);
+ long startTime = System.nanoTime();
+ optimization = fs.getContentSummary(getTestPath(prefix));
+ delay2 = System.nanoTime() - startTime;
+ } finally {
+ fs.close();
+ fs = null;
+ }
+
+ // test optimization
+ try {
+ init(true, true);
+ long startTime = System.nanoTime();
+ optimizationWithDFS = fs.getContentSummary(getTestPath(prefix));
+ delay3 = System.nanoTime() - startTime;
+ assertTrue(fs.delete(getTestPath(prefix), true));
+ } finally {
+ if (fs != null) {
+ fs.close();
+ fs = null;
+ }
+ }
+
+ LOG.info("delay (ns): {} vs. {} vs. {}", delay1, delay2, delay3);
+
+ // compare
+ assertTrue(compareContentSummary(primitive, optimization));
+ assertTrue(compareContentSummary(primitive, optimizationWithDFS));
+ }
+
+ private void testGetContentSummarySubPath(String prefix,
+ boolean isPosixTest) throws Exception {
+ // test primitive
+ ContentSummary primitive = null;
+ Map summarys = new HashMap<>();
+ try {
+ init(false, false);
+ if (isPosixTest != fs.isFsBucket()) {
+ return;
+ }
+ fs.delete(getTestPath(prefix), true);
+ constructTestPaths(prefix, 3, 5, 5);
+ FileStatus[] status = fs.listStatus(getTestPath(prefix));
+ long length = 0;
+ long fileCount = 0;
+ long dirCount = 0;
+ for (FileStatus s : status) {
+ ContentSummary summary = fs.getContentSummary(s.getPath());
+ summarys.put(s.getPath().toString(), summary);
+ length += summary.getLength();
+ fileCount += summary.getFileCount();
+ dirCount += summary.getDirectoryCount();
+ }
+ primitive = new ContentSummary(length, fileCount, dirCount);
+ } finally {
+ if (fs != null) {
+ fs.close();
+ fs = null;
+ }
+ }
+
+ // test optimization
+ ContentSummary optimization = null;
+ Map optSummarys = new HashMap<>();
+ try {
+ init(true, true);
+ FileStatus[] status = fs.listStatus(getTestPath(prefix));
+ long length = 0;
+ long fileCount = 0;
+ long dirCount = 0;
+ for (FileStatus s : status) {
+ ContentSummary summary = fs.getContentSummary(s.getPath());
+ optSummarys.put(s.getPath().toString(), summary);
+ length += summary.getLength();
+ fileCount += summary.getFileCount();
+ dirCount += summary.getDirectoryCount();
+ }
+ optimization = new ContentSummary(length, fileCount, dirCount);
+ assertTrue(fs.delete(getTestPath(prefix), true));
+ } finally {
+ if (fs != null) {
+ fs.close();
+ fs = null;
+ }
+ }
+
+ // compare
+ boolean isSameResult = compareContentSummary(primitive, optimization);
+ assertTrue(isSameResult);
+ }
+
+ private boolean compareContentSummary(ContentSummary s1,
+ ContentSummary s2) {
+ LOG.info(String.format("s1{len[%s], fileCount[%s], dirCount[%s]",
+ s1.getLength(), s1.getFileCount(), s1.getDirectoryCount()));
+
+ LOG.info(String.format("s2{len[%s], fileCount[%s], dirCount[%s]",
+ s2.getLength(), s2.getFileCount(), s2.getDirectoryCount()));
+
+ if (s1.getLength() != s2.getLength()
+ || s1.getDirectoryCount() != s2.getDirectoryCount()
+ || s1.getFileCount() != s2.getFileCount()) {
+ return false;
+ }
+ return true;
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetContentSummaryV2.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetContentSummaryV2.java
new file mode 100644
index 0000000..f19c6cd
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetContentSummaryV2.java
@@ -0,0 +1,518 @@
+package org.apache.hadoop.fs.obs;
+
+import static org.apache.hadoop.fs.obs.OBSTestConstants.TEST_FS_OBS_NAME;
+
+import com.obs.services.IObsCredentialsProvider;
+import com.obs.services.ObsConfiguration;
+import com.obs.services.exception.ObsException;
+import com.obs.services.model.fs.ContentSummaryFsRequest;
+import com.obs.services.model.fs.ContentSummaryFsResult;
+import com.obs.services.model.fs.DirContentSummary;
+import com.obs.services.model.fs.DirSummary;
+import com.obs.services.model.fs.ListContentSummaryFsRequest;
+import com.obs.services.model.fs.ListContentSummaryFsResult;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.fs.obs.mock.MockObsClient;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+
+public class ITestOBSGetContentSummaryV2 {
+
+ private static boolean failure;
+
+ private static final Logger LOG = LoggerFactory.getLogger(ITestOBSGetContentSummaryV2.class);
+
+ private OBSFileSystem fs;
+
+ private Configuration conf;
+
+ private MockObsClient mockObsClient;
+
+ private OBSFileSystem ofs;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ private boolean useMock = true;
+
+ @Before
+ public void setUp() throws IOException, URISyntaxException, NoSuchFieldException, IllegalAccessException {
+ if (failure) return;
+ conf = OBSContract.getConfiguration(null);
+ conf.setInt(OBSConstants.CORE_LIST_THREADS, 100);
+ conf.setInt(OBSConstants.MAX_LIST_THREADS, 100);
+ conf.setInt(OBSConstants.RETRY_MAXTIME, 5000);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ ofs = OBSTestUtils.createTestFileSystem(conf);
+ if (useMock) {
+ mock();
+ }
+ }
+
+ private void mock() throws NoSuchFieldException, IOException, IllegalAccessException, URISyntaxException {
+ // create mock client
+ String fsname = conf.getTrimmed(TEST_FS_OBS_NAME, "");
+ URI uri = new URI(fsname);
+ ObsConfiguration obsConf = new ObsConfiguration();
+ String endPoint = conf.getTrimmed(OBSConstants.ENDPOINT, "");
+ obsConf.setEndPoint(endPoint);
+ IObsCredentialsProvider securityProvider = OBSSecurityProviderUtil.createObsSecurityProvider(conf, uri);
+ mockObsClient = new MockObsClient(securityProvider, obsConf);
+
+ // mock
+ Field obsFiled = OBSFileSystem.class.getDeclaredField("obs");
+ obsFiled.setAccessible(true);
+ obsFiled.set(this.fs, mockObsClient);
+ }
+
+ @After
+ public void cleanUp() throws IOException {
+ if (fs != null) {
+ fs.delete(new Path(testRootPath), true);
+ }
+ }
+
+ private void createFile(Path path, long fileSize) throws IOException {
+ StringBuilder stringBuilder = new StringBuilder();
+ for (long i = 0; i < fileSize; i ++) {
+ stringBuilder.append((char)1);
+ }
+ FSDataOutputStream outputStream = fs.create(path);
+ outputStream.write(stringBuilder.toString().getBytes());
+ outputStream.close();
+ }
+
+ private void createFiles(String basePath, int fileNum, int parDirLevel, long fileSize) throws IOException {
+ if (parDirLevel < 1) {
+ throw new IllegalArgumentException("parDirLevel should >= 1");
+ }
+ for (int i = 0; i < fileNum; i ++) {
+ String path = String.format("%s/sub-dir%d", basePath, i);
+ for (int j = 0; j < parDirLevel - 1; j ++) {
+ path = String.format("%s/sub-dir-level%d", path, j);
+ }
+ path = path + "/file";
+ createFile(new Path(path), fileSize);
+ }
+ }
+
+ @Test
+ public void testFsContentSummaryV2_1() throws IOException {
+ fs.mkdirs(new Path(testRootPath));
+ FileStatus fileStatus = fs.getFileStatus(new Path(testRootPath));
+ if (!useMock) {
+ try {
+ Thread.sleep(30 * 1000);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus);
+ assertTrue("contentSummary correctness",
+ contentSummary.getDirectoryCount() == 0 + 1 &&
+ contentSummary.getFileCount() == 0 &&
+ contentSummary.getLength() == 0);
+ }
+
+ @Test
+ public void testFsContentSummaryV2_2() throws IOException {
+ Path path = new Path(testRootPath + "/dir1");
+ fs.mkdirs(path);
+ FileStatus fileStatus = fs.getFileStatus(path);
+ if (!useMock) {
+ try {
+ Thread.sleep(30 * 1000);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus);
+ assertTrue("contentSummary correctness",
+ contentSummary.getDirectoryCount() == 0 + 1 &&
+ contentSummary.getFileCount() == 0 &&
+ contentSummary.getLength() == 0);
+ }
+
+ @Test
+ public void testFsContentSummaryV2_3() throws IOException {
+ Path path = new Path(testRootPath + "/dir1/dir2");
+ fs.mkdirs(path);
+ FileStatus fileStatus = fs.getFileStatus(new Path(testRootPath + "/dir1"));
+ if (!useMock) {
+ try {
+ Thread.sleep(30 * 1000);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus);
+ assertTrue("contentSummary correctness",
+ contentSummary.getDirectoryCount() == 1 + 1 &&
+ contentSummary.getFileCount() == 0 &&
+ contentSummary.getLength() == 0);
+ }
+
+ @Test
+ public void testFsContentSummaryV2_4() throws IOException {
+ Path path = new Path(testRootPath + "/dir1/dir2");
+ fs.mkdirs(path);
+ FileStatus fileStatus = fs.getFileStatus(path);
+ if (!useMock) {
+ try {
+ Thread.sleep(30 * 1000);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus);
+ assertTrue("contentSummary correctness",
+ contentSummary.getDirectoryCount() == 0 + 1 &&
+ contentSummary.getFileCount() == 0 &&
+ contentSummary.getLength() == 0);
+ }
+
+ @Test
+ public void testFsContentSummaryV2_5() throws IOException {
+ Path path = new Path(testRootPath + "/dir1/dir2/file1");
+ createFile(path, 1);
+ FileStatus fileStatus = fs.getFileStatus(new Path(testRootPath + "/dir1"));
+ if (!useMock) {
+ try {
+ Thread.sleep(30 * 1000);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus);
+ assertTrue("contentSummary correctness",
+ contentSummary.getDirectoryCount() == 1 + 1 &&
+ contentSummary.getFileCount() == 1 &&
+ contentSummary.getLength() == 1);
+ }
+
+ @Test
+ public void testFsContentSummaryV2_6() throws IOException {
+ Path path = new Path(testRootPath + "/dir1/dir2/file1");
+ createFile(path, 1);
+ FileStatus fileStatus = fs.getFileStatus(new Path(testRootPath + "/dir1/dir2"));
+ if (!useMock) {
+ try {
+ Thread.sleep(30 * 1000);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus);
+ assertTrue("contentSummary correctness",
+ contentSummary.getDirectoryCount() == 0 + 1 &&
+ contentSummary.getFileCount() == 1 &&
+ contentSummary.getLength() == 1);
+ }
+
+ @Test
+ public void testFsContentSummaryV2_7() throws IOException {
+ createFile(new Path(testRootPath + "/dir1/file1"), 1);
+ createFile(new Path(testRootPath + "/dir1/dir2/file2"), 2);
+ FileStatus fileStatus = fs.getFileStatus(new Path(testRootPath + "/dir1"));
+ if (!useMock) {
+ try {
+ Thread.sleep(30 * 1000);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus);
+ assertTrue("contentSummary correctness",
+ contentSummary.getDirectoryCount() == 1 + 1 &&
+ contentSummary.getFileCount() == 2 &&
+ contentSummary.getLength() == 3);
+ }
+
+ private void testFsContentSummaryV2MultiDirsAndFiles(int fileNum, int parDirLevel,
+ Function listCSResultCallback) throws IOException {
+ long fileSize = 3;
+ createFiles(testRootPath + "/dir1", fileNum, parDirLevel, fileSize);
+ FileStatus fileStatus = fs.getFileStatus(new Path(testRootPath + "/dir1"));
+ if (!useMock) {
+ try {
+ Thread.sleep(300 * 1000);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ this.mockObsClient.setListCSResultCallback(listCSResultCallback);
+ ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus);
+ LOG.info("V2: dirNum: {}, fileNum: {}, fileSize: {}, should be: ({}, {}, {})",
+ contentSummary.getDirectoryCount(), contentSummary.getFileCount(), contentSummary.getLength(),
+ fileNum * parDirLevel + 1, fileNum, fileNum * fileSize);
+ boolean pass = contentSummary.getDirectoryCount() == (long) fileNum * parDirLevel + 1 &&
+ contentSummary.getFileCount() == fileNum &&
+ contentSummary.getLength() == fileNum * fileSize;
+ if (!pass) {
+ failure = true;
+ }
+
+ assertTrue("contentSummary correctness", pass);
+
+ // compare with v1
+ ContentSummary contentSummary1 = OBSPosixBucketUtils.fsGetDirectoryContentSummary(fs,
+ OBSCommonUtils.pathToKey(fs, fileStatus.getPath()));
+ LOG.info("V1: dirNum: {}, fileNum: {}, fileSize: {}",
+ contentSummary1.getDirectoryCount(), contentSummary1.getFileCount(), contentSummary1.getLength());
+ assertTrue("v2 should align to v1",
+ contentSummary.getDirectoryCount() == contentSummary1.getDirectoryCount() &&
+ contentSummary.getFileCount() == contentSummary1.getFileCount() &&
+ contentSummary.getLength() == contentSummary1.getLength());
+ }
+
+ @Test
+ public void testFsContentSummaryV2_8() throws IOException {
+ testFsContentSummaryV2MultiDirsAndFiles(1100, 1, null);
+ }
+
+ @Test
+ public void testFsContentSummaryV2_9() throws IOException {
+ testFsContentSummaryV2MultiDirsAndFiles(900, 1, null);
+ }
+
+ @Test
+ public void testFsContentSummaryV2_10() throws IOException {
+ testFsContentSummaryV2MultiDirsAndFiles(1000, 1, null);
+ }
+
+ @Test
+ public void testFsContentSummaryV2_11() throws IOException {
+ testFsContentSummaryV2MultiDirsAndFiles(1100, 2, null);
+ }
+
+ @Test
+ public void testFsContentSummaryV2_12() throws IOException {
+ testFsContentSummaryV2MultiDirsAndFiles(900, 2, null);
+ }
+
+ @Test
+ public void testFsContentSummaryV2_13() throws IOException {
+ testFsContentSummaryV2MultiDirsAndFiles(1000, 2, null);
+ }
+
+ @Test
+ public void testFsContentSummaryV2_14() throws IOException {
+ Path root = new Path("/");
+ FileStatus fileStatus = fs.getFileStatus(root);
+ try {
+ OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus);
+ } catch (Exception e) {
+ assertTrue(e.getMessage(), false);
+ }
+ assertTrue(true);
+ }
+
+ private static class ErrorPattern1 {
+ private volatile AtomicInteger count;
+
+ private int[] errorPoint;
+
+ private int errorDirNum;
+
+ public ErrorPattern1(int[] errorPoint, int errorDirNum) {
+ this.errorPoint = errorPoint;
+ this.errorDirNum = errorDirNum;
+ this.count = new AtomicInteger();
+ }
+
+ public ListContentSummaryFsResult callbackFunc(ListContentSummaryFsResult listCSResult) {
+ this.count.incrementAndGet();
+ boolean needInject = false;
+ for (int i : errorPoint) {
+ if (this.count.get() == i) {
+ needInject = true;
+ break;
+ }
+ }
+ if (!needInject) {
+ return listCSResult;
+ }
+ List retErrors = new ArrayList<>();
+ List retDirs = new ArrayList<>();
+ List dirs = listCSResult.getDirContentSummaries();
+ int nNum = Math.min(dirs.size(), errorDirNum);
+ for (int i = 0; i < dirs.size() - nNum; i ++) {
+ retDirs.add(dirs.get(i));
+ }
+ for (int i = dirs.size() - nNum; i < dirs.size(); i ++) {
+ DirContentSummary dir = dirs.get(i);
+ ListContentSummaryFsResult.ErrorResult err = new ListContentSummaryFsResult.ErrorResult();
+ err.setStatusCode("500");
+ err.setErrorCode("TestErrorCode");
+ err.setKey(dir.getKey());
+ err.setInode(dir.getInode());
+ err.setMessage("InjectError");
+ retErrors.add(err);
+ }
+ listCSResult.setDirContentSummaries(retDirs);
+ listCSResult.setErrorResults(retErrors);
+ return listCSResult;
+ }
+ }
+
+ @Test
+ public void testFsContentSummaryV2_15() throws IOException {
+ if (!useMock) {
+ return;
+ }
+ ErrorPattern1 ep = new ErrorPattern1(new int[]{1, 3, 5}, 3);
+ testFsContentSummaryV2MultiDirsAndFiles(1100, 1,
+ ep::callbackFunc);
+ }
+
+ @Test
+ public void testFsContentSummaryV2_16() throws IOException {
+ if (!useMock) {
+ return;
+ }
+ ErrorPattern1 ep = new ErrorPattern1(new int[]{1, 3, 5}, 3);
+ testFsContentSummaryV2MultiDirsAndFiles(900, 1,
+ ep::callbackFunc);
+ }
+
+ @Test
+ public void testFsContentSummaryV2_17() throws IOException {
+ if (!useMock) {
+ return;
+ }
+ ErrorPattern1 ep = new ErrorPattern1(new int[]{1, 3, 5}, 3);
+ testFsContentSummaryV2MultiDirsAndFiles(1000, 1,
+ ep::callbackFunc);
+ }
+
+ @Test
+ public void testFsContentSummaryV2_18() throws IOException {
+ if (!useMock) {
+ return;
+ }
+ ErrorPattern1 ep = new ErrorPattern1(new int[]{1, 3, 5}, 3);
+ testFsContentSummaryV2MultiDirsAndFiles(1100, 2,
+ ep::callbackFunc);
+ }
+
+ @Test
+ public void testFsContentSummaryV2_19() throws IOException {
+ if (!useMock) {
+ return;
+ }
+ ErrorPattern1 ep = new ErrorPattern1(new int[]{1, 3, 5}, 3);
+ testFsContentSummaryV2MultiDirsAndFiles(900, 2,
+ ep::callbackFunc);
+ }
+
+ @Test
+ public void testFsContentSummaryV2_20() throws IOException {
+ if (!useMock) {
+ return;
+ }
+ ErrorPattern1 ep = new ErrorPattern1(new int[]{1, 3, 5}, 3);
+ testFsContentSummaryV2MultiDirsAndFiles(1000, 2,
+ ep::callbackFunc);
+ }
+
+ @Test
+ public void test405FallbackWithoutRetry() throws IOException {
+ if (useMock) {
+ mockObsClient.setGetCSUnsupported(true);
+ mockObsClient.setGetCSNum(0);
+ mockObsClient.setResponseCode(405);
+ mockObsClient.setErrorMsg("mock unsupported");
+ createFile(new Path(testRootPath + "/dir1/file1"), 1);
+ createFile(new Path(testRootPath + "/dir1/dir2/file2"), 2);
+ Path path = new Path(testRootPath + "/dir1");
+ ContentSummary contentSummary = fs.getContentSummary(path);
+ assertTrue("contentSummary correctness",
+ contentSummary.getDirectoryCount() == 1 + 1 &&
+ contentSummary.getFileCount() == 2 &&
+ contentSummary.getLength() == 3);
+ assertEquals("escape without retry", 1, mockObsClient.getGetCSNum());
+ mockObsClient.setGetCSUnsupported(false);
+ mockObsClient.setGetCSNum(0);
+ }
+ }
+
+ @Test
+ public void test503FallbackShouldRetry() throws IOException {
+ if (useMock) {
+ mockObsClient.setGetCSUnsupported(true);
+ mockObsClient.setGetCSNum(0);
+ mockObsClient.setResponseCode(503);
+ mockObsClient.setErrorMsg("mock service unavailable");
+ createFile(new Path(testRootPath + "/dir1/file1"), 1);
+ createFile(new Path(testRootPath + "/dir1/dir2/file2"), 2);
+ Path path = new Path(testRootPath + "/dir1");
+ ContentSummary contentSummary = fs.getContentSummary(path);
+ assertTrue("contentSummary correctness",
+ contentSummary.getDirectoryCount() == 1 + 1 &&
+ contentSummary.getFileCount() == 2 &&
+ contentSummary.getLength() == 3);
+ assertTrue("escape with retry", mockObsClient.getGetCSNum() > 1);
+ mockObsClient.setGetCSUnsupported(false);
+ mockObsClient.setGetCSNum(0);
+ }
+ }
+
+ // @Test
+ public void testFsContentSummaryV2() throws IOException {
+ FileStatus fileStatus = ofs.getFileStatus(new Path("/"));
+ ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(ofs, fileStatus);
+ long dirNum = contentSummary.getDirectoryCount();
+ long fileNum = contentSummary.getFileCount();
+ long fileSize = contentSummary.getLength();
+ System.out.printf("%d %d %d", dirNum, fileNum, fileSize);
+ }
+
+ // @Test
+ public void testSDK1() {
+ List dirs = new ArrayList<>();
+ ListContentSummaryFsRequest.DirLayer dir = new ListContentSummaryFsRequest.DirLayer();
+ dir.setKey("test/dir1/");
+ // dir.setInode(4611752345613959168L);
+ dirs.add(dir);
+
+ ListContentSummaryFsRequest req = new ListContentSummaryFsRequest();
+ req.setBucketName("sanfangliantiao-0218");
+ req.setMaxKeys(1000);
+ req.setDirLayers(dirs);
+
+ ListContentSummaryFsResult res = ofs.getObsClient().listContentSummaryFs(req);
+ System.out.println(res);
+ }
+
+ // @Test
+ public void testSDK2() {
+ ContentSummaryFsRequest req = new ContentSummaryFsRequest();
+ req.setBucketName("jianantest02");
+ req.setDirName("test/dir1/sub-dir908/");
+
+ ContentSummaryFsResult res = ofs.getObsClient().getContentSummaryFs(req);
+ DirSummary summary = res.getContentSummary();
+ System.out.printf("%d %d %d", summary.getDirCount(), summary.getFileCount(), summary.getFileSize());
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetFileStatusAndExist.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetFileStatusAndExist.java
new file mode 100644
index 0000000..302b26f
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetFileStatusAndExist.java
@@ -0,0 +1,164 @@
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import static org.junit.Assert.*;
+
+public class ITestOBSGetFileStatusAndExist {
+ private OBSFileSystem fs;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.setClass(OBSConstants.OBS_METRICS_CONSUMER,
+ MockMetricsConsumer.class, BasicMetricsConsumer.class);
+ conf.setBoolean(OBSConstants.METRICS_SWITCH, true);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(new Path(testRootPath), true);
+ }
+ }
+
+ private Path getTestPath(String relativePath) {
+ return new Path(testRootPath + "/" + relativePath);
+ }
+
+ @Test
+ // 获取根目录的信息,exist返回true
+ public void testGetFileStatusAndExistNormal001() throws IOException {
+ FileStatus status = fs.getFileStatus(new Path("/"));
+ System.out.println(status.toString());
+ assertTrue("Root path should be directory.", status.isDirectory());
+
+ assertEquals(fs.exists(new Path("/")), status != null);
+ }
+
+ @Test
+ // 获取文件的元数据信息,exist返回true
+ public void testGetFileStatusAndExistNormal002() throws IOException {
+ Path testPath = getTestPath("a001/test_file");
+ FSDataOutputStream outputStream = null;
+ try {
+ outputStream = fs.create(testPath, false);
+ byte[] data = ContractTestUtils.dataset(8, 'a', 26);
+ outputStream.write(data);
+ outputStream.close();
+
+ FileStatus status = fs.getFileStatus(testPath);
+ if (fs.getMetricSwitch()) {
+ MockMetricsConsumer mmc
+ = (MockMetricsConsumer) fs.getMetricsConsumer();
+ assertEquals(BasicMetricsConsumer.MetricKind.normal, mmc.getMr().getKind());
+ }
+ System.out.println(status.toString());
+ assertTrue("test path should be a file.", status.isFile());
+ assertEquals(fs.exists(testPath), status != null);
+ } finally {
+ if (outputStream != null) {
+ outputStream.close();
+ }
+ }
+ OBSFSTestUtil.deletePathRecursive(fs, testPath);
+ }
+
+ @Test
+ // 获取目录的元数据信息, exist返回true
+ public void testGetFileStatusAndExistNormal003() throws IOException {
+ Path testPath = getTestPath("a001/b001");
+ fs.mkdirs(testPath);
+
+ FileStatus status = fs.getFileStatus(testPath);
+ System.out.println(status.toString());
+ assertTrue("test path should be directory.", status.isDirectory());
+ assertEquals(fs.exists(testPath), status != null);
+ OBSFSTestUtil.deletePathRecursive(fs, testPath);
+ }
+
+ @Test
+ // 路径不存在,getFileStatus抛FileNotFoundException, exist返回false
+ public void testGetFileStatusAndExistAbnormal001() throws IOException {
+ Path testPath = getTestPath("a001/test_file");
+ if (fs.exists(testPath)) {
+ fs.delete(testPath, true);
+ }
+
+ boolean hasException = false;
+ try {
+ fs.getFileStatus(testPath);
+ } catch (FileNotFoundException fnfe) {
+ hasException = true;
+ }
+ assertTrue(hasException);
+ assertFalse(fs.exists(testPath));
+ }
+
+ @Test
+ // 路径的父目录及上级目录不存在,getFileStatus抛FileNotFoundException, exist返回false
+ public void testGetFileStatusAndExistAbnormal002() throws IOException {
+ Path testPath = getTestPath("a001/b001/test_file");
+ OBSFSTestUtil.deletePathRecursive(fs, testPath.getParent());
+
+ boolean hasException = false;
+ try {
+ fs.getFileStatus(testPath);
+ } catch (FileNotFoundException e) {
+ hasException = true;
+ }
+ assertTrue(hasException);
+ assertFalse(fs.exists(testPath));
+ }
+
+ @Test
+ // 路径的父目录及上级目录非目录,getFileStatus抛FileNotFoundException,exist返false
+ public void testGetFileStatusAndExistAbnormal003() throws IOException {
+ Path testPath = getTestPath("a001/b001/test_file");
+ OBSFSTestUtil.deletePathRecursive(fs, testPath.getParent());
+
+ FSDataOutputStream outputStream = fs.create(testPath.getParent(),
+ false);
+ outputStream.close();
+
+ boolean hasException = false;
+ try {
+ fs.getFileStatus(testPath);
+ if (fs.getMetricSwitch()) {
+ MockMetricsConsumer mmc
+ = (MockMetricsConsumer) fs.getMetricsConsumer();
+ assertEquals(BasicMetricsConsumer.MetricKind.normal, mmc.getMr().getKind());
+ }
+ } catch (FileNotFoundException e) {
+ hasException = true;
+ }
+ assertTrue(hasException);
+ assertFalse(fs.exists(testPath));
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHDFSWrapper.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHDFSWrapper.java
new file mode 100644
index 0000000..6d1d6f9
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHDFSWrapper.java
@@ -0,0 +1,79 @@
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+public class ITestOBSHDFSWrapper {
+
+ private Configuration conf;
+ private MiniDFSCluster dfsCluster;
+
+ @Rule
+ public TemporaryFolder folder = new TemporaryFolder();
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @Before
+ public void setUp() throws Exception {
+ conf = new Configuration();
+ conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName());
+ dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (dfsCluster != null) {
+ dfsCluster.close();
+ }
+ }
+
+ @Test
+ public void testStatus() throws IOException {
+ String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress();
+
+ // hdfs native file
+ DistributedFileSystem rawHdfsFS = dfsCluster.getFileSystem();
+ rawHdfsFS.create(new Path("/hdfsfile1")).close();
+
+ // wrapped local file system files
+ folder.newFile("newFile1");
+ File subfolder = folder.newFolder("folder1");
+ subfolder.toPath().resolve("subFile1").toFile().createNewFile();
+
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/",
+ folder.getRoot().toURI().toString());
+ conf.set("fs.hdfs.impl.disable.cache", "true");
+
+ // If not set abfs, will still use Hdfs to resole path which will cause FileNotFoundException on hdfs.
+ Path wrapperFile1 = new Path("hdfs://" + fsAuthority + "/wrapper/newFile1");
+ FileContext fc = FileContext.getFileContext(wrapperFile1.toUri(), conf);
+ try {
+ fc.rename(wrapperFile1, new Path("hdfs://" + fsAuthority + "/wrapper/newFile1_r1"));
+ } catch (Exception e) {
+ Assert.assertTrue(e instanceof FileNotFoundException);
+ }
+
+ // set abfs to MRSHDFSWrapper impl hdfs.
+ conf.set("fs.AbstractFileSystem.hdfs.impl", "org.apache.hadoop.fs.obs.OBSHDFSWrapper");
+ Path wrapperFile1New = new Path("hdfs://" + fsAuthority + "/wrapper/newFile1");
+ Path newPath = new Path("hdfs://" + fsAuthority + "/wrapper/newFile1_r1");
+ FileContext fcNew = FileContext.getFileContext(wrapperFile1New.toUri(), conf);
+ fcNew.rename(wrapperFile1, newPath);
+ fcNew.open(newPath).close();
+ }
+
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHDFSWrapperFileSystem.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHDFSWrapperFileSystem.java
new file mode 100644
index 0000000..b2ed19d
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHDFSWrapperFileSystem.java
@@ -0,0 +1,986 @@
+package org.apache.hadoop.fs.obs;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSMainOperationsBaseTest;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.TrashPolicy;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class ITestOBSHDFSWrapperFileSystem extends FSMainOperationsBaseTest {
+
+ static Configuration conf = OBSContract.getConfiguration(null);
+ static String fsAuthority;
+ static MiniDFSCluster dfsCluster;
+
+ @BeforeClass
+ public static void setUpper() throws Exception {
+ conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName());
+ conf.setLong("dfs.namenode.fs-limits.min-block-size",1000);
+ dfsCluster =
+ new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+ fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress();
+ }
+
+ @Override
+ protected FileSystem createFileSystem() throws Exception {
+
+ // conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName());
+ // conf.setLong("dfs.namenode.fs-limits.min-block-size",1000);
+ // dfsCluster =
+ // new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+ // fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress();
+ // conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./test/",
+ // folder.getRoot().toURI().toString());
+
+ // // hdfs native file
+ DistributedFileSystem fSys = dfsCluster.getFileSystem();
+ String testRootDir = getTestRootPath(fSys).toUri().getPath();
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link."+testRootDir+
+ "/test/", conf.get(OBSTestConstants.TEST_FS_OBS_NAME) + "/test/");
+ return fSys;
+ }
+
+
+ @AfterClass
+ public static void tearDowner() {
+ if (dfsCluster != null) {
+ dfsCluster.close();
+ }
+ }
+
+ @Rule
+ public TemporaryFolder folder = new TemporaryFolder();
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @Test
+ public void testTrash() throws IOException {
+ // Configuration conf = new HdfsConfiguration();
+ // conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName());
+ // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) {
+ // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress();
+ //
+ // // hdfs native file
+ // DistributedFileSystem fSys = fSys;
+ fSys.create(new Path("/hdfsfile1")).close();
+ fSys.mkdirs(new Path("/hdfsdir"), FsPermission.createImmutable((short) 777));
+ fSys.create(new Path("/hdfsdir/subfile1")).close();
+
+ // wrapped local file system files
+ folder.newFile("newFile1");
+ File subfolder = folder.newFolder("folder1");
+ subfolder.toPath().resolve("subFile1").toFile().createNewFile();
+
+
+ conf.set("fs.trash.interval", "30");
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/",
+ folder.getRoot().toURI().toString());
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./abc/",
+ folder.getRoot().toURI() + "/folder1");
+ conf.set("fs.hdfs.impl.disable.cache", "true");
+ //file:C:\Users\L30002~1\AppData\Local\Temp\junit1625000454693617450/folder1
+ FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf);
+ TrashPolicy trashPolicy = TrashPolicy.getInstance(conf, fs);
+
+ // trash wrapper file
+ {
+ Assert.assertTrue(trashPolicy.moveToTrash(new Path("hdfs://" + fsAuthority + "/wrapper/newFile1")));
+ Assert.assertTrue(trashPolicy.moveToTrash(new Path("hdfs://" + fsAuthority + "/abc/subFile1")));
+ Assert.assertTrue(trashPolicy.moveToTrash(new Path("hdfs://" + fsAuthority + "/hdfsfile1")));
+ Assert.assertTrue(trashPolicy.moveToTrash(new Path("hdfs://" + fsAuthority + "/hdfsdir/subfile1")));
+ }
+
+ }
+
+ // trash target folder is wrapped to another filesystem
+ @Test
+ public void testTrashUser() throws IOException {
+ // Configuration conf = new HdfsConfiguration();
+ // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) {
+ // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress();
+ //
+ // // hdfs native file
+ // DistributedFileSystem fSys = fSys;
+ fSys.create(new Path("/hdfsfile1")).close();
+ fSys.mkdirs(new Path("/hdfsdir"),
+ FsPermission.createImmutable((short) 777));
+ fSys.create(new Path("/hdfsdir/subfile1")).close();
+
+ // wrapped local file system files
+ folder.newFile("newFile1");
+ File subfolder = folder.newFolder("folder1");
+ subfolder.toPath().resolve("subFile1").toFile().createNewFile();
+
+ conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName());
+ conf.set("fs.trash.interval", "30");
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/",
+ folder.getRoot().toURI().toString());
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./user/",
+ folder.getRoot().toURI().toString() + "/folder1");
+ conf.set("fs.hdfs.impl.disable.cache", "true");
+
+ FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf);
+ TrashPolicy trashPolicy = TrashPolicy.getInstance(conf, fs);
+ Path hdfsTrashRoot = fs.getTrashRoot(new Path("hdfs://" + fsAuthority + "/hdfsfile1"));
+ Path localFSTrashRoot = fs.getTrashRoot(new Path("hdfs://" + fsAuthority + "/wrapper/newFile1"));
+ fs.delete(localFSTrashRoot, true);
+ fs.delete(hdfsTrashRoot, true);
+
+ // trash wrapper file
+ {
+ Assert.assertTrue(trashPolicy.moveToTrash(new Path("hdfs://" + fsAuthority + "/wrapper/newFile1")));
+ Assert.assertTrue(trashPolicy.moveToTrash(new Path("hdfs://" + fsAuthority + "/user/subFile1")));
+ Assert.assertTrue(trashPolicy.moveToTrash(new Path("hdfs://" + fsAuthority + "/hdfsfile1")));
+ Assert.assertTrue(trashPolicy.moveToTrash(new Path("hdfs://" + fsAuthority + "/hdfsdir/subfile1")));
+ }
+ // list trash root
+ {
+ RemoteIterator hdfsTrashItr = fs.listFiles(hdfsTrashRoot, true);
+ int countHdfs = 0;
+ while (hdfsTrashItr.hasNext()) {
+ System.out.println(hdfsTrashItr.next());
+ countHdfs++;
+ }
+ Assert.assertEquals(2, countHdfs);
+
+ RemoteIterator localTrashItr = fs.listFiles(localFSTrashRoot, true);
+ int countLocal = 0;
+ while (localTrashItr.hasNext()) {
+ System.out.println(localTrashItr.next());
+ countLocal++;
+ }
+ Assert.assertEquals(2, countLocal);
+ }
+ // }
+ }
+
+ @Test
+ public void testRename() throws IOException {
+ // Configuration conf = new HdfsConfiguration();
+ // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) {
+ // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress();
+ //
+ // // hdfs native file
+ // DistributedFileSystem fSys = fSys;
+ fSys.create(new Path("/hdfsfile1")).close();
+ fSys.create(new Path("/hdfsfile2")).close();
+ fSys.create(new Path("/hdfsfile3")).close();
+ fSys.mkdirs(new Path("/hdfsdir"),
+ FsPermission.createImmutable((short) 777));
+ fSys.create(new Path("/hdfsdir/subfile1")).close();
+ fSys.create(new Path("/hdfsdir/subfile2")).close();
+
+ // wrapped local file system files
+ folder.newFile("newFile1");
+ folder.newFile("newFile2");
+ folder.newFile("newFile3");
+ File subfolder = folder.newFolder("folder1");
+ System.out.println("---"+subfolder.toPath());
+ subfolder.toPath().resolve("subFile1").toFile().createNewFile();
+ subfolder.toPath().resolve("subFile2").toFile().createNewFile();
+
+ conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName());
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/",
+ folder.getRoot().toURI().toString());
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./abc/",
+ folder.getRoot().toURI().toString() + "/folder1");
+ conf.set("fs.hdfs.impl.disable.cache", "true");
+
+ FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf);
+
+ // cross fs rename
+ {
+ Path from = new Path("hdfs://" + fsAuthority + "/hdfsfile1");
+ Path to = new Path("hdfs://" + fsAuthority + "/wrapper/hdfsfile1");
+ try {
+ fs.rename(from, to);
+ fail("cross fs rename should fail.");
+ } catch (IOException e) {
+ // 20210729: [BUG2021070600630] change UnsupportedOperationException to IOException
+ if (!(e.getCause() instanceof UnsupportedOperationException)) {
+ throw e;
+ }
+ // ignore UnsupportedOperationException cause
+ }
+ }
+
+ // same fs rename in hdfs
+ {
+ Path from = new Path("hdfs://" + fsAuthority + "/hdfsfile1");
+ Path to = new Path("hdfs://" + fsAuthority + "/hdfsfile1");
+ Assert.assertTrue(fs.rename(from, to));
+ Assert.assertTrue(fSys.exists(to));
+ }
+
+ // same fs rename in wrapper fs
+ {
+ Path from = new Path("hdfs://" + fsAuthority + "/wrapper/newFile1");
+ Path to = new Path("hdfs://" + fsAuthority + "/wrapper/newfile11");
+ Assert.assertTrue(fs.rename(from, to));
+ Assert.assertTrue(folder.getRoot().toPath().resolve("newfile11").toFile().exists());
+ }
+
+ // same fs rename in wrapper fs to different mount point
+ {
+ Path from = new Path("hdfs://" + fsAuthority + "/wrapper/newFile2");
+ Path to = new Path("hdfs://" + fsAuthority + "/abc/newFile2");
+ Assert.assertTrue(fs.rename(from, to));
+ Assert.assertTrue(folder.getRoot().toPath().resolve("folder1/newFile2").toFile().exists());
+ }
+ // }
+ }
+
+ @Test
+ public void testRename1() throws IOException {
+ // Configuration conf = new HdfsConfiguration();
+ // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) {
+ // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress();
+ //
+ // // hdfs native file
+ // DistributedFileSystem fSys = fSys;
+ fSys.create(new Path("/hdfsfile1")).close();
+ fSys.create(new Path("/hdfsfile2")).close();
+ fSys.create(new Path("/hdfsfile3")).close();
+ fSys.mkdirs(new Path("/hdfsdir"),
+ FsPermission.createImmutable((short) 777));
+ fSys.create(new Path("/hdfsdir/subfile1")).close();
+ fSys.create(new Path("/hdfsdir/subfile2")).close();
+
+ // wrapped local file system files
+ folder.newFile("newFile1");
+ folder.newFile("newFile2");
+ folder.newFile("newFile3");
+ folder.newFile("newFile4");
+ folder.newFile("newFile5");
+ File subfolder = folder.newFolder("folder1");
+ subfolder.toPath().resolve("subFile1").toFile().createNewFile();
+ subfolder.toPath().resolve("subFile2").toFile().createNewFile();
+ subfolder.toPath().resolve("newFile5").toFile().createNewFile();
+
+ conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName());
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/",
+ folder.getRoot().toURI().toString());
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./abc/",
+ folder.getRoot().toURI().toString() + "/folder1");
+ conf.set("fs.hdfs.impl.disable.cache", "true");
+
+ FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf);
+
+ // cross fs rename
+ {
+ Path from = new Path("hdfs://" + fsAuthority + "/hdfsfile1");
+ Path to = new Path("hdfs://" + fsAuthority + "/wrapper/hdfsfile1");
+ try {
+ if (fs instanceof OBSHDFSFileSystem) {
+ ((OBSHDFSFileSystem) fs).rename(from, to, Options.Rename.OVERWRITE);
+ } else {
+ fs.delete(to, false);
+ fs.rename(from, to);
+ }
+ fail("cross fs rename should fail.");
+ } catch (IOException e) {
+ // 20210729: [BUG2021070600630] change UnsupportedOperationException to IOException
+ if (!(e.getCause() instanceof UnsupportedOperationException)) {
+ throw e;
+ }
+ // ignore UnsupportedOperationException cause
+ }
+ }
+
+ // same fs rename in hdfs
+ {
+ Path from = new Path("hdfs://" + fsAuthority + "/hdfsfile1");
+ Path to = new Path("hdfs://" + fsAuthority + "/hdfsfile11");
+ boolean success = false;
+ if (fs instanceof OBSHDFSFileSystem) {
+ ((OBSHDFSFileSystem) fs).rename(from, to, Options.Rename.OVERWRITE);
+ success = true;
+ } else {
+ fs.delete(to, false);
+ success = fs.rename(from, to);
+ }
+ Assert.assertTrue(success);
+ Assert.assertTrue(fSys.exists(to));
+ }
+
+ // same fs rename in wrapper fs
+ {
+ Path from = new Path("hdfs://" + fsAuthority + "/wrapper/newFile1");
+ Path to = new Path("hdfs://" + fsAuthority + "/wrapper/newfile11");
+ boolean success = false;
+ if (fs instanceof OBSHDFSFileSystem) {
+ ((OBSHDFSFileSystem) fs).rename(from, to, Options.Rename.OVERWRITE);
+ success = true;
+ } else {
+ fs.delete(to, false);
+ success = fs.rename(from, to);
+ }
+ Assert.assertTrue(success);
+ Assert.assertTrue(folder.getRoot().toPath().resolve("newfile11").toFile().exists());
+ }
+
+ // same fs rename in wrapper fs, to path is exist
+ {
+ Path from = new Path("hdfs://" + fsAuthority + "/wrapper/newFile4");
+ Path to = new Path("hdfs://" + fsAuthority + "/wrapper/newFile4");
+ boolean success = false;
+ if (fs instanceof OBSHDFSFileSystem) {
+ ((OBSHDFSFileSystem) fs).rename(from, to, Options.Rename.OVERWRITE);
+ success = true;
+ } else {
+ fs.delete(to, false);
+ success = fs.rename(from, to);
+ }
+ Assert.assertTrue(success);
+ Assert.assertTrue(folder.getRoot().toPath().resolve("newFile4").toFile().exists());
+ }
+
+ // same fs rename in wrapper fs to different mount point
+ {
+ Path from = new Path("hdfs://" + fsAuthority + "/wrapper/newFile2");
+ Path to = new Path("hdfs://" + fsAuthority + "/abc/newFile2");
+ boolean success = false;
+ if (fs instanceof OBSHDFSFileSystem) {
+ ((OBSHDFSFileSystem) fs).rename(from, to, Options.Rename.OVERWRITE);
+ success = true;
+ } else {
+ fs.delete(to, false);
+ success = fs.rename(from, to);
+ }
+ Assert.assertTrue(success);
+ Assert.assertTrue(folder.getRoot().toPath().resolve("folder1/newFile2").toFile().exists());
+ }
+
+ // same fs rename in wrapper fs to different mount point, to path is exist
+ {
+ Path from = new Path("hdfs://" + fsAuthority + "/wrapper/newFile5");
+ Path to = new Path("hdfs://" + fsAuthority + "/abc/newFile5");
+ boolean success = false;
+ if (fs instanceof OBSHDFSFileSystem) {
+ ((OBSHDFSFileSystem) fs).rename(from, to, Options.Rename.OVERWRITE);
+ success = true;
+ } else {
+ fs.delete(to, false);
+ success = fs.rename(from, to);
+ }
+ Assert.assertTrue(success);
+ Assert.assertTrue(folder.getRoot().toPath().resolve("folder1/newFile5").toFile().exists());
+ }
+
+
+ // }
+ }
+
+ @Test
+ public void testDelete() throws IOException {
+ // Configuration conf = new HdfsConfiguration();
+ // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) {
+ // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress();
+
+ conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName());
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/",
+ folder.getRoot().toURI().toString());
+ conf.set("fs.hdfs.impl.disable.cache", "true");
+
+ FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf);
+
+ File dir = folder.newFolder("toBeDelete");
+ dir.toPath().resolve("subFile1").toFile().createNewFile();
+ dir.toPath().resolve("subFile2").toFile().createNewFile();
+
+ Path deletePath = new Path("hdfs://" + fsAuthority + "/wrapper/toBeDelete");
+ Assert.assertTrue(fs.delete(deletePath, true));
+
+ try {
+ fs.listStatus(deletePath);
+ fail("the path " + deletePath + " should not exists");
+ } catch (FileNotFoundException e) {
+ // success
+ }
+
+ FileStatus[] status = fs.listStatus(deletePath.getParent());
+ Assert.assertEquals(0, status.length);
+ // }
+ }
+
+ @Test
+ public void testReadWriteFiles() throws IOException {
+ // Configuration conf = new HdfsConfiguration();
+ // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) {
+ // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress();
+
+ // conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName());
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/",
+ folder.getRoot().toURI().toString());
+ // conf.set("fs.hdfs.impl.disable.cache", "true");
+
+ FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf);
+
+ Path fileInWrapper = new Path("/wrapper/file1");
+ Path fileInHDFS = new Path("/hdfsfile1");
+
+ byte[] bytes = generateByteArry(4096);
+
+ try (FSDataOutputStream out = fs.create(fileInWrapper)) {
+ out.write(bytes);
+ } catch (IOException e) {
+ fail("write file failed");
+ }
+
+ try (FSDataOutputStream out = fs.create(fileInHDFS)) {
+ out.write(bytes);
+ } catch (IOException e) {
+ fail("write file failed");
+ }
+
+ byte[] resultWrapper = new byte[4096];
+ try (FSDataInputStream in = fs.open(fileInHDFS)) {
+ in.read(resultWrapper);
+ }
+
+ byte[] resultHdfs = new byte[4096];
+ try (FSDataInputStream in = fs.open(fileInHDFS)) {
+ in.read(resultHdfs);
+ }
+
+ Assert.assertArrayEquals(bytes, resultHdfs);
+ Assert.assertArrayEquals(bytes, resultWrapper);
+ // }
+ }
+
+
+
+ @Test
+ public void testListStatus() throws IOException {
+ // Configuration conf = new HdfsConfiguration();
+ // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) {
+ // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress();
+
+ conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName());
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/",
+ folder.getRoot().toURI().toString());
+ conf.set("fs.hdfs.impl.disable.cache", "true");
+
+ // hdfs native file
+ fSys.create(new Path("/hdfsfile1")).close();
+ fSys.create(new Path("/hdfsfile2")).close();
+ fSys.create(new Path("/hdfsfile3")).close();
+ fSys.mkdirs(new Path("/hdfsdir"),
+ FsPermission.createImmutable((short) 777));
+ fSys.create(new Path("/hdfsdir/subfile1")).close();
+ fSys.create(new Path("/hdfsdir/subfile2")).close();
+
+ // wrapped local file system files
+ folder.newFile("newFile1");
+ folder.newFile("newFile2");
+ folder.newFile("newFile3");
+ File subfolder = folder.newFolder("folder1");
+ subfolder.toPath().resolve("subFile1").toFile().createNewFile();
+ subfolder.toPath().resolve("subFile2").toFile().createNewFile();
+
+ FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf);
+ Path listPath = new Path("hdfs://" + fsAuthority + "/wrapper");
+ Path listPathWithTailSlash = new Path("hdfs://" + fsAuthority + "/wrapper/");
+ FileStatus[] files = fs.listStatus(listPath);
+ Set filesSet = Arrays.stream(files).map(FileStatus::getPath).map(Path::toString).collect(Collectors.toSet());
+ Assert.assertEquals(4, files.length);
+ Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile1").toString()));
+ Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile2").toString()));
+ Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile3").toString()));
+ Assert.assertTrue(filesSet.contains(new Path(listPath, "folder1").toString()));
+
+ FileStatus[] files2 = fs.listStatus(listPathWithTailSlash);
+ Set filesSet2 = Arrays.stream(files2).map(FileStatus::getPath).map(Path::toString).collect(Collectors.toSet());
+ Assert.assertEquals(4, files2.length);
+ Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile1").toString()));
+ Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile2").toString()));
+ Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile3").toString()));
+ Assert.assertTrue(filesSet2.contains(new Path(listPath, "folder1").toString()));
+
+
+ RemoteIterator recursivelistIrt = fs.listFiles(listPath, true);
+ Set recursivelistPaths = new HashSet<>();
+ while (recursivelistIrt.hasNext()) {
+ recursivelistPaths.add(recursivelistIrt.next().getPath().toString());
+ }
+
+ Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "newFile1").toString()));
+ Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "newFile2").toString()));
+ Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "newFile3").toString()));
+ Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "folder1/subFile1").toString()));
+ Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "folder1/subFile2").toString()));
+
+ RemoteIterator nonRecursivelistIrt = fs.listFiles(listPath, false);
+ Set nonRecursivelistPaths = new HashSet<>();
+ while (nonRecursivelistIrt.hasNext()) {
+ nonRecursivelistPaths.add(nonRecursivelistIrt.next().getPath().toString());
+ }
+
+ Assert.assertTrue(nonRecursivelistPaths.contains(new Path(listPath, "newFile1").toString()));
+ Assert.assertTrue(nonRecursivelistPaths.contains(new Path(listPath, "newFile2").toString()));
+ Assert.assertTrue(nonRecursivelistPaths.contains(new Path(listPath, "newFile3").toString()));
+
+ Path listHDFSPath = new Path("hdfs://" + fsAuthority + "/");
+ FileStatus[] hdfsFiles = fs.listStatus(listHDFSPath);
+ Set filesSetHdfs = Arrays.stream(hdfsFiles).map(FileStatus::getPath).map(Path::toString).collect(Collectors.toSet());
+ Assert.assertEquals(5, filesSetHdfs.size());
+ Assert.assertTrue(filesSetHdfs.contains(new Path(listHDFSPath, "hdfsfile1").toString()));
+ Assert.assertTrue(filesSetHdfs.contains(new Path(listHDFSPath, "hdfsfile2").toString()));
+ Assert.assertTrue(filesSetHdfs.contains(new Path(listHDFSPath, "hdfsfile3").toString()));
+ Assert.assertTrue(filesSetHdfs.contains(new Path(listHDFSPath, "hdfsdir").toString()));
+
+ RemoteIterator recursivelistHDFSIrt = fs.listFiles(listHDFSPath, true);
+ Set recursivelistHDFSPaths = new HashSet<>();
+ while (recursivelistHDFSIrt.hasNext()) {
+ recursivelistHDFSPaths.add(recursivelistHDFSIrt.next().getPath().toString());
+ }
+ Assert.assertEquals(7, recursivelistHDFSPaths.size());
+ Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "hdfsfile1").toString()));
+ Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "hdfsfile2").toString()));
+ Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "hdfsfile3").toString() ));
+ Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "hdfsdir/subfile1").toString()));
+ Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "hdfsdir/subfile2").toString()));
+ fSys.deleteOnExit(new Path("/wrapper"));
+ // }
+ }
+
+ @Test
+ public void testListStatus1() throws IOException {
+ // Configuration conf = new HdfsConfiguration();
+ // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) {
+ // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress();
+
+ conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName());
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link"
+ + "./wrapper1234/", folder.getRoot().toURI().toString() + "/test1234");
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper",
+ folder.getRoot().toURI().toString() + "/test1234");
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper"
+ + "/1234/", folder.getRoot().toURI().toString() + "/test12345");
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./yy/",
+ folder.getRoot().toURI().toString() + "/folder1");
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/",
+ folder.getRoot().toURI().toString());
+ conf.set("fs.hdfs.impl.disable.cache", "true");
+
+ // hdfs native file
+ fSys.create(new Path("/hdfsfile1")).close();
+ fSys.create(new Path("/hdfsfile2")).close();
+ fSys.create(new Path("/hdfsfile3")).close();
+ fSys.mkdirs(new Path("/hdfsdir"),
+ FsPermission.createImmutable((short) 777));
+ fSys.create(new Path("/hdfsdir/subfile1")).close();
+ fSys.create(new Path("/hdfsdir/subfile2")).close();
+
+ // wrapped local file system files
+ folder.newFile("newFile1");
+ folder.newFile("newFile2");
+ folder.newFile("newFile3");
+ File subfolder = folder.newFolder("folder1");
+ subfolder.toPath().resolve("subFile1").toFile().createNewFile();
+ subfolder.toPath().resolve("subFile2").toFile().createNewFile();
+
+ FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf);
+ Path listPath = new Path("hdfs://" + fsAuthority + "/yy");
+
+ FileStatus[] status = fs.listStatus(listPath);
+ Assert.assertEquals(2, status.length);
+ fSys.deleteOnExit(new Path("/wrapper"));
+ // }
+ }
+
+ @Test
+ public void testListLocatedStatus() throws IOException, NoSuchMethodException, InvocationTargetException, IllegalAccessException {
+ // Configuration conf = new HdfsConfiguration();
+ // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) {
+ // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress();
+
+ conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName());
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/",
+ folder.getRoot().toURI().toString());
+ conf.set("fs.hdfs.impl.disable.cache", "true");
+
+ // hdfs native file
+ fSys.create(new Path("/hdfsfile1")).close();
+ fSys.create(new Path("/hdfsfile2")).close();
+ fSys.create(new Path("/hdfsfile3")).close();
+ fSys.mkdirs(new Path("/hdfsdir"),
+ FsPermission.createImmutable((short) 777));
+ fSys.create(new Path("/hdfsdir/subfile1")).close();
+ fSys.create(new Path("/hdfsdir/subfile2")).close();
+
+ // wrapped local file system files
+ folder.newFile("newFile1");
+ folder.newFile("newFile2");
+ folder.newFile("newFile3");
+ File subfolder = folder.newFolder("folder1");
+ subfolder.toPath().resolve("subFile1").toFile().createNewFile();
+ subfolder.toPath().resolve("subFile2").toFile().createNewFile();
+
+ FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf);
+ Path listPath = new Path("hdfs://" + fsAuthority + "/wrapper");
+ Path listPathWithTailSlash = new Path("hdfs://" + fsAuthority + "/wrapper/");
+ RemoteIterator files = fs.listLocatedStatus(listPath);
+ List locatedStatusList = new ArrayList<>();
+ while (files.hasNext()) {
+ locatedStatusList.add(files.next());
+ }
+
+ Set filesSet = locatedStatusList.stream().map(FileStatus::getPath).map(Path::toString).collect(Collectors.toSet());
+ Assert.assertEquals(4, locatedStatusList.size());
+ Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile1").toString()));
+ Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile2").toString()));
+ Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile3").toString()));
+ Assert.assertTrue(filesSet.contains(new Path(listPath, "folder1").toString()));
+
+ RemoteIterator files2 = fs.listLocatedStatus(listPathWithTailSlash);
+ List locatedStatusList2 = new ArrayList<>();
+ while (files2.hasNext()) {
+ locatedStatusList2.add(files2.next());
+ }
+ Set filesSet2 = locatedStatusList2.stream().map(FileStatus::getPath).map(Path::toString).collect(Collectors.toSet());
+ Assert.assertEquals(4, locatedStatusList2.size());
+ Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile1").toString()));
+ Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile2").toString()));
+ Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile3").toString()));
+ Assert.assertTrue(filesSet2.contains(new Path(listPath, "folder1").toString()));
+ fSys.deleteOnExit(new Path("/wrapper"));
+ // }
+ }
+
+ @Test
+ public void testListStatus2() throws IOException {
+ // Configuration conf = new HdfsConfiguration();
+ // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) {
+ // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress();
+
+ conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName());
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./ ",
+ folder.getRoot().toURI().toString());
+ conf.set("fs.hdfs.impl.disable.cache", "true");
+
+ // hdfs native file
+ fSys.create(new Path("/hdfsfile1")).close();
+ fSys.create(new Path("/hdfsfile2")).close();
+ fSys.create(new Path("/hdfsfile3")).close();
+ fSys.mkdirs(new Path("/hdfsdir"),
+ FsPermission.createImmutable((short) 777));
+ fSys.create(new Path("/hdfsdir/subfile1")).close();
+ fSys.create(new Path("/hdfsdir/subfile2")).close();
+
+ // wrapped local file system files
+ folder.newFile("newFile1");
+ folder.newFile("newFile2");
+ folder.newFile("newFile3");
+ File subfolder = folder.newFolder("folder1");
+ subfolder.toPath().resolve("subFile1").toFile().createNewFile();
+ subfolder.toPath().resolve("subFile2").toFile().createNewFile();
+
+ FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf);
+ Path listPath = new Path("hdfs://" + fsAuthority + "/");
+ Path listPathWithTailSlash = new Path("hdfs://" + fsAuthority + "/");
+ FileStatus[] files = fs.listStatus(listPath);
+ Set filesSet = Arrays.stream(files).map(FileStatus::getPath).map(Path::toString).collect(Collectors.toSet());
+ Assert.assertEquals(4, files.length);
+ Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile1").toString()));
+ Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile2").toString()));
+ Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile3").toString()));
+ Assert.assertTrue(filesSet.contains(new Path(listPath, "folder1").toString()));
+
+ FileStatus[] files2 = fs.listStatus(listPathWithTailSlash);
+ Set filesSet2 = Arrays.stream(files2).map(FileStatus::getPath).map(Path::toString).collect(Collectors.toSet());
+ Assert.assertEquals(4, files2.length);
+ Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile1").toString()));
+ Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile2").toString()));
+ Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile3").toString()));
+ Assert.assertTrue(filesSet2.contains(new Path(listPath, "folder1").toString()));
+
+
+ RemoteIterator recursivelistIrt = fs.listFiles(listPath, true);
+ Set recursivelistPaths = new HashSet<>();
+ while (recursivelistIrt.hasNext()) {
+ recursivelistPaths.add(recursivelistIrt.next().getPath().toString());
+ }
+
+ Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "newFile1").toString()));
+ Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "newFile2").toString()));
+ Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "newFile3").toString()));
+ Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "folder1/subFile1").toString()));
+ Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "folder1/subFile2").toString()));
+
+ RemoteIterator nonRecursivelistIrt = fs.listFiles(listPath, false);
+ Set nonRecursivelistPaths = new HashSet<>();
+ while (nonRecursivelistIrt.hasNext()) {
+ nonRecursivelistPaths.add(nonRecursivelistIrt.next().getPath().toString());
+ }
+
+ Assert.assertTrue(nonRecursivelistPaths.contains(new Path(listPath, "newFile1").toString()));
+ Assert.assertTrue(nonRecursivelistPaths.contains(new Path(listPath, "newFile2").toString()));
+ Assert.assertTrue(nonRecursivelistPaths.contains(new Path(listPath, "newFile3").toString()));
+
+ Path listHDFSPath = new Path("hdfs://" + fsAuthority + "/");
+ FileStatus[] hdfsFiles = fs.listStatus(listHDFSPath);
+ Set filesSetHdfs = Arrays.stream(hdfsFiles).map(FileStatus::getPath).map(Path::toString).collect(Collectors.toSet());
+ Assert.assertEquals(4, filesSetHdfs.size());
+ Assert.assertTrue(filesSetHdfs.contains(new Path(listHDFSPath, "folder1").toString()));
+ Assert.assertTrue(filesSetHdfs.contains(new Path(listHDFSPath, "newFile3").toString()));
+ Assert.assertTrue(filesSetHdfs.contains(new Path(listHDFSPath, "newFile2").toString()));
+ Assert.assertTrue(filesSetHdfs.contains(new Path(listHDFSPath, "newFile1").toString()));
+
+ RemoteIterator recursivelistHDFSIrt = fs.listFiles(listHDFSPath, true);
+ Set recursivelistHDFSPaths = new HashSet<>();
+ while (recursivelistHDFSIrt.hasNext()) {
+ recursivelistHDFSPaths.add(recursivelistHDFSIrt.next().getPath().toString());
+ }
+ Assert.assertEquals(5, recursivelistHDFSPaths.size());
+ Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "newFile1").toString()));
+ Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "newFile3").toString()));
+ Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "newFile2").toString()));
+ Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "folder1/subFile1").toString()));
+ Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "folder1/subFile2").toString()));
+
+ fSys.deleteOnExit(new Path("/wrapper"));
+ // }
+ }
+
+ @Test
+ public void testListStatus3() throws IOException {
+ // Configuration conf = new HdfsConfiguration();
+ // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) {
+ // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress();
+
+ conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName());
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/",
+ folder.getRoot().toURI().toString());
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./abc/",
+ folder.getRoot().toURI().toString() + "/folder1");
+ conf.set("fs.hdfs.impl.disable.cache", "true");
+
+ // hdfs native file
+ fSys.create(new Path("/hdfsfile1")).close();
+ fSys.create(new Path("/hdfsfile2")).close();
+ fSys.create(new Path("/hdfsfile3")).close();
+ fSys.mkdirs(new Path("/hdfsdir"),
+ FsPermission.createImmutable((short) 777));
+ fSys.create(new Path("/hdfsdir/subfile1")).close();
+ fSys.create(new Path("/hdfsdir/subfile2")).close();
+
+ // wrapped local file system files
+ folder.newFile("newFile1");
+ folder.newFile("newFile2");
+ folder.newFile("newFile3");
+ File subfolder = folder.newFolder("folder1");
+ subfolder.toPath().resolve("subFile1").toFile().createNewFile();
+ subfolder.toPath().resolve("subFile2").toFile().createNewFile();
+
+ FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf);
+ // Path listPath = new Path("hdfs://" + fsAuthority + "/");
+ Path listPathLocal = new Path("hdfs://" + fsAuthority + "/wrapper/");
+ Path listPathLocal2 = new Path("hdfs://" + fsAuthority + "/abc/");
+
+ RemoteIterator filesLocalItr2 = fs.listStatusIterator(listPathLocal2);
+ Set filesLocal2Set = remoteItrToList(filesLocalItr2).stream().map(FileStatus::getPath)
+ .map(Path::toString).collect(Collectors.toSet());
+ Assert.assertEquals(2, filesLocal2Set.size());
+ Assert.assertTrue(filesLocal2Set.contains(new Path(listPathLocal2, "subFile1").toString()));
+ Assert.assertTrue(filesLocal2Set.contains(new Path(listPathLocal2, "subFile2").toString()));
+
+ RemoteIterator locatedFilesLocalItr2 = fs.listLocatedStatus(listPathLocal2);
+ Set locatedFilesLocal2Set = remoteItrToList(locatedFilesLocalItr2).stream().map(FileStatus::getPath)
+ .map(Path::toString).collect(Collectors.toSet());
+ Assert.assertEquals(2, locatedFilesLocal2Set.size());
+ Assert.assertTrue(locatedFilesLocal2Set.contains(new Path(listPathLocal2, "subFile1").toString()));
+ Assert.assertTrue(locatedFilesLocal2Set.contains(new Path(listPathLocal2, "subFile2").toString()));
+
+ RemoteIterator recursiveFilesLocalItr2 = fs.listFiles(listPathLocal2, true);
+ Set recursiveFilesLocal2Set = remoteItrToList(recursiveFilesLocalItr2).stream().map(FileStatus::getPath)
+ .map(Path::toString).collect(Collectors.toSet());
+ Assert.assertEquals(2, recursiveFilesLocal2Set.size());
+ Assert.assertTrue(recursiveFilesLocal2Set.contains(new Path(listPathLocal2, "subFile1").toString()));
+ Assert.assertTrue(recursiveFilesLocal2Set.contains(new Path(listPathLocal2, "subFile2").toString()));
+ fSys.deleteOnExit(new Path("/wrapper"));
+ // }
+ }
+
+ @Test
+ public void testDeleteFile() throws IOException, NoSuchMethodException, InvocationTargetException, IllegalAccessException {
+ // Configuration conf = new HdfsConfiguration();
+ // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) {
+ // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress();
+
+ conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName());
+ conf.set("fs.trash.interval", "30");
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/",
+ folder.getRoot().toURI().toString());
+ conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./abc/",
+ folder.getRoot().toURI().toString() + "/folder1");
+ conf.set("fs.hdfs.impl.disable.cache", "true");
+
+ // hdfs native file
+ fSys.create(new Path("/hdfsfile1")).close();
+ fSys.create(new Path("/hdfsfile2")).close();
+ fSys.create(new Path("/hdfsfile3")).close();
+ fSys.mkdirs(new Path("/hdfsdir"),
+ FsPermission.createImmutable((short) 777));
+ fSys.create(new Path("/hdfsdir/subfile1")).close();
+ fSys.create(new Path("/hdfsdir/subfile2")).close();
+
+ // wrapped local file system files
+ folder.newFile("newFile1");
+ folder.newFile("newFile2");
+ folder.newFile("newFile3");
+ File subfolder = folder.newFolder("folder1");
+ subfolder.toPath().resolve("subFile1").toFile().createNewFile();
+ subfolder.toPath().resolve("subFile2").toFile().createNewFile();
+
+ FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf);
+ Path file1 = new Path("hdfs://" + fsAuthority + "/hdfsfile1");
+ Assert.assertTrue(fs.delete(file1, false));
+ try {
+ fs.getFileStatus(file1);
+ fail("should throw FileNotFoundException");
+ } catch (IOException e) {
+ Assert.assertTrue(e instanceof FileNotFoundException);
+ Assert.assertFalse(fs.exists(file1));
+ }
+
+ Path fileFolder = new Path("hdfs://" + fsAuthority + "/hdfsdir");
+ Assert.assertTrue(fs.delete(fileFolder, true));
+ Assert.assertFalse(fs.exists(fileFolder));
+
+ Path fileLocal1 = new Path("hdfs://" + fsAuthority + "/wrapper/newFile1");
+ Assert.assertTrue(fs.delete(fileLocal1, false));
+
+ try {
+ fs.getFileStatus(fileLocal1);
+ fail("should throw FileNotFoundException");
+ } catch (IOException e) {
+ Assert.assertTrue(e instanceof FileNotFoundException);
+ Assert.assertFalse(fs.exists(fileLocal1));
+ }
+
+ Path fileLocalFolder = new Path("hdfs://" + fsAuthority + "/wrapper/folder1");
+ Assert.assertTrue(fs.delete(fileLocalFolder, true));
+ Assert.assertFalse(fs.exists(fileLocalFolder));
+ // }
+ }
+
+ private List remoteItrToList(RemoteIterator itr) throws IOException {
+ List list = new ArrayList<>();
+ if (itr != null) {
+ while (itr.hasNext()) {
+ list.add(itr.next());
+ }
+ }
+ return list;
+ }
+
+ Random r = new Random();
+ private byte[] generateByteArry(int length) {
+ byte[] arr = new byte[length];
+ r.nextBytes(arr);
+ return arr;
+ }
+
+ @Override
+ public void testFsStatus() {
+ skip("Unsupport.");
+ }
+
+ @Override
+ public void testWorkingDirectory() {
+ skip("Unsupport.");
+ }
+
+ @Override
+ public void testListStatusThrowsExceptionForUnreadableDir() {
+ skip("Unsupport.");
+ }
+
+ @Override
+ public void testRenameDirectoryAsEmptyDirectory() {
+ skip("Unsupport.");
+ }
+
+ @Override
+ public void testRenameDirectoryAsFile() {
+ skip("Unsupport.");
+ }
+
+ @Override
+ public void testRenameDirectoryAsNonEmptyDirectory() {
+ skip("Unsupport.");
+ }
+
+ @Override
+ public void testRenameDirectoryToItself(){
+ skip("Unsupport.");
+ }
+
+ @Override
+ public void testRenameDirectoryToNonExistentParent(){
+ skip("Unsupport.");
+ }
+
+ @Override
+ public void testRenameFileAsExistingDirectory(){
+ skip("Unsupport.");
+ }
+
+ @Override
+ public void testRenameFileAsExistingFile(){
+ skip("Unsupport.");
+ }
+
+ @Override
+ public void testRenameFileToItself(){
+ skip("Unsupport.");
+ }
+
+ @Override
+ public void testRenameFileToNonExistentDirectory(){
+ skip("Unsupport.");
+ }
+
+ @Override
+ public void testRenameNonExistentPath(){
+ skip("Unsupport.");
+ }
+
+ @Override
+ public void testWDAbsolute(){
+ skip("Unsupport.");
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHFlush.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHFlush.java
new file mode 100644
index 0000000..01ae3fc
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHFlush.java
@@ -0,0 +1,249 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.EnumSet;
+
+/**
+ * OBS tests for configuring block size.
+ */
+@RunWith(Parameterized.class)
+public class ITestOBSHFlush {
+ private OBSFileSystem fs;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ private static final Logger LOG = LoggerFactory.getLogger(
+ ITestOBSHFlush.class);
+
+ @Parameterized.Parameters
+ public static Collection inputStreams() {
+ return Arrays.asList(
+ OBSConstants.OUTPUT_STREAM_HFLUSH_POLICY_SYNC,
+ OBSConstants.OUTPUT_STREAM_HFLUSH_POLICY_EMPTY,
+ OBSConstants.OUTPUT_STREAM_HFLUSH_POLICY_FLUSH
+ );
+ }
+
+ private boolean downgrade;
+
+ public ITestOBSHFlush(String downgradePolicy) throws IOException {
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.set(OBSConstants.MULTIPART_SIZE, String.valueOf(5 * 1024 * 1024));
+ conf.set(OBSConstants.OUTPUT_STREAM_HFLUSH_POLICY, downgradePolicy);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ if (fs.exists(getTestPath("testFlush"))) {
+ fs.delete(getTestPath("testFlush"), true);
+ }
+ downgrade = (!downgradePolicy.equals(OBSConstants.OUTPUT_STREAM_HFLUSH_POLICY_SYNC));
+ }
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(new Path(testRootPath), true);
+ fs.close();
+ fs = null;
+ }
+ }
+
+ private Path getTestPath(String relativePath) {
+ return new Path(testRootPath + "/" + relativePath);
+ }
+
+ @Test
+ // 200M数据,分10次写,每次写完hflush,校验文件大小
+ public void testFlush01() {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ try {
+ doTheJob("testFlush", OBSAppendTestUtil.BLOCK_SIZE, (short) 2, false, downgrade,
+ EnumSet.noneOf(HdfsDataOutputStream.SyncFlag.class),
+ 1024 * 1024 * 200);
+ } catch (Exception e) {
+ LOG.error(e.getMessage());
+ assertNull("cache exception", e);
+ }
+ }
+
+ @Test
+ // 20M数据,分10次写,每次写完hflush,校验文件大小
+ public void testFlush02() {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ try {
+ doTheJob("testFlush", OBSAppendTestUtil.BLOCK_SIZE, (short) 2, false, downgrade,
+ EnumSet.noneOf(HdfsDataOutputStream.SyncFlag.class),
+ 1024 * 1024 * 20);
+ } catch (Exception e) {
+ LOG.error(e.getMessage());
+ assertNull("cache exception", e);
+ }
+ }
+
+ @Test
+ // 51M数据,分10次写,每次写完hflush,校验文件大小
+ public void testFlush03() {
+ if (!fs.isFsBucket()) {
+ return;
+ }
+ try {
+ doTheJob("testFlush", OBSAppendTestUtil.BLOCK_SIZE, (short) 2, false, downgrade,
+ EnumSet.noneOf(HdfsDataOutputStream.SyncFlag.class),
+ 1024 * 1024 * 51);
+ } catch (Exception e) {
+ LOG.error(e.getMessage());
+ assertNull("cache exception", e);
+ }
+ }
+
+ /**
+ * The method starts new cluster with defined Configuration; creates a file
+ * with specified block_size and writes 10 equal sections in it; it also
+ * calls hflush/hsync after each write and throws an IOException in case of
+ * an error.
+ *
+ * @param fileName of the file to be created and processed as required
+ * @param block_size value to be used for the file's creation
+ * @param replicas is the number of replicas
+ * @param isSync hsync or hflush
+ * @param syncFlags specify the semantic of the sync/flush
+ * @throws IOException in case of any errors
+ */
+ public void doTheJob(final String fileName,
+ long block_size, short replicas, boolean isSync, boolean downgrade,
+ EnumSet syncFlags, int size)
+ throws IOException {
+ byte[] fileContent;
+ final int SECTIONS = 10;
+
+ fileContent = OBSAppendTestUtil.initBuffer(size);
+
+ FSDataInputStream is;
+ try {
+ Path path = getTestPath(fileName);
+ FSDataOutputStream stm = fs.create(path, false, 4096, replicas,
+ block_size);
+
+ int tenth = size / SECTIONS;
+ int rounding = size - tenth * SECTIONS;
+ for (int i = 0; i < SECTIONS; i++) {
+ // write to the file
+ stm.write(fileContent, tenth * i, tenth);
+
+ // Wait while hflush/hsync pushes all packets through built pipeline
+ if (isSync) {
+ ((OBSBlockOutputStream) stm.getWrappedStream()).hsync();
+ } else {
+ ((OBSBlockOutputStream) stm.getWrappedStream()).hflush();
+ }
+
+ if (downgrade) {
+ // skip check flushed block
+ continue;
+ }
+
+ // Check file length if update length is required
+ if (isSync && syncFlags.contains(HdfsDataOutputStream.SyncFlag.UPDATE_LENGTH)) {
+ long currentFileLength = fs.getFileStatus(path).getLen();
+ assertEquals("File size doesn't match for hsync/hflush with updating the length", tenth * (i + 1), currentFileLength);
+ } else if (!isSync || (!syncFlags.contains(HdfsDataOutputStream.SyncFlag.END_BLOCK)
+ && !syncFlags.contains(HdfsDataOutputStream.SyncFlag.UPDATE_LENGTH))) {
+ long currentFileLength = fs.getFileStatus(path).getLen();
+ assertEquals("File size doesn't match for hsync/hflush with updating the length", tenth * (i + 1), currentFileLength);
+ }
+
+ byte[] toRead = new byte[tenth];
+ byte[] expected = new byte[tenth];
+ System.arraycopy(fileContent, tenth * i, expected, 0, tenth);
+ // Open the same file for read. Need to create new reader after every write operation(!)
+ is = fs.open(path);
+ is.seek(tenth * i);
+ int readBytes = is.read(toRead, 0, tenth);
+ System.out.println("Has read " + readBytes);
+ Assert.assertTrue("Should've get more bytes",
+ (readBytes > 0) && (readBytes <= tenth));
+ is.close();
+ checkData(toRead, 0, readBytes, expected,
+ "Partial verification");
+ }
+
+ stm.write(fileContent, tenth * SECTIONS, rounding);
+ stm.close();
+
+ OBSAppendTestUtil.checkFullFile(fs, path, fileContent.length,
+ fileContent, "hflush()");
+ } finally {
+ fs.delete(new Path(testRootPath), true);
+ fs.close();
+ fs = null;
+ }
+ }
+
+ void checkData(final byte[] actual, int from, int len,
+ final byte[] expected, String message) {
+ for (int idx = 0; idx < len; idx++) {
+ assertEquals(
+ message + " byte " + (from + idx) + " differs. expected " +
+ expected[from + idx] + " actual " + actual[idx],
+ expected[from + idx], actual[idx]);
+ actual[idx] = 0;
+ }
+ }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInputStream.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInputStream.java
new file mode 100644
index 0000000..8d880c9
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInputStream.java
@@ -0,0 +1,669 @@
+/**
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.fs.obs.input.InputPolicyFactory;
+import org.apache.hadoop.fs.obs.input.InputPolicys;
+import org.apache.hadoop.fs.obs.input.ReadAheadBuffer;
+import org.apache.hadoop.fs.obs.input.ReadAheadTask;
+import org.apache.hadoop.fs.obs.mock.MockMemArtsCCClient;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.mockito.Mockito;
+import org.mockito.stubbing.Answer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Random;
+
+/**
+ * Tests basic functionality for OBSInputStream, including seeking and
+ * reading files.
+ */
+@RunWith(Parameterized.class)
+public class ITestOBSInputStream {
+
+ private OBSFileSystem fs;
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ITestOBSInputStream.class);
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ private String inputPolicy;
+
+ @Parameterized.Parameters
+ public static Collection inputStreams() {
+ return Arrays.asList(
+ OBSConstants.READAHEAD_POLICY_PRIMARY,
+ // OBSConstants.READAHEAD_POLICY_ADVANCE,
+ OBSConstants.READAHEAD_POLICY_MEMARTSCC
+ );
+ }
+
+ public ITestOBSInputStream(String inputPolicy) throws IOException, NoSuchFieldException, IllegalAccessException {
+ this.inputPolicy = inputPolicy;
+ if (inputPolicy.equals(OBSConstants.READAHEAD_POLICY_PRIMARY)) {
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.setClass(OBSConstants.OBS_METRICS_CONSUMER,
+ MockMetricsConsumer.class, BasicMetricsConsumer.class);
+ conf.setBoolean(OBSConstants.METRICS_SWITCH, true);
+ conf.set(OBSConstants.READAHEAD_POLICY, OBSConstants.READAHEAD_POLICY_PRIMARY);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ }
+ if (inputPolicy.equals(OBSConstants.READAHEAD_POLICY_MEMARTSCC)) {
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, MockMetricsConsumer.class, BasicMetricsConsumer.class);
+ conf.setBoolean(OBSConstants.METRICS_SWITCH, true);
+ conf.set(OBSConstants.READAHEAD_POLICY, OBSConstants.READAHEAD_POLICY_MEMARTSCC);
+ conf.set(OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_TYPE, OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_TYPE_BIND);
+ this.fs = OBSTestUtils.createTestFileSystem(conf);
+
+ MockMemArtsCCClient mockMemArtsCCClient;
+ mockMemArtsCCClient = new MockMemArtsCCClient(fs, false, this.fs.getBucket());
+ mockMemArtsCCClient.init("", "");
+
+ InputPolicyFactory inputPolicyFactory = InputPolicys.createFactory(OBSConstants.READAHEAD_POLICY_MEMARTSCC);
+
+ // mock memartscc client
+ Field ccClient = OBSFileSystem.class.getDeclaredField("memArtsCCClient");
+ ccClient.setAccessible(true);
+ ccClient.set(this.fs, mockMemArtsCCClient);
+
+ // mock input policy factory
+ Field fInputPolicyFactory = OBSFileSystem.class.getDeclaredField("inputPolicyFactory");
+ fInputPolicyFactory.setAccessible(true);
+ fInputPolicyFactory.set(this.fs, inputPolicyFactory);
+ }
+ }
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @Rule
+ public Timeout testTimeout = new Timeout(30 * 60 * 1000);
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(new Path(testRootPath), true);
+ }
+ }
+
+ private Path setPath(String path) {
+ if (path.startsWith("/")) {
+ return new Path(testRootPath + path);
+ } else {
+ return new Path(testRootPath + "/" + path);
+ }
+ }
+
+ private static void genTestBuffer(byte[] buffer, long lenth) {
+ for (int i = 0; i < lenth; i++) {
+ buffer[i] = (byte) (i % 255);
+ }
+ }
+
+ @Test
+ // 校验inputstream默认readahead大小为1M
+ public void testDefaultReadAheadSize() {
+ long readAHeadSize = fs.getReadAheadRange();
+ assertTrue("Default read ahead size must 1MB",
+ readAHeadSize == 1 * 1024 * 1024);
+ }
+
+ @Test
+ public void testFileReaderTask() throws Exception {
+ Path smallSeekFile = setPath("/test/testFileReaderTask.txt");
+ long size = 5 * 1024 * 1024;
+
+ ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256, 255);
+ LOG.info("5MB file created: smallSeekFileOSSFileReader.txt");
+ ReadAheadBuffer readBuffer = new ReadAheadBuffer(12, 24);
+ ReadAheadTask task = new ReadAheadTask(fs.getBucket(),"1",
+ fs.getObsClient(), readBuffer);
+ //NullPointerException, fail
+ task.run();
+ assertEquals(readBuffer.getStatus(), ReadAheadBuffer.STATUS.ERROR);
+ //OK
+ task = new ReadAheadTask(fs.getBucket(),"test/test/testFileReaderTask.txt",
+ fs.getObsClient(), readBuffer);
+ task.run();
+ assertEquals(readBuffer.getStatus(), ReadAheadBuffer.STATUS.SUCCESS);
+ }
+
+ @Test
+ // 打开文件流,按固定长度和随机长度分别向前seek 5次
+ public void testInSeekFile() throws Exception {
+ Path smallSeekFile = setPath("/test/smallSeekFile.txt");
+ long size = 5 * 1024 * 1024;
+
+ ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256,
+ 255);
+ LOG.info("5MB file created: smallSeekFile.txt");
+
+ FSDataInputStream instream = this.fs.open(smallSeekFile);
+ int seekTimes = 5;
+ LOG.info("multiple fold position seeking test...:");
+ for (int i = 0; i < seekTimes; i++) {
+ long pos = size / (seekTimes - i) - 1;
+ LOG.info("begin seeking for pos: " + pos);
+ instream.seek(pos);
+ assertTrue("expected position at:" + pos + ", but got:"
+ + instream.getPos(), instream.getPos() == pos);
+ LOG.info("completed seeking at pos: " + instream.getPos());
+ }
+ LOG.info("random position seeking test...:");
+ Random rand = new Random();
+ for (int i = 0; i < seekTimes; i++) {
+ long pos = Math.abs(rand.nextLong()) % size;
+ LOG.info("begin seeking for pos: " + pos);
+ instream.seek(pos);
+ assertTrue("expected position at:" + pos + ", but got:"
+ + instream.getPos(), instream.getPos() == pos);
+ LOG.info("completed seeking at pos: " + instream.getPos());
+ }
+ IOUtils.closeStream(instream);
+ }
+
+ @Test
+ // 打开文件输入流,分别测试随机和顺序读取功能正确性
+ public void testSequentialAndRandomRead() throws Exception {
+ Path smallSeekFile = setPath("/test/smallSeekFile.txt");
+ int size = 5 * 1024 * 1024;
+
+ ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256,
+ 255);
+ LOG.info("5MB file created: smallSeekFile.txt");
+
+ long now = System.currentTimeMillis();
+ FSDataInputStream fsDataInputStream = this.fs.open(smallSeekFile);
+ // OBSInputStream in =
+ // (OBSInputStream) fsDataInputStream.getWrappedStream();
+ assertTrue("expected position at:" + 0 + ", but got:"
+ + fsDataInputStream.getPos(), fsDataInputStream.getPos() == 0);
+
+ byte[] bytes = new byte[(int) size];
+ int bytesRead = 0;
+ int bufOffset = 0;
+ int bytesRemaining = size;
+ int ret = 0;
+ while (bytesRead < size) {
+ ret = fsDataInputStream.read(bytes, bufOffset, bytesRemaining);
+ // System.out.println(ret);
+
+ if (ret < 0) {
+ break;
+ }
+
+ bufOffset += ret;
+ bytesRemaining -= ret;
+ bytesRead += ret;
+ }
+
+ LOG.warn("time consumed mills:" + String.valueOf(
+ System.currentTimeMillis() - now));
+
+ byte[] testBuffer = new byte[256];
+ genTestBuffer(testBuffer, 256);
+
+ byte[] equalSizeBuffer = new byte[(int) size];
+
+ for (int i = 0; i < equalSizeBuffer.length; i++) {
+ equalSizeBuffer[i] = testBuffer[(i) % 256];
+ }
+ assertTrue(Arrays.equals(bytes, equalSizeBuffer));
+
+ fsDataInputStream.close();
+ //fsDataInputStream.seek(4 * 1024 * 1024);
+
+ }
+
+ @Test
+ // 打开文件输入流,测试read byteBuffer功能正确性
+ public void testReadWithByteBuffer() throws Exception {
+ final int bufLen = 256 * 1024;
+ final int sizeMB = 100;
+ String filename = "readTestFile_" + sizeMB + ".txt";
+ Path readTestFile = setPath("/test/" + filename);
+ long size = sizeMB * 1024 * 1024;
+
+ ContractTestUtils.generateTestFile(this.fs, readTestFile, size, 256,
+ 255);
+ LOG.info(sizeMB + "MB file created: /test/" + filename);
+
+ FSDataInputStream instream = this.fs.open(readTestFile);
+ ByteBuffer byteBuffer = ByteBuffer.allocateDirect(bufLen);
+ long expectedRound = size / bufLen + (size % bufLen == 0 ? 0 : 1);
+ long bytesRead = 0;
+ long round = 0;
+ while (bytesRead < size) {
+ byteBuffer.clear();
+ round++;
+ int bytes = instream.read(byteBuffer);
+ if (bytes == -1) {
+ break;
+ }
+
+ if (round < expectedRound) {
+ assertEquals(bufLen, bytes);
+ } else {
+ assertEquals(size - bytesRead, bytes);
+ }
+
+ bytesRead += bytes;
+
+ if (bytesRead % (1024 * 1024) == 0) {
+ int available = instream.available();
+ int remaining = (int) (size - bytesRead);
+ assertTrue("expected remaining:" + remaining + ", but got:"
+ + available,
+ remaining == available);
+ LOG.info("Bytes read: " + Math.round(
+ (double) bytesRead / (1024 * 1024))
+ + " MB");
+ }
+
+ byteBuffer.flip();
+ for (int i = 0; i < bytes; i++) {
+ byteBuffer.get();
+ }
+ assertEquals(0, byteBuffer.remaining());
+ }
+ assertEquals(0, instream.available());
+ assertEquals(expectedRound, round);
+ assertEquals(size, bytesRead);
+ IOUtils.closeStream(instream);
+ }
+
+ @Test
+ // 打开文件流,测试read(byte[] buf, final int off, final int len)接口正确性
+ public void testReadFile() throws Exception {
+ final int bufLen = 256;
+ final int sizeFlag = 5;
+ String filename = "readTestFile_" + sizeFlag + ".txt";
+ Path readTestFile = setPath("/test/" + filename);
+ long size = sizeFlag * 1024 * 1024;
+
+ ContractTestUtils.generateTestFile(this.fs, readTestFile, size, 256,
+ 255);
+ LOG.info(sizeFlag + "MB file created: /test/" + filename);
+
+ FSDataInputStream instream = this.fs.open(readTestFile);
+ byte[] buf = new byte[bufLen];
+ long bytesRead = 0;
+ while (bytesRead < size) {
+ int bytes;
+ if (size - bytesRead < bufLen) {
+ int remaining = (int) (size - bytesRead);
+ bytes = instream.read(buf, 0, remaining);
+ } else {
+ bytes = instream.read(buf, 0, bufLen);
+ }
+ bytesRead += bytes;
+
+ if (bytesRead % (1024 * 1024) == 0) {
+ int available = instream.available();
+ int remaining = (int) (size - bytesRead);
+ assertTrue("expected remaining:" + remaining + ", but got:"
+ + available,
+ remaining == available);
+ LOG.info("Bytes read: " + Math.round(
+ (double) bytesRead / (1024 * 1024))
+ + " MB");
+ }
+ }
+ assertTrue(instream.available() == 0);
+ IOUtils.closeStream(instream);
+ }
+
+ @Test
+ // 打开文件输入流,测试read(long position, byte[] buffer, int offset, int length)接口正确性
+ public void testReadByPosistion() throws IOException {
+ Path smallSeekFile = setPath("/test/smallSeekFile.txt");
+ int size = 5 * 1024 * 1024;
+
+ ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256,
+ 255);
+ LOG.info("5MB file created: smallSeekFile.txt");
+
+ long now = System.currentTimeMillis();
+ FSDataInputStream fsDataInputStream = this.fs.open(smallSeekFile);
+ // OBSInputStream in
+ // = (OBSInputStream) fsDataInputStream.getWrappedStream();
+ assertTrue("expected position at:" + 0 + ", but got:"
+ + fsDataInputStream.getPos(),
+ fsDataInputStream.getPos() == 0);
+
+ byte[] testBuffer = new byte[256];
+ genTestBuffer(testBuffer, 256);
+
+ int length = 1024 * 1024;
+ int pos = 0;
+ byte[] randomBytes = new byte[length];
+ int len = fsDataInputStream.read(pos, randomBytes, 0, length);
+ // if (fs.getMetricSwitch()) {
+ // MockMetricsConsumer mmc
+ // = (MockMetricsConsumer) fs.getMetricsConsumer();
+ // assertEquals("read", mmc.getOpName());
+ // assertTrue(mmc.isSuccess());
+ // assertEquals("random", mmc.getOpType());
+ //
+ // assertTrue(length == len);
+ // LOG.warn("random read len: " + len);
+ // }
+ byte[] equalsRandomBuffer = new byte[length];
+ for (int i = 0; i < equalsRandomBuffer.length; i++) {
+ equalsRandomBuffer[i] = testBuffer[(i + pos) % 256];
+ }
+ assertTrue(Arrays.equals(randomBytes, equalsRandomBuffer));
+
+ int overlapSize = 1024;
+ len = fsDataInputStream.read(size - overlapSize, randomBytes, 0, length);
+ assertEquals(overlapSize, len);
+
+ len = fsDataInputStream.read(size, randomBytes, 0, length);
+ assertEquals(-1, len);
+
+ LOG.warn("time consumed mills:" + String.valueOf(
+ System.currentTimeMillis() - now));
+
+ fsDataInputStream.close();
+ }
+
+ @Test
+ // 读取位置超过文件大小时,校验是否返回-1
+ public void testReadPositionExceedFileLength() throws IOException {
+ Path smallSeekFile = setPath("/test/smallSeekFile.txt");
+ int size = 5 * 1024 * 1024;
+
+ ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256,
+ 255);
+ LOG.info("5MB file created: smallSeekFile.txt");
+
+ long now = System.currentTimeMillis();
+ FSDataInputStream fsDataInputStream = this.fs.open(smallSeekFile);
+ // OBSInputStream in
+ // = (OBSInputStream) fsDataInputStream.getWrappedStream();
+ assertTrue("expected position at:" + 0 + ", but got:"
+ + fsDataInputStream.getPos(),
+ fsDataInputStream.getPos() == 0);
+
+ byte[] testBuffer = new byte[256];
+ genTestBuffer(testBuffer, 256);
+
+ int length = 10 * 1024 * 1024;
+ int pos = size;
+ byte[] randomBytes = new byte[length];
+ int len = fsDataInputStream.read(pos, randomBytes, 0, length);
+ assertTrue(len == -1);
+
+ pos = size + 1;
+ len = fsDataInputStream.read(pos, randomBytes, 0, length);
+ assertTrue(len == -1);
+ fsDataInputStream.close();
+
+ }
+
+ @Test
+ // 读取长度超过文件大小时,校验是否成功读到文件末尾
+ public void testReadLengthExceedFileLength() throws IOException {
+ if (inputPolicy.equals(OBSConstants.READAHEAD_POLICY_MEMARTSCC)) {
+ return;
+ /**
+ * 争议用例:hadoop PositionedReadable接口中的
+ * int read(long position, byte[] buffer, int offset, int length)方法
+ * 没有规定:返回值必需要等于min(buffer.size, length),即必需要读满buffer
+ * 而该用例假设:读取5M数据,返回读取成功大小一定等于5M
+ */
+ }
+ Path smallSeekFile = setPath("/test/smallSeekFile.txt");
+ int size = 5 * 1024 * 1024;
+
+ ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256,
+ 255);
+ LOG.info("5MB file created: smallSeekFile.txt");
+
+ long now = System.currentTimeMillis();
+ FSDataInputStream fsDataInputStream = this.fs.open(smallSeekFile);
+ // OBSInputStream in
+ // = (OBSInputStream) fsDataInputStream.getWrappedStream();
+ assertTrue("expected position at:" + 0 + ", but got:"
+ + fsDataInputStream.getPos(),
+ fsDataInputStream.getPos() == 0);
+
+ byte[] testBuffer = new byte[256];
+ genTestBuffer(testBuffer, 256);
+
+ int length = 10 * 1024 * 1024;
+ int pos = 0;
+ byte[] randomBytes = new byte[length];
+ int len = fsDataInputStream.read(pos, randomBytes, 0, length);
+ assertTrue(size == len);
+ fsDataInputStream.close();
+ }
+
+ @Test
+ // 读取长度超过文件大小时,校验是否成功读到文件末尾
+ public void testReadLengthExceedFileLength1() throws IOException {
+ Path smallSeekFile = setPath("/test/smallSeekFile1.txt");
+ int size = 5 * 1024;
+
+ ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256,
+ 255);
+ LOG.info("1MB file created: smallSeekFile1.txt");
+
+ long now = System.currentTimeMillis();
+ FSDataInputStream fsDataInputStream = this.fs.open(smallSeekFile);
+ // OBSInputStream in
+ // = (OBSInputStream) fsDataInputStream.getWrappedStream();
+ assertTrue("expected position at:" + 0 + ", but got:"
+ + fsDataInputStream.getPos(),
+ fsDataInputStream.getPos() == 0);
+
+ byte[] testBuffer = new byte[256];
+ genTestBuffer(testBuffer, 256);
+
+ int length = 1 * 1024 * 1024;
+ int pos = 0;
+ byte[] randomBytes = new byte[length];
+ int len = fsDataInputStream.read(pos, randomBytes, 0, length);
+ assertTrue(size == len);
+ fsDataInputStream.close();
+ }
+
+ @Test
+ // 校验四个参数read转三个参数read开关是否能正确配置生效
+ public void testReadTransformSwitch() throws Exception {
+ final int sizeFlag = 5;
+ String filename = "readTestFile_" + sizeFlag + ".txt";
+ Path readTestFile = setPath("/test/" + filename);
+ long size = sizeFlag * 1024 * 1024;
+
+ ContractTestUtils.generateTestFile(this.fs, readTestFile, size, 256,
+ 255);
+ LOG.info(sizeFlag + "MB file created: /test/" + filename);
+
+ byte[] buffer = new byte[32];
+ FSDataInputStream fsDataInputStream = this.fs.open(readTestFile);
+ FSInputStream obsInputStream
+ = (FSInputStream) fsDataInputStream.getWrappedStream();
+ FSInputStream mockInputStream = Mockito.spy(obsInputStream);
+ System.out.println(mockInputStream);
+ Mockito.doReturn(-100)
+ .when((FSInputStream) mockInputStream)
+ .read(buffer, 0, 10);
+ int readLen = mockInputStream.read(0, buffer, 0, 10);
+ assertTrue(readLen == -100);
+ IOUtils.closeStream(fsDataInputStream);
+ fs.close();
+ fs = null;
+
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.setBoolean(OBSConstants.READAHEAD_TRANSFORM_ENABLE, false);
+ OBSFileSystem obsFs = OBSTestUtils.createTestFileSystem(conf);
+ fsDataInputStream = obsFs.open(readTestFile);
+ obsInputStream = (FSInputStream) fsDataInputStream.getWrappedStream();
+ mockInputStream = Mockito.spy(obsInputStream);
+ Mockito.when(mockInputStream.read(buffer, 0, 10))
+ .then((Answer) invocationOnMock -> -100);
+ readLen = mockInputStream.read(0, buffer, 0, 10);
+ assertTrue(readLen == 10);
+ IOUtils.closeStream(fsDataInputStream);
+ obsFs.close();
+ }
+
+ @Test
+ public void testSeekOutOfRange1() throws IOException {
+ Path smallSeekFile = setPath("/test/seekOutOfRangeFile1.txt");
+ long size = 100;
+
+ ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256,
+ 255);
+ LOG.info("5MB file created: smallSeekFile.txt");
+
+ FSDataInputStream instream = this.fs.open(smallSeekFile);
+ instream.seek(size);
+ }
+
+ @Test(expected = EOFException.class)
+ public void testSeekOutOfRange2() throws IOException {
+ Path smallSeekFile = setPath("/test/seekOutOfRangeFile2.txt");
+ long size = 100;
+
+ ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256,
+ 255);
+ LOG.info("5MB file created: smallSeekFile.txt");
+
+ FSDataInputStream instream = this.fs.open(smallSeekFile);
+ instream.seek(size + 1);
+ }
+
+
+ // @Test
+ // public void testRead() throws Exception {
+ // if (fs.getMetricSwitch()) {
+ // if (!fs.isFsBucket()) {
+ // return;
+ // }
+ // Path testFile = setPath("test_file");
+ // if (fs.exists(testFile)) {
+ // fs.delete(testFile);
+ // }
+ // final int fileSize = 1024;
+ //
+ // FSDataOutputStream outputStream = fs.create(testFile);
+ // byte[] data5 = ContractTestUtils.dataset(fileSize, 'a',
+ // 26);
+ // outputStream.write(data5);
+ // outputStream.hsync();
+ // FSDataInputStream inputStream;
+ // inputStream = fs.open(testFile);
+ // int res = inputStream.read();
+ //
+ // MockMetricsConsumer mmc
+ // = (MockMetricsConsumer) fs.getMetricsConsumer();
+ // assertEquals("read", mmc.getOpName());
+ // assertEquals("1byte", mmc.getOpType());
+ // assertTrue(mmc.isSuccess());
+ // inputStream.close();
+ //
+ // inputStream = fs.open(testFile);
+ // ByteBuffer byteBuffer = ByteBuffer.allocateDirect(fileSize);
+ // int res1 = inputStream.read(byteBuffer);
+ // MockMetricsConsumer mmc1 =
+ // (MockMetricsConsumer) fs.getMetricsConsumer();
+ // assertEquals("read", mmc1.getOpName());
+ // assertEquals("byteBuf", mmc1.getOpType());
+ // assertTrue(mmc1.isSuccess());
+ // inputStream.close();
+ //
+ // inputStream = fs.open(testFile);
+ // byte[] bytes = new byte[fileSize];
+ // int res2 = inputStream.read(bytes, 0, 1);
+ // MockMetricsConsumer mmc2 =
+ // (MockMetricsConsumer) fs.getMetricsConsumer();
+ // assertEquals("read", mmc2.getOpName());
+ // assertEquals("seq", mmc2.getOpType());
+ // assertTrue(mmc2.isSuccess());
+ // inputStream.close();
+ //
+ // inputStream = fs.open(testFile);
+ // byte[] bytes2 = new byte[fileSize];
+ // long position = 2;
+ // int offset = 1;
+ // int length = 4;
+ // inputStream.readFully(position, bytes2, offset, length);
+ // MockMetricsConsumer mmc3 =
+ // (MockMetricsConsumer) fs.getMetricsConsumer();
+ // assertEquals("readFully", mmc3.getOpName());
+ // assertTrue(mmc3.isSuccess());
+ // inputStream.close();
+ //
+ // inputStream = fs.open(testFile);
+ // byte[] bytes3 = new byte[fileSize];
+ // long position1 = 3;
+ // int offset1 = 4;
+ // int length1 = 5;
+ // int readSize = inputStream.read(position1, bytes3, offset1,
+ // length1);
+ // MockMetricsConsumer mmc4 =
+ // (MockMetricsConsumer) fs.getMetricsConsumer();
+ // assertEquals("read", mmc4.getOpName());
+ // assertEquals("random", mmc4.getOpType());
+ // assertTrue(mmc4.isSuccess());
+ // inputStream.close();
+ //
+ // fs.delete(testFile, false);
+ // }
+ // }
+}
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInputStreamSimulator.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInputStreamSimulator.java
new file mode 100644
index 0000000..bf94b99
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInputStreamSimulator.java
@@ -0,0 +1,45 @@
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.fs.obs.memartscc.OBSInputStreamSimulator;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class ITestOBSInputStreamSimulator {
+
+ @Before
+ public void setUp() throws Exception {
+
+ }
+
+ @After
+ public void tearDown() throws Exception {
+
+ }
+
+ @Test
+ public void testRead() throws Exception{
+ long fileStatusLength = 100 * 1024 * 1024;
+ long readAheadRangeValue = 1024 * 1024;
+ OBSInputStreamSimulator obsInputStreamSimulator =
+ new OBSInputStreamSimulator(fileStatusLength, readAheadRangeValue);
+ int willReadBytes;
+ long readBytesFromOBS;
+
+ willReadBytes = 5 *1024 * 1024;
+ readBytesFromOBS = obsInputStreamSimulator.read(willReadBytes);
+ assertEquals("First read", willReadBytes, readBytesFromOBS);
+
+ willReadBytes = 512 * 1024;
+ readBytesFromOBS = obsInputStreamSimulator.read(willReadBytes);
+ assertEquals("Second read", readBytesFromOBS, readAheadRangeValue);
+
+ readBytesFromOBS = obsInputStreamSimulator.read(willReadBytes);
+ assertEquals("Second read", readBytesFromOBS, 0);
+
+ obsInputStreamSimulator.close();
+ }
+}
+
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInvoker.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInvoker.java
new file mode 100644
index 0000000..9c8c744
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInvoker.java
@@ -0,0 +1,244 @@
+package org.apache.hadoop.fs.obs;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import com.obs.services.exception.ObsException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.AccessControlException;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.SocketTimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+@RunWith(MockitoJUnitRunner.class)
+public class ITestOBSInvoker {
+ private static final int COMMON_RETRY_LIMIT = 5;
+
+ private static final int QOS_RETRY_LIMIT = 7;
+
+ private int retryCount;
+
+ private OBSInvoker invoker;
+ private OBSInvoker invoker_fail;
+ private OBSFileSystem fs;
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+
+ @Before
+ public void setup() {
+ retryCount = 0;
+ Configuration conf = new Configuration();
+ conf.setInt(OBSConstants.RETRY_LIMIT, COMMON_RETRY_LIMIT);
+ conf.setInt(OBSConstants.RETRY_QOS_LIMIT, QOS_RETRY_LIMIT);
+
+ conf.setLong(OBSConstants.RETRY_SLEEP_BASETIME, 5);
+ conf.setLong(OBSConstants.RETRY_SLEEP_MAXTIME, 10);
+ conf.setLong(OBSConstants.RETRY_QOS_SLEEP_BASETIME, 5);
+ conf.setLong(OBSConstants.RETRY_QOS_SLEEP_MAXTIME, 10);
+
+ try {
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+
+ OBSRetryPolicy RETRY_POLICY = new OBSRetryPolicy(conf);
+ invoker = new OBSInvoker(fs, RETRY_POLICY, (text, e, retries, idempotent) -> retryCount++);
+
+ conf.setLong(OBSConstants.RETRY_MAXTIME, 10);
+ conf.setLong(OBSConstants.RETRY_QOS_MAXTIME, 10);
+ OBSRetryPolicy RETRY_POLICY1 = new OBSRetryPolicy(conf);
+ invoker_fail = new OBSInvoker(fs, RETRY_POLICY1, (text, e, retries, idempotent) -> retryCount++);
+ }
+
+ @Test
+ public void testNonIOException() {
+ //can not process non IOException
+ }
+
+ @Test
+ public void testAccessControlException() throws Exception {
+ ObsException obsException = new ObsException("test");
+ obsException.setResponseCode(403);
+ try {
+ invoker.retryByMaxTime(OBSOperateAction.write,"test",
+ () -> {
+ throw obsException;
+ }, true);
+ } catch (AccessControlException e) {
+ Assert.assertEquals("AccessControlException",0,retryCount);
+ }
+ }
+
+ @Test
+ public void testFileNotFoundException() throws Exception {
+ ObsException obsException = new ObsException("test");
+ obsException.setResponseCode(404);
+ try {
+ invoker.retryByMaxTime(OBSOperateAction.write,"test",
+ () -> {
+ throw obsException;
+ }, true);
+ } catch (FileNotFoundException e) {
+ Assert.assertEquals("FileNotFoundException",0,retryCount);
+ }
+ }
+
+ @Test
+ public void testOBSFileConflictException() throws Exception {
+ ObsException obsException = new ObsException("test");
+ obsException.setResponseCode(409);
+ try {
+ invoker.retryByMaxTime(OBSOperateAction.write,"test",
+ () -> {
+ throw obsException;
+ }, true);
+ } catch (OBSFileConflictException e) {
+ Assert.assertEquals("OBSFileConflictException",0,retryCount);
+ }
+ }
+
+ @Test
+ public void testOBSIllegalArgumentException() throws Exception {
+ ObsException obsException = new ObsException("test");
+ obsException.setResponseCode(400);
+ try {
+ invoker.retryByMaxTime(OBSOperateAction.write,"test",
+ () -> {
+ throw obsException;
+ }, true);
+ } catch (OBSIllegalArgumentException e) {
+ Assert.assertEquals("OBSIllegalArgumentException",0,retryCount);
+ }
+ }
+
+ @Test(expected = OBSIOException.class)
+ public void testOBSIOException() throws Exception {
+ ObsException obsException = new ObsException("test");
+ obsException.setResponseCode(500);
+ invoker_fail.retryByMaxTime(OBSOperateAction.write,"test",
+ () -> {
+ throw obsException;
+ }, true);
+
+ }
+
+ @Test
+ public void testOBSIOExceptionSuccess() throws Exception {
+ ObsException obsException = new ObsException("test");
+ obsException.setResponseCode(500);
+ final AtomicInteger counter = new AtomicInteger(0);
+ invoker.retryByMaxTime(OBSOperateAction.write,"test",
+ () -> {
+ if (counter.incrementAndGet() < COMMON_RETRY_LIMIT) {
+ throw obsException;
+ }
+ return null;
+ }, true);
+ assertEquals(COMMON_RETRY_LIMIT-1, retryCount);
+ }
+
+ @Test(expected = SocketTimeoutException.class)
+ public void testIOException() throws IOException {
+ ObsException obsException = new ObsException("test");
+ obsException.setResponseCode(500);
+ invoker_fail.retryByMaxTime(OBSOperateAction.write,"test",
+ () -> {
+ throw new SocketTimeoutException("test");
+ }, true);
+ }
+
+ @Test
+ public void testIOExceptionSuccess() throws IOException {
+ final AtomicInteger counter = new AtomicInteger(0);
+ invoker.retryByMaxTime(OBSOperateAction.write,"test",
+ () -> {
+ if (counter.incrementAndGet() < QOS_RETRY_LIMIT) {
+ throw new SocketTimeoutException("test");
+ }
+ return null;
+ }, true);
+ assertEquals( QOS_RETRY_LIMIT-1, retryCount);
+ }
+
+ @Test(expected = OBSQosException.class)
+ public void testOBSQosException() throws Exception {
+ ObsException obsException = new ObsException("test");
+ obsException.setResponseCode(503);
+ obsException.setErrorCode(OBSCommonUtils.DETAIL_QOS_CODE);
+ invoker_fail.retryByMaxTime(OBSOperateAction.write,"test",
+ () -> {
+ throw obsException;
+ }, true);
+
+ }
+
+ @Test
+ public void testOBSQosExceptionSuccess() throws Exception {
+ ObsException obsException = new ObsException("test");
+ obsException.setResponseCode(503);
+ obsException.setErrorCode(OBSCommonUtils.DETAIL_QOS_CODE);
+ final AtomicInteger counter = new AtomicInteger(0);
+ invoker.retryByMaxTime(OBSOperateAction.write,"test",
+ () -> {
+ if (counter.incrementAndGet() < QOS_RETRY_LIMIT) {
+ throw obsException;
+ }
+ return null;
+ }, true);
+ assertEquals(QOS_RETRY_LIMIT-1, retryCount);
+ }
+
+ //translate exception
+ @Test(expected = OBSQosException.class)
+ public void testTranslateOBSException() throws Exception {
+ ObsException obsException = new ObsException("test");
+ obsException.setResponseCode(503);
+ OBSQosException qosException = new OBSQosException("test", obsException);
+ obsException.setErrorCode(OBSCommonUtils.DETAIL_QOS_CODE);
+ invoker_fail.retryByMaxTime(OBSOperateAction.write,"test",
+ () -> {
+ throw qosException;
+ }, true);
+ }
+
+ //translate exception
+ @Test
+ public void testTranslateOBSExceptionSuccess() throws Exception {
+ ObsException obsException = new ObsException("test");
+ obsException.setResponseCode(503);
+ OBSQosException qosException = new OBSQosException("test", obsException);
+ obsException.setErrorCode(OBSCommonUtils.DETAIL_QOS_CODE);
+ final AtomicInteger counter = new AtomicInteger(0);
+ invoker.retryByMaxTime(OBSOperateAction.write,"test",
+ () -> {
+ if (counter.incrementAndGet() < QOS_RETRY_LIMIT) {
+ throw qosException;
+ }
+ return null;
+ }, true);
+ assertEquals(QOS_RETRY_LIMIT-1, retryCount);
+ }
+
+ //nonIdempotent exception
+ @Test(expected = OBSIOException.class)
+ public void testNonIdempotentOBSException() throws Exception {
+ ObsException obsException = new ObsException("test");
+ obsException.setResponseCode(500);
+ invoker.retryByMaxTime(OBSOperateAction.write,"test",
+ () -> {
+ throw obsException;
+ }, false);
+ }
+}
\ No newline at end of file
diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSListFiles.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSListFiles.java
new file mode 100644
index 0000000..e5bb26f
--- /dev/null
+++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSListFiles.java
@@ -0,0 +1,237 @@
+package org.apache.hadoop.fs.obs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.obs.contract.OBSContract;
+import org.apache.hadoop.security.AccessControlException;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class ITestOBSListFiles {
+ private OBSFileSystem fs;
+
+ private static String testRootPath =
+ OBSTestUtils.generateUniqueTestPath();
+
+ private static final String SUB_DIR_PREFIX = "sub_dir-";
+
+ private static final String SUB_DIR_FILE_SURFIX = "sub_file-";
+
+ @Rule
+ public OBSTestRule testRule = new OBSTestRule();
+
+ @BeforeClass
+ public static void skipTestCheck() {
+ Assume.assumeTrue(OBSContract.isContractTestEnabled());
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ Configuration conf = OBSContract.getConfiguration(null);
+ conf.set(OBSConstants.MULTIPART_SIZE,
+ String.valueOf(5 * 1024 * 1024));
+ conf.setClass(OBSConstants.OBS_METRICS_CONSUMER,
+ MockMetricsConsumer.class, BasicMetricsConsumer.class);
+ conf.setBoolean(OBSConstants.METRICS_SWITCH, true);
+ fs = OBSTestUtils.createTestFileSystem(conf);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ if (fs != null) {
+ fs.delete(new Path(testRootPath), true);
+ }
+ }
+
+ private Path getTestPath(String relativePath) {
+ return new Path(testRootPath + "/" + relativePath);
+ }
+
+ @Test
+ // 路径是根目录,结果数组大小大于等于0
+ public void testListFiles001() throws Exception {
+ RemoteIterator