You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by dr...@apache.org on 2016/10/10 21:31:43 UTC

[01/23] hadoop git commit: HADOOP-13483. File create should throw error rather than overwrite directories. Contributed by Genmao Yu.

Repository: hadoop
Updated Branches:
  refs/heads/trunk c874fa914 -> 669d6f13e


HADOOP-13483. File create should throw error rather than overwrite directories. Contributed by Genmao Yu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/bd2d97ad
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bd2d97ad
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bd2d97ad

Branch: refs/heads/trunk
Commit: bd2d97adeea55bf2c7e4ab475bcc90f3a14e751a
Parents: 4d84c81
Author: Mingfei <mi...@intel.com>
Authored: Tue Aug 23 16:18:34 2016 +0800
Committer: Mingfei <mi...@intel.com>
Committed: Wed Sep 7 11:15:47 2016 +0800

----------------------------------------------------------------------
 .../fs/aliyun/oss/AliyunOSSFileSystem.java      | 25 +++++++++++++++++---
 .../oss/contract/TestOSSContractCreate.java     |  6 -----
 2 files changed, 22 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd2d97ad/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
index 6923b95..99a60db 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
@@ -63,6 +63,8 @@ import com.aliyun.oss.model.ObjectMetadata;
 import com.aliyun.oss.model.PartETag;
 import com.aliyun.oss.model.UploadPartCopyRequest;
 import com.aliyun.oss.model.UploadPartCopyResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Implementation of {@link FileSystem} for <a href="https://oss.aliyun.com">
@@ -70,6 +72,8 @@ import com.aliyun.oss.model.UploadPartCopyResult;
  */
 public class AliyunOSSFileSystem extends FileSystem {
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AliyunOSSFileSystem.class);
   private URI uri;
   private Path workingDir;
   private OSSClient ossClient;
@@ -101,9 +105,24 @@ public class AliyunOSSFileSystem extends FileSystem {
       boolean overwrite, int bufferSize, short replication, long blockSize,
       Progressable progress) throws IOException {
     String key = pathToKey(path);
+    FileStatus status = null;
 
-    if (!overwrite && exists(path)) {
-      throw new FileAlreadyExistsException(path + " already exists");
+    try {
+      // get the status or throw a FNFE
+      status = getFileStatus(path);
+
+      // if the thread reaches here, there is something at the path
+      if (status.isDirectory()) {
+        // path references a directory
+        throw new FileAlreadyExistsException(path + " is a directory");
+      }
+      if (!overwrite) {
+        // path references a file and overwrite is disabled
+        throw new FileAlreadyExistsException(path + " already exists");
+      }
+      LOG.debug("Overwriting file {}", path);
+    } catch (FileNotFoundException e) {
+      // this means the file is not found
     }
 
     return new FSDataOutputStream(new AliyunOSSOutputStream(getConf(),
@@ -540,7 +559,7 @@ public class AliyunOSSFileSystem extends FileSystem {
   /**
    * Used to create an empty file that represents an empty directory.
    *
-   * @param bucketName the bucket this directory belongs to
+   * @param bucket the bucket this directory belongs to
    * @param objectName directory path
    * @return true if directory successfully created
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd2d97ad/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractCreate.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractCreate.java
index cc5a2d1..ce927a9 100644
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractCreate.java
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractCreate.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.fs.aliyun.oss.contract;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
 
 /**
  * OSS contract creating tests.
@@ -33,9 +32,4 @@ public class TestOSSContractCreate extends AbstractContractCreateTest {
     return new OSSContract(conf);
   }
 
-  @Override
-  public void testOverwriteEmptyDirectory() throws Throwable {
-    ContractTestUtils.skip(
-        "blobstores can't distinguish empty directories from files");
-  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[15/23] hadoop git commit: HADOOP-13609. Refine credential provider related codes for AliyunOss integration. Contributed by Genmao Yu

Posted by dr...@apache.org.
HADOOP-13609. Refine credential provider related codes for AliyunOss integration. Contributed by Genmao Yu


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9cd47602
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9cd47602
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9cd47602

Branch: refs/heads/trunk
Commit: 9cd47602576cd01a905e27642b685905a88eee72
Parents: 005f452
Author: Kai Zheng <ka...@intel.com>
Authored: Tue Sep 20 11:39:02 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Tue Sep 20 11:39:02 2016 +0800

----------------------------------------------------------------------
 .../aliyun/oss/AliyunCredentialsProvider.java   | 87 ++++++++++++++++++++
 .../fs/aliyun/oss/AliyunOSSFileSystem.java      |  4 +-
 .../fs/aliyun/oss/AliyunOSSFileSystemStore.java | 20 ++++-
 .../fs/aliyun/oss/AliyunOSSInputStream.java     |  4 +-
 .../fs/aliyun/oss/AliyunOSSOutputStream.java    |  4 +-
 .../hadoop/fs/aliyun/oss/AliyunOSSUtils.java    | 14 +---
 .../apache/hadoop/fs/aliyun/oss/Constants.java  |  4 +-
 .../oss/TemporaryAliyunCredentialsProvider.java | 64 --------------
 .../fs/aliyun/oss/TestAliyunCredentials.java    | 78 ++++++++++++++++++
 .../oss/TestAliyunOSSFileSystemStore.java       |  4 +-
 .../oss/TestAliyunOSSTemporaryCredentials.java  | 65 ---------------
 11 files changed, 197 insertions(+), 151 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9cd47602/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunCredentialsProvider.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunCredentialsProvider.java
new file mode 100644
index 0000000..b46c67a
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunCredentialsProvider.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.aliyun.oss;
+
+import com.aliyun.oss.common.auth.Credentials;
+import com.aliyun.oss.common.auth.CredentialsProvider;
+import com.aliyun.oss.common.auth.DefaultCredentials;
+import com.aliyun.oss.common.auth.InvalidCredentialsException;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
+
+/**
+ * Support session credentials for authenticating with Aliyun.
+ */
+public class AliyunCredentialsProvider implements CredentialsProvider {
+  private Credentials credentials = null;
+
+  public AliyunCredentialsProvider(Configuration conf)
+      throws IOException {
+    String accessKeyId;
+    String accessKeySecret;
+    String securityToken;
+    try {
+      accessKeyId = AliyunOSSUtils.getValueWithKey(conf, ACCESS_KEY_ID);
+      accessKeySecret = AliyunOSSUtils.getValueWithKey(conf, ACCESS_KEY_SECRET);
+    } catch (IOException e) {
+      throw new InvalidCredentialsException(e);
+    }
+
+    try {
+      securityToken = AliyunOSSUtils.getValueWithKey(conf, SECURITY_TOKEN);
+    } catch (IOException e) {
+      securityToken = null;
+    }
+
+    if (StringUtils.isEmpty(accessKeyId)
+        || StringUtils.isEmpty(accessKeySecret)) {
+      throw new InvalidCredentialsException(
+          "AccessKeyId and AccessKeySecret should not be null or empty.");
+    }
+
+    if (StringUtils.isNotEmpty(securityToken)) {
+      credentials = new DefaultCredentials(accessKeyId, accessKeySecret,
+          securityToken);
+    } else {
+      credentials = new DefaultCredentials(accessKeyId, accessKeySecret);
+    }
+  }
+
+  @Override
+  public void setCredentials(Credentials creds) {
+    if (creds == null) {
+      throw new InvalidCredentialsException("Credentials should not be null.");
+    }
+
+    credentials = creds;
+  }
+
+  @Override
+  public Credentials getCredentials() {
+    if (credentials == null) {
+      throw new InvalidCredentialsException("Invalid credentials");
+    }
+
+    return credentials;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9cd47602/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
index ad321bd..d40fcd0 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.fs.aliyun.oss;
 
-import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
-
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
@@ -43,6 +41,8 @@ import com.aliyun.oss.model.ObjectMetadata;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
+
 /**
  * Implementation of {@link FileSystem} for <a href="https://oss.aliyun.com">
  * Aliyun OSS</a>, used to access OSS blob system in a filesystem style.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9cd47602/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
index b3cd1bd..d53f930 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
@@ -23,7 +23,25 @@ import com.aliyun.oss.OSSClient;
 import com.aliyun.oss.OSSException;
 import com.aliyun.oss.common.auth.CredentialsProvider;
 import com.aliyun.oss.common.comm.Protocol;
-import com.aliyun.oss.model.*;
+import com.aliyun.oss.model.AbortMultipartUploadRequest;
+import com.aliyun.oss.model.CannedAccessControlList;
+import com.aliyun.oss.model.CompleteMultipartUploadRequest;
+import com.aliyun.oss.model.CompleteMultipartUploadResult;
+import com.aliyun.oss.model.CopyObjectResult;
+import com.aliyun.oss.model.DeleteObjectsRequest;
+import com.aliyun.oss.model.GetObjectRequest;
+import com.aliyun.oss.model.InitiateMultipartUploadRequest;
+import com.aliyun.oss.model.InitiateMultipartUploadResult;
+import com.aliyun.oss.model.ListObjectsRequest;
+import com.aliyun.oss.model.ObjectMetadata;
+import com.aliyun.oss.model.ObjectListing;
+import com.aliyun.oss.model.OSSObjectSummary;
+import com.aliyun.oss.model.PartETag;
+import com.aliyun.oss.model.PutObjectResult;
+import com.aliyun.oss.model.UploadPartCopyRequest;
+import com.aliyun.oss.model.UploadPartCopyResult;
+import com.aliyun.oss.model.UploadPartRequest;
+import com.aliyun.oss.model.UploadPartResult;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9cd47602/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
index 69265fb..b87a3a7 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.fs.aliyun.oss;
 
-import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
-
 import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
@@ -31,6 +29,8 @@ import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 
+import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
+
 /**
  * The input stream for OSS blob system.
  * The class uses multi-part downloading to read data from the object content

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9cd47602/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
index c952d0a..c75ee18 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.fs.aliyun.oss;
 
-import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
-
 import java.io.BufferedOutputStream;
 import java.io.File;
 import java.io.FileOutputStream;
@@ -33,6 +31,8 @@ import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.util.Progressable;
 
+import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
+
 /**
  * The output stream for OSS blob system.
  * Data will be buffered on local disk, then uploaded to OSS in

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9cd47602/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
index b96aea7..fe70d53 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
@@ -23,8 +23,6 @@ import java.io.InputStream;
 import java.net.URI;
 
 import com.aliyun.oss.common.auth.CredentialsProvider;
-import com.aliyun.oss.common.auth.DefaultCredentialProvider;
-import com.aliyun.oss.common.auth.DefaultCredentials;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.ProviderUtils;
@@ -32,7 +30,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
-import static org.apache.hadoop.fs.aliyun.oss.Constants.ALIYUN_OSS_CREDENTIALS_PROVIDER_KEY;
 
 /**
  * Utility methods for Aliyun OSS code.
@@ -52,7 +49,7 @@ final public class AliyunOSSUtils {
    * @return the value for the key
    * @throws IOException if failed to get password from configuration
    */
-  static public String getPassword(Configuration conf, String key)
+  public static String getValueWithKey(Configuration conf, String key)
       throws IOException {
     try {
       final char[] pass = conf.getPassword(key);
@@ -126,12 +123,7 @@ final public class AliyunOSSUtils {
       Configuration newConf =
           ProviderUtils.excludeIncompatibleCredentialProviders(conf,
               AliyunOSSFileSystem.class);
-      String accessKey =
-          AliyunOSSUtils.getPassword(newConf, ACCESS_KEY);
-      String secretKey =
-          AliyunOSSUtils.getPassword(newConf, SECRET_KEY);
-      credentials = new DefaultCredentialProvider(
-          new DefaultCredentials(accessKey, secretKey));
+      credentials = new AliyunCredentialsProvider(newConf);
     } else {
       try {
         LOG.debug("Credential provider class is:" + className);
@@ -139,7 +131,7 @@ final public class AliyunOSSUtils {
         try {
           credentials =
               (CredentialsProvider)credClass.getDeclaredConstructor(
-                  URI.class, Configuration.class).newInstance(uri, conf);
+                  Configuration.class).newInstance(conf);
         } catch (NoSuchMethodException | SecurityException e) {
           credentials =
               (CredentialsProvider)credClass.getDeclaredConstructor()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9cd47602/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
index e0c05ed..04a2ccd 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
@@ -31,8 +31,8 @@ public final class Constants {
       "fs.oss.credentials.provider";
 
   // OSS access verification
-  public static final String ACCESS_KEY = "fs.oss.accessKeyId";
-  public static final String SECRET_KEY = "fs.oss.accessKeySecret";
+  public static final String ACCESS_KEY_ID = "fs.oss.accessKeyId";
+  public static final String ACCESS_KEY_SECRET = "fs.oss.accessKeySecret";
   public static final String SECURITY_TOKEN = "fs.oss.securityToken";
 
   // Number of simultaneous connections to oss

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9cd47602/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/TemporaryAliyunCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/TemporaryAliyunCredentialsProvider.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/TemporaryAliyunCredentialsProvider.java
deleted file mode 100644
index ec8e7fe..0000000
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/TemporaryAliyunCredentialsProvider.java
+++ /dev/null
@@ -1,64 +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.aliyun.oss;
-
-import com.aliyun.oss.common.auth.Credentials;
-import com.aliyun.oss.common.auth.CredentialsProvider;
-import com.aliyun.oss.common.auth.DefaultCredentials;
-import com.aliyun.oss.common.auth.InvalidCredentialsException;
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-
-import java.net.URI;
-
-import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
-
-/**
- * Support session credentials for authenticating with ALiyun.
- */
-public class TemporaryAliyunCredentialsProvider implements CredentialsProvider {
-  public static final String NAME
-      = "org.apache.hadoop.fs.aliyun.oss.TemporaryAliyunCredentialsProvider";
-  private final String accessKeyId;
-  private final String accessKeySecret;
-  private final String securityToken;
-
-  public TemporaryAliyunCredentialsProvider(URI uri, Configuration conf) {
-    this.accessKeyId = conf.get(ACCESS_KEY, null);
-    this.accessKeySecret = conf.get(SECRET_KEY, null);
-    this.securityToken = conf.get(SECURITY_TOKEN, null);
-  }
-
-  @Override
-  public void setCredentials(Credentials creds) {
-
-  }
-
-  @Override
-  public Credentials getCredentials() {
-    if (!StringUtils.isEmpty(accessKeyId)
-        && !StringUtils.isEmpty(accessKeySecret)
-        && !StringUtils.isEmpty(securityToken)) {
-      return new DefaultCredentials(accessKeyId, accessKeySecret,
-          securityToken);
-    }
-    throw new InvalidCredentialsException(
-        "AccessKeyId, AccessKeySecret or SecurityToken is unset");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9cd47602/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunCredentials.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunCredentials.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunCredentials.java
new file mode 100644
index 0000000..e08a4dc
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunCredentials.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.aliyun.oss;
+
+import com.aliyun.oss.common.auth.Credentials;
+import com.aliyun.oss.common.auth.InvalidCredentialsException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.aliyun.oss.contract.AliyunOSSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.aliyun.oss.Constants.ACCESS_KEY_ID;
+import static org.apache.hadoop.fs.aliyun.oss.Constants.ACCESS_KEY_SECRET;
+import static org.apache.hadoop.fs.aliyun.oss.Constants.SECURITY_TOKEN;
+
+/**
+ * Tests use of temporary credentials (for example, Aliyun STS & Aliyun OSS).
+ * This test extends a class that "does things to the root directory", and
+ * should only be used against transient filesystems where you don't care about
+ * the data.
+ */
+public class TestAliyunCredentials extends AbstractFSContractTestBase {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new AliyunOSSContract(conf);
+  }
+
+  @Test
+  public void testCredentialMissingAccessKeyId() throws Throwable {
+    Configuration conf = new Configuration();
+    conf.set(ACCESS_KEY_ID, "");
+    conf.set(ACCESS_KEY_SECRET, "accessKeySecret");
+    conf.set(SECURITY_TOKEN, "token");
+    validateCredential(conf);
+  }
+
+  @Test
+  public void testCredentialMissingAccessKeySecret() throws Throwable {
+    Configuration conf = new Configuration();
+    conf.set(ACCESS_KEY_ID, "accessKeyId");
+    conf.set(ACCESS_KEY_SECRET, "");
+    conf.set(SECURITY_TOKEN, "token");
+    validateCredential(conf);
+  }
+
+  private void validateCredential(Configuration conf) {
+    try {
+      AliyunCredentialsProvider provider
+          = new AliyunCredentialsProvider(conf);
+      Credentials credentials = provider.getCredentials();
+      fail("Expected a CredentialInitializationException, got " + credentials);
+    } catch (InvalidCredentialsException expected) {
+      // expected
+    } catch (IOException e) {
+      fail("Unexpected exception.");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9cd47602/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemStore.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemStore.java
index c849804..dee4ccf 100644
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemStore.java
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemStore.java
@@ -67,8 +67,8 @@ public class TestAliyunOSSFileSystemStore {
   @BeforeClass
   public static void checkSettings() throws Exception {
     Configuration conf = new Configuration();
-    assumeNotNull(conf.get(Constants.ACCESS_KEY));
-    assumeNotNull(conf.get(Constants.SECRET_KEY));
+    assumeNotNull(conf.get(Constants.ACCESS_KEY_ID));
+    assumeNotNull(conf.get(Constants.ACCESS_KEY_SECRET));
     assumeNotNull(conf.get("test.fs.oss.name"));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9cd47602/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSTemporaryCredentials.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSTemporaryCredentials.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSTemporaryCredentials.java
deleted file mode 100644
index e106aae..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSTemporaryCredentials.java
+++ /dev/null
@@ -1,65 +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.aliyun.oss;
-
-import com.aliyun.oss.common.auth.Credentials;
-import com.aliyun.oss.common.auth.InvalidCredentialsException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.aliyun.oss.contract.AliyunOSSContract;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
-import org.junit.Test;
-
-import java.net.URI;
-
-import static org.apache.hadoop.fs.aliyun.oss.Constants.ACCESS_KEY;
-import static org.apache.hadoop.fs.aliyun.oss.Constants.SECRET_KEY;
-import static org.apache.hadoop.fs.aliyun.oss.Constants.SECURITY_TOKEN;
-
-/**
- * Tests use of temporary credentials (for example, Aliyun STS & Aliyun OSS).
- * This test extends a class that "does things to the root directory", and
- * should only be used against transient filesystems where you don't care about
- * the data.
- */
-public class TestAliyunOSSTemporaryCredentials
-    extends AbstractFSContractTestBase {
-
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new AliyunOSSContract(conf);
-  }
-
-  @Test
-  public void testTemporaryCredentialValidation() throws Throwable {
-    Configuration conf = new Configuration();
-    conf.set(ACCESS_KEY, "accessKeyId");
-    conf.set(SECRET_KEY, "accessKeySecret");
-    conf.set(SECURITY_TOKEN, "");
-    URI uri = getFileSystem().getUri();
-    TemporaryAliyunCredentialsProvider provider
-        = new TemporaryAliyunCredentialsProvider(uri, conf);
-    try {
-      Credentials credentials = provider.getCredentials();
-      fail("Expected a CredentialInitializationException, got " + credentials);
-    } catch (InvalidCredentialsException expected) {
-      // expected
-    }
-  }
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[09/23] hadoop git commit: Merge branch 'trunk' into HADOOP-12756

Posted by dr...@apache.org.
Merge branch 'trunk' into HADOOP-12756


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/60f66a93
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/60f66a93
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/60f66a93

Branch: refs/heads/trunk
Commit: 60f66a930626255d60dea7acc828975061ec37de
Parents: e671a0f f0876b8
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Sep 14 16:12:40 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Wed Sep 14 16:12:40 2016 +0800

----------------------------------------------------------------------
 .../dev-support/findbugsExcludeFile.xml         |  10 +-
 .../hadoop-common/src/main/bin/hadoop           |  13 +-
 .../src/main/bin/hadoop-functions.sh            | 144 +++++-
 .../hadoop-common/src/main/conf/hadoop-env.sh   |  31 +-
 .../org/apache/hadoop/conf/ConfServlet.java     |  21 +-
 .../hadoop/crypto/CryptoOutputStream.java       |  22 +-
 .../hadoop/fs/CommonConfigurationKeys.java      |  12 +-
 .../main/java/org/apache/hadoop/fs/Path.java    |  20 +-
 .../fs/crypto/CryptoFSDataOutputStream.java     |   8 +-
 .../main/java/org/apache/hadoop/ipc/Client.java | 178 +++++--
 .../apache/hadoop/ipc/ProtobufRpcEngine.java    |   5 +-
 .../main/java/org/apache/hadoop/ipc/RPC.java    |  15 +-
 .../main/java/org/apache/hadoop/ipc/Server.java |   4 +-
 .../apache/hadoop/security/SaslRpcClient.java   |  25 +-
 .../hadoop/security/UserGroupInformation.java   |  26 +-
 .../apache/hadoop/util/AutoCloseableLock.java   |  36 +-
 .../org/apache/hadoop/util/DataChecksum.java    |   2 +-
 .../hadoop/util/NodeHealthScriptRunner.java     |   3 +-
 .../src/main/resources/core-default.xml         |  17 +-
 .../src/site/markdown/ClusterSetup.md           |  19 +-
 .../src/site/markdown/UnixShellGuide.md         |  34 +-
 .../hadoop/cli/util/ExactLineComparator.java    |  44 ++
 .../org/apache/hadoop/conf/TestConfServlet.java |  24 +
 .../crypto/TestCryptoOutputStreamClosing.java   |  57 +++
 .../fs/TestLocalFileSystemPermission.java       | 149 +++---
 .../java/org/apache/hadoop/fs/TestPath.java     |  20 +
 .../org/apache/hadoop/ipc/RPCCallBenchmark.java |  38 +-
 .../java/org/apache/hadoop/ipc/TestIPC.java     |  86 +++-
 .../hadoop/ipc/TestMultipleProtocolServer.java  | 236 ++++++++-
 .../apache/hadoop/ipc/TestRPCCallBenchmark.java |  13 +
 .../apache/hadoop/ipc/TestRPCCompatibility.java | 242 ++++++++-
 .../apache/hadoop/ipc/TestRPCWaitForProxy.java  |  37 +-
 .../java/org/apache/hadoop/ipc/TestRpcBase.java |  50 +-
 .../java/org/apache/hadoop/ipc/TestSaslRPC.java |  74 +--
 .../hadoop/security/TestDoAsEffectiveUser.java  | 291 +++++++----
 .../security/TestUserGroupInformation.java      |  52 +-
 .../hadoop/util/TestNodeHealthScriptRunner.java |   7 -
 .../hadoop-common/src/test/proto/test.proto     |   4 +-
 .../src/test/proto/test_rpc_service.proto       |   4 +-
 .../test/scripts/hadoop_add_client_opts.bats    |  40 ++
 .../test/scripts/hadoop_subcommand_opts.bats    |  68 +++
 .../scripts/hadoop_subcommand_secure_opts.bats  |  52 ++
 .../src/test/scripts/hadoop_verify_user.bats    |  53 ++
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |  68 ++-
 .../hadoop/hdfs/DFSStripedInputStream.java      |  24 +-
 .../hadoop/hdfs/util/StripedBlockUtil.java      | 177 ++++---
 .../hadoop/hdfs/web/URLConnectionFactory.java   |   1 +
 .../hadoop/fs/http/client/HttpFSFileSystem.java |  21 +-
 .../hadoop/fs/http/server/FSOperations.java     | 186 ++-----
 .../fs/http/client/BaseTestHttpFSWith.java      | 227 +++++----
 .../org/apache/hadoop/test/TestHdfsHelper.java  |  23 +-
 .../src/main/native/libhdfs/hdfs.c              |  10 +-
 .../hadoop-hdfs/src/main/bin/hdfs               |  47 +-
 .../hadoop-hdfs/src/main/bin/hdfs-config.sh     |  28 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   5 +
 .../apache/hadoop/hdfs/InstrumentedLock.java    | 185 +++++++
 .../server/blockmanagement/BlockManager.java    |   4 +-
 .../hdfs/server/datanode/BPServiceActor.java    |   7 +-
 .../hdfs/server/datanode/DiskBalancer.java      |  50 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  16 +-
 .../datanode/fsdataset/impl/FsVolumeList.java   |  18 +-
 .../diskbalancer/DiskBalancerException.java     |   3 +-
 .../diskbalancer/command/CancelCommand.java     |  23 +-
 .../server/diskbalancer/command/Command.java    |  20 +-
 .../diskbalancer/command/ExecuteCommand.java    |  15 +-
 .../diskbalancer/command/HelpCommand.java       |  22 +-
 .../diskbalancer/command/PlanCommand.java       |  63 +--
 .../diskbalancer/command/QueryCommand.java      |  19 +-
 .../diskbalancer/command/ReportCommand.java     | 133 ++---
 .../hdfs/server/namenode/NameNodeRpcServer.java |   3 +
 .../apache/hadoop/hdfs/tools/DiskBalancer.java  | 482 ------------------
 .../hadoop/hdfs/tools/DiskBalancerCLI.java      | 482 ++++++++++++++++++
 .../FileDistributionCalculator.java             |  20 +-
 .../FileDistributionVisitor.java                |  28 +-
 .../offlineImageViewer/OfflineImageViewer.java  | 116 ++---
 .../OfflineImageViewerPB.java                   |  78 +--
 .../src/main/resources/hdfs-default.xml         |  34 +-
 .../src/site/markdown/ArchivalStorage.md        |   2 +-
 .../src/site/markdown/HDFSCommands.md           |   4 +
 .../src/site/markdown/HDFSDiskbalancer.md       |   4 +-
 .../markdown/HDFSHighAvailabilityWithNFS.md     |   4 +-
 .../src/site/markdown/HdfsImageViewer.md        |  13 +-
 .../src/site/markdown/HdfsMultihoming.md        |   4 +-
 .../src/site/markdown/HdfsNfsGateway.md         |   4 +-
 .../hadoop-hdfs/src/site/markdown/ViewFs.md     |   2 +-
 .../hadoop/hdfs/TestDFSStripedInputStream.java  | 121 ++++-
 .../hadoop/hdfs/TestInstrumentedLock.java       | 166 +++++++
 .../TestClientProtocolWithDelegationToken.java  | 119 +++++
 .../fsdataset/impl/TestFsDatasetImpl.java       |  78 +--
 .../server/diskbalancer/TestDiskBalancer.java   | 492 ++++++++++++-------
 .../TestDiskBalancerWithMockMover.java          |  17 +-
 .../command/TestDiskBalancerCommand.java        |  65 ++-
 .../namenode/TestAddStripedBlockInFBR.java      | 109 ++++
 .../TestOfflineImageViewer.java                 |  24 +-
 .../hadoop/hdfs/util/TestStripedBlockUtil.java  |  22 +-
 .../src/test/resources/testAclCLI.xml           |  34 +-
 hadoop-mapreduce-project/bin/mapred             |  25 +-
 hadoop-mapreduce-project/bin/mapred-config.sh   |  10 +-
 hadoop-mapreduce-project/conf/mapred-env.sh     |   4 +-
 .../java/org/apache/hadoop/mapred/MapTask.java  |  40 +-
 .../apache/hadoop/mapreduce/CryptoUtils.java    |  56 ++-
 .../mapreduce/v2/hs/server/HSAdminServer.java   |   3 +
 hadoop-project/pom.xml                          |   7 +
 .../main/shellprofile.d/hadoop-archive-logs.sh  |   2 -
 hadoop-tools/hadoop-aws/pom.xml                 |   4 +
 .../src/site/markdown/tools/hadoop-aws/index.md | 101 +++-
 .../java/org/apache/hadoop/tools/DistCp.java    |   4 +
 .../org/apache/hadoop/tools/DistCpOptions.java  |   8 +-
 .../src/main/shellprofile.d/hadoop-distcp.sh    |   4 -
 .../apache/hadoop/tools/TestOptionsParser.java  |  64 ++-
 .../src/main/shellprofile.d/hadoop-extras.sh    |   2 -
 .../src/main/shellprofile.d/hadoop-rumen.sh     |   4 -
 .../hadoop-sls/src/main/bin/rumen2sls.sh        |   3 +-
 hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh  |   3 +-
 .../src/main/shellprofile.d/hadoop-streaming.sh |   4 -
 hadoop-yarn-project/hadoop-yarn/bin/yarn        |  53 +-
 .../scheduler/fair/FairScheduler.java           |   5 -
 .../scheduler/fair/NewAppWeightBooster.java     |  60 ---
 .../scheduler/fair/WeightAdjuster.java          |  36 --
 119 files changed, 4521 insertions(+), 2189 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/60f66a93/hadoop-project/pom.xml
----------------------------------------------------------------------


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[17/23] hadoop git commit: HADOOP-13624. Rename TestAliyunOSSContractDispCp. Contributed by Genmao Yu

Posted by dr...@apache.org.
HADOOP-13624. Rename TestAliyunOSSContractDispCp. Contributed by Genmao Yu


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/22af6f8d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/22af6f8d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/22af6f8d

Branch: refs/heads/trunk
Commit: 22af6f8db3a44cd51514b4851b99adcfad42751d
Parents: 08b3760
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Sep 21 14:02:44 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Wed Sep 21 14:02:44 2016 +0800

----------------------------------------------------------------------
 .../fs/aliyun/oss/AliyunOSSFileSystemStore.java |  6 ++-
 .../oss/TestAliyunOSSFileSystemStore.java       |  6 ++-
 .../fs/aliyun/oss/TestAliyunOSSInputStream.java |  5 ++-
 .../contract/TestAliyunOSSContractDispCp.java   | 44 --------------------
 .../contract/TestAliyunOSSContractDistCp.java   | 44 ++++++++++++++++++++
 5 files changed, 58 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/22af6f8d/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
index be87fa9..9792a78 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
@@ -49,7 +49,11 @@ import org.apache.hadoop.fs.FileSystem;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.*;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22af6f8d/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemStore.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemStore.java
index dee4ccf..7f4bac2 100644
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemStore.java
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemStore.java
@@ -25,7 +25,11 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.io.*;
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.net.URI;
 import java.security.DigestInputStream;
 import java.security.DigestOutputStream;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22af6f8d/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java
index 892eda0..37af28f 100644
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java
@@ -24,7 +24,10 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.io.IOUtils;
-import org.junit.*;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
 import org.junit.rules.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22af6f8d/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDispCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDispCp.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDispCp.java
deleted file mode 100644
index 4b482fc..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDispCp.java
+++ /dev/null
@@ -1,44 +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.aliyun.oss.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
-
-import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
-
-/**
- * Contract test suite covering Aliyun OSS integration with DistCp.
- */
-public class TestAliyunOSSContractDispCp extends AbstractContractDistCpTest {
-
-  private static final long MULTIPART_SETTING = 8 * 1024 * 1024; // 8 MB
-
-  @Override
-  protected Configuration createConfiguration() {
-    Configuration newConf = super.createConfiguration();
-    newConf.setLong(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY, MULTIPART_SETTING);
-    newConf.setLong(MULTIPART_UPLOAD_SIZE_KEY, MULTIPART_SETTING);
-    return newConf;
-  }
-
-  @Override
-  protected AliyunOSSContract createContract(Configuration conf) {
-    return new AliyunOSSContract(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22af6f8d/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDistCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDistCp.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDistCp.java
new file mode 100644
index 0000000..18d09d5
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDistCp.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
+
+import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
+
+/**
+ * Contract test suite covering Aliyun OSS integration with DistCp.
+ */
+public class TestAliyunOSSContractDistCp extends AbstractContractDistCpTest {
+
+  private static final long MULTIPART_SETTING = 8 * 1024 * 1024; // 8 MB
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration newConf = super.createConfiguration();
+    newConf.setLong(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY, MULTIPART_SETTING);
+    newConf.setLong(MULTIPART_UPLOAD_SIZE_KEY, MULTIPART_SETTING);
+    return newConf;
+  }
+
+  @Override
+  protected AliyunOSSContract createContract(Configuration conf) {
+    return new AliyunOSSContract(conf);
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[16/23] hadoop git commit: HADOOP-13591. Unit test failure in TestOSSContractGetFileStatus and TestOSSContractRootDir. Contributed by Genmao Yu

Posted by dr...@apache.org.
HADOOP-13591. Unit test failure in TestOSSContractGetFileStatus and TestOSSContractRootDir. Contributed by Genmao Yu


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/08b37603
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/08b37603
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/08b37603

Branch: refs/heads/trunk
Commit: 08b37603d9c0be67c4e0790c1ad266551ef21f5e
Parents: 9cd4760
Author: Kai Zheng <ka...@intel.com>
Authored: Tue Sep 20 15:12:02 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Tue Sep 20 15:12:02 2016 +0800

----------------------------------------------------------------------
 .../fs/aliyun/oss/AliyunOSSFileSystem.java      | 88 +++++++++-----------
 .../fs/aliyun/oss/AliyunOSSFileSystemStore.java | 36 ++++----
 .../hadoop/fs/aliyun/oss/AliyunOSSUtils.java    | 24 ++++--
 .../fs/aliyun/oss/AliyunOSSTestUtils.java       |  2 +-
 .../src/test/resources/contract/aliyun-oss.xml  | 10 +++
 5 files changed, 89 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/08b37603/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
index d40fcd0..81e038d 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
@@ -24,6 +24,8 @@ import java.net.URI;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -124,9 +126,6 @@ public class AliyunOSSFileSystem extends FileSystem {
     Path f = status.getPath();
     String key = pathToKey(f);
     if (status.isDirectory()) {
-      if (!key.endsWith("/")) {
-        key += "/";
-      }
       if (!recursive) {
         FileStatus[] statuses = listStatus(status.getPath());
         // Check whether it is an empty directory or not
@@ -135,6 +134,7 @@ public class AliyunOSSFileSystem extends FileSystem {
               ": It is not empty!");
         } else {
           // Delete empty directory without '-r'
+          key = AliyunOSSUtils.maybeAddTrailingSlash(key);
           store.deleteObject(key);
         }
       } else {
@@ -149,15 +149,9 @@ public class AliyunOSSFileSystem extends FileSystem {
   }
 
   private void createFakeDirectoryIfNecessary(Path f) throws IOException {
-    try {
-      Path pPath = f.getParent();
-      FileStatus pStatus = getFileStatus(pPath);
-      if (pStatus.isFile()) {
-        throw new IOException("Path " + pPath +
-            " is assumed to be a directory!");
-      }
-    } catch (FileNotFoundException fnfe) {
-      // Make sure the parent directory exists
+    String key = pathToKey(f);
+    if (StringUtils.isNotEmpty(key) && !exists(f)) {
+      LOG.debug("Creating new fake directory at {}", f);
       mkdir(pathToKey(f.getParent()));
     }
   }
@@ -175,14 +169,14 @@ public class AliyunOSSFileSystem extends FileSystem {
     ObjectMetadata meta = store.getObjectMetadata(key);
     // If key not found and key does not end with "/"
     if (meta == null && !key.endsWith("/")) {
-      // Case: dir + "/"
+      // In case of 'dir + "/"'
       key += "/";
       meta = store.getObjectMetadata(key);
     }
     if (meta == null) {
-      ObjectListing listing = store.listObjects(key, 1, "/", null);
-      if (!listing.getObjectSummaries().isEmpty() ||
-          !listing.getCommonPrefixes().isEmpty()) {
+      ObjectListing listing = store.listObjects(key, 1, null, false);
+      if (CollectionUtils.isNotEmpty(listing.getObjectSummaries()) ||
+          CollectionUtils.isNotEmpty(listing.getCommonPrefixes())) {
         return new FileStatus(0, true, 1, 0, 0, qualifiedPath);
       } else {
         throw new FileNotFoundException(path + ": No such file or directory!");
@@ -251,7 +245,7 @@ public class AliyunOSSFileSystem extends FileSystem {
    */
   private boolean objectRepresentsDirectory(final String name,
       final long size) {
-    return !name.isEmpty() && name.endsWith("/") && size == 0L;
+    return StringUtils.isNotEmpty(name) && name.endsWith("/") && size == 0L;
   }
 
   /**
@@ -265,10 +259,6 @@ public class AliyunOSSFileSystem extends FileSystem {
       path = new Path(workingDir, path);
     }
 
-    if (path.toUri().getScheme() != null && path.toUri().getPath().isEmpty()) {
-      return "";
-    }
-
     return path.toUri().getPath().substring(1);
   }
 
@@ -287,26 +277,23 @@ public class AliyunOSSFileSystem extends FileSystem {
     final FileStatus fileStatus = getFileStatus(path);
 
     if (fileStatus.isDirectory()) {
-      if (!key.endsWith("/")) {
-        key = key + "/";
-      }
-
       if (LOG.isDebugEnabled()) {
         LOG.debug("listStatus: doing listObjects for directory " + key);
       }
 
-      ObjectListing objects = store.listObjects(key, maxKeys, "/", null);
+      ObjectListing objects = store.listObjects(key, maxKeys, null, false);
       while (true) {
         statistics.incrementReadOps(1);
         for (OSSObjectSummary objectSummary : objects.getObjectSummaries()) {
-          Path keyPath = keyToPath(objectSummary.getKey())
-              .makeQualified(uri, workingDir);
-          if (keyPath.equals(path)) {
+          String objKey = objectSummary.getKey();
+          if (objKey.equals(key + "/")) {
             if (LOG.isDebugEnabled()) {
-              LOG.debug("Ignoring: " + keyPath);
+              LOG.debug("Ignoring: " + objKey);
             }
             continue;
           } else {
+            Path keyPath = keyToPath(objectSummary.getKey())
+                .makeQualified(uri, workingDir);
             if (LOG.isDebugEnabled()) {
               LOG.debug("Adding: fi: " + keyPath);
             }
@@ -317,10 +304,13 @@ public class AliyunOSSFileSystem extends FileSystem {
         }
 
         for (String prefix : objects.getCommonPrefixes()) {
-          Path keyPath = keyToPath(prefix).makeQualified(uri, workingDir);
-          if (keyPath.equals(path)) {
+          if (prefix.equals(key + "/")) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Ignoring: " + prefix);
+            }
             continue;
           } else {
+            Path keyPath = keyToPath(prefix).makeQualified(uri, workingDir);
             if (LOG.isDebugEnabled()) {
               LOG.debug("Adding: rd: " + keyPath);
             }
@@ -332,8 +322,8 @@ public class AliyunOSSFileSystem extends FileSystem {
           if (LOG.isDebugEnabled()) {
             LOG.debug("listStatus: list truncated - getting next batch");
           }
-          objects = store.listObjects(key, maxKeys, "/",
-              objects.getNextMarker());
+          String nextMarker = objects.getNextMarker();
+          objects = store.listObjects(key, maxKeys, nextMarker, false);
           statistics.incrementReadOps(1);
         } else {
           break;
@@ -358,10 +348,12 @@ public class AliyunOSSFileSystem extends FileSystem {
    */
   private boolean mkdir(final String key) throws IOException {
     String dirName = key;
-    if (!key.endsWith("/")) {
-      dirName += "/";
+    if (StringUtils.isNotEmpty(key)) {
+      if (!key.endsWith("/")) {
+        dirName += "/";
+      }
+      store.storeEmptyFile(dirName);
     }
-    store.storeEmptyFile(dirName);
     return true;
   }
 
@@ -506,16 +498,11 @@ public class AliyunOSSFileSystem extends FileSystem {
    * @param dstPath destination path.
    * @return true if directory is successfully copied.
    */
-  private boolean copyDirectory(Path srcPath, Path dstPath) {
-    String srcKey = pathToKey(srcPath);
-    String dstKey = pathToKey(dstPath);
-
-    if (!srcKey.endsWith("/")) {
-      srcKey = srcKey + "/";
-    }
-    if (!dstKey.endsWith("/")) {
-      dstKey = dstKey + "/";
-    }
+  private boolean copyDirectory(Path srcPath, Path dstPath) throws IOException {
+    String srcKey = AliyunOSSUtils
+        .maybeAddTrailingSlash(pathToKey(srcPath));
+    String dstKey = AliyunOSSUtils
+        .maybeAddTrailingSlash(pathToKey(dstPath));
 
     if (dstKey.startsWith(srcKey)) {
       if (LOG.isDebugEnabled()) {
@@ -524,7 +511,8 @@ public class AliyunOSSFileSystem extends FileSystem {
       return false;
     }
 
-    ObjectListing objects = store.listObjects(srcKey, maxKeys, null, null);
+    store.storeEmptyFile(dstKey);
+    ObjectListing objects = store.listObjects(srcKey, maxKeys, null, true);
     statistics.incrementReadOps(1);
     // Copy files from src folder to dst
     while (true) {
@@ -534,8 +522,8 @@ public class AliyunOSSFileSystem extends FileSystem {
         store.copyFile(objectSummary.getKey(), newKey);
       }
       if (objects.isTruncated()) {
-        objects = store.listObjects(srcKey, maxKeys, null,
-            objects.getNextMarker());
+        String nextMarker = objects.getNextMarker();
+        objects = store.listObjects(srcKey, maxKeys, nextMarker, true);
         statistics.incrementReadOps(1);
       } else {
         break;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08b37603/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
index d53f930..be87fa9 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
@@ -42,6 +42,8 @@ import com.aliyun.oss.model.UploadPartCopyRequest;
 import com.aliyun.oss.model.UploadPartCopyResult;
 import com.aliyun.oss.model.UploadPartRequest;
 import com.aliyun.oss.model.UploadPartResult;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.slf4j.Logger;
@@ -89,7 +91,7 @@ public class AliyunOSSFileSystemStore {
 
     String proxyHost = conf.getTrimmed(PROXY_HOST_KEY, "");
     int proxyPort = conf.getInt(PROXY_PORT_KEY, -1);
-    if (!proxyHost.isEmpty()) {
+    if (StringUtils.isNotEmpty(proxyHost)) {
       clientConf.setProxyHost(proxyHost);
       if (proxyPort >= 0) {
         clientConf.setProxyPort(proxyPort);
@@ -123,7 +125,7 @@ public class AliyunOSSFileSystemStore {
 
     String endPoint = conf.getTrimmed(ENDPOINT_KEY, "");
     CredentialsProvider provider =
-        AliyunOSSUtils.getCredentialsProvider(uri, conf);
+        AliyunOSSUtils.getCredentialsProvider(conf);
     ossClient = new OSSClient(endPoint, provider, clientConf);
     uploadPartSize = conf.getLong(MULTIPART_UPLOAD_SIZE_KEY,
         MULTIPART_UPLOAD_SIZE_DEFAULT);
@@ -153,7 +155,7 @@ public class AliyunOSSFileSystemStore {
     }
 
     String cannedACLName = conf.get(CANNED_ACL_KEY, CANNED_ACL_DEFAULT);
-    if (!cannedACLName.isEmpty()) {
+    if (StringUtils.isNotEmpty(cannedACLName)) {
       CannedAccessControlList cannedACL =
           CannedAccessControlList.valueOf(cannedACLName);
       ossClient.setBucketAcl(bucketName, cannedACL);
@@ -179,11 +181,13 @@ public class AliyunOSSFileSystemStore {
    * @param keysToDelete collection of keys to delete.
    */
   public void deleteObjects(List<String> keysToDelete) {
-    DeleteObjectsRequest deleteRequest =
-        new DeleteObjectsRequest(bucketName);
-    deleteRequest.setKeys(keysToDelete);
-    ossClient.deleteObjects(deleteRequest);
-    statistics.incrementWriteOps(keysToDelete.size());
+    if (CollectionUtils.isNotEmpty(keysToDelete)) {
+      DeleteObjectsRequest deleteRequest =
+          new DeleteObjectsRequest(bucketName);
+      deleteRequest.setKeys(keysToDelete);
+      ossClient.deleteObjects(deleteRequest);
+      statistics.incrementWriteOps(keysToDelete.size());
+    }
   }
 
   /**
@@ -192,8 +196,10 @@ public class AliyunOSSFileSystemStore {
    * @param key directory key to delete.
    */
   public void deleteDirs(String key) {
+    key = AliyunOSSUtils.maybeAddTrailingSlash(key);
     ListObjectsRequest listRequest = new ListObjectsRequest(bucketName);
     listRequest.setPrefix(key);
+    listRequest.setDelimiter(null);
     listRequest.setMaxKeys(maxKeys);
 
     while (true) {
@@ -299,7 +305,7 @@ public class AliyunOSSFileSystemStore {
     InitiateMultipartUploadRequest initiateMultipartUploadRequest =
         new InitiateMultipartUploadRequest(bucketName, dstKey);
     ObjectMetadata meta = new ObjectMetadata();
-    if (!serverSideEncryptionAlgorithm.isEmpty()) {
+    if (StringUtils.isNotEmpty(serverSideEncryptionAlgorithm)) {
       meta.setServerSideEncryption(serverSideEncryptionAlgorithm);
     }
     initiateMultipartUploadRequest.setObjectMetadata(meta);
@@ -353,7 +359,7 @@ public class AliyunOSSFileSystemStore {
     FileInputStream fis = new FileInputStream(object);
     ObjectMetadata meta = new ObjectMetadata();
     meta.setContentLength(object.length());
-    if (!serverSideEncryptionAlgorithm.isEmpty()) {
+    if (StringUtils.isNotEmpty(serverSideEncryptionAlgorithm)) {
       meta.setServerSideEncryption(serverSideEncryptionAlgorithm);
     }
     try {
@@ -384,7 +390,7 @@ public class AliyunOSSFileSystemStore {
     InitiateMultipartUploadRequest initiateMultipartUploadRequest =
         new InitiateMultipartUploadRequest(bucketName, key);
     ObjectMetadata meta = new ObjectMetadata();
-    if (!serverSideEncryptionAlgorithm.isEmpty()) {
+    if (StringUtils.isNotEmpty(serverSideEncryptionAlgorithm)) {
       meta.setServerSideEncryption(serverSideEncryptionAlgorithm);
     }
     initiateMultipartUploadRequest.setObjectMetadata(meta);
@@ -435,12 +441,14 @@ public class AliyunOSSFileSystemStore {
    *
    * @param prefix prefix.
    * @param maxListingLength max no. of entries
-   * @param delimiter delimiter.
    * @param marker last key in any previous search.
+   * @param recursive whether to list directory recursively.
    * @return a list of matches.
    */
   public ObjectListing listObjects(String prefix, int maxListingLength,
-                                   String delimiter, String marker) {
+                                   String marker, boolean recursive) {
+    String delimiter = recursive ? null : "/";
+    prefix = AliyunOSSUtils.maybeAddTrailingSlash(prefix);
     ListObjectsRequest listRequest = new ListObjectsRequest(bucketName);
     listRequest.setPrefix(prefix);
     listRequest.setDelimiter(delimiter);
@@ -488,7 +496,7 @@ public class AliyunOSSFileSystemStore {
   public void purge(String prefix) {
     String key;
     try {
-      ObjectListing objects = listObjects(prefix, maxKeys, null, null);
+      ObjectListing objects = listObjects(prefix, maxKeys, null, true);
       for (OSSObjectSummary object : objects.getObjectSummaries()) {
         key = object.getKey();
         ossClient.deleteObject(bucketName, key);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08b37603/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
index fe70d53..cae9749 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.fs.aliyun.oss;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.net.URI;
 
 import com.aliyun.oss.common.auth.CredentialsProvider;
 import org.apache.commons.lang.StringUtils;
@@ -106,16 +105,13 @@ final public class AliyunOSSUtils {
    * Create credential provider specified by configuration, or create default
    * credential provider if not specified.
    *
-   * @param name the uri of the file system
    * @param conf configuration
    * @return a credential provider
    * @throws IOException on any problem. Class construction issues may be
    * nested inside the IOE.
    */
-  public static CredentialsProvider getCredentialsProvider(URI name,
-      Configuration conf) throws IOException {
-    URI uri = java.net.URI.create(
-        name.getScheme() + "://" + name.getAuthority());
+  public static CredentialsProvider getCredentialsProvider(Configuration conf)
+      throws IOException {
     CredentialsProvider credentials;
 
     String className = conf.getTrimmed(ALIYUN_OSS_CREDENTIALS_PROVIDER_KEY);
@@ -152,4 +148,20 @@ final public class AliyunOSSUtils {
 
     return credentials;
   }
+
+  /**
+   * Turns a path (relative or otherwise) into an OSS key, adding a trailing
+   * "/" if the path is not the root <i>and</i> does not already have a "/"
+   * at the end.
+   *
+   * @param key OSS key or ""
+   * @return the with a trailing "/", or, if it is the root key, "".
+   */
+  public static String maybeAddTrailingSlash(String key) {
+    if (StringUtils.isNotEmpty(key) && !key.endsWith("/")) {
+      return key + '/';
+    } else {
+      return key;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08b37603/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSTestUtils.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSTestUtils.java
index 84dba6a..901cb2b 100644
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSTestUtils.java
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSTestUtils.java
@@ -48,7 +48,7 @@ public final class AliyunOSSTestUtils {
     String fsname = conf.getTrimmed(
         TestAliyunOSSFileSystemContract.TEST_FS_OSS_NAME, "");
 
-    boolean liveTest = !StringUtils.isEmpty(fsname);
+    boolean liveTest = StringUtils.isNotEmpty(fsname);
     URI testURI = null;
     if (liveTest) {
       testURI = URI.create(fsname);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08b37603/hadoop-tools/hadoop-aliyun/src/test/resources/contract/aliyun-oss.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/resources/contract/aliyun-oss.xml b/hadoop-tools/hadoop-aliyun/src/test/resources/contract/aliyun-oss.xml
index 2bc34b7..7bbbf46 100644
--- a/hadoop-tools/hadoop-aliyun/src/test/resources/contract/aliyun-oss.xml
+++ b/hadoop-tools/hadoop-aliyun/src/test/resources/contract/aliyun-oss.xml
@@ -99,6 +99,16 @@
     </property>
 
     <property>
+        <name>fs.contract.test.root-tests-enabled</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-getfilestatus</name>
+        <value>true</value>
+    </property>
+
+    <property>
         <name>fs.oss.multipart.download.size</name>
         <value>102400</value>
     </property>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[06/23] hadoop git commit: HADOOP-13498. The number of multi-part upload part should not bigger than 10000. Contributed by Genmao Yu.

Posted by dr...@apache.org.
HADOOP-13498. The number of multi-part upload part should not bigger than 10000. Contributed by Genmao Yu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/cdb77110
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cdb77110
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cdb77110

Branch: refs/heads/trunk
Commit: cdb77110e77b70ed0c1125b2a6a422a8c7c28ec7
Parents: 6bb741b
Author: Mingfei <mi...@intel.com>
Authored: Wed Aug 24 10:09:37 2016 +0800
Committer: Mingfei <mi...@intel.com>
Committed: Wed Sep 7 11:15:48 2016 +0800

----------------------------------------------------------------------
 .../fs/aliyun/oss/AliyunOSSFileSystem.java      |  9 ++++----
 .../fs/aliyun/oss/AliyunOSSOutputStream.java    | 23 ++++++++++----------
 .../hadoop/fs/aliyun/oss/AliyunOSSUtils.java    | 15 +++++++++++++
 .../apache/hadoop/fs/aliyun/oss/Constants.java  |  4 +++-
 .../fs/aliyun/oss/TestOSSOutputStream.java      | 19 ++++++++++++++++
 5 files changed, 53 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdb77110/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
index 99a60db..afe7242 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
@@ -71,7 +71,6 @@ import org.slf4j.LoggerFactory;
  * Aliyun OSS</a>, used to access OSS blob system in a filesystem style.
  */
 public class AliyunOSSFileSystem extends FileSystem {
-
   private static final Logger LOG =
       LoggerFactory.getLogger(AliyunOSSFileSystem.class);
   private URI uri;
@@ -560,18 +559,18 @@ public class AliyunOSSFileSystem extends FileSystem {
    * Used to create an empty file that represents an empty directory.
    *
    * @param bucket the bucket this directory belongs to
-   * @param objectName directory path
+   * @param key directory path
    * @return true if directory successfully created
    * @throws IOException
    */
-  private boolean mkdir(final String bucket, final String objectName)
+  private boolean mkdir(final String bucket, final String key)
       throws IOException {
-    String dirName = objectName;
+    String dirName = key;
     ObjectMetadata dirMeta = new ObjectMetadata();
     byte[] buffer = new byte[0];
     ByteArrayInputStream in = new ByteArrayInputStream(buffer);
     dirMeta.setContentLength(0);
-    if (!objectName.endsWith("/")) {
+    if (!key.endsWith("/")) {
       dirName += "/";
     }
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdb77110/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
index 654b81d..1e16df9 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
@@ -84,6 +84,9 @@ public class AliyunOSSOutputStream extends OutputStream {
 
     partSize = conf.getLong(MULTIPART_UPLOAD_SIZE_KEY,
         MULTIPART_UPLOAD_SIZE_DEFAULT);
+    if (partSize < MIN_MULTIPART_UPLOAD_PART_SIZE) {
+      partSize = MIN_MULTIPART_UPLOAD_PART_SIZE;
+    }
     partSizeThreshold = conf.getLong(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY,
         MIN_MULTIPART_UPLOAD_THRESHOLD_DEFAULT);
 
@@ -151,6 +154,12 @@ public class AliyunOSSOutputStream extends OutputStream {
   private void multipartUploadObject() throws IOException {
     File object = tmpFile.getAbsoluteFile();
     long dataLen = object.length();
+    long realPartSize = AliyunOSSUtils.calculatePartSize(dataLen, partSize);
+    int partNum = (int)(dataLen / realPartSize);
+    if (dataLen % realPartSize != 0) {
+      partNum += 1;
+    }
+
     InitiateMultipartUploadRequest initiateMultipartUploadRequest =
         new InitiateMultipartUploadRequest(bucketName, key);
     ObjectMetadata meta = new ObjectMetadata();
@@ -161,14 +170,6 @@ public class AliyunOSSOutputStream extends OutputStream {
     initiateMultipartUploadRequest.setObjectMetadata(meta);
     InitiateMultipartUploadResult initiateMultipartUploadResult =
         ossClient.initiateMultipartUpload(initiateMultipartUploadRequest);
-    int partNum = (int)(dataLen / partSize);
-    if (dataLen % partSize != 0) {
-      partNum += 1;
-    }
-    if (partNum > MULTIPART_UPLOAD_PART_NUM_LIMIT) {
-      throw new IOException("Number of parts " + partNum + " should not be " +
-          "bigger than limit " + MULTIPART_UPLOAD_PART_NUM_LIMIT);
-    }
     List<PartETag> partETags = new ArrayList<PartETag>();
     String uploadId = initiateMultipartUploadResult.getUploadId();
 
@@ -177,10 +178,10 @@ public class AliyunOSSOutputStream extends OutputStream {
         // TODO: Optimize this, avoid opening the object multiple times
         FileInputStream fis = new FileInputStream(object);
         try {
-          long skipBytes = partSize * i;
+          long skipBytes = realPartSize * i;
           AliyunOSSUtils.skipFully(fis, skipBytes);
-          long size = (partSize < dataLen - skipBytes) ?
-              partSize : dataLen - skipBytes;
+          long size = (realPartSize < dataLen - skipBytes) ?
+              realPartSize : dataLen - skipBytes;
           UploadPartRequest uploadPartRequest = new UploadPartRequest();
           uploadPartRequest.setBucketName(bucketName);
           uploadPartRequest.setKey(key);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdb77110/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
index 9acde00..d54dd9c 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
@@ -28,6 +28,8 @@ import java.util.Objects;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 
+import static org.apache.hadoop.fs.aliyun.oss.Constants.MULTIPART_UPLOAD_PART_NUM_LIMIT;
+
 /**
  * Utility methods for Aliyun OSS code.
  */
@@ -172,4 +174,17 @@ final public class AliyunOSSUtils {
               "to EOF.");
     }
   }
+
+  /**
+   * Calculate a proper size of multipart piece. If <code>minPartSize</code>
+   * is too small, the number of multipart pieces may exceed the limit of
+   * {@link Constants#MULTIPART_UPLOAD_PART_NUM_LIMIT}.
+   * @param contentLength the size of file.
+   * @param minPartSize the minimum size of multipart piece.
+   * @return a revisional size of multipart piece.
+     */
+  public static long calculatePartSize(long contentLength, long minPartSize) {
+    long tmpPartSize = contentLength / MULTIPART_UPLOAD_PART_NUM_LIMIT + 1;
+    return Math.max(minPartSize, tmpPartSize);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdb77110/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
index 0bc6d57..9902275 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
@@ -79,7 +79,7 @@ public final class Constants {
       "fs.oss.multipart.upload.size";
 
   public static final long MULTIPART_UPLOAD_SIZE_DEFAULT = 10 * 1024 * 1024;
-  public static final int MULTIPART_UPLOAD_PART_NUM_LIMIT = 1000;
+  public static final int MULTIPART_UPLOAD_PART_NUM_LIMIT = 10000;
 
   // Minimum size in bytes before we start a multipart uploads or copy
   public static final String MIN_MULTIPART_UPLOAD_THRESHOLD_KEY =
@@ -108,4 +108,6 @@ public final class Constants {
   public static final int FS_OSS_BLOCK_SIZE_DEFAULT = 64 * 1024 * 1024;
   public static final String FS_OSS = "oss";
 
+  public static final long MIN_MULTIPART_UPLOAD_PART_SIZE = 100 * 1024L;
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdb77110/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSOutputStream.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSOutputStream.java
index 3951529..b33ab99 100644
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSOutputStream.java
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSOutputStream.java
@@ -68,4 +68,23 @@ public class TestOSSOutputStream {
   public void testMultiPartUpload() throws IOException {
     ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 6 * 1024 * 1024);
   }
+
+  @Test
+  public void testMultiPartUploadLimit() throws IOException {
+    long partSize1 = AliyunOSSUtils.calculatePartSize(10 * 1024, 100 * 1024);
+    assert(10 * 1024 / partSize1 < Constants.MULTIPART_UPLOAD_PART_NUM_LIMIT);
+
+    long partSize2 = AliyunOSSUtils.calculatePartSize(200 * 1024, 100 * 1024);
+    assert(200 * 1024 / partSize2 < Constants.MULTIPART_UPLOAD_PART_NUM_LIMIT);
+
+    long partSize3 = AliyunOSSUtils.calculatePartSize(10000 * 100 * 1024,
+        100 * 1024);
+    assert(10000 * 100 * 1024 / partSize3
+        < Constants.MULTIPART_UPLOAD_PART_NUM_LIMIT);
+
+    long partSize4 = AliyunOSSUtils.calculatePartSize(10001 * 100 * 1024,
+        100 * 1024);
+    assert(10001 * 100 * 1024 / partSize4
+        < Constants.MULTIPART_UPLOAD_PART_NUM_LIMIT);
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[02/23] hadoop git commit: HADOOP-13491. Fix several warnings from findbugs. Contributed by Genmao Yu.

Posted by dr...@apache.org.
HADOOP-13491. Fix several warnings from findbugs. Contributed by Genmao Yu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/4d84c814
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4d84c814
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4d84c814

Branch: refs/heads/trunk
Commit: 4d84c814fcaf074022593c057d8f8dec4cd461fa
Parents: a5d5342
Author: Mingfei <mi...@intel.com>
Authored: Thu Aug 18 16:06:57 2016 +0800
Committer: Mingfei <mi...@intel.com>
Committed: Wed Sep 7 11:15:47 2016 +0800

----------------------------------------------------------------------
 .../fs/aliyun/oss/AliyunOSSFileSystem.java      |  3 ++-
 .../fs/aliyun/oss/AliyunOSSInputStream.java     |  6 ++---
 .../fs/aliyun/oss/AliyunOSSOutputStream.java    | 10 +++++---
 .../hadoop/fs/aliyun/oss/AliyunOSSUtils.java    | 26 +++++++++++++++++++-
 4 files changed, 37 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d84c814/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
index 30ddf8c..6923b95 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.aliyun.oss;
 
 import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
 
+import com.aliyun.oss.ClientException;
 import com.aliyun.oss.common.auth.CredentialsProvider;
 import com.aliyun.oss.common.auth.DefaultCredentialProvider;
 import com.aliyun.oss.common.auth.DefaultCredentials;
@@ -782,7 +783,7 @@ public class AliyunOSSFileSystem extends FileSystem {
           ossClient.completeMultipartUpload(completeMultipartUploadRequest);
       LOG.debug(completeMultipartUploadResult.getETag());
       return true;
-    } catch (Exception e) {
+    } catch (OSSException | ClientException e) {
       AbortMultipartUploadRequest abortMultipartUploadRequest =
           new AbortMultipartUploadRequest(bucketName, dstKey, uploadId);
       ossClient.abortMultipartUpload(abortMultipartUploadRequest);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d84c814/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
index bcd00dc..b12e3f0 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
@@ -227,12 +227,12 @@ public class AliyunOSSInputStream extends FSInputStream {
   }
 
   @Override
-  public void seek(long pos) throws IOException {
+  public synchronized void seek(long pos) throws IOException {
     checkNotClosed();
     if (position == pos) {
       return;
     } else if (pos > position && pos < position + partRemaining) {
-      wrappedStream.skip(pos - position);
+      AliyunOSSUtils.skipFully(wrappedStream, pos - position);
       position = pos;
     } else {
       reopen(pos);
@@ -240,7 +240,7 @@ public class AliyunOSSInputStream extends FSInputStream {
   }
 
   @Override
-  public long getPos() throws IOException {
+  public synchronized long getPos() throws IOException {
     checkNotClosed();
     return position;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d84c814/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
index 589e014..654b81d 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
@@ -29,6 +29,8 @@ import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.List;
 
+import com.aliyun.oss.ClientException;
+import com.aliyun.oss.OSSException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -113,7 +115,9 @@ public class AliyunOSSOutputStream extends OutputStream {
         multipartUploadObject();
       }
     } finally {
-      tmpFile.delete();
+      if (!tmpFile.delete()) {
+        LOG.warn("Can not delete file: " + tmpFile);
+      }
     }
   }
 
@@ -174,7 +178,7 @@ public class AliyunOSSOutputStream extends OutputStream {
         FileInputStream fis = new FileInputStream(object);
         try {
           long skipBytes = partSize * i;
-          fis.skip(skipBytes);
+          AliyunOSSUtils.skipFully(fis, skipBytes);
           long size = (partSize < dataLen - skipBytes) ?
               partSize : dataLen - skipBytes;
           UploadPartRequest uploadPartRequest = new UploadPartRequest();
@@ -198,7 +202,7 @@ public class AliyunOSSOutputStream extends OutputStream {
       CompleteMultipartUploadResult completeMultipartUploadResult =
           ossClient.completeMultipartUpload(completeMultipartUploadRequest);
       LOG.debug(completeMultipartUploadResult.getETag());
-    } catch (Exception e) {
+    } catch (OSSException | ClientException e) {
       AbortMultipartUploadRequest abortMultipartUploadRequest =
           new AbortMultipartUploadRequest(bucketName, key, uploadId);
       ossClient.abortMultipartUpload(abortMultipartUploadRequest);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d84c814/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
index 3f66a4f..9acde00 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.aliyun.oss;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.UnsupportedEncodingException;
 import java.net.URI;
 import java.net.URLDecoder;
@@ -116,7 +117,7 @@ final public class AliyunOSSUtils {
 
   /**
    * Extract the user information details from a URI.
-   * @param name URI of the filesystem
+   * @param name URI of the filesystem.
    * @return a login tuple, possibly empty.
    */
   public static UserInfo extractLoginDetails(URI name) {
@@ -148,4 +149,27 @@ final public class AliyunOSSUtils {
       throw new RuntimeException(e);
     }
   }
+
+  /**
+   * Skips the requested number of bytes or fail if there are not enough left.
+   * This allows for the possibility that {@link InputStream#skip(long)} may not
+   * skip as many bytes as requested (most likely because of reaching EOF).
+   * @param is the input stream to skip.
+   * @param n the number of bytes to skip.
+   * @throws IOException thrown when skipped less number of bytes.
+   */
+  public static void skipFully(InputStream is, long n) throws IOException {
+    long total = 0;
+    long cur = 0;
+
+    do {
+      cur = is.skip(n - total);
+      total += cur;
+    } while((total < n) && (cur > 0));
+
+    if (total < n) {
+      throw new IOException("Failed to skip " + n + " bytes, possibly due " +
+              "to EOF.");
+    }
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[21/23] hadoop git commit: HADOOP-13701. AbstractContractRootDirectoryTest can fail when handling delete "/". Contributed by Genmao Yu

Posted by dr...@apache.org.
HADOOP-13701. AbstractContractRootDirectoryTest can fail when handling delete "/". Contributed by Genmao Yu


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c31b5e61
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c31b5e61
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c31b5e61

Branch: refs/heads/trunk
Commit: c31b5e61b1f09949548116309218a2b3e9c0beda
Parents: a57bba4
Author: Kai Zheng <ka...@intel.com>
Authored: Sat Oct 8 17:35:59 2016 +0600
Committer: Kai Zheng <ka...@intel.com>
Committed: Sat Oct 8 17:35:59 2016 +0600

----------------------------------------------------------------------
 .../fs/aliyun/oss/AliyunOSSFileSystem.java      | 39 +++++++++++++++++++-
 1 file changed, 38 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c31b5e61/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
index 81e038d..3b266c8 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.Progressable;
 
@@ -53,6 +54,7 @@ public class AliyunOSSFileSystem extends FileSystem {
   private static final Logger LOG =
       LoggerFactory.getLogger(AliyunOSSFileSystem.class);
   private URI uri;
+  private String bucket;
   private Path workingDir;
   private AliyunOSSFileSystemStore store;
   private int maxKeys;
@@ -124,11 +126,20 @@ public class AliyunOSSFileSystem extends FileSystem {
   private boolean innerDelete(FileStatus status, boolean recursive)
       throws IOException {
     Path f = status.getPath();
+    String p = f.toUri().getPath();
+    FileStatus[] statuses;
+    // indicating root directory "/".
+    if (p.equals("/")) {
+      statuses = listStatus(status.getPath());
+      boolean isEmptyDir = statuses.length <= 0;
+      return rejectRootDirectoryDelete(isEmptyDir, recursive);
+    }
+
     String key = pathToKey(f);
     if (status.isDirectory()) {
       if (!recursive) {
-        FileStatus[] statuses = listStatus(status.getPath());
         // Check whether it is an empty directory or not
+        statuses = listStatus(status.getPath());
         if (statuses.length > 0) {
           throw new IOException("Cannot remove directory " + f +
               ": It is not empty!");
@@ -148,6 +159,31 @@ public class AliyunOSSFileSystem extends FileSystem {
     return true;
   }
 
+  /**
+   * Implements the specific logic to reject root directory deletion.
+   * The caller must return the result of this call, rather than
+   * attempt to continue with the delete operation: deleting root
+   * directories is never allowed. This method simply implements
+   * the policy of when to return an exit code versus raise an exception.
+   * @param isEmptyDir empty directory or not
+   * @param recursive recursive flag from command
+   * @return a return code for the operation
+   * @throws PathIOException if the operation was explicitly rejected.
+   */
+  private boolean rejectRootDirectoryDelete(boolean isEmptyDir,
+      boolean recursive) throws IOException {
+    LOG.info("oss delete the {} root directory of {}", bucket, recursive);
+    if (isEmptyDir) {
+      return true;
+    }
+    if (recursive) {
+      return false;
+    } else {
+      // reject
+      throw new PathIOException(bucket, "Cannot delete root path");
+    }
+  }
+
   private void createFakeDirectoryIfNecessary(Path f) throws IOException {
     String key = pathToKey(f);
     if (StringUtils.isNotEmpty(key) && !exists(f)) {
@@ -226,6 +262,7 @@ public class AliyunOSSFileSystem extends FileSystem {
   public void initialize(URI name, Configuration conf) throws IOException {
     super.initialize(name, conf);
 
+    bucket = name.getHost();
     uri = java.net.URI.create(name.getScheme() + "://" + name.getAuthority());
     workingDir = new Path("/user",
         System.getProperty("user.name")).makeQualified(uri, null);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[22/23] hadoop git commit: Merge branch 'trunk' into HADOOP-12756

Posted by dr...@apache.org.
Merge branch 'trunk' into HADOOP-12756


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a57bba47
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a57bba47
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a57bba47

Branch: refs/heads/trunk
Commit: a57bba470b396c163baef7ac9447c063180ec15b
Parents: 26d5df3 6a38d11
Author: Kai Zheng <ka...@intel.com>
Authored: Sun Oct 9 10:29:40 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Sun Oct 9 10:29:40 2016 +0800

----------------------------------------------------------------------
 .../IncludePublicAnnotationsJDiffDoclet.java    |    64 +
 .../util/RolloverSignerSecretProvider.java      |     2 +-
 .../util/TestZKSignerSecretProvider.java        |   221 +-
 .../dev-support/findbugsExcludeFile.xml         |     5 +
 .../jdiff/Apache_Hadoop_Common_2.7.2.xml        | 41149 ++++++-----------
 .../org/apache/hadoop/conf/ConfServlet.java     |    19 +-
 .../org/apache/hadoop/conf/Configuration.java   |   284 +-
 .../apache/hadoop/fs/DFCachingGetSpaceUsed.java |    48 +
 .../src/main/java/org/apache/hadoop/fs/DU.java  |     8 +-
 .../apache/hadoop/fs/FileEncryptionInfo.java    |    21 +
 .../java/org/apache/hadoop/fs/FileSystem.java   |    13 +-
 .../org/apache/hadoop/fs/ftp/FTPFileSystem.java |     6 +-
 .../apache/hadoop/fs/permission/AclEntry.java   |    24 +-
 .../hadoop/fs/permission/AclEntryScope.java     |     2 +-
 .../hadoop/fs/permission/AclEntryType.java      |    23 +-
 .../apache/hadoop/fs/permission/AclStatus.java  |     2 +-
 .../org/apache/hadoop/fs/shell/AclCommands.java |     6 +-
 .../hadoop/fs/shell/CommandWithDestination.java |     5 +-
 .../org/apache/hadoop/fs/viewfs/ViewFs.java     |     2 +-
 .../java/org/apache/hadoop/io/BloomMapFile.java |    11 +-
 .../main/java/org/apache/hadoop/io/IOUtils.java |     9 +-
 .../main/java/org/apache/hadoop/io/MapFile.java |    10 +-
 .../java/org/apache/hadoop/io/SequenceFile.java |    16 +-
 .../apache/hadoop/io/compress/BZip2Codec.java   |     9 +-
 .../apache/hadoop/io/compress/DefaultCodec.java |     9 +-
 .../apache/hadoop/io/compress/GzipCodec.java    |     9 +-
 .../hadoop/io/file/tfile/Compression.java       |    14 +-
 .../org/apache/hadoop/ipc/ExternalCall.java     |    91 +
 .../main/java/org/apache/hadoop/ipc/Server.java |    88 +-
 .../org/apache/hadoop/net/NetworkTopology.java  |     2 +-
 .../apache/hadoop/net/SocksSocketFactory.java   |     4 +-
 .../org/apache/hadoop/security/Credentials.java |     8 +-
 .../hadoop/security/KerberosAuthException.java  |   118 +
 .../hadoop/security/UGIExceptionMessages.java   |    46 +
 .../hadoop/security/UserGroupInformation.java   |   105 +-
 .../org/apache/hadoop/security/token/Token.java |    60 +-
 .../java/org/apache/hadoop/util/LineReader.java |     6 +-
 .../org/apache/hadoop/util/SysInfoWindows.java  |    58 +-
 .../java/org/apache/hadoop/util/hash/Hash.java  |     6 +-
 .../src/main/resources/core-default.xml         |     6 +-
 .../src/site/markdown/FileSystemShell.md        |     3 +-
 .../src/site/markdown/filesystem/filesystem.md  |    77 +-
 .../org/apache/hadoop/conf/TestConfServlet.java |   122 +-
 .../apache/hadoop/conf/TestConfiguration.java   |   140 +-
 .../apache/hadoop/fs/FileContextURIBase.java    |     4 +-
 .../hadoop/fs/TestDFCachingGetSpaceUsed.java    |    75 +
 .../hadoop/fs/TestDelegationTokenRenewer.java   |     3 +-
 .../hadoop/fs/TestFileSystemInitialization.java |    12 +-
 .../AbstractContractRootDirectoryTest.java      |    34 +-
 .../fs/contract/AbstractFSContractTestBase.java |     2 +-
 .../hadoop/fs/contract/ContractTestUtils.java   |    48 +-
 .../java/org/apache/hadoop/ipc/TestRPC.java     |    85 +
 .../org/apache/hadoop/net/ServerSocketUtil.java |    23 +
 .../security/TestUserGroupInformation.java      |    33 +-
 .../apache/hadoop/util/TestSysInfoWindows.java  |     7 +-
 .../hadoop/crypto/key/kms/server/KMS.java       |    76 +-
 .../hadoop/crypto/key/kms/server/KMSWebApp.java |     2 +
 .../hadoop/crypto/key/kms/server/TestKMS.java   |    76 +-
 hadoop-hdfs-project/hadoop-hdfs-client/pom.xml  |     4 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |     9 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    |   146 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |    30 +
 .../hdfs/client/CreateEncryptionZoneFlag.java   |    70 +
 .../apache/hadoop/hdfs/client/HdfsAdmin.java    |   536 +
 .../apache/hadoop/hdfs/client/HdfsUtils.java    |    86 +
 .../apache/hadoop/hdfs/client/package-info.java |    27 +
 .../server/datanode/DiskBalancerWorkItem.java   |     2 +-
 .../hdfs/shortcircuit/ShortCircuitCache.java    |    88 +-
 .../hdfs/web/resources/AclPermissionParam.java  |    23 +-
 hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml  |     1 -
 .../jdiff/Apache_Hadoop_HDFS_2.7.2.xml          | 21704 +--------
 .../src/contrib/bkjournal/README.txt            |    66 -
 .../dev-support/findbugsExcludeFile.xml         |     5 -
 .../hadoop-hdfs/src/contrib/bkjournal/pom.xml   |   175 -
 .../bkjournal/BookKeeperEditLogInputStream.java |   264 -
 .../BookKeeperEditLogOutputStream.java          |   188 -
 .../bkjournal/BookKeeperJournalManager.java     |   893 -
 .../contrib/bkjournal/CurrentInprogress.java    |   160 -
 .../bkjournal/EditLogLedgerMetadata.java        |   217 -
 .../hadoop/contrib/bkjournal/MaxTxId.java       |   103 -
 .../bkjournal/src/main/proto/bkjournal.proto    |    49 -
 .../hadoop/contrib/bkjournal/BKJMUtil.java      |   184 -
 .../bkjournal/TestBookKeeperAsHASharedDir.java  |   414 -
 .../bkjournal/TestBookKeeperConfiguration.java  |   174 -
 .../bkjournal/TestBookKeeperEditLogStreams.java |    92 -
 .../bkjournal/TestBookKeeperHACheckpoints.java  |   109 -
 .../bkjournal/TestBookKeeperJournalManager.java |   984 -
 .../TestBookKeeperSpeculativeRead.java          |   167 -
 .../bkjournal/TestBootstrapStandbyWithBKJM.java |   170 -
 .../bkjournal/TestCurrentInprogress.java        |   160 -
 .../hdfs/server/namenode/FSEditLogTestUtil.java |    40 -
 .../src/test/resources/log4j.properties         |    55 -
 .../java/org/apache/hadoop/hdfs/HAUtil.java     |     5 +-
 .../hdfs/client/CreateEncryptionZoneFlag.java   |    71 -
 .../apache/hadoop/hdfs/client/HdfsAdmin.java    |   524 -
 .../apache/hadoop/hdfs/client/HdfsUtils.java    |    86 -
 .../apache/hadoop/hdfs/client/package-info.java |    27 -
 .../block/BlockPoolTokenSecretManager.java      |     3 +-
 .../token/block/BlockTokenSecretManager.java    |     6 +
 .../hadoop/hdfs/server/balancer/Dispatcher.java |     2 +-
 .../server/blockmanagement/BlockManager.java    |    70 +-
 .../BlockPlacementPolicyDefault.java            |     5 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |     5 +
 .../hdfs/server/datanode/DiskBalancer.java      |   210 +-
 .../erasurecode/StripedBlockReader.java         |     2 +-
 .../datanode/fsdataset/impl/BlockPoolSlice.java |     3 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |    11 +-
 .../datanode/fsdataset/impl/ReplicaMap.java     |    41 +-
 .../server/diskbalancer/planner/MoveStep.java   |     6 +-
 .../hdfs/server/namenode/CacheManager.java      |    12 +-
 .../ContentSummaryComputationContext.java       |    94 +-
 .../namenode/EncryptionFaultInjector.java       |     6 +
 .../server/namenode/EncryptionZoneManager.java  |    25 +-
 .../hadoop/hdfs/server/namenode/FSDirAclOp.java |     4 +-
 .../hdfs/server/namenode/FSDirAttrOp.java       |   122 +-
 .../hdfs/server/namenode/FSDirDeleteOp.java     |    67 +-
 .../server/namenode/FSDirEncryptionZoneOp.java  |   148 +-
 .../server/namenode/FSDirErasureCodingOp.java   |     2 +-
 .../hdfs/server/namenode/FSDirMkdirOp.java      |     2 +-
 .../hdfs/server/namenode/FSDirRenameOp.java     |   200 +-
 .../hdfs/server/namenode/FSDirSnapshotOp.java   |    22 +-
 .../server/namenode/FSDirStatAndListingOp.java  |   281 +-
 .../hdfs/server/namenode/FSDirSymlinkOp.java    |     2 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |   163 +-
 .../hdfs/server/namenode/FSDirXAttrOp.java      |    27 +-
 .../hdfs/server/namenode/FSDirectory.java       |    94 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |    94 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   244 +-
 .../hdfs/server/namenode/FSNamesystemLock.java  |   187 +-
 .../hadoop/hdfs/server/namenode/INode.java      |     1 +
 .../server/namenode/INodeAttributeProvider.java |    10 +-
 .../hdfs/server/namenode/INodeDirectory.java    |    11 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |     1 +
 .../hadoop/hdfs/server/namenode/INodeId.java    |    16 -
 .../hdfs/server/namenode/INodeReference.java    |     2 +
 .../hdfs/server/namenode/INodeSymlink.java      |     1 +
 .../hdfs/server/namenode/INodesInPath.java      |     7 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |    23 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java |     2 +
 .../hdfs/server/namenode/XAttrStorage.java      |     7 +-
 .../ha/RequestHedgingProxyProvider.java         |    25 +-
 .../snapshot/DirectorySnapshottableFeature.java |     9 +-
 .../snapshot/DirectoryWithSnapshotFeature.java  |    14 +-
 .../hdfs/server/namenode/snapshot/Snapshot.java |     1 +
 .../server/protocol/RemoteEditLogManifest.java  |     7 +-
 .../apache/hadoop/hdfs/tools/CryptoAdmin.java   |    51 +-
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |     2 +-
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    |     2 +-
 .../hadoop-hdfs/src/main/proto/HdfsServer.proto |     2 +-
 .../src/main/resources/hdfs-default.xml         |     6 +-
 .../src/site/markdown/FaultInjectFramework.md   |   254 -
 .../src/site/markdown/HDFSCommands.md           |     8 +-
 .../src/site/markdown/HDFSDiskbalancer.md       |     5 +-
 .../src/site/markdown/HDFSErasureCoding.md      |    11 +-
 .../markdown/HDFSHighAvailabilityWithNFS.md     |   114 -
 .../src/site/markdown/TransparentEncryption.md  |    16 +
 .../hadoop-hdfs/src/site/markdown/ViewFs.md     |     2 +-
 .../hadoop-hdfs/src/site/markdown/WebHDFS.md    |     2 +-
 .../hadoop/fs/TestEnhancedByteBufferAccess.java |    17 +-
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |   103 +-
 .../hadoop/hdfs/TestDFSClientRetries.java       |     4 +-
 .../org/apache/hadoop/hdfs/TestDFSShell.java    |  2021 +-
 .../hadoop/hdfs/TestEncryptedTransfer.java      |   742 +-
 .../apache/hadoop/hdfs/TestEncryptionZones.java |   109 +-
 .../apache/hadoop/hdfs/TestFileCorruption.java  |    61 +
 .../hadoop/hdfs/TestLeaseRecoveryStriped.java   |     1 +
 .../apache/hadoop/hdfs/TestMiniDFSCluster.java  |   119 +
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |    30 +-
 .../TestPendingInvalidateBlock.java             |    37 +-
 .../fsdataset/impl/FsDatasetImplTestUtils.java  |     2 +-
 .../fsdataset/impl/TestDatanodeRestart.java     |    13 +-
 .../impl/TestInterDatanodeProtocol.java         |     3 +-
 .../datanode/fsdataset/impl/TestReplicaMap.java |     3 +-
 .../fsdataset/impl/TestWriteToReplica.java      |     3 +-
 .../server/diskbalancer/TestDiskBalancer.java   |   156 +
 .../server/namenode/TestCacheDirectives.java    |    10 +
 .../namenode/TestDiskspaceQuotaUpdate.java      |   301 +-
 .../hdfs/server/namenode/TestFSDirectory.java   |    48 +
 .../hdfs/server/namenode/TestFSNamesystem.java  |   251 -
 .../server/namenode/TestFSNamesystemLock.java   |   317 +
 .../hadoop/hdfs/server/namenode/TestFsck.java   |  2403 +-
 .../server/namenode/TestNameNodeMXBean.java     |    34 +-
 .../server/namenode/TestSnapshotPathINodes.java |    22 +
 .../server/namenode/ha/TestEditLogTailer.java   |    42 +-
 .../ha/TestRequestHedgingProxyProvider.java     |    18 +-
 .../snapshot/TestRenameWithSnapshots.java       |   199 +
 .../shortcircuit/TestShortCircuitCache.java     |     9 +-
 .../hadoop/hdfs/web/TestWebHdfsTokens.java      |    27 +-
 .../apache/hadoop/security/TestPermission.java  |   131 +-
 .../src/test/resources/testCryptoConf.xml       |    90 +
 hadoop-hdfs-project/pom.xml                     |     1 -
 .../Apache_Hadoop_MapReduce_Common_2.7.2.xml    |   727 +-
 .../Apache_Hadoop_MapReduce_Core_2.7.2.xml      |  7059 +--
 .../Apache_Hadoop_MapReduce_JobClient_2.7.2.xml |   962 +-
 .../hadoop/mapreduce/v2/app/MRAppMaster.java    |    90 +-
 .../mapreduce/v2/app/TaskHeartbeatHandler.java  |    24 +-
 .../v2/app/rm/RMContainerAllocator.java         |    51 +-
 .../hadoop/mapreduce/v2/app/TestRecovery.java   |    66 +
 .../v2/app/TestTaskHeartbeatHandler.java        |    67 +
 .../v2/app/launcher/TestContainerLauncher.java  |    30 +
 .../app/launcher/TestContainerLauncherImpl.java |    30 +
 .../v2/app/rm/TestRMContainerAllocator.java     |    46 +
 .../mapreduce/v2/app/webapp/TestAMWebApp.java   |     8 +-
 .../hadoop-mapreduce-client-core/pom.xml        |     6 +-
 .../java/org/apache/hadoop/mapred/Master.java   |    70 +-
 .../java/org/apache/hadoop/mapred/Task.java     |     8 +-
 .../apache/hadoop/mapreduce/MRJobConfig.java    |    11 +-
 .../hadoop/mapreduce/util/MRJobConfUtil.java    |    16 +
 .../src/main/resources/mapred-default.xml       |    10 +-
 .../org/apache/hadoop/mapred/TestMaster.java    |    56 +-
 .../mapreduce/v2/hs/HistoryFileManager.java     |    21 +
 .../hadoop/mapred/ResourceMgrDelegate.java      |     6 +
 .../apache/hadoop/mapred/JobClientUnitTest.java |    34 +-
 .../hadoop-mapreduce-client/pom.xml             |     4 +-
 .../maven/plugin/cmakebuilder/CompileMojo.java  |     4 +-
 .../maven/plugin/cmakebuilder/TestMojo.java     |     4 +-
 .../hadoop/maven/plugin/protoc/ProtocMojo.java  |     4 +
 .../apache/hadoop/maven/plugin/util/Exec.java   |     6 +-
 .../plugin/versioninfo/VersionInfoMojo.java     |    55 +-
 hadoop-project-dist/pom.xml                     |    21 +-
 hadoop-project/pom.xml                          |     8 +-
 hadoop-tools/hadoop-aws/pom.xml                 |    12 +
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |   171 +-
 .../hadoop/fs/s3a/S3AInstrumentation.java       |    10 +
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java |    17 +
 .../org/apache/hadoop/fs/s3a/Statistic.java     |     4 +
 .../src/site/markdown/tools/hadoop-aws/index.md |    42 +-
 .../hadoop/fs/s3a/ITestS3AConfiguration.java    |     8 +
 .../hadoop/fs/s3a/ITestS3AFailureHandling.java  |    55 -
 .../fs/s3a/ITestS3AFileOperationCost.java       |    85 +
 .../hadoop/fs/s3a/ITestS3AMiscOperations.java   |    63 +
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java  |    13 +-
 .../fs/s3a/TestS3AExceptionTranslation.java     |   127 +
 hadoop-tools/hadoop-azure-datalake/pom.xml      |     4 +
 ...ClientCredentialBasedAccesTokenProvider.java |     5 +-
 hadoop-tools/hadoop-azure/pom.xml               |     6 +-
 .../hadoop/fs/azure/NativeAzureFileSystem.java  |    16 +-
 .../tools/mapred/CopyMapper_Counter.properties  |    24 +
 .../contract/AbstractContractDistCpTest.java    |     2 +-
 hadoop-tools/hadoop-kafka/pom.xml               |     2 +-
 hadoop-tools/hadoop-openstack/pom.xml           |    18 +-
 .../swift/auth/ApiKeyAuthenticationRequest.java |     2 +-
 .../fs/swift/auth/entities/AccessToken.java     |     2 +-
 .../hadoop/fs/swift/auth/entities/Catalog.java  |     2 +-
 .../hadoop/fs/swift/auth/entities/Endpoint.java |     2 +-
 .../hadoop/fs/swift/auth/entities/Tenant.java   |     2 +-
 .../hadoop/fs/swift/auth/entities/User.java     |     2 +-
 .../snative/SwiftNativeFileSystemStore.java     |     3 +-
 .../apache/hadoop/fs/swift/util/JSONUtil.java   |    24 +-
 hadoop-tools/hadoop-rumen/pom.xml               |     9 +
 .../apache/hadoop/tools/rumen/Anonymizer.java   |    23 +-
 .../hadoop/tools/rumen/HadoopLogsAnalyzer.java  |     3 +-
 .../tools/rumen/JsonObjectMapperParser.java     |    17 +-
 .../tools/rumen/JsonObjectMapperWriter.java     |    21 +-
 .../apache/hadoop/tools/rumen/LoggedJob.java    |     2 +-
 .../hadoop/tools/rumen/LoggedLocation.java      |     2 +-
 .../tools/rumen/LoggedNetworkTopology.java      |     2 +-
 .../rumen/LoggedSingleRelativeRanking.java      |     4 +-
 .../apache/hadoop/tools/rumen/LoggedTask.java   |     2 +-
 .../hadoop/tools/rumen/LoggedTaskAttempt.java   |     2 +-
 .../hadoop/tools/rumen/datatypes/NodeName.java  |     2 +-
 .../rumen/serializers/BlockingSerializer.java   |    10 +-
 .../DefaultAnonymizingRumenSerializer.java      |     8 +-
 .../serializers/DefaultRumenSerializer.java     |     9 +-
 .../serializers/ObjectStringSerializer.java     |    10 +-
 .../apache/hadoop/tools/rumen/state/State.java  |     2 +-
 .../tools/rumen/state/StateDeserializer.java    |    14 +-
 .../hadoop/tools/rumen/state/StatePool.java     |    36 +-
 .../hadoop/tools/rumen/TestHistograms.java      |    13 +-
 hadoop-tools/hadoop-sls/pom.xml                 |     4 +
 .../hadoop/yarn/sls/RumenToSLSConverter.java    |     8 +-
 .../org/apache/hadoop/yarn/sls/SLSRunner.java   |     7 +-
 .../apache/hadoop/yarn/sls/utils/SLSUtils.java  |    10 +-
 .../jdiff/Apache_Hadoop_YARN_API_2.7.2.xml      |   530 +-
 .../jdiff/Apache_Hadoop_YARN_Client_2.7.2.xml   |   613 +-
 .../jdiff/Apache_Hadoop_YARN_Common_2.7.2.xml   |  1185 +-
 .../Apache_Hadoop_YARN_Server_Common_2.7.2.xml  |  1056 +-
 .../yarn/api/ContainerManagementProtocol.java   |    54 +
 .../api/protocolrecords/CommitResponse.java     |    42 +
 .../protocolrecords/KillApplicationRequest.java |    18 +
 .../ReInitializeContainerRequest.java           |   110 +
 .../ReInitializeContainerResponse.java          |    38 +
 .../RestartContainerResponse.java               |    38 +
 .../api/protocolrecords/RollbackResponse.java   |    42 +
 .../records/ApplicationSubmissionContext.java   |    21 +
 .../api/records/ApplicationTimeoutType.java     |    41 +
 .../yarn/api/records/ReservationDefinition.java |    44 +-
 .../org/apache/hadoop/yarn/api/records/URL.java |    58 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |    24 +-
 .../ReplaceLabelsOnNodeRequest.java             |     8 +
 .../proto/containermanagement_protocol.proto    |     6 +
 ..._server_resourcemanager_service_protos.proto |     2 +-
 .../src/main/proto/yarn_protos.proto            |    11 +
 .../src/main/proto/yarn_service_protos.proto    |    19 +
 .../apache/hadoop/yarn/api/records/TestURL.java |    99 +
 .../hadoop/yarn/client/api/YarnClient.java      |    14 +
 .../yarn/client/api/impl/AMRMClientImpl.java    |    10 +-
 .../impl/ContainerManagementProtocolProxy.java  |     7 +-
 .../yarn/client/api/impl/YarnClientImpl.java    |    35 +-
 .../hadoop/yarn/client/cli/RMAdminCLI.java      |    39 +-
 .../TestOpportunisticContainerAllocation.java   |   398 +
 .../yarn/client/api/impl/TestYarnClient.java    |    52 +
 .../hadoop/yarn/client/cli/TestRMAdminCLI.java  |     3 +-
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |    10 +-
 ...ContainerManagementProtocolPBClientImpl.java |    73 +
 ...ontainerManagementProtocolPBServiceImpl.java |    86 +-
 .../impl/pb/CommitResponsePBImpl.java           |    67 +
 .../impl/pb/KillApplicationRequestPBImpl.java   |    18 +
 .../pb/ReInitializeContainerRequestPBImpl.java  |   173 +
 .../pb/ReInitializeContainerResponsePBImpl.java |    68 +
 .../impl/pb/RestartContainerResponsePBImpl.java |    67 +
 .../impl/pb/RollbackResponsePBImpl.java         |    67 +
 .../pb/ApplicationSubmissionContextPBImpl.java  |    83 +
 .../records/impl/pb/ContainerStatusPBImpl.java  |     2 +
 .../yarn/api/records/impl/pb/ProtoUtils.java    |    19 +
 .../impl/pb/ReservationDefinitionPBImpl.java    |    31 +
 .../client/api/impl/TimelineClientImpl.java     |     3 +-
 .../pb/ReplaceLabelsOnNodeRequestPBImpl.java    |    14 +-
 .../yarn/util/AbstractLivelinessMonitor.java    |    32 +-
 .../hadoop/yarn/webapp/YarnWebParams.java       |     3 +-
 .../src/main/resources/yarn-default.xml         |    28 +-
 .../hadoop/yarn/TestContainerLaunchRPC.java     |    30 +
 .../yarn/TestContainerResourceIncreaseRPC.java  |    30 +
 .../hadoop/yarn/api/BasePBImplRecordsTest.java  |   264 +
 .../hadoop/yarn/api/TestPBImplRecords.java      |   269 +-
 ...pplicationHistoryManagerOnTimelineStore.java |    14 +-
 ...pplicationHistoryManagerOnTimelineStore.java |    14 +-
 .../metrics/ApplicationMetricsConstants.java    |     3 +
 .../OpportunisticContainerAllocator.java        |    22 +-
 .../OpportunisticContainerContext.java          |    49 +-
 .../java/org/apache/hadoop/yarn/TestRPC.java    |    30 +
 .../yarn/server/nodemanager/NMAuditLogger.java  |     4 +
 .../yarn/server/nodemanager/NodeManager.java    |     3 +-
 .../amrmproxy/DefaultRequestInterceptor.java    |     4 +-
 .../containermanager/ContainerManagerImpl.java  |    53 +-
 .../container/ContainerImpl.java                |    92 +-
 .../monitor/ContainersMonitorImpl.java          |    31 +-
 .../nodemanager/metrics/NodeManagerMetrics.java |    26 +
 .../scheduler/DistributedScheduler.java         |    59 +-
 .../impl/container-executor.c                   |    46 +-
 .../impl/container-executor.h                   |    10 +-
 .../main/native/container-executor/impl/main.c  |   264 +-
 .../TestContainerManagerWithLCE.java            |    12 +
 .../containermanager/TestContainerManager.java  |   103 +-
 .../monitor/TestContainersMonitor.java          |    39 +
 .../server/resourcemanager/AdminService.java    |    46 +
 .../server/resourcemanager/ClientRMService.java |    20 +-
 ...pportunisticContainerAllocatorAMService.java |   215 +-
 .../resourcemanager/RMActiveServiceContext.java |    16 +
 .../server/resourcemanager/RMAppManager.java    |     4 +
 .../server/resourcemanager/RMAuditLogger.java   |     3 +
 .../yarn/server/resourcemanager/RMContext.java  |     5 +
 .../server/resourcemanager/RMContextImpl.java   |    12 +
 .../server/resourcemanager/RMServerUtils.java   |    21 +-
 .../yarn/server/resourcemanager/RMZKUtils.java  |    19 +-
 .../server/resourcemanager/ResourceManager.java |    23 +-
 .../resourcemanager/amlauncher/AMLauncher.java  |    21 -
 .../metrics/TimelineServiceV1Publisher.java     |     6 +
 .../metrics/TimelineServiceV2Publisher.java     |     5 +
 .../recovery/ZKRMStateStore.java                |   269 +-
 .../server/resourcemanager/rmapp/RMAppImpl.java |    49 +-
 .../rmapp/monitor/RMAppLifetimeMonitor.java     |   130 +
 .../rmapp/monitor/RMAppToMonitor.java           |    77 +
 .../rmapp/monitor/package-info.java             |    28 +
 .../scheduler/AbstractYarnScheduler.java        |   416 +-
 .../scheduler/AppSchedulingInfo.java            |   619 +-
 .../scheduler/SchedulerApplicationAttempt.java  |    66 +-
 .../scheduler/capacity/CapacityScheduler.java   |  1729 +-
 .../scheduler/capacity/LeafQueue.java           |    16 +
 .../scheduler/common/fica/FiCaSchedulerApp.java |     4 +
 .../distributed/NodeQueueLoadMonitor.java       |    45 +-
 .../scheduler/fair/FSLeafQueue.java             |     2 +-
 .../scheduler/fair/FSParentQueue.java           |     6 +-
 .../scheduler/fair/FairScheduler.java           |  1047 +-
 .../resourcemanager/webapp/ErrorBlock.java      |    39 +
 .../server/resourcemanager/webapp/RMWebApp.java |     1 +
 .../resourcemanager/webapp/RMWebServices.java   |    24 +-
 .../webapp/RedirectionErrorPage.java            |    47 +
 .../resourcemanager/webapp/RmController.java    |     4 +
 .../resourcemanager/webapp/dao/AppState.java    |     8 +
 .../webapp/dao/ReservationDefinitionInfo.java   |    11 +
 .../yarn/server/resourcemanager/MockRM.java     |    22 +-
 .../server/resourcemanager/NodeManager.java     |    29 +
 .../resourcemanager/TestAMAuthorization.java    |    31 +
 .../TestApplicationMasterLauncher.java          |    30 +
 .../resourcemanager/TestClientRMService.java    |     7 +-
 ...pportunisticContainerAllocatorAMService.java |    10 +-
 .../resourcemanager/TestRMAdminService.java     |   103 +-
 .../metrics/TestSystemMetricsPublisher.java     |    18 +
 .../TestSystemMetricsPublisherForV2.java        |     9 +
 .../reservation/ReservationSystemTestUtil.java  |    10 +-
 .../rmapp/TestApplicationLifetimeMonitor.java   |   165 +
 .../TestRMWebServicesAppsModification.java      |     4 +
 .../webapp/TestRedirectionErrorPage.java        |    68 +
 .../yarn/server/webproxy/ProxyUriUtils.java     |    53 +-
 .../server/webproxy/WebAppProxyServlet.java     |   274 +-
 .../server/webproxy/amfilter/AmIpFilter.java    |    64 +-
 .../server/webproxy/TestWebAppProxyServlet.java |    24 +-
 .../server/webproxy/amfilter/TestAmFilter.java  |    29 +-
 .../src/site/markdown/CapacityScheduler.md      |     2 +-
 .../src/site/markdown/NodeLabel.md              |     2 +-
 .../src/site/markdown/ReservationSystem.md      |     2 +-
 .../src/site/markdown/ResourceManagerRest.md    |     3 +
 .../site/markdown/WritingYarnApplications.md    |     6 +-
 .../hadoop-yarn-site/src/site/markdown/YARN.md  |     6 +-
 hadoop-yarn-project/hadoop-yarn/pom.xml         |     4 +-
 pom.xml                                         |     1 -
 407 files changed, 32657 insertions(+), 72369 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a57bba47/hadoop-project/pom.xml
----------------------------------------------------------------------


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[08/23] hadoop git commit: HADOOP-13481. User documents for Aliyun OSS FileSystem. Contributed by Genmao Yu.

Posted by dr...@apache.org.
HADOOP-13481. User documents for Aliyun OSS FileSystem. Contributed by Genmao Yu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e671a0f5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e671a0f5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e671a0f5

Branch: refs/heads/trunk
Commit: e671a0f52b5488b8453e1a3258ea5e6477995648
Parents: d33e928
Author: Mingfei <mi...@intel.com>
Authored: Sun Aug 28 10:37:52 2016 +0800
Committer: Mingfei <mi...@intel.com>
Committed: Wed Sep 7 11:17:43 2016 +0800

----------------------------------------------------------------------
 .../apache/hadoop/fs/aliyun/oss/Constants.java  |   3 +-
 .../site/markdown/tools/hadoop-aliyun/index.md  | 299 +++++++++++++++++++
 2 files changed, 300 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e671a0f5/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
index 243fdd4..e0c05ed 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
@@ -95,8 +95,7 @@ public final class Constants {
   // Comma separated list of directories
   public static final String BUFFER_DIR_KEY = "fs.oss.buffer.dir";
 
-  // private | public-read | public-read-write | authenticated-read |
-  // log-delivery-write | bucket-owner-read | bucket-owner-full-control
+  // private | public-read | public-read-write
   public static final String CANNED_ACL_KEY = "fs.oss.acl.default";
   public static final String CANNED_ACL_DEFAULT = "";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e671a0f5/hadoop-tools/hadoop-aliyun/src/site/markdown/tools/hadoop-aliyun/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/site/markdown/tools/hadoop-aliyun/index.md b/hadoop-tools/hadoop-aliyun/src/site/markdown/tools/hadoop-aliyun/index.md
new file mode 100644
index 0000000..4095e06
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/site/markdown/tools/hadoop-aliyun/index.md
@@ -0,0 +1,299 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# Hadoop-Aliyun module: Integration with Aliyun Web Services
+
+<!-- MACRO{toc|fromDepth=0|toDepth=5} -->
+
+## Overview
+
+The `hadoop-aliyun` module provides support for Aliyun integration with
+[Aliyun Object Storage Service (Aliyun OSS)](https://www.aliyun.com/product/oss).
+The generated JAR file, `hadoop-aliyun.jar` also declares a transitive
+dependency on all external artifacts which are needed for this support \u2014 enabling
+downstream applications to easily use this support.
+
+To make it part of Apache Hadoop's default classpath, simply make sure
+that HADOOP_OPTIONAL_TOOLS in hadoop-env.sh has 'hadoop-aliyun' in the list.
+
+### Features
+
+* Read and write data stored in Aliyun OSS.
+* Present a hierarchical file system view by implementing the standard Hadoop
+[`FileSystem`](../api/org/apache/hadoop/fs/FileSystem.html) interface.
+* Can act as a source of data in a MapReduce job, or a sink.
+
+### Warning #1: Object Stores are not filesystems.
+
+Aliyun OSS is an example of "an object store". In order to achieve scalability
+and especially high availability, Aliyun OSS has relaxed some of the constraints
+which classic "POSIX" filesystems promise.
+
+
+
+Specifically
+
+1. Atomic operations: `delete()` and `rename()` are implemented by recursive
+file-by-file operations. They take time at least proportional to the number of files,
+during which time partial updates may be visible. `delete()` and `rename()`
+can not guarantee atomicity. If the operations are interrupted, the filesystem
+is left in an intermediate state.
+2. File owner and group are persisted, but the permissions model is not enforced.
+Authorization occurs at the level of the entire Aliyun account via
+[Aliyun Resource Access Management (Aliyun RAM)](https://www.aliyun.com/product/ram).
+3. Directory last access time is not tracked.
+4. The append operation is not supported.
+
+### Warning #2: Directory last access time is not tracked,
+features of Hadoop relying on this can have unexpected behaviour. E.g. the
+AggregatedLogDeletionService of YARN will not remove the appropriate logfiles.
+
+### Warning #3: Your Aliyun credentials are valuable
+
+Your Aliyun credentials not only pay for services, they offer read and write
+access to the data. Anyone with the account can not only read your datasets
+\u2014they can delete them.
+
+Do not inadvertently share these credentials through means such as
+1. Checking in to SCM any configuration files containing the secrets.
+2. Logging them to a console, as they invariably end up being seen.
+3. Defining filesystem URIs with the credentials in the URL, such as
+`oss://accessKeyId:accessKeySecret@directory/file`. They will end up in
+logs and error messages.
+4. Including the secrets in bug reports.
+
+If you do any of these: change your credentials immediately!
+
+### Warning #4: The Aliyun OSS client provided by Aliyun E-MapReduce are different from this implementation
+
+Specifically: on Aliyun E-MapReduce, `oss://` is also supported but with
+a different implementation. If you are using Aliyun E-MapReduce,
+follow these instructions \u2014and be aware that all issues related to Aliyun
+OSS integration in E-MapReduce can only be addressed by Aliyun themselves:
+please raise your issues with them.
+
+## OSS
+
+### Authentication properties
+
+    <property>
+      <name>fs.oss.accessKeyId</name>
+      <description>Aliyun access key ID</description>
+    </property>
+
+    <property>
+      <name>fs.oss.accessKeySecret</name>
+      <description>Aliyun access key secret</description>
+    </property>
+
+    <property>
+      <name>fs.oss.credentials.provider</name>
+      <description>
+        Class name of a credentials provider that implements
+        com.aliyun.oss.common.auth.CredentialsProvider. Omit if using access/secret keys
+        or another authentication mechanism. The specified class must provide an
+        accessible constructor accepting java.net.URI and
+        org.apache.hadoop.conf.Configuration, or an accessible default constructor.
+      </description>
+    </property>
+
+### Other properties
+
+    <property>
+      <name>fs.oss.endpoint</name>
+      <description>Aliyun OSS endpoint to connect to. An up-to-date list is
+        provided in the Aliyun OSS Documentation.
+       </description>
+    </property>
+
+    <property>
+      <name>fs.oss.proxy.host</name>
+      <description>Hostname of the (optinal) proxy server for Aliyun OSS connection</description>
+    </property>
+
+    <property>
+      <name>fs.oss.proxy.port</name>
+      <description>Proxy server port</description>
+    </property>
+
+    <property>
+      <name>fs.oss.proxy.username</name>
+      <description>Username for authenticating with proxy server</description>
+    </property>
+
+    <property>
+      <name>fs.oss.proxy.password</name>
+      <description>Password for authenticating with proxy server.</description>
+    </property>
+
+    <property>
+      <name>fs.oss.proxy.domain</name>
+      <description>Domain for authenticating with proxy server.</description>
+    </property>
+
+    <property>
+      <name>fs.oss.proxy.workstation</name>
+      <description>Workstation for authenticating with proxy server.</description>
+    </property>
+
+    <property>
+      <name>fs.oss.attempts.maximum</name>
+      <value>20</value>
+      <description>How many times we should retry commands on transient errors.</description>
+    </property>
+
+    <property>
+      <name>fs.oss.connection.establish.timeout</name>
+      <value>50000</value>
+      <description>Connection setup timeout in milliseconds.</description>
+    </property>
+
+    <property>
+      <name>fs.oss.connection.timeout</name>
+      <value>200000</value>
+      <description>Socket connection timeout in milliseconds.</description>
+    </property>
+
+    <property>
+      <name>fs.oss.paging.maximum</name>
+      <value>500</value>
+      <description>How many keys to request from Aliyun OSS when doing directory listings at a time.
+      </description>
+    </property>
+
+    <property>
+      <name>fs.oss.multipart.upload.size</name>
+      <value>10485760</value>
+      <description>Size of each of multipart pieces in bytes.</description>
+    </property>
+
+    <property>
+      <name>fs.oss.multipart.upload.threshold</name>
+      <value>20971520</value>
+      <description>Minimum size in bytes before we start a multipart uploads or copy.</description>
+    </property>
+
+    <property>
+      <name>fs.oss.multipart.download.size</name>
+      <value>102400/value>
+      <description>Size in bytes in each request from ALiyun OSS.</description>
+    </property>
+
+    <property>
+      <name>fs.oss.buffer.dir</name>
+      <description>Comma separated list of directories to buffer OSS data before uploading to Aliyun OSS</description>
+    </property>
+
+    <property>
+      <name>fs.oss.buffer.dir</name>
+      <description>Comma separated list of directories to buffer OSS data before uploading to Aliyun OSS</description>
+    </property>
+
+    <property>
+      <name>fs.oss.acl.default</name>
+      <value></vaule>
+      <description>Set a canned ACL for bucket. Value may be private, public-read, public-read-write.
+      </description>
+    </property>
+
+    <property>
+      <name>fs.oss.server-side-encryption-algorithm</name>
+      <value></vaule>
+      <description>Specify a server-side encryption algorithm for oss: file system.
+         Unset by default, and the only other currently allowable value is AES256.
+      </description>
+    </property>
+
+    <property>
+      <name>fs.oss.connection.maximum</name>
+      <value>32</value>
+      <description>Number of simultaneous connections to oss.</description>
+    </property>
+
+    <property>
+      <name>fs.oss.connection.secure.enabled</name>
+      <value>true</value>
+      <description>Connect to oss over ssl or not, true by default.</description>
+    </property>
+
+## Testing the hadoop-aliyun Module
+
+To test `oss://` filesystem client, two files which pass in authentication
+details to the test runner are needed.
+
+1. `auth-keys.xml`
+2. `core-site.xml`
+
+Those two configuration files must be put into
+`hadoop-tools/hadoop-aliyun/src/test/resources`.
+
+### `core-site.xml`
+
+This file pre-exists and sources the configurations created in `auth-keys.xml`.
+
+For most cases, no modification is needed, unless a specific, non-default property
+needs to be set during the testing.
+
+### `auth-keys.xml`
+
+This file triggers the testing of Aliyun OSS module. Without this file,
+*none of the tests in this module will be executed*
+
+It contains the access key Id/secret and proxy information that are needed to
+connect to Aliyun OSS, and an OSS bucket URL should be also provided.
+
+1. `test.fs.oss.name` : the URL of the bucket for Aliyun OSS tests
+
+The contents of the bucket will be cleaned during the testing process, so
+do not use the bucket for any purpose other than testing.
+
+### Run Hadoop contract tests
+Create file `contract-test-options.xml` under `/test/resources`. If a
+specific file `fs.contract.test.fs.oss` test path is not defined, those
+tests will be skipped. Credentials are also needed to run any of those
+tests, they can be copied from `auth-keys.xml` or through direct
+XInclude inclusion. Here is an example of `contract-test-options.xml`:
+
+    <?xml version="1.0"?>
+    <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+    <configuration>
+
+    <include xmlns="http://www.w3.org/2001/XInclude"
+    href="auth-keys.xml"/>
+
+      <property>
+        <name>fs.contract.test.fs.oss</name>
+        <value>oss://spark-tests</value>
+      </property>
+
+      <property>
+        <name>fs.oss.impl</name>
+        <value>org.apache.hadoop.fs.aliyun.AliyunOSSFileSystem</value>
+      </property>
+
+      <property>
+        <name>fs.oss.endpoint</name>
+        <value>oss-cn-hangzhou.aliyuncs.com</value>
+      </property>
+
+      <property>
+        <name>fs.oss.buffer.dir</name>
+        <value>/tmp/oss</value>
+      </property>
+
+      <property>
+        <name>fs.oss.multipart.download.size</name>
+        <value>102400</value>
+      </property>
+    </configuration>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[23/23] hadoop git commit: Merge branch 'HADOOP-12756' into trunk

Posted by dr...@apache.org.
Merge branch 'HADOOP-12756' into trunk


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/669d6f13
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/669d6f13
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/669d6f13

Branch: refs/heads/trunk
Commit: 669d6f13ec48a90d4ba7e4ed1dd0e9687580f8f3
Parents: c874fa9 c31b5e6
Author: Kai Zheng <ka...@intel.com>
Authored: Tue Oct 11 03:22:11 2016 +0600
Committer: Kai Zheng <ka...@intel.com>
Committed: Tue Oct 11 03:22:11 2016 +0600

----------------------------------------------------------------------
 .gitignore                                      |   2 +
 hadoop-project/pom.xml                          |  22 +
 .../dev-support/findbugs-exclude.xml            |  18 +
 hadoop-tools/hadoop-aliyun/pom.xml              | 154 +++++
 .../aliyun/oss/AliyunCredentialsProvider.java   |  87 +++
 .../fs/aliyun/oss/AliyunOSSFileSystem.java      | 580 +++++++++++++++++++
 .../fs/aliyun/oss/AliyunOSSFileSystemStore.java | 516 +++++++++++++++++
 .../fs/aliyun/oss/AliyunOSSInputStream.java     | 260 +++++++++
 .../fs/aliyun/oss/AliyunOSSOutputStream.java    | 111 ++++
 .../hadoop/fs/aliyun/oss/AliyunOSSUtils.java    | 167 ++++++
 .../apache/hadoop/fs/aliyun/oss/Constants.java  | 113 ++++
 .../hadoop/fs/aliyun/oss/package-info.java      |  22 +
 .../site/markdown/tools/hadoop-aliyun/index.md  | 294 ++++++++++
 .../fs/aliyun/oss/AliyunOSSTestUtils.java       |  77 +++
 .../fs/aliyun/oss/TestAliyunCredentials.java    |  78 +++
 .../oss/TestAliyunOSSFileSystemContract.java    | 239 ++++++++
 .../oss/TestAliyunOSSFileSystemStore.java       | 125 ++++
 .../fs/aliyun/oss/TestAliyunOSSInputStream.java | 145 +++++
 .../aliyun/oss/TestAliyunOSSOutputStream.java   |  91 +++
 .../aliyun/oss/contract/AliyunOSSContract.java  |  49 ++
 .../contract/TestAliyunOSSContractCreate.java   |  35 ++
 .../contract/TestAliyunOSSContractDelete.java   |  34 ++
 .../contract/TestAliyunOSSContractDistCp.java   |  44 ++
 .../TestAliyunOSSContractGetFileStatus.java     |  35 ++
 .../contract/TestAliyunOSSContractMkdir.java    |  34 ++
 .../oss/contract/TestAliyunOSSContractOpen.java |  34 ++
 .../contract/TestAliyunOSSContractRename.java   |  35 ++
 .../contract/TestAliyunOSSContractRootDir.java  |  69 +++
 .../oss/contract/TestAliyunOSSContractSeek.java |  34 ++
 .../src/test/resources/contract/aliyun-oss.xml  | 115 ++++
 .../src/test/resources/core-site.xml            |  46 ++
 .../src/test/resources/log4j.properties         |  23 +
 hadoop-tools/hadoop-tools-dist/pom.xml          |   6 +
 hadoop-tools/pom.xml                            |   1 +
 34 files changed, 3695 insertions(+)
----------------------------------------------------------------------



---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[05/23] hadoop git commit: HADOOP-13529. Do some code refactoring. Contributed by Genmao Yu.

Posted by dr...@apache.org.
HADOOP-13529. Do some code refactoring. Contributed by Genmao Yu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d33e928f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d33e928f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d33e928f

Branch: refs/heads/trunk
Commit: d33e928fbeb1764a724c8f3c051bb0d8be82bbff
Parents: cdb7711
Author: Mingfei <mi...@intel.com>
Authored: Fri Aug 26 11:00:03 2016 +0800
Committer: Mingfei <mi...@intel.com>
Committed: Wed Sep 7 11:15:48 2016 +0800

----------------------------------------------------------------------
 hadoop-tools/hadoop-aliyun/pom.xml              |  23 +-
 .../fs/aliyun/oss/AliyunOSSFileSystem.java      | 455 +++--------------
 .../fs/aliyun/oss/AliyunOSSFileSystemStore.java | 486 +++++++++++++++++++
 .../fs/aliyun/oss/AliyunOSSInputStream.java     |  60 +--
 .../fs/aliyun/oss/AliyunOSSOutputStream.java    | 129 +----
 .../hadoop/fs/aliyun/oss/AliyunOSSUtils.java    | 193 ++++----
 .../apache/hadoop/fs/aliyun/oss/Constants.java  |   3 +-
 .../aliyun/oss/TestOSSFileSystemContract.java   |  10 -
 .../fs/aliyun/oss/TestOSSFileSystemStore.java   | 121 +++++
 .../fs/aliyun/oss/contract/OSSContract.java     |   1 -
 .../oss/contract/TestOSSContractDispCp.java     |  44 ++
 .../contract/TestOSSContractGetFileStatus.java  |  35 ++
 .../oss/contract/TestOSSContractRootDir.java    |  69 +++
 13 files changed, 968 insertions(+), 661 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d33e928f/hadoop-tools/hadoop-aliyun/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/pom.xml b/hadoop-tools/hadoop-aliyun/pom.xml
index c87d13f..358b18b 100644
--- a/hadoop-tools/hadoop-aliyun/pom.xml
+++ b/hadoop-tools/hadoop-aliyun/pom.xml
@@ -128,6 +128,27 @@
       <scope>test</scope>
       <type>test-jar</type>
     </dependency>
-
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-distcp</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-distcp</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-tests</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d33e928f/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
index afe7242..ad321bd 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
@@ -20,18 +20,12 @@ package org.apache.hadoop.fs.aliyun.oss;
 
 import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
 
-import com.aliyun.oss.ClientException;
-import com.aliyun.oss.common.auth.CredentialsProvider;
-import com.aliyun.oss.common.auth.DefaultCredentialProvider;
-import com.aliyun.oss.common.auth.DefaultCredentials;
-import java.io.ByteArrayInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -39,30 +33,13 @@ import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.aliyun.oss.AliyunOSSUtils.UserInfo;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.util.Progressable;
 
-import com.aliyun.oss.ClientConfiguration;
-import com.aliyun.oss.OSSClient;
-import com.aliyun.oss.OSSException;
-import com.aliyun.oss.common.comm.Protocol;
-import com.aliyun.oss.model.AbortMultipartUploadRequest;
-import com.aliyun.oss.model.CannedAccessControlList;
-import com.aliyun.oss.model.CompleteMultipartUploadRequest;
-import com.aliyun.oss.model.CompleteMultipartUploadResult;
-import com.aliyun.oss.model.CopyObjectResult;
-import com.aliyun.oss.model.DeleteObjectsRequest;
-import com.aliyun.oss.model.InitiateMultipartUploadRequest;
-import com.aliyun.oss.model.InitiateMultipartUploadResult;
-import com.aliyun.oss.model.ListObjectsRequest;
 import com.aliyun.oss.model.OSSObjectSummary;
 import com.aliyun.oss.model.ObjectListing;
 import com.aliyun.oss.model.ObjectMetadata;
-import com.aliyun.oss.model.PartETag;
-import com.aliyun.oss.model.UploadPartCopyRequest;
-import com.aliyun.oss.model.UploadPartCopyResult;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -75,12 +52,8 @@ public class AliyunOSSFileSystem extends FileSystem {
       LoggerFactory.getLogger(AliyunOSSFileSystem.class);
   private URI uri;
   private Path workingDir;
-  private OSSClient ossClient;
-  private String bucketName;
-  private long uploadPartSize;
-  private long multipartThreshold;
+  private AliyunOSSFileSystemStore store;
   private int maxKeys;
-  private String serverSideEncryptionAlgorithm;
 
   @Override
   public FSDataOutputStream append(Path path, int bufferSize,
@@ -91,9 +64,7 @@ public class AliyunOSSFileSystem extends FileSystem {
   @Override
   public void close() throws IOException {
     try {
-      if (ossClient != null) {
-        ossClient.shutdown();
-      }
+      store.close();
     } finally {
       super.close();
     }
@@ -125,23 +96,33 @@ public class AliyunOSSFileSystem extends FileSystem {
     }
 
     return new FSDataOutputStream(new AliyunOSSOutputStream(getConf(),
-        ossClient, bucketName, key, progress, statistics,
-        serverSideEncryptionAlgorithm), (Statistics)(null));
+        store, key, progress, statistics), (Statistics)(null));
   }
 
   @Override
   public boolean delete(Path path, boolean recursive) throws IOException {
-    FileStatus status;
     try {
-      status = getFileStatus(path);
+      return innerDelete(getFileStatus(path), recursive);
     } catch (FileNotFoundException e) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Couldn't delete " + path + ": Does not exist!");
-      }
+      LOG.debug("Couldn't delete {} - does not exist", path);
       return false;
     }
+  }
 
-    String key = pathToKey(status.getPath());
+  /**
+   * Delete an object. See {@link #delete(Path, boolean)}.
+   *
+   * @param status fileStatus object
+   * @param recursive if path is a directory and set to
+   * true, the directory is deleted else throws an exception. In
+   * case of a file the recursive can be set to either true or false.
+   * @return  true if delete is successful else false.
+   * @throws IOException due to inability to delete a directory or file.
+   */
+  private boolean innerDelete(FileStatus status, boolean recursive)
+      throws IOException {
+    Path f = status.getPath();
+    String key = pathToKey(f);
     if (status.isDirectory()) {
       if (!key.endsWith("/")) {
         key += "/";
@@ -150,54 +131,34 @@ public class AliyunOSSFileSystem extends FileSystem {
         FileStatus[] statuses = listStatus(status.getPath());
         // Check whether it is an empty directory or not
         if (statuses.length > 0) {
-          throw new IOException("Cannot remove directory" + path +
+          throw new IOException("Cannot remove directory " + f +
               ": It is not empty!");
         } else {
           // Delete empty directory without '-r'
-          ossClient.deleteObject(bucketName, key);
-          statistics.incrementWriteOps(1);
+          store.deleteObject(key);
         }
       } else {
-        ListObjectsRequest listRequest = new ListObjectsRequest(bucketName);
-        listRequest.setPrefix(key);
-        listRequest.setMaxKeys(maxKeys);
-
-        while (true) {
-          ObjectListing objects = ossClient.listObjects(listRequest);
-          statistics.incrementReadOps(1);
-          List<String> keysToDelete = new ArrayList<String>();
-          for (OSSObjectSummary objectSummary : objects.getObjectSummaries()) {
-            keysToDelete.add(objectSummary.getKey());
-          }
-          DeleteObjectsRequest deleteRequest =
-              new DeleteObjectsRequest(bucketName);
-          deleteRequest.setKeys(keysToDelete);
-          ossClient.deleteObjects(deleteRequest);
-          statistics.incrementWriteOps(1);
-          if (objects.isTruncated()) {
-            listRequest.setMarker(objects.getNextMarker());
-          } else {
-            break;
-          }
-        }
+        store.deleteDirs(key);
       }
     } else {
-      ossClient.deleteObject(bucketName, key);
-      statistics.incrementWriteOps(1);
+      store.deleteObject(key);
     }
-    //TODO: optimize logic here
+
+    createFakeDirectoryIfNecessary(f);
+    return true;
+  }
+
+  private void createFakeDirectoryIfNecessary(Path f) throws IOException {
     try {
-      Path pPath = status.getPath().getParent();
+      Path pPath = f.getParent();
       FileStatus pStatus = getFileStatus(pPath);
-      if (pStatus.isDirectory()) {
-        return true;
-      } else {
+      if (pStatus.isFile()) {
         throw new IOException("Path " + pPath +
             " is assumed to be a directory!");
       }
     } catch (FileNotFoundException fnfe) {
       // Make sure the parent directory exists
-      return mkdir(bucketName, pathToKey(status.getPath().getParent()));
+      mkdir(pathToKey(f.getParent()));
     }
   }
 
@@ -211,22 +172,15 @@ public class AliyunOSSFileSystem extends FileSystem {
       return new FileStatus(0, true, 1, 0, 0, qualifiedPath);
     }
 
-    ObjectMetadata meta = getObjectMetadata(key);
+    ObjectMetadata meta = store.getObjectMetadata(key);
     // If key not found and key does not end with "/"
     if (meta == null && !key.endsWith("/")) {
       // Case: dir + "/"
       key += "/";
-      meta = getObjectMetadata(key);
+      meta = store.getObjectMetadata(key);
     }
     if (meta == null) {
-      // Case: dir + "/" + file
-      ListObjectsRequest listRequest = new ListObjectsRequest(bucketName);
-      listRequest.setPrefix(key);
-      listRequest.setDelimiter("/");
-      listRequest.setMaxKeys(1);
-
-      ObjectListing listing = ossClient.listObjects(listRequest);
-      statistics.incrementReadOps(1);
+      ObjectListing listing = store.listObjects(key, 1, "/", null);
       if (!listing.getObjectSummaries().isEmpty() ||
           !listing.getCommonPrefixes().isEmpty()) {
         return new FileStatus(0, true, 1, 0, 0, qualifiedPath);
@@ -242,22 +196,6 @@ public class AliyunOSSFileSystem extends FileSystem {
     }
   }
 
-  /**
-   * Return object metadata given object key.
-   *
-   * @param key object key
-   * @return return null if key does not exist
-   */
-  private ObjectMetadata getObjectMetadata(String key) {
-    try {
-      return ossClient.getObjectMetadata(bucketName, key);
-    } catch (OSSException osse) {
-      return null;
-    } finally {
-      statistics.incrementReadOps(1);
-    }
-  }
-
   @Override
   public String getScheme() {
     return "oss";
@@ -288,7 +226,6 @@ public class AliyunOSSFileSystem extends FileSystem {
    * Initialize new FileSystem.
    *
    * @param name the uri of the file system, including host, port, etc.
-   *
    * @param conf configuration of the file system
    * @throws IOException IO problems
    */
@@ -296,154 +233,16 @@ public class AliyunOSSFileSystem extends FileSystem {
     super.initialize(name, conf);
 
     uri = java.net.URI.create(name.getScheme() + "://" + name.getAuthority());
-    workingDir =
-        new Path("/user",
-            System.getProperty("user.name")).makeQualified(uri, null);
-
-    bucketName = name.getHost();
-
-    ClientConfiguration clientConf = new ClientConfiguration();
-    clientConf.setMaxConnections(conf.getInt(MAXIMUM_CONNECTIONS_KEY,
-        MAXIMUM_CONNECTIONS_DEFAULT));
-    boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS_KEY,
-        SECURE_CONNECTIONS_DEFAULT);
-    clientConf.setProtocol(secureConnections ? Protocol.HTTPS : Protocol.HTTP);
-    clientConf.setMaxErrorRetry(conf.getInt(MAX_ERROR_RETRIES_KEY,
-        MAX_ERROR_RETRIES_DEFAULT));
-    clientConf.setConnectionTimeout(conf.getInt(ESTABLISH_TIMEOUT_KEY,
-        ESTABLISH_TIMEOUT_DEFAULT));
-    clientConf.setSocketTimeout(conf.getInt(SOCKET_TIMEOUT_KEY,
-        SOCKET_TIMEOUT_DEFAULT));
-
-    String proxyHost = conf.getTrimmed(PROXY_HOST_KEY, "");
-    int proxyPort = conf.getInt(PROXY_PORT_KEY, -1);
-    if (!proxyHost.isEmpty()) {
-      clientConf.setProxyHost(proxyHost);
-      if (proxyPort >= 0) {
-        clientConf.setProxyPort(proxyPort);
-      } else {
-        if (secureConnections) {
-          LOG.warn("Proxy host set without port. Using HTTPS default 443");
-          clientConf.setProxyPort(443);
-        } else {
-          LOG.warn("Proxy host set without port. Using HTTP default 80");
-          clientConf.setProxyPort(80);
-        }
-      }
-      String proxyUsername = conf.getTrimmed(PROXY_USERNAME_KEY);
-      String proxyPassword = conf.getTrimmed(PROXY_PASSWORD_KEY);
-      if ((proxyUsername == null) != (proxyPassword == null)) {
-        String msg = "Proxy error: " + PROXY_USERNAME_KEY + " or " +
-            PROXY_PASSWORD_KEY + " set without the other.";
-        LOG.error(msg);
-        throw new IllegalArgumentException(msg);
-      }
-      clientConf.setProxyUsername(proxyUsername);
-      clientConf.setProxyPassword(proxyPassword);
-      clientConf.setProxyDomain(conf.getTrimmed(PROXY_DOMAIN_KEY));
-      clientConf.setProxyWorkstation(conf.getTrimmed(PROXY_WORKSTATION_KEY));
-    } else if (proxyPort >= 0) {
-      String msg = "Proxy error: " + PROXY_PORT_KEY + " set without " +
-          PROXY_HOST_KEY;
-      LOG.error(msg);
-      throw new IllegalArgumentException(msg);
-    }
-
-    String endPoint = conf.getTrimmed(ENDPOINT_KEY, "");
-    ossClient =
-        new OSSClient(endPoint, getCredentialsProvider(name, conf), clientConf);
+    workingDir = new Path("/user",
+        System.getProperty("user.name")).makeQualified(uri, null);
 
+    store = new AliyunOSSFileSystemStore();
+    store.initialize(name, conf, statistics);
     maxKeys = conf.getInt(MAX_PAGING_KEYS_KEY, MAX_PAGING_KEYS_DEFAULT);
-    uploadPartSize = conf.getLong(MULTIPART_UPLOAD_SIZE_KEY,
-        MULTIPART_UPLOAD_SIZE_DEFAULT);
-    multipartThreshold = conf.getLong(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY,
-        MIN_MULTIPART_UPLOAD_THRESHOLD_DEFAULT);
-
-    if (uploadPartSize < 5 * 1024 * 1024) {
-      LOG.warn(MULTIPART_UPLOAD_SIZE_KEY + " must be at least 5 MB");
-      uploadPartSize = 5 * 1024 * 1024;
-    }
-
-    if (multipartThreshold < 5 * 1024 * 1024) {
-      LOG.warn(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY + " must be at least 5 MB");
-      multipartThreshold = 5 * 1024 * 1024;
-    }
-
-    if (multipartThreshold > 1024 * 1024 * 1024) {
-      LOG.warn(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY + " must be less than 1 GB");
-      multipartThreshold = 1024 * 1024 * 1024;
-    }
-
-    String cannedACLName = conf.get(CANNED_ACL_KEY, CANNED_ACL_DEFAULT);
-    if (!cannedACLName.isEmpty()) {
-      CannedAccessControlList cannedACL =
-          CannedAccessControlList.valueOf(cannedACLName);
-      ossClient.setBucketAcl(bucketName, cannedACL);
-    }
-
-    serverSideEncryptionAlgorithm =
-        conf.get(SERVER_SIDE_ENCRYPTION_ALGORITHM_KEY, "");
-
     setConf(conf);
   }
 
   /**
-   * Create the default credential provider, or load in one explicitly
-   * identified in the configuration.
-   * @param name the uri of the file system
-   * @param conf configuration
-   * @return a credential provider
-   * @throws IOException on any problem. Class construction issues may be
-   * nested inside the IOE.
-   */
-  private CredentialsProvider getCredentialsProvider(URI name,
-      Configuration conf) throws IOException {
-    CredentialsProvider credentials;
-
-    String className = conf.getTrimmed(ALIYUN_OSS_CREDENTIALS_PROVIDER_KEY);
-    if (StringUtils.isEmpty(className)) {
-      Configuration newConf =
-          ProviderUtils.excludeIncompatibleCredentialProviders(conf,
-              AliyunOSSFileSystem.class);
-      String accessKey =
-          AliyunOSSUtils.getPassword(newConf, ACCESS_KEY,
-              UserInfo.EMPTY.getUser());
-      String secretKey =
-          AliyunOSSUtils.getPassword(newConf, SECRET_KEY,
-              UserInfo.EMPTY.getPassword());
-      credentials =
-          new DefaultCredentialProvider(
-              new DefaultCredentials(accessKey, secretKey));
-
-    } else {
-      try {
-        LOG.debug("Credential provider class is:" + className);
-        Class<?> credClass = Class.forName(className);
-        try {
-          credentials =
-              (CredentialsProvider)credClass.getDeclaredConstructor(
-                  URI.class, Configuration.class).newInstance(this.uri, conf);
-        } catch (NoSuchMethodException | SecurityException e) {
-          credentials =
-              (CredentialsProvider)credClass.getDeclaredConstructor()
-              .newInstance();
-        }
-      } catch (ClassNotFoundException e) {
-        throw new IOException(className + " not found.", e);
-      } catch (NoSuchMethodException | SecurityException e) {
-        throw new IOException(String.format("%s constructor exception.  A " +
-            "class specified in %s must provide an accessible constructor " +
-            "accepting URI and Configuration, or an accessible default " +
-            "constructor.", className, ALIYUN_OSS_CREDENTIALS_PROVIDER_KEY), e);
-      } catch (ReflectiveOperationException | IllegalArgumentException e) {
-        throw new IOException(className + " instantiation exception.", e);
-      }
-    }
-
-    return credentials;
-  }
-
-  /**
    * Check if OSS object represents a directory.
    *
    * @param name object key
@@ -456,10 +255,10 @@ public class AliyunOSSFileSystem extends FileSystem {
   }
 
   /**
-   * Turns a path (relative or otherwise) into an OSS key.
+   * Turn a path (relative or otherwise) into an OSS key.
    *
-   * @param path the path of the file
-   * @return the key of the object that represent the file
+   * @param path the path of the file.
+   * @return the key of the object that represents the file.
    */
   private String pathToKey(Path path) {
     if (!path.isAbsolute()) {
@@ -492,18 +291,12 @@ public class AliyunOSSFileSystem extends FileSystem {
         key = key + "/";
       }
 
-      ListObjectsRequest listObjectsRequest =
-          new ListObjectsRequest(bucketName);
-      listObjectsRequest.setPrefix(key);
-      listObjectsRequest.setDelimiter("/");
-      listObjectsRequest.setMaxKeys(maxKeys);
-
       if (LOG.isDebugEnabled()) {
         LOG.debug("listStatus: doing listObjects for directory " + key);
       }
 
+      ObjectListing objects = store.listObjects(key, maxKeys, "/", null);
       while (true) {
-        ObjectListing objects = ossClient.listObjects(listObjectsRequest);
         statistics.incrementReadOps(1);
         for (OSSObjectSummary objectSummary : objects.getObjectSummaries()) {
           Path keyPath = keyToPath(objectSummary.getKey())
@@ -539,7 +332,8 @@ public class AliyunOSSFileSystem extends FileSystem {
           if (LOG.isDebugEnabled()) {
             LOG.debug("listStatus: list truncated - getting next batch");
           }
-          listObjectsRequest.setMarker(objects.getNextMarker());
+          objects = store.listObjects(key, maxKeys, "/",
+              objects.getNextMarker());
           statistics.incrementReadOps(1);
         } else {
           break;
@@ -558,27 +352,17 @@ public class AliyunOSSFileSystem extends FileSystem {
   /**
    * Used to create an empty file that represents an empty directory.
    *
-   * @param bucket the bucket this directory belongs to
    * @param key directory path
-   * @return true if directory successfully created
+   * @return true if directory is successfully created
    * @throws IOException
    */
-  private boolean mkdir(final String bucket, final String key)
-      throws IOException {
+  private boolean mkdir(final String key) throws IOException {
     String dirName = key;
-    ObjectMetadata dirMeta = new ObjectMetadata();
-    byte[] buffer = new byte[0];
-    ByteArrayInputStream in = new ByteArrayInputStream(buffer);
-    dirMeta.setContentLength(0);
     if (!key.endsWith("/")) {
       dirName += "/";
     }
-    try {
-      ossClient.putObject(bucket, dirName, in, dirMeta);
-      return true;
-    } finally {
-      in.close();
-    }
+    store.storeEmptyFile(dirName);
+    return true;
   }
 
   @Override
@@ -595,14 +379,14 @@ public class AliyunOSSFileSystem extends FileSystem {
     } catch (FileNotFoundException e) {
       validatePath(path);
       String key = pathToKey(path);
-      return mkdir(bucketName, key);
+      return mkdir(key);
     }
   }
 
   /**
    * Check whether the path is a valid path.
    *
-   * @param path the path to be checked
+   * @param path the path to be checked.
    * @throws IOException
    */
   private void validatePath(Path path) throws IOException {
@@ -631,8 +415,8 @@ public class AliyunOSSFileSystem extends FileSystem {
           " because it is a directory");
     }
 
-    return new FSDataInputStream(new AliyunOSSInputStream(getConf(), ossClient,
-        bucketName, pathToKey(path), fileStatus.getLen(), statistics));
+    return new FSDataInputStream(new AliyunOSSInputStream(getConf(), store,
+        pathToKey(path), fileStatus.getLen(), statistics));
   }
 
   @Override
@@ -696,126 +480,31 @@ public class AliyunOSSFileSystem extends FileSystem {
     } else {
       copyFile(srcPath, dstPath);
     }
-    if (srcPath.equals(dstPath)) {
-      return true;
-    } else {
-      return delete(srcPath, true);
-    }
+
+    return srcPath.equals(dstPath) || delete(srcPath, true);
   }
 
   /**
    * Copy file from source path to destination path.
-   * (the caller should make sure srcPath is a file and dstPath is valid.)
+   * (the caller should make sure srcPath is a file and dstPath is valid)
    *
-   * @param srcPath source path
-   * @param dstPath destination path
-   * @return true if successfully copied
+   * @param srcPath source path.
+   * @param dstPath destination path.
+   * @return true if file is successfully copied.
    */
   private boolean copyFile(Path srcPath, Path dstPath) {
     String srcKey = pathToKey(srcPath);
     String dstKey = pathToKey(dstPath);
-    return copyFile(srcKey, dstKey);
-  }
-
-  /**
-   * Copy an object from source key to destination key.
-   *
-   * @param srcKey source key
-   * @param dstKey destination key
-   * @return true if successfully copied
-   */
-  private boolean copyFile(String srcKey, String dstKey) {
-    ObjectMetadata objectMeta =
-        ossClient.getObjectMetadata(bucketName, srcKey);
-    long dataLen = objectMeta.getContentLength();
-    if (dataLen <= multipartThreshold) {
-      return singleCopy(srcKey, dstKey);
-    } else {
-      return multipartCopy(srcKey, dataLen, dstKey);
-    }
-  }
-
-  /**
-   * Use single copy to copy an oss object.
-   *
-   * @param srcKey source key
-   * @param dstKey destination key
-   * @return true if successfully copied
-   * (the caller should make sure srcPath is a file and dstPath is valid)
-   */
-  private boolean singleCopy(String srcKey, String dstKey) {
-    CopyObjectResult copyResult =
-        ossClient.copyObject(bucketName, srcKey, bucketName, dstKey);
-    LOG.debug(copyResult.getETag());
-    return true;
-  }
-
-  /**
-   * Use multipart copy to copy an oss object.
-   * (the caller should make sure srcPath is a file and dstPath is valid)
-   *
-   * @param srcKey source key
-   * @param dataLen data size of the object to copy
-   * @param dstKey destination key
-   * @return true if successfully copied, or false if upload is aborted
-   */
-  private boolean multipartCopy(String srcKey, long dataLen, String dstKey) {
-    int partNum = (int)(dataLen / uploadPartSize);
-    if (dataLen % uploadPartSize != 0) {
-      partNum++;
-    }
-    InitiateMultipartUploadRequest initiateMultipartUploadRequest =
-        new InitiateMultipartUploadRequest(bucketName, dstKey);
-    ObjectMetadata meta = new ObjectMetadata();
-    if (!serverSideEncryptionAlgorithm.isEmpty()) {
-      meta.setServerSideEncryption(serverSideEncryptionAlgorithm);
-    }
-    initiateMultipartUploadRequest.setObjectMetadata(meta);
-    InitiateMultipartUploadResult initiateMultipartUploadResult =
-        ossClient.initiateMultipartUpload(initiateMultipartUploadRequest);
-    String uploadId = initiateMultipartUploadResult.getUploadId();
-    List<PartETag> partETags = new ArrayList<PartETag>();
-    try {
-      for (int i = 0; i < partNum; i++) {
-        long skipBytes = uploadPartSize * i;
-        long size = (uploadPartSize < dataLen - skipBytes) ?
-            uploadPartSize : dataLen - skipBytes;
-        UploadPartCopyRequest partCopyRequest = new UploadPartCopyRequest();
-        partCopyRequest.setSourceBucketName(bucketName);
-        partCopyRequest.setSourceKey(srcKey);
-        partCopyRequest.setBucketName(bucketName);
-        partCopyRequest.setKey(dstKey);
-        partCopyRequest.setUploadId(uploadId);
-        partCopyRequest.setPartSize(size);
-        partCopyRequest.setBeginIndex(skipBytes);
-        partCopyRequest.setPartNumber(i + 1);
-        UploadPartCopyResult partCopyResult =
-            ossClient.uploadPartCopy(partCopyRequest);
-        statistics.incrementWriteOps(1);
-        partETags.add(partCopyResult.getPartETag());
-      }
-      CompleteMultipartUploadRequest completeMultipartUploadRequest =
-          new CompleteMultipartUploadRequest(bucketName, dstKey,
-          uploadId, partETags);
-      CompleteMultipartUploadResult completeMultipartUploadResult =
-          ossClient.completeMultipartUpload(completeMultipartUploadRequest);
-      LOG.debug(completeMultipartUploadResult.getETag());
-      return true;
-    } catch (OSSException | ClientException e) {
-      AbortMultipartUploadRequest abortMultipartUploadRequest =
-          new AbortMultipartUploadRequest(bucketName, dstKey, uploadId);
-      ossClient.abortMultipartUpload(abortMultipartUploadRequest);
-      return false;
-    }
+    return store.copyFile(srcKey, dstKey);
   }
 
   /**
    * Copy a directory from source path to destination path.
    * (the caller should make sure srcPath is a directory, and dstPath is valid)
    *
-   * @param srcPath source path
-   * @param dstPath destination path
-   * @return true if successfully copied
+   * @param srcPath source path.
+   * @param dstPath destination path.
+   * @return true if directory is successfully copied.
    */
   private boolean copyDirectory(Path srcPath, Path dstPath) {
     String srcKey = pathToKey(srcPath);
@@ -835,21 +524,18 @@ public class AliyunOSSFileSystem extends FileSystem {
       return false;
     }
 
-    ListObjectsRequest listObjectsRequest = new ListObjectsRequest(bucketName);
-    listObjectsRequest.setPrefix(srcKey);
-    listObjectsRequest.setMaxKeys(maxKeys);
-
-    ObjectListing objects = ossClient.listObjects(listObjectsRequest);
+    ObjectListing objects = store.listObjects(srcKey, maxKeys, null, null);
     statistics.incrementReadOps(1);
     // Copy files from src folder to dst
     while (true) {
       for (OSSObjectSummary objectSummary : objects.getObjectSummaries()) {
         String newKey =
             dstKey.concat(objectSummary.getKey().substring(srcKey.length()));
-        copyFile(objectSummary.getKey(), newKey);
+        store.copyFile(objectSummary.getKey(), newKey);
       }
       if (objects.isTruncated()) {
-        listObjectsRequest.setMarker(objects.getNextMarker());
+        objects = store.listObjects(srcKey, maxKeys, null,
+            objects.getNextMarker());
         statistics.incrementReadOps(1);
       } else {
         break;
@@ -863,4 +549,7 @@ public class AliyunOSSFileSystem extends FileSystem {
     this.workingDir = dir;
   }
 
+  public AliyunOSSFileSystemStore getStore() {
+    return store;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d33e928f/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
new file mode 100644
index 0000000..b3cd1bd
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
@@ -0,0 +1,486 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.aliyun.oss;
+
+import com.aliyun.oss.ClientConfiguration;
+import com.aliyun.oss.ClientException;
+import com.aliyun.oss.OSSClient;
+import com.aliyun.oss.OSSException;
+import com.aliyun.oss.common.auth.CredentialsProvider;
+import com.aliyun.oss.common.comm.Protocol;
+import com.aliyun.oss.model.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.*;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
+
+/**
+ * Core implementation of Aliyun OSS Filesystem for Hadoop.
+ * Provides the bridging logic between Hadoop's abstract filesystem and
+ * Aliyun OSS.
+ */
+public class AliyunOSSFileSystemStore {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(AliyunOSSFileSystemStore.class);
+  private FileSystem.Statistics statistics;
+  private OSSClient ossClient;
+  private String bucketName;
+  private long uploadPartSize;
+  private long multipartThreshold;
+  private long partSize;
+  private int maxKeys;
+  private String serverSideEncryptionAlgorithm;
+
+  public void initialize(URI uri, Configuration conf,
+                         FileSystem.Statistics stat) throws IOException {
+    statistics = stat;
+    ClientConfiguration clientConf = new ClientConfiguration();
+    clientConf.setMaxConnections(conf.getInt(MAXIMUM_CONNECTIONS_KEY,
+        MAXIMUM_CONNECTIONS_DEFAULT));
+    boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS_KEY,
+        SECURE_CONNECTIONS_DEFAULT);
+    clientConf.setProtocol(secureConnections ? Protocol.HTTPS : Protocol.HTTP);
+    clientConf.setMaxErrorRetry(conf.getInt(MAX_ERROR_RETRIES_KEY,
+        MAX_ERROR_RETRIES_DEFAULT));
+    clientConf.setConnectionTimeout(conf.getInt(ESTABLISH_TIMEOUT_KEY,
+        ESTABLISH_TIMEOUT_DEFAULT));
+    clientConf.setSocketTimeout(conf.getInt(SOCKET_TIMEOUT_KEY,
+        SOCKET_TIMEOUT_DEFAULT));
+
+    String proxyHost = conf.getTrimmed(PROXY_HOST_KEY, "");
+    int proxyPort = conf.getInt(PROXY_PORT_KEY, -1);
+    if (!proxyHost.isEmpty()) {
+      clientConf.setProxyHost(proxyHost);
+      if (proxyPort >= 0) {
+        clientConf.setProxyPort(proxyPort);
+      } else {
+        if (secureConnections) {
+          LOG.warn("Proxy host set without port. Using HTTPS default 443");
+          clientConf.setProxyPort(443);
+        } else {
+          LOG.warn("Proxy host set without port. Using HTTP default 80");
+          clientConf.setProxyPort(80);
+        }
+      }
+      String proxyUsername = conf.getTrimmed(PROXY_USERNAME_KEY);
+      String proxyPassword = conf.getTrimmed(PROXY_PASSWORD_KEY);
+      if ((proxyUsername == null) != (proxyPassword == null)) {
+        String msg = "Proxy error: " + PROXY_USERNAME_KEY + " or " +
+            PROXY_PASSWORD_KEY + " set without the other.";
+        LOG.error(msg);
+        throw new IllegalArgumentException(msg);
+      }
+      clientConf.setProxyUsername(proxyUsername);
+      clientConf.setProxyPassword(proxyPassword);
+      clientConf.setProxyDomain(conf.getTrimmed(PROXY_DOMAIN_KEY));
+      clientConf.setProxyWorkstation(conf.getTrimmed(PROXY_WORKSTATION_KEY));
+    } else if (proxyPort >= 0) {
+      String msg = "Proxy error: " + PROXY_PORT_KEY + " set without " +
+          PROXY_HOST_KEY;
+      LOG.error(msg);
+      throw new IllegalArgumentException(msg);
+    }
+
+    String endPoint = conf.getTrimmed(ENDPOINT_KEY, "");
+    CredentialsProvider provider =
+        AliyunOSSUtils.getCredentialsProvider(uri, conf);
+    ossClient = new OSSClient(endPoint, provider, clientConf);
+    uploadPartSize = conf.getLong(MULTIPART_UPLOAD_SIZE_KEY,
+        MULTIPART_UPLOAD_SIZE_DEFAULT);
+    multipartThreshold = conf.getLong(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY,
+        MIN_MULTIPART_UPLOAD_THRESHOLD_DEFAULT);
+    partSize = conf.getLong(MULTIPART_UPLOAD_SIZE_KEY,
+        MULTIPART_UPLOAD_SIZE_DEFAULT);
+    if (partSize < MIN_MULTIPART_UPLOAD_PART_SIZE) {
+      partSize = MIN_MULTIPART_UPLOAD_PART_SIZE;
+    }
+    serverSideEncryptionAlgorithm =
+        conf.get(SERVER_SIDE_ENCRYPTION_ALGORITHM_KEY, "");
+
+    if (uploadPartSize < 5 * 1024 * 1024) {
+      LOG.warn(MULTIPART_UPLOAD_SIZE_KEY + " must be at least 5 MB");
+      uploadPartSize = 5 * 1024 * 1024;
+    }
+
+    if (multipartThreshold < 5 * 1024 * 1024) {
+      LOG.warn(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY + " must be at least 5 MB");
+      multipartThreshold = 5 * 1024 * 1024;
+    }
+
+    if (multipartThreshold > 1024 * 1024 * 1024) {
+      LOG.warn(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY + " must be less than 1 GB");
+      multipartThreshold = 1024 * 1024 * 1024;
+    }
+
+    String cannedACLName = conf.get(CANNED_ACL_KEY, CANNED_ACL_DEFAULT);
+    if (!cannedACLName.isEmpty()) {
+      CannedAccessControlList cannedACL =
+          CannedAccessControlList.valueOf(cannedACLName);
+      ossClient.setBucketAcl(bucketName, cannedACL);
+    }
+
+    maxKeys = conf.getInt(MAX_PAGING_KEYS_KEY, MAX_PAGING_KEYS_DEFAULT);
+    bucketName = uri.getHost();
+  }
+
+  /**
+   * Delete an object, and update write operation statistics.
+   *
+   * @param key key to blob to delete.
+   */
+  public void deleteObject(String key) {
+    ossClient.deleteObject(bucketName, key);
+    statistics.incrementWriteOps(1);
+  }
+
+  /**
+   * Delete a list of keys, and update write operation statistics.
+   *
+   * @param keysToDelete collection of keys to delete.
+   */
+  public void deleteObjects(List<String> keysToDelete) {
+    DeleteObjectsRequest deleteRequest =
+        new DeleteObjectsRequest(bucketName);
+    deleteRequest.setKeys(keysToDelete);
+    ossClient.deleteObjects(deleteRequest);
+    statistics.incrementWriteOps(keysToDelete.size());
+  }
+
+  /**
+   * Delete a directory from Aliyun OSS.
+   *
+   * @param key directory key to delete.
+   */
+  public void deleteDirs(String key) {
+    ListObjectsRequest listRequest = new ListObjectsRequest(bucketName);
+    listRequest.setPrefix(key);
+    listRequest.setMaxKeys(maxKeys);
+
+    while (true) {
+      ObjectListing objects = ossClient.listObjects(listRequest);
+      statistics.incrementReadOps(1);
+      List<String> keysToDelete = new ArrayList<String>();
+      for (OSSObjectSummary objectSummary : objects.getObjectSummaries()) {
+        keysToDelete.add(objectSummary.getKey());
+      }
+      deleteObjects(keysToDelete);
+      if (objects.isTruncated()) {
+        listRequest.setMarker(objects.getNextMarker());
+      } else {
+        break;
+      }
+    }
+  }
+
+  /**
+   * Return metadata of a given object key.
+   *
+   * @param key object key.
+   * @return return null if key does not exist.
+   */
+  public ObjectMetadata getObjectMetadata(String key) {
+    try {
+      return ossClient.getObjectMetadata(bucketName, key);
+    } catch (OSSException osse) {
+      return null;
+    } finally {
+      statistics.incrementReadOps(1);
+    }
+  }
+
+  /**
+   * Upload an empty file as an OSS object, using single upload.
+   *
+   * @param key object key.
+   * @throws IOException if failed to upload object.
+   */
+  public void storeEmptyFile(String key) throws IOException {
+    ObjectMetadata dirMeta = new ObjectMetadata();
+    byte[] buffer = new byte[0];
+    ByteArrayInputStream in = new ByteArrayInputStream(buffer);
+    dirMeta.setContentLength(0);
+    try {
+      ossClient.putObject(bucketName, key, in, dirMeta);
+    } finally {
+      in.close();
+    }
+  }
+
+  /**
+   * Copy an object from source key to destination key.
+   *
+   * @param srcKey source key.
+   * @param dstKey destination key.
+   * @return true if file is successfully copied.
+   */
+  public boolean copyFile(String srcKey, String dstKey) {
+    ObjectMetadata objectMeta =
+        ossClient.getObjectMetadata(bucketName, srcKey);
+    long contentLength = objectMeta.getContentLength();
+    if (contentLength <= multipartThreshold) {
+      return singleCopy(srcKey, dstKey);
+    } else {
+      return multipartCopy(srcKey, contentLength, dstKey);
+    }
+  }
+
+  /**
+   * Use single copy to copy an OSS object.
+   * (The caller should make sure srcPath is a file and dstPath is valid)
+   *
+   * @param srcKey source key.
+   * @param dstKey destination key.
+   * @return true if object is successfully copied.
+   */
+  private boolean singleCopy(String srcKey, String dstKey) {
+    CopyObjectResult copyResult =
+        ossClient.copyObject(bucketName, srcKey, bucketName, dstKey);
+    LOG.debug(copyResult.getETag());
+    return true;
+  }
+
+  /**
+   * Use multipart copy to copy an OSS object.
+   * (The caller should make sure srcPath is a file and dstPath is valid)
+   *
+   * @param srcKey source key.
+   * @param contentLength data size of the object to copy.
+   * @param dstKey destination key.
+   * @return true if success, or false if upload is aborted.
+   */
+  private boolean multipartCopy(String srcKey, long contentLength,
+      String dstKey) {
+    long realPartSize =
+        AliyunOSSUtils.calculatePartSize(contentLength, uploadPartSize);
+    int partNum = (int) (contentLength / realPartSize);
+    if (contentLength % realPartSize != 0) {
+      partNum++;
+    }
+    InitiateMultipartUploadRequest initiateMultipartUploadRequest =
+        new InitiateMultipartUploadRequest(bucketName, dstKey);
+    ObjectMetadata meta = new ObjectMetadata();
+    if (!serverSideEncryptionAlgorithm.isEmpty()) {
+      meta.setServerSideEncryption(serverSideEncryptionAlgorithm);
+    }
+    initiateMultipartUploadRequest.setObjectMetadata(meta);
+    InitiateMultipartUploadResult initiateMultipartUploadResult =
+        ossClient.initiateMultipartUpload(initiateMultipartUploadRequest);
+    String uploadId = initiateMultipartUploadResult.getUploadId();
+    List<PartETag> partETags = new ArrayList<PartETag>();
+    try {
+      for (int i = 0; i < partNum; i++) {
+        long skipBytes = realPartSize * i;
+        long size = (realPartSize < contentLength - skipBytes) ?
+            realPartSize : contentLength - skipBytes;
+        UploadPartCopyRequest partCopyRequest = new UploadPartCopyRequest();
+        partCopyRequest.setSourceBucketName(bucketName);
+        partCopyRequest.setSourceKey(srcKey);
+        partCopyRequest.setBucketName(bucketName);
+        partCopyRequest.setKey(dstKey);
+        partCopyRequest.setUploadId(uploadId);
+        partCopyRequest.setPartSize(size);
+        partCopyRequest.setBeginIndex(skipBytes);
+        partCopyRequest.setPartNumber(i + 1);
+        UploadPartCopyResult partCopyResult =
+            ossClient.uploadPartCopy(partCopyRequest);
+        statistics.incrementWriteOps(1);
+        partETags.add(partCopyResult.getPartETag());
+      }
+      CompleteMultipartUploadRequest completeMultipartUploadRequest =
+          new CompleteMultipartUploadRequest(bucketName, dstKey,
+              uploadId, partETags);
+      CompleteMultipartUploadResult completeMultipartUploadResult =
+          ossClient.completeMultipartUpload(completeMultipartUploadRequest);
+      LOG.debug(completeMultipartUploadResult.getETag());
+      return true;
+    } catch (OSSException | ClientException e) {
+      AbortMultipartUploadRequest abortMultipartUploadRequest =
+          new AbortMultipartUploadRequest(bucketName, dstKey, uploadId);
+      ossClient.abortMultipartUpload(abortMultipartUploadRequest);
+      return false;
+    }
+  }
+
+  /**
+   * Upload a file as an OSS object, using single upload.
+   *
+   * @param key object key.
+   * @param file local file to upload.
+   * @throws IOException if failed to upload object.
+   */
+  public void uploadObject(String key, File file) throws IOException {
+    File object = file.getAbsoluteFile();
+    FileInputStream fis = new FileInputStream(object);
+    ObjectMetadata meta = new ObjectMetadata();
+    meta.setContentLength(object.length());
+    if (!serverSideEncryptionAlgorithm.isEmpty()) {
+      meta.setServerSideEncryption(serverSideEncryptionAlgorithm);
+    }
+    try {
+      PutObjectResult result = ossClient.putObject(bucketName, key, fis, meta);
+      LOG.debug(result.getETag());
+      statistics.incrementWriteOps(1);
+    } finally {
+      fis.close();
+    }
+  }
+
+  /**
+   * Upload a file as an OSS object, using multipart upload.
+   *
+   * @param key object key.
+   * @param file local file to upload.
+   * @throws IOException if failed to upload object.
+   */
+  public void multipartUploadObject(String key, File file) throws IOException {
+    File object = file.getAbsoluteFile();
+    long dataLen = object.length();
+    long realPartSize = AliyunOSSUtils.calculatePartSize(dataLen, partSize);
+    int partNum = (int) (dataLen / realPartSize);
+    if (dataLen % realPartSize != 0) {
+      partNum += 1;
+    }
+
+    InitiateMultipartUploadRequest initiateMultipartUploadRequest =
+        new InitiateMultipartUploadRequest(bucketName, key);
+    ObjectMetadata meta = new ObjectMetadata();
+    if (!serverSideEncryptionAlgorithm.isEmpty()) {
+      meta.setServerSideEncryption(serverSideEncryptionAlgorithm);
+    }
+    initiateMultipartUploadRequest.setObjectMetadata(meta);
+    InitiateMultipartUploadResult initiateMultipartUploadResult =
+        ossClient.initiateMultipartUpload(initiateMultipartUploadRequest);
+    List<PartETag> partETags = new ArrayList<PartETag>();
+    String uploadId = initiateMultipartUploadResult.getUploadId();
+
+    try {
+      for (int i = 0; i < partNum; i++) {
+        // TODO: Optimize this, avoid opening the object multiple times
+        FileInputStream fis = new FileInputStream(object);
+        try {
+          long skipBytes = realPartSize * i;
+          AliyunOSSUtils.skipFully(fis, skipBytes);
+          long size = (realPartSize < dataLen - skipBytes) ?
+              realPartSize : dataLen - skipBytes;
+          UploadPartRequest uploadPartRequest = new UploadPartRequest();
+          uploadPartRequest.setBucketName(bucketName);
+          uploadPartRequest.setKey(key);
+          uploadPartRequest.setUploadId(uploadId);
+          uploadPartRequest.setInputStream(fis);
+          uploadPartRequest.setPartSize(size);
+          uploadPartRequest.setPartNumber(i + 1);
+          UploadPartResult uploadPartResult =
+              ossClient.uploadPart(uploadPartRequest);
+          statistics.incrementWriteOps(1);
+          partETags.add(uploadPartResult.getPartETag());
+        } finally {
+          fis.close();
+        }
+      }
+      CompleteMultipartUploadRequest completeMultipartUploadRequest =
+          new CompleteMultipartUploadRequest(bucketName, key,
+              uploadId, partETags);
+      CompleteMultipartUploadResult completeMultipartUploadResult =
+          ossClient.completeMultipartUpload(completeMultipartUploadRequest);
+      LOG.debug(completeMultipartUploadResult.getETag());
+    } catch (OSSException | ClientException e) {
+      AbortMultipartUploadRequest abortMultipartUploadRequest =
+          new AbortMultipartUploadRequest(bucketName, key, uploadId);
+      ossClient.abortMultipartUpload(abortMultipartUploadRequest);
+    }
+  }
+
+  /**
+   * list objects.
+   *
+   * @param prefix prefix.
+   * @param maxListingLength max no. of entries
+   * @param delimiter delimiter.
+   * @param marker last key in any previous search.
+   * @return a list of matches.
+   */
+  public ObjectListing listObjects(String prefix, int maxListingLength,
+                                   String delimiter, String marker) {
+    ListObjectsRequest listRequest = new ListObjectsRequest(bucketName);
+    listRequest.setPrefix(prefix);
+    listRequest.setDelimiter(delimiter);
+    listRequest.setMaxKeys(maxListingLength);
+    listRequest.setMarker(marker);
+
+    ObjectListing listing = ossClient.listObjects(listRequest);
+    statistics.incrementReadOps(1);
+    return listing;
+  }
+
+  /**
+   * Retrieve a part of an object.
+   *
+   * @param key the object name that is being retrieved from the Aliyun OSS.
+   * @param byteStart start position.
+   * @param byteEnd end position.
+   * @return This method returns null if the key is not found.
+   */
+  public InputStream retrieve(String key, long byteStart, long byteEnd) {
+    try {
+      GetObjectRequest request = new GetObjectRequest(bucketName, key);
+      request.setRange(byteStart, byteEnd);
+      return ossClient.getObject(request).getObjectContent();
+    } catch (OSSException | ClientException e) {
+      return null;
+    }
+  }
+
+  /**
+   * Close OSS client properly.
+   */
+  public void close() {
+    if (ossClient != null) {
+      ossClient.shutdown();
+      ossClient = null;
+    }
+  }
+
+  /**
+   * Clean up all objects matching the prefix.
+   *
+   * @param prefix Aliyun OSS object prefix.
+   */
+  public void purge(String prefix) {
+    String key;
+    try {
+      ObjectListing objects = listObjects(prefix, maxKeys, null, null);
+      for (OSSObjectSummary object : objects.getObjectSummaries()) {
+        key = object.getKey();
+        ossClient.deleteObject(bucketName, key);
+      }
+
+      for (String dir: objects.getCommonPrefixes()) {
+        deleteDirs(dir);
+      }
+    } catch (OSSException | ClientException e) {
+      LOG.error("Failed to purge " + prefix);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d33e928f/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
index b12e3f0..69265fb 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
@@ -27,12 +27,10 @@ import java.io.InputStream;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 
-import com.aliyun.oss.OSSClient;
-import com.aliyun.oss.model.GetObjectRequest;
-
 /**
  * The input stream for OSS blob system.
  * The class uses multi-part downloading to read data from the object content
@@ -40,27 +38,23 @@ import com.aliyun.oss.model.GetObjectRequest;
  */
 public class AliyunOSSInputStream extends FSInputStream {
   public static final Log LOG = LogFactory.getLog(AliyunOSSInputStream.class);
-  private static final int MAX_RETRIES = 10;
   private final long downloadPartSize;
-
-  private String bucketName;
-  private String key;
-  private OSSClient ossClient;
+  private AliyunOSSFileSystemStore store;
+  private final String key;
   private Statistics statistics;
   private boolean closed;
   private InputStream wrappedStream = null;
-  private long dataLen;
+  private long contentLength;
   private long position;
   private long partRemaining;
 
-  public AliyunOSSInputStream(Configuration conf, OSSClient client,
-      String bucketName, String key, Long dataLen, Statistics statistics)
-      throws IOException {
-    this.bucketName = bucketName;
+  public AliyunOSSInputStream(Configuration conf,
+      AliyunOSSFileSystemStore store, String key, Long contentLength,
+      Statistics statistics) throws IOException {
+    this.store = store;
     this.key = key;
-    ossClient = client;
     this.statistics = statistics;
-    this.dataLen = dataLen;
+    this.contentLength = contentLength;
     downloadPartSize = conf.getLong(MULTIPART_DOWNLOAD_SIZE_KEY,
         MULTIPART_DOWNLOAD_SIZE_DEFAULT);
     reopen(0);
@@ -75,18 +69,17 @@ public class AliyunOSSInputStream extends FSInputStream {
    * @throws IOException if failed to reopen
    */
   private synchronized void reopen(long pos) throws IOException {
-
-    long partLen;
+    long partSize;
 
     if (pos < 0) {
-      throw new EOFException("Cannot seek at negtive position:" + pos);
-    } else if (pos > dataLen) {
-      throw new EOFException("Cannot seek after EOF, fileLen:" + dataLen +
-          " position:" + pos);
-    } else if (pos + downloadPartSize > dataLen) {
-      partLen = dataLen - pos;
+      throw new EOFException("Cannot seek at negative position:" + pos);
+    } else if (pos > contentLength) {
+      throw new EOFException("Cannot seek after EOF, contentLength:" +
+          contentLength + " position:" + pos);
+    } else if (pos + downloadPartSize > contentLength) {
+      partSize = contentLength - pos;
     } else {
-      partLen = downloadPartSize;
+      partSize = downloadPartSize;
     }
 
     if (wrappedStream != null) {
@@ -96,21 +89,19 @@ public class AliyunOSSInputStream extends FSInputStream {
       wrappedStream.close();
     }
 
-    GetObjectRequest request = new GetObjectRequest(bucketName, key);
-    request.setRange(pos, pos + partLen - 1);
-    wrappedStream = ossClient.getObject(request).getObjectContent();
+    wrappedStream = store.retrieve(key, pos, pos + partSize -1);
     if (wrappedStream == null) {
       throw new IOException("Null IO stream");
     }
     position = pos;
-    partRemaining = partLen;
+    partRemaining = partSize;
   }
 
   @Override
   public synchronized int read() throws IOException {
     checkNotClosed();
 
-    if (partRemaining <= 0 && position < dataLen) {
+    if (partRemaining <= 0 && position < contentLength) {
       reopen(position);
     }
 
@@ -139,13 +130,14 @@ public class AliyunOSSInputStream extends FSInputStream {
 
 
   /**
-   * Check whether the input stream is closed.
+   * Verify that the input stream is open. Non blocking; this gives
+   * the last state of the volatile {@link #closed} field.
    *
-   * @throws IOException if stream is closed
+   * @throws IOException if the connection is closed.
    */
   private void checkNotClosed() throws IOException {
     if (closed) {
-      throw new IOException("Stream is closed!");
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
     }
   }
 
@@ -164,7 +156,7 @@ public class AliyunOSSInputStream extends FSInputStream {
 
     int bytesRead = 0;
     // Not EOF, and read not done
-    while (position < dataLen && bytesRead < len) {
+    while (position < contentLength && bytesRead < len) {
       if (partRemaining == 0) {
         reopen(position);
       }
@@ -219,7 +211,7 @@ public class AliyunOSSInputStream extends FSInputStream {
   public synchronized int available() throws IOException {
     checkNotClosed();
 
-    long remaining = dataLen - position;
+    long remaining = contentLength - position;
     if (remaining > Integer.MAX_VALUE) {
       return Integer.MAX_VALUE;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d33e928f/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
index 1e16df9..c952d0a 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
@@ -22,15 +22,10 @@ import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
 
 import java.io.BufferedOutputStream;
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.List;
 
-import com.aliyun.oss.ClientException;
-import com.aliyun.oss.OSSException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -38,18 +33,6 @@ import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.util.Progressable;
 
-import com.aliyun.oss.OSSClient;
-import com.aliyun.oss.model.AbortMultipartUploadRequest;
-import com.aliyun.oss.model.CompleteMultipartUploadRequest;
-import com.aliyun.oss.model.CompleteMultipartUploadResult;
-import com.aliyun.oss.model.InitiateMultipartUploadRequest;
-import com.aliyun.oss.model.InitiateMultipartUploadResult;
-import com.aliyun.oss.model.ObjectMetadata;
-import com.aliyun.oss.model.PartETag;
-import com.aliyun.oss.model.PutObjectResult;
-import com.aliyun.oss.model.UploadPartRequest;
-import com.aliyun.oss.model.UploadPartResult;
-
 /**
  * The output stream for OSS blob system.
  * Data will be buffered on local disk, then uploaded to OSS in
@@ -57,36 +40,24 @@ import com.aliyun.oss.model.UploadPartResult;
  */
 public class AliyunOSSOutputStream extends OutputStream {
   public static final Log LOG = LogFactory.getLog(AliyunOSSOutputStream.class);
-  private String bucketName;
-  private String key;
+  private AliyunOSSFileSystemStore store;
+  private final String key;
   private Statistics statistics;
   private Progressable progress;
-  private String serverSideEncryptionAlgorithm;
-  private long partSize;
   private long partSizeThreshold;
   private LocalDirAllocator dirAlloc;
   private boolean closed;
   private File tmpFile;
   private BufferedOutputStream backupStream;
-  private OSSClient ossClient;
 
-  public AliyunOSSOutputStream(Configuration conf, OSSClient client,
-      String bucketName, String key, Progressable progress,
-      Statistics statistics, String serverSideEncryptionAlgorithm)
-      throws IOException {
-    this.bucketName = bucketName;
+  public AliyunOSSOutputStream(Configuration conf,
+      AliyunOSSFileSystemStore store, String key, Progressable progress,
+      Statistics statistics) throws IOException {
+    this.store = store;
     this.key = key;
     // The caller cann't get any progress information
     this.progress = progress;
-    ossClient = client;
     this.statistics = statistics;
-    this.serverSideEncryptionAlgorithm = serverSideEncryptionAlgorithm;
-
-    partSize = conf.getLong(MULTIPART_UPLOAD_SIZE_KEY,
-        MULTIPART_UPLOAD_SIZE_DEFAULT);
-    if (partSize < MIN_MULTIPART_UPLOAD_PART_SIZE) {
-      partSize = MIN_MULTIPART_UPLOAD_PART_SIZE;
-    }
     partSizeThreshold = conf.getLong(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY,
         MIN_MULTIPART_UPLOAD_THRESHOLD_DEFAULT);
 
@@ -113,9 +84,9 @@ public class AliyunOSSOutputStream extends OutputStream {
     long dataLen = tmpFile.length();
     try {
       if (dataLen <= partSizeThreshold) {
-        uploadObject();
+        store.uploadObject(key, tmpFile);
       } else {
-        multipartUploadObject();
+        store.multipartUploadObject(key, tmpFile);
       }
     } finally {
       if (!tmpFile.delete()) {
@@ -124,91 +95,7 @@ public class AliyunOSSOutputStream extends OutputStream {
     }
   }
 
-  /**
-   * Upload temporary file as an OSS object, using single upload.
-   *
-   * @throws IOException
-   */
-  private void uploadObject() throws IOException {
-    File object = tmpFile.getAbsoluteFile();
-    FileInputStream fis = new FileInputStream(object);
-    ObjectMetadata meta = new ObjectMetadata();
-    meta.setContentLength(object.length());
-    if (!serverSideEncryptionAlgorithm.isEmpty()) {
-      meta.setServerSideEncryption(serverSideEncryptionAlgorithm);
-    }
-    try {
-      PutObjectResult result = ossClient.putObject(bucketName, key, fis, meta);
-      LOG.debug(result.getETag());
-      statistics.incrementWriteOps(1);
-    } finally {
-      fis.close();
-    }
-  }
-
-  /**
-   * Upload temporary file as an OSS object, using multipart upload.
-   *
-   * @throws IOException
-   */
-  private void multipartUploadObject() throws IOException {
-    File object = tmpFile.getAbsoluteFile();
-    long dataLen = object.length();
-    long realPartSize = AliyunOSSUtils.calculatePartSize(dataLen, partSize);
-    int partNum = (int)(dataLen / realPartSize);
-    if (dataLen % realPartSize != 0) {
-      partNum += 1;
-    }
-
-    InitiateMultipartUploadRequest initiateMultipartUploadRequest =
-        new InitiateMultipartUploadRequest(bucketName, key);
-    ObjectMetadata meta = new ObjectMetadata();
-    //    meta.setContentLength(dataLen);
-    if (!serverSideEncryptionAlgorithm.isEmpty()) {
-      meta.setServerSideEncryption(serverSideEncryptionAlgorithm);
-    }
-    initiateMultipartUploadRequest.setObjectMetadata(meta);
-    InitiateMultipartUploadResult initiateMultipartUploadResult =
-        ossClient.initiateMultipartUpload(initiateMultipartUploadRequest);
-    List<PartETag> partETags = new ArrayList<PartETag>();
-    String uploadId = initiateMultipartUploadResult.getUploadId();
 
-    try {
-      for (int i = 0; i < partNum; i++) {
-        // TODO: Optimize this, avoid opening the object multiple times
-        FileInputStream fis = new FileInputStream(object);
-        try {
-          long skipBytes = realPartSize * i;
-          AliyunOSSUtils.skipFully(fis, skipBytes);
-          long size = (realPartSize < dataLen - skipBytes) ?
-              realPartSize : dataLen - skipBytes;
-          UploadPartRequest uploadPartRequest = new UploadPartRequest();
-          uploadPartRequest.setBucketName(bucketName);
-          uploadPartRequest.setKey(key);
-          uploadPartRequest.setUploadId(uploadId);
-          uploadPartRequest.setInputStream(fis);
-          uploadPartRequest.setPartSize(size);
-          uploadPartRequest.setPartNumber(i + 1);
-          UploadPartResult uploadPartResult =
-              ossClient.uploadPart(uploadPartRequest);
-          statistics.incrementWriteOps(1);
-          partETags.add(uploadPartResult.getPartETag());
-        } finally {
-          fis.close();
-        }
-      }
-      CompleteMultipartUploadRequest completeMultipartUploadRequest =
-          new CompleteMultipartUploadRequest(bucketName, key,
-          uploadId, partETags);
-      CompleteMultipartUploadResult completeMultipartUploadResult =
-          ossClient.completeMultipartUpload(completeMultipartUploadRequest);
-      LOG.debug(completeMultipartUploadResult.getETag());
-    } catch (OSSException | ClientException e) {
-      AbortMultipartUploadRequest abortMultipartUploadRequest =
-          new AbortMultipartUploadRequest(bucketName, key, uploadId);
-      ossClient.abortMultipartUpload(abortMultipartUploadRequest);
-    }
-  }
 
   @Override
   public synchronized void flush() throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d33e928f/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
index d54dd9c..b96aea7 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
@@ -20,142 +20,58 @@ package org.apache.hadoop.fs.aliyun.oss;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.UnsupportedEncodingException;
 import java.net.URI;
-import java.net.URLDecoder;
-import java.util.Objects;
 
+import com.aliyun.oss.common.auth.CredentialsProvider;
+import com.aliyun.oss.common.auth.DefaultCredentialProvider;
+import com.aliyun.oss.common.auth.DefaultCredentials;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.ProviderUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import static org.apache.hadoop.fs.aliyun.oss.Constants.MULTIPART_UPLOAD_PART_NUM_LIMIT;
+import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
+import static org.apache.hadoop.fs.aliyun.oss.Constants.ALIYUN_OSS_CREDENTIALS_PROVIDER_KEY;
 
 /**
  * Utility methods for Aliyun OSS code.
  */
 final public class AliyunOSSUtils {
-  private AliyunOSSUtils() {
-  }
-
-  /**
-   * User information includes user name and password.
-   */
-  static public class UserInfo {
-    private final String user;
-    private final String password;
-
-    public static final UserInfo EMPTY = new UserInfo("", "");
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AliyunOSSUtils.class);
 
-    public UserInfo(String user, String password) {
-      this.user = user;
-      this.password = password;
-    }
-
-    /**
-     * Predicate to verify user information is set.
-     * @return true if the username is defined (not null, not empty).
-     */
-    public boolean hasLogin() {
-      return StringUtils.isNotEmpty(user);
-    }
-
-    /**
-     * Equality test matches user and password.
-     * @param o other object
-     * @return true if the objects are considered equivalent.
-     */
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-      UserInfo that = (UserInfo) o;
-      return Objects.equals(user, that.user) &&
-          Objects.equals(password, that.password);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hash(user, password);
-    }
-
-    public String getUser() {
-      return user;
-    }
-
-    public String getPassword() {
-      return password;
-    }
+  private AliyunOSSUtils() {
   }
 
   /**
-   * Used to get password from configuration, if default value is not available.
+   * Used to get password from configuration.
+   *
    * @param conf configuration that contains password information
    * @param key the key of the password
-   * @param val the default value of the key
    * @return the value for the key
    * @throws IOException if failed to get password from configuration
    */
-  static public String getPassword(Configuration conf, String key, String val)
+  static public String getPassword(Configuration conf, String key)
       throws IOException {
-    if (StringUtils.isEmpty(val)) {
-      try {
-        final char[] pass = conf.getPassword(key);
-        if (pass != null) {
-          return (new String(pass)).trim();
-        } else {
-          return "";
-        }
-      } catch (IOException ioe) {
-        throw new IOException("Cannot find password option " + key, ioe);
-      }
-    } else {
-      return val;
-    }
-  }
-
-  /**
-   * Extract the user information details from a URI.
-   * @param name URI of the filesystem.
-   * @return a login tuple, possibly empty.
-   */
-  public static UserInfo extractLoginDetails(URI name) {
     try {
-      String authority = name.getAuthority();
-      if (authority == null) {
-        return UserInfo.EMPTY;
-      }
-      int loginIndex = authority.indexOf('@');
-      if (loginIndex < 0) {
-        // No user information
-        return UserInfo.EMPTY;
-      }
-      String login = authority.substring(0, loginIndex);
-      int loginSplit = login.indexOf(':');
-      if (loginSplit > 0) {
-        String user = login.substring(0, loginSplit);
-        String password = URLDecoder.decode(login.substring(loginSplit + 1),
-            "UTF-8");
-        return new UserInfo(user, password);
-      } else if (loginSplit == 0) {
-        // There is no user, just a password.
-        return UserInfo.EMPTY;
+      final char[] pass = conf.getPassword(key);
+      if (pass != null) {
+        return (new String(pass)).trim();
       } else {
-        return new UserInfo(login, "");
+        return "";
       }
-    } catch (UnsupportedEncodingException e) {
-      // This should never happen; translate it if it does.
-      throw new RuntimeException(e);
+    } catch (IOException ioe) {
+      throw new IOException("Cannot find password option " + key, ioe);
     }
   }
 
   /**
-   * Skips the requested number of bytes or fail if there are not enough left.
-   * This allows for the possibility that {@link InputStream#skip(long)} may not
-   * skip as many bytes as requested (most likely because of reaching EOF).
+   * Skip the requested number of bytes or fail if there are no enough bytes
+   * left. This allows for the possibility that {@link InputStream#skip(long)}
+   * may not skip as many bytes as requested (most likely because of reaching
+   * EOF).
+   *
    * @param is the input stream to skip.
    * @param n the number of bytes to skip.
    * @throws IOException thrown when skipped less number of bytes.
@@ -179,12 +95,69 @@ final public class AliyunOSSUtils {
    * Calculate a proper size of multipart piece. If <code>minPartSize</code>
    * is too small, the number of multipart pieces may exceed the limit of
    * {@link Constants#MULTIPART_UPLOAD_PART_NUM_LIMIT}.
+   *
    * @param contentLength the size of file.
    * @param minPartSize the minimum size of multipart piece.
    * @return a revisional size of multipart piece.
-     */
+   */
   public static long calculatePartSize(long contentLength, long minPartSize) {
     long tmpPartSize = contentLength / MULTIPART_UPLOAD_PART_NUM_LIMIT + 1;
     return Math.max(minPartSize, tmpPartSize);
   }
+
+  /**
+   * Create credential provider specified by configuration, or create default
+   * credential provider if not specified.
+   *
+   * @param name the uri of the file system
+   * @param conf configuration
+   * @return a credential provider
+   * @throws IOException on any problem. Class construction issues may be
+   * nested inside the IOE.
+   */
+  public static CredentialsProvider getCredentialsProvider(URI name,
+      Configuration conf) throws IOException {
+    URI uri = java.net.URI.create(
+        name.getScheme() + "://" + name.getAuthority());
+    CredentialsProvider credentials;
+
+    String className = conf.getTrimmed(ALIYUN_OSS_CREDENTIALS_PROVIDER_KEY);
+    if (StringUtils.isEmpty(className)) {
+      Configuration newConf =
+          ProviderUtils.excludeIncompatibleCredentialProviders(conf,
+              AliyunOSSFileSystem.class);
+      String accessKey =
+          AliyunOSSUtils.getPassword(newConf, ACCESS_KEY);
+      String secretKey =
+          AliyunOSSUtils.getPassword(newConf, SECRET_KEY);
+      credentials = new DefaultCredentialProvider(
+          new DefaultCredentials(accessKey, secretKey));
+    } else {
+      try {
+        LOG.debug("Credential provider class is:" + className);
+        Class<?> credClass = Class.forName(className);
+        try {
+          credentials =
+              (CredentialsProvider)credClass.getDeclaredConstructor(
+                  URI.class, Configuration.class).newInstance(uri, conf);
+        } catch (NoSuchMethodException | SecurityException e) {
+          credentials =
+              (CredentialsProvider)credClass.getDeclaredConstructor()
+              .newInstance();
+        }
+      } catch (ClassNotFoundException e) {
+        throw new IOException(className + " not found.", e);
+      } catch (NoSuchMethodException | SecurityException e) {
+        throw new IOException(String.format("%s constructor exception.  A " +
+            "class specified in %s must provide an accessible constructor " +
+            "accepting URI and Configuration, or an accessible default " +
+            "constructor.", className, ALIYUN_OSS_CREDENTIALS_PROVIDER_KEY),
+            e);
+      } catch (ReflectiveOperationException | IllegalArgumentException e) {
+        throw new IOException(className + " instantiation exception.", e);
+      }
+    }
+
+    return credentials;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d33e928f/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
index 9902275..243fdd4 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
@@ -72,7 +72,7 @@ public final class Constants {
 
   // Number of records to get while paging through a directory listing
   public static final String MAX_PAGING_KEYS_KEY = "fs.oss.paging.maximum";
-  public static final int MAX_PAGING_KEYS_DEFAULT = 500;
+  public static final int MAX_PAGING_KEYS_DEFAULT = 1000;
 
   // Size of each of or multipart pieces in bytes
   public static final String MULTIPART_UPLOAD_SIZE_KEY =
@@ -109,5 +109,6 @@ public final class Constants {
   public static final String FS_OSS = "oss";
 
   public static final long MIN_MULTIPART_UPLOAD_PART_SIZE = 100 * 1024L;
+  public static final int MAX_RETRIES = 10;
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d33e928f/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemContract.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemContract.java
index de4e5a9..f234d50 100644
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemContract.java
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemContract.java
@@ -73,11 +73,6 @@ public class TestOSSFileSystemContract extends FileSystemContractBaseTest {
     // not supported
   }
 
-  /**
-   * Assert that root directory renames are not allowed.
-   *
-   * @throws Exception on failures
-   */
   @Override
   public void testRootDirAlwaysExists() throws Exception {
     //this will throw an exception if the path is not found
@@ -88,11 +83,6 @@ public class TestOSSFileSystemContract extends FileSystemContractBaseTest {
         fs.exists(super.path("/")));
   }
 
-  /**
-   * Assert that root directory renames are not allowed.
-   *
-   * @throws Exception on failures
-   */
   @Override
   public void testRenameRootDirForbidden() throws Exception {
     if (!renameSupported()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d33e928f/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemStore.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemStore.java
new file mode 100644
index 0000000..6331ed8
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemStore.java
@@ -0,0 +1,121 @@
+/**
+ * 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.aliyun.oss;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.*;
+import java.net.URI;
+import java.security.DigestInputStream;
+import java.security.DigestOutputStream;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeNotNull;
+
+/**
+ * Test the bridging logic between Hadoop's abstract filesystem and
+ * Aliyun OSS.
+ */
+public class TestOSSFileSystemStore {
+  private Configuration conf;
+  private AliyunOSSFileSystemStore store;
+  private AliyunOSSFileSystem fs;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();
+    fs = new AliyunOSSFileSystem();
+    fs.initialize(URI.create(conf.get("test.fs.oss.name")), conf);
+    store = fs.getStore();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    try {
+      store.purge("test");
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+  }
+
+  @BeforeClass
+  public static void checkSettings() throws Exception {
+    Configuration conf = new Configuration();
+    assumeNotNull(conf.get("fs.oss.accessKeyId"));
+    assumeNotNull(conf.get("fs.oss.accessKeySecret"));
+    assumeNotNull(conf.get("test.fs.oss.name"));
+  }
+
+  protected void writeRenameReadCompare(Path path, long len)
+      throws IOException, NoSuchAlgorithmException {
+    // If len > fs.oss.multipart.upload.threshold,
+    // we'll use a multipart upload copy
+    MessageDigest digest = MessageDigest.getInstance("MD5");
+    OutputStream out = new BufferedOutputStream(
+        new DigestOutputStream(fs.create(path, false), digest));
+    for (long i = 0; i < len; i++) {
+      out.write('Q');
+    }
+    out.flush();
+    out.close();
+
+    assertTrue("Exists", fs.exists(path));
+
+    Path copyPath = path.suffix(".copy");
+    fs.rename(path, copyPath);
+
+    assertTrue("Copy exists", fs.exists(copyPath));
+
+    // Download file from Aliyun OSS and compare the digest against the original
+    MessageDigest digest2 = MessageDigest.getInstance("MD5");
+    InputStream in = new BufferedInputStream(
+        new DigestInputStream(fs.open(copyPath), digest2));
+    long copyLen = 0;
+    while (in.read() != -1) {
+      copyLen++;
+    }
+    in.close();
+
+    assertEquals("Copy length matches original", len, copyLen);
+    assertArrayEquals("Digests match", digest.digest(), digest2.digest());
+  }
+
+  @Test
+  public void testSmallUpload() throws IOException, NoSuchAlgorithmException {
+    // Regular upload, regular copy
+    writeRenameReadCompare(new Path("/test/small"), 16384);
+  }
+
+  @Test
+  public void testLargeUpload()
+      throws IOException, NoSuchAlgorithmException {
+    // Multipart upload, multipart copy
+    writeRenameReadCompare(new Path("/test/xlarge"), 52428800L); // 50MB byte
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d33e928f/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/OSSContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/OSSContract.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/OSSContract.java
index 8214b9f..f90a8bb 100644
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/OSSContract.java
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/OSSContract.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.fs.aliyun.oss.contract;
 
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.aliyun.oss.OSSTestUtils;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d33e928f/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractDispCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractDispCp.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractDispCp.java
new file mode 100644
index 0000000..eb0c5e0
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractDispCp.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
+
+import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
+
+/**
+ * Contract test suite covering Aliyun OSS integration with DistCp.
+ */
+public class TestOSSContractDispCp extends AbstractContractDistCpTest {
+
+  private static final long MULTIPART_SETTING = 8 * 1024 * 1024; // 8 MB
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration newConf = super.createConfiguration();
+    newConf.setLong(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY, MULTIPART_SETTING);
+    newConf.setLong(MULTIPART_UPLOAD_SIZE_KEY, MULTIPART_SETTING);
+    return newConf;
+  }
+
+  @Override
+  protected OSSContract createContract(Configuration conf) {
+    return new OSSContract(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d33e928f/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractGetFileStatus.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractGetFileStatus.java
new file mode 100644
index 0000000..cc21a2e
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractGetFileStatus.java
@@ -0,0 +1,35 @@
+/**
+ * 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.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Test getFileStatus and related listing operations.
+ */
+public class TestOSSContractGetFileStatus
+    extends AbstractContractGetFileStatusTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new OSSContract(conf);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d33e928f/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractRootDir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractRootDir.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractRootDir.java
new file mode 100644
index 0000000..cbc262c
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractRootDir.java
@@ -0,0 +1,69 @@
+/*
+ * 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.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+/**
+ * Root dir operations against an Aliyun OSS bucket.
+ */
+public class TestOSSContractRootDir extends
+    AbstractContractRootDirectoryTest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOSSContractRootDir.class);
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new OSSContract(conf);
+  }
+
+  @Override
+  public void testListEmptyRootDirectory() throws IOException {
+    for (int attempt = 1, maxAttempts = 10; attempt <= maxAttempts; ++attempt) {
+      try {
+        super.testListEmptyRootDirectory();
+        break;
+      } catch (AssertionError | FileNotFoundException e) {
+        if (attempt < maxAttempts) {
+          LOG.info("Attempt {} of {} for empty root directory test failed.  "
+              + "Attempting retry.", attempt, maxAttempts);
+          try {
+            Thread.sleep(1000);
+          } catch (InterruptedException e2) {
+            Thread.currentThread().interrupt();
+            fail("Test interrupted.");
+            break;
+          }
+        } else {
+          LOG.error(
+              "Empty root directory test failed {} attempts.  Failing test.",
+              maxAttempts);
+          throw e;
+        }
+      }
+    }
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[18/23] hadoop git commit: Merge branch 'trunk' into HADOOP-12756

Posted by dr...@apache.org.
Merge branch 'trunk' into HADOOP-12756


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a49b3be3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a49b3be3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a49b3be3

Branch: refs/heads/trunk
Commit: a49b3be38ed97a27f215afb996c6db516f5857d7
Parents: 22af6f8 e52d6e7
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Sep 21 14:03:21 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Wed Sep 21 14:03:21 2016 +0800

----------------------------------------------------------------------
 .../hdfs/client/HdfsClientConfigKeys.java       |   3 -
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   4 -
 .../hdfs/server/namenode/INodeDirectory.java    |   9 +-
 .../hadoop/tools/TestHdfsConfigFields.java      |   2 -
 .../apache/hadoop/tools/DistCpConstants.java    |   4 +
 .../apache/hadoop/tools/SimpleCopyListing.java  | 114 ++-
 .../apache/hadoop/tools/TestCopyListing.java    |  83 ++-
 .../scheduler/AppSchedulingInfo.java            |  96 +--
 .../scheduler/SchedulerApplicationAttempt.java  | 744 +++++++++++--------
 .../allocator/RegularContainerAllocator.java    |   2 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java | 418 ++++++-----
 .../scheduler/fair/FSAppAttempt.java            | 465 ++++++------
 .../scheduler/TestAppSchedulingInfo.java        |  65 ++
 .../src/site/markdown/TimelineServiceV2.md      |   6 +
 14 files changed, 1240 insertions(+), 775 deletions(-)
----------------------------------------------------------------------



---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[03/23] hadoop git commit: HADOOP-12756. Incorporate Aliyun OSS file system implementation. Contributed by Mingfei Shi and Lin Zhou

Posted by dr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/OSSContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/OSSContract.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/OSSContract.java
new file mode 100644
index 0000000..8214b9f
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/OSSContract.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.aliyun.oss.contract;
+
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.aliyun.oss.OSSTestUtils;
+import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
+
+/**
+ * The contract of OSS: only enabled if the test bucket is provided.
+ */
+public class OSSContract extends AbstractBondedFSContract {
+
+  public static final String CONTRACT_XML = "contract/oss.xml";
+  public static final String CONTRACT_TEST_OSS_FS_NAME =
+      "fs.contract.test.fs.oss";
+
+  private static String testPath = OSSTestUtils.generateUniqueTestPath();
+
+  public OSSContract(Configuration conf) {
+    super(conf);
+    //insert the base features
+    addConfResource(CONTRACT_XML);
+  }
+
+  @Override
+  public String getScheme() {
+    return "oss";
+  }
+
+  @Override
+  public Path getTestPath() {
+    Path path = new Path(testPath);
+    return path;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractCreate.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractCreate.java
new file mode 100644
index 0000000..cc5a2d1
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractCreate.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+
+/**
+ * OSS contract creating tests.
+ */
+public class TestOSSContractCreate extends AbstractContractCreateTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new OSSContract(conf);
+  }
+
+  @Override
+  public void testOverwriteEmptyDirectory() throws Throwable {
+    ContractTestUtils.skip(
+        "blobstores can't distinguish empty directories from files");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractDelete.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractDelete.java
new file mode 100644
index 0000000..6a1eb40
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractDelete.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * OSS contract deleting tests.
+ */
+public class TestOSSContractDelete extends AbstractContractDeleteTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new OSSContract(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractMkdir.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractMkdir.java
new file mode 100644
index 0000000..1dcb7f0
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractMkdir.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * OSS contract directory tests.
+ */
+public class TestOSSContractMkdir extends AbstractContractMkdirTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new OSSContract(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractOpen.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractOpen.java
new file mode 100644
index 0000000..ee0c055
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractOpen.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * OSS contract opening file tests.
+ */
+public class TestOSSContractOpen extends AbstractContractOpenTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new OSSContract(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractRename.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractRename.java
new file mode 100644
index 0000000..634fcf1
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractRename.java
@@ -0,0 +1,35 @@
+/**
+ * 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.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * OSS contract renaming tests.
+ */
+public class TestOSSContractRename extends AbstractContractRenameTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new OSSContract(conf);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractSeek.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractSeek.java
new file mode 100644
index 0000000..40ea772
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractSeek.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * OSS contract seeking tests.
+ */
+public class TestOSSContractSeek extends AbstractContractSeekTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new OSSContract(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/test/resources/contract/oss.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/resources/contract/oss.xml b/hadoop-tools/hadoop-aliyun/src/test/resources/contract/oss.xml
new file mode 100644
index 0000000..2bc34b7
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/resources/contract/oss.xml
@@ -0,0 +1,105 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  ~ 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.
+  -->
+<configuration>
+    <property>
+        <name>fs.contract.test.random-seek-count</name>
+        <value>10</value>
+    </property>
+
+    <property>
+        <name>fs.contract.is-blobstore</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.is-case-sensitive</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.rename-returns-false-if-source-missing</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.rename-remove-dest-if-empty-dir</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-append</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-atomic-directory-delete</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-atomic-rename</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-block-locality</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-concat</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-seek</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-seek-on-closed-file</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.rejects-seek-past-eof</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-strict-exceptions</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-unix-permissions</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.rename-overwrites-dest</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.oss.multipart.download.size</name>
+        <value>102400</value>
+    </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/test/resources/core-site.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/resources/core-site.xml b/hadoop-tools/hadoop-aliyun/src/test/resources/core-site.xml
new file mode 100644
index 0000000..fa4118c
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/resources/core-site.xml
@@ -0,0 +1,46 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  ~ 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.
+  -->
+<configuration>
+
+    <property>
+        <name>hadoop.tmp.dir</name>
+        <value>target/build/test</value>
+        <description>A base for other temporary directories.</description>
+        <final>true</final>
+    </property>
+
+    <!-- Turn security off for tests by default -->
+    <property>
+        <name>hadoop.security.authentication</name>
+        <value>simple</value>
+    </property>
+
+    <!--
+    To run these tests.
+
+    # Create a file auth-keys.xml  - DO NOT ADD TO REVISION CONTROL
+    # add the property test.fs.oss.name to point to an OSS filesystem URL
+    # Add the credentials for the service you are testing against
+    -->
+    <include xmlns="http://www.w3.org/2001/XInclude" href="auth-keys.xml">
+        <fallback/>
+    </include>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aliyun/src/test/resources/log4j.properties
new file mode 100644
index 0000000..bb5cbe5
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/resources/log4j.properties
@@ -0,0 +1,23 @@
+#
+#   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.
+#
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=INFO,stdout
+log4j.threshold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-tools-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-tools-dist/pom.xml b/hadoop-tools/hadoop-tools-dist/pom.xml
index 899a945..14fa9f0 100644
--- a/hadoop-tools/hadoop-tools-dist/pom.xml
+++ b/hadoop-tools/hadoop-tools-dist/pom.xml
@@ -102,6 +102,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-aliyun</artifactId>
+      <scope>compile</scope>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-sls</artifactId>
       <scope>compile</scope>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/pom.xml b/hadoop-tools/pom.xml
index db002f4..e7e876b 100644
--- a/hadoop-tools/pom.xml
+++ b/hadoop-tools/pom.xml
@@ -47,6 +47,7 @@
     <module>hadoop-aws</module>
     <module>hadoop-kafka</module>
     <module>hadoop-azure-datalake</module>
+    <module>hadoop-aliyun</module>
   </modules>
 
   <build>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[10/23] hadoop git commit: Merge branch 'trunk' into HADOOP-12756

Posted by dr...@apache.org.
Merge branch 'trunk' into HADOOP-12756


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/dbb28eb1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/dbb28eb1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/dbb28eb1

Branch: refs/heads/trunk
Commit: dbb28eb1f2125b4a2e43977d9add80a45f359c28
Parents: 60f66a9 501a778
Author: Kai Zheng <ka...@intel.com>
Authored: Sun Sep 18 19:03:39 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Sun Sep 18 19:03:39 2016 +0800

----------------------------------------------------------------------
 .gitattributes                                  |  18 +-
 dev-support/checkstyle/suppressions.xml         |  21 -
 .../main/resources/checkstyle/checkstyle.xml    |  10 +-
 .../main/resources/checkstyle/suppressions.xml  |  21 +
 .../server/AuthenticationFilter.java            |   3 +
 .../java/org/apache/hadoop/fs/FileSystem.java   | 125 ++-
 .../apache/hadoop/io/retry/RetryPolicies.java   |  14 +
 .../org/apache/hadoop/io/retry/RetryUtils.java  | 118 ++-
 .../apache/hadoop/ipc/ProtobufRpcEngine.java    |   5 +-
 .../main/java/org/apache/hadoop/ipc/RPC.java    |  13 +-
 .../main/java/org/apache/hadoop/ipc/Server.java |   4 +-
 .../hadoop/security/UserGroupInformation.java   |   4 +-
 .../java/org/apache/hadoop/util/RunJar.java     |   6 +-
 .../src/main/resources/core-default.xml         |   6 -
 .../src/site/markdown/Benchmarking.md           |   1 +
 .../apache/hadoop/fs/TestFilterFileSystem.java  |   1 +
 .../org/apache/hadoop/fs/TestHarFileSystem.java |   1 +
 .../io/retry/TestConnectionRetryPolicy.java     | 154 ++++
 .../org/apache/hadoop/ipc/RPCCallBenchmark.java |  38 +-
 .../hadoop/ipc/TestMultipleProtocolServer.java  | 236 +----
 .../apache/hadoop/ipc/TestRPCCallBenchmark.java |  13 -
 .../apache/hadoop/ipc/TestRPCCompatibility.java | 242 +----
 .../apache/hadoop/ipc/TestRPCWaitForProxy.java  |  44 +-
 .../hadoop/ipc/TestReuseRpcConnections.java     | 166 ++++
 .../java/org/apache/hadoop/ipc/TestRpcBase.java |  77 +-
 .../java/org/apache/hadoop/ipc/TestSaslRPC.java |  74 +-
 .../hadoop/security/TestDoAsEffectiveUser.java  | 291 +++---
 .../security/TestUserGroupInformation.java      |  28 +-
 .../hadoop-common/src/test/proto/test.proto     |   4 +-
 .../src/test/proto/test_rpc_service.proto       |   8 +-
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |  42 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  87 +-
 .../hadoop/fs/http/client/HttpFSFileSystem.java |  56 +-
 .../hadoop/fs/http/client/HttpFSUtils.java      |   2 +
 .../hadoop/fs/http/server/FSOperations.java     |  62 ++
 .../http/server/HttpFSParametersProvider.java   |  20 +
 .../hadoop/fs/http/server/HttpFSServer.java     |  17 +
 .../service/hadoop/FileSystemAccessService.java |   4 +-
 .../fs/http/client/BaseTestHttpFSWith.java      |  62 +-
 .../hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java    |   2 +-
 .../server/datanode/BlockPoolSliceStorage.java  |  16 +-
 .../hdfs/server/datanode/BlockReceiver.java     |   2 +-
 .../hdfs/server/datanode/BlockScanner.java      |   4 +
 .../hdfs/server/datanode/BlockSender.java       |   7 +-
 .../hadoop/hdfs/server/datanode/DNConf.java     |  95 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   | 135 +--
 .../server/datanode/DataNodeFaultInjector.java  |   2 +-
 .../hdfs/server/datanode/DataStorage.java       |   4 +-
 .../hdfs/server/datanode/DirectoryScanner.java  |  10 +-
 .../hdfs/server/datanode/FinalizedReplica.java  |  27 +-
 .../hdfs/server/datanode/LocalReplica.java      | 479 ++++++++++
 .../server/datanode/LocalReplicaInPipeline.java | 417 +++++++++
 .../server/datanode/ReplicaBeingWritten.java    |  16 +-
 .../hdfs/server/datanode/ReplicaBuilder.java    | 252 +++++
 .../hdfs/server/datanode/ReplicaHandler.java    |   6 +-
 .../hdfs/server/datanode/ReplicaInPipeline.java | 324 ++-----
 .../datanode/ReplicaInPipelineInterface.java    |  86 --
 .../hdfs/server/datanode/ReplicaInfo.java       | 370 ++++----
 .../server/datanode/ReplicaUnderRecovery.java   |  30 +-
 .../datanode/ReplicaWaitingToBeRecovered.java   |  27 +-
 .../server/datanode/fsdataset/FsDatasetSpi.java |   9 +-
 .../datanode/fsdataset/impl/BlockPoolSlice.java |  74 +-
 .../impl/FsDatasetAsyncDiskService.java         |  71 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  | 908 +++++++------------
 .../datanode/fsdataset/impl/FsDatasetUtil.java  |  18 +
 .../datanode/fsdataset/impl/FsVolumeImpl.java   | 154 +++-
 .../datanode/fsdataset/impl/FsVolumeList.java   |   4 +-
 .../impl/RamDiskAsyncLazyPersistService.java    |  34 +-
 .../hdfs/server/namenode/FSDirectory.java       |   2 +-
 .../hadoop/hdfs/server/namenode/NNStorage.java  |   2 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |   3 -
 .../hadoop/hdfs/tools/DiskBalancerCLI.java      |  25 +-
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    |  34 +-
 .../src/site/markdown/HDFSDiskbalancer.md       | 109 ++-
 .../hadoop-hdfs/src/site/markdown/WebHDFS.md    | 183 ++--
 .../org/apache/hadoop/hdfs/AppendTestUtil.java  |  18 +-
 .../TestClientProtocolForPipelineRecovery.java  |   4 +-
 .../apache/hadoop/hdfs/TestCrcCorruption.java   |   6 +-
 .../TestClientProtocolWithDelegationToken.java  | 119 ---
 .../server/datanode/SimulatedFSDataset.java     |  30 +-
 .../server/datanode/TestBPOfferService.java     |   4 +-
 .../datanode/TestBlockPoolSliceStorage.java     |   6 +-
 .../hdfs/server/datanode/TestBlockRecovery.java |   2 +-
 .../datanode/TestDataNodeRollingUpgrade.java    |   6 +-
 .../TestDataXceiverLazyPersistHint.java         |   5 +-
 .../server/datanode/TestDirectoryScanner.java   |  17 +-
 .../server/datanode/TestSimulatedFSDataset.java |   2 +-
 .../hdfs/server/datanode/TestTransferRbw.java   |   4 +-
 .../extdataset/ExternalDatasetImpl.java         |   6 +-
 .../extdataset/ExternalReplicaInPipeline.java   |  26 +-
 .../extdataset/TestExternalDataset.java         |   4 +-
 .../fsdataset/impl/FsDatasetImplTestUtils.java  |  43 +-
 .../fsdataset/impl/FsDatasetTestUtil.java       |  20 +-
 .../fsdataset/impl/TestFsDatasetImpl.java       |   6 +-
 .../fsdataset/impl/TestWriteToReplica.java      |   4 +-
 .../command/TestDiskBalancerCommand.java        |  29 +-
 .../mapreduce/v2/hs/server/HSAdminServer.java   |   3 -
 .../java/org/apache/hadoop/fs/TestDFSIO.java    | 159 +++-
 .../services/org.apache.hadoop.fs.FileSystem    |  16 +
 .../hadoop/yarn/client/cli/RMAdminCLI.java      |   1 +
 .../hadoop/yarn/client/cli/TestRMAdminCLI.java  |  10 +-
 .../nodemanager/DefaultContainerExecutor.java   |   2 +-
 .../containermanager/ContainerManagerImpl.java  |  51 +-
 .../containermanager/container/Container.java   |   6 +
 .../container/ContainerEventType.java           |   3 +-
 .../container/ContainerImpl.java                | 274 +++++-
 .../container/ContainerReInitEvent.java         |  62 ++
 .../container/ContainerState.java               |   2 +-
 .../launcher/ContainersLauncher.java            |   1 +
 .../launcher/ContainersLauncherEventType.java   |   1 +
 .../localizer/ResourceLocalizationService.java  |   3 +-
 .../containermanager/localizer/ResourceSet.java |  43 +-
 .../ContainerLocalizationRequestEvent.java      |   4 +-
 .../recovery/NMLeveldbStateStoreService.java    |  23 +-
 .../TestContainerManagerWithLCE.java            |  36 +
 .../BaseContainerManagerTest.java               |   6 +-
 .../containermanager/TestContainerManager.java  | 238 ++++-
 .../TestNMLeveldbStateStoreService.java         |  84 +-
 .../nodemanager/webapp/MockContainer.java       |  15 +
 .../security/TestClientToAMTokens.java          | 108 ++-
 pom.xml                                         |   2 +-
 121 files changed, 4772 insertions(+), 2781 deletions(-)
----------------------------------------------------------------------



---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[13/23] hadoop git commit: Merge branch 'trunk' into HADOOP-12756

Posted by dr...@apache.org.
Merge branch 'trunk' into HADOOP-12756


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7d4431c9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7d4431c9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7d4431c9

Branch: refs/heads/trunk
Commit: 7d4431c93bf4faaa8ca54e2f9a219bf7e1d506b7
Parents: a194046 f67237c
Author: Kai Zheng <ka...@intel.com>
Authored: Mon Sep 19 08:34:08 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Mon Sep 19 08:34:08 2016 +0800

----------------------------------------------------------------------
 .../hadoop/mapred/TaskAttemptListenerImpl.java  |   2 +-
 .../jobhistory/JobHistoryEventHandler.java      |   2 +-
 .../java/org/apache/hadoop/mapred/Task.java     |   2 +-
 .../apache/hadoop/mapred/ShuffleHandler.java    |   2 +-
 .../apache/hadoop/yarn/client/cli/TopCLI.java   | 114 ++++++++++++++++---
 .../hadoop/yarn/client/cli/TestTopCLI.java      | 106 +++++++++++++++++
 .../hadoop/yarn/event/AsyncDispatcher.java      |   2 +-
 .../security/YarnAuthorizationProvider.java     |   2 +-
 .../hadoop/yarn/webapp/util/WebAppUtils.java    |  35 +++---
 .../FileSystemApplicationHistoryStore.java      |   2 +-
 .../monitor/ContainersMonitorImpl.java          |   2 +-
 .../TestDefaultContainerExecutor.java           |   8 +-
 .../reservation/AbstractReservationSystem.java  |   2 +-
 .../scheduler/capacity/LeafQueue.java           |   2 +-
 .../scheduler/capacity/ParentQueue.java         |   4 +-
 .../scheduler/fair/FairScheduler.java           |   2 +-
 16 files changed, 242 insertions(+), 47 deletions(-)
----------------------------------------------------------------------



---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[07/23] hadoop git commit: HADOOP-13499. Support session credentials for authenticating with Aliyun. Contributed by Genmao Yu.

Posted by dr...@apache.org.
HADOOP-13499. Support session credentials for authenticating with Aliyun. Contributed by Genmao Yu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6bb741b9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6bb741b9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6bb741b9

Branch: refs/heads/trunk
Commit: 6bb741b9f811d3a1c0ce4ecc91a78ac47513bb8e
Parents: bd2d97a
Author: Mingfei <mi...@intel.com>
Authored: Tue Aug 23 17:10:00 2016 +0800
Committer: Mingfei <mi...@intel.com>
Committed: Wed Sep 7 11:15:48 2016 +0800

----------------------------------------------------------------------
 .../apache/hadoop/fs/aliyun/oss/Constants.java  |  5 +-
 .../oss/TemporaryAliyunCredentialsProvider.java | 64 ++++++++++++++++++++
 .../aliyun/oss/TestOSSTemporaryCredentials.java | 64 ++++++++++++++++++++
 3 files changed, 131 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6bb741b9/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
index 4ee4cd4..0bc6d57 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
@@ -31,8 +31,9 @@ public final class Constants {
       "fs.oss.credentials.provider";
 
   // OSS access verification
-  public static final String ACCESS_KEY = "fs.oss.access.key";
-  public static final String SECRET_KEY = "fs.oss.secret.key";
+  public static final String ACCESS_KEY = "fs.oss.accessKeyId";
+  public static final String SECRET_KEY = "fs.oss.accessKeySecret";
+  public static final String SECURITY_TOKEN = "fs.oss.securityToken";
 
   // Number of simultaneous connections to oss
   public static final String MAXIMUM_CONNECTIONS_KEY =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6bb741b9/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/TemporaryAliyunCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/TemporaryAliyunCredentialsProvider.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/TemporaryAliyunCredentialsProvider.java
new file mode 100644
index 0000000..ec8e7fe
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/TemporaryAliyunCredentialsProvider.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.aliyun.oss;
+
+import com.aliyun.oss.common.auth.Credentials;
+import com.aliyun.oss.common.auth.CredentialsProvider;
+import com.aliyun.oss.common.auth.DefaultCredentials;
+import com.aliyun.oss.common.auth.InvalidCredentialsException;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+
+import java.net.URI;
+
+import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
+
+/**
+ * Support session credentials for authenticating with ALiyun.
+ */
+public class TemporaryAliyunCredentialsProvider implements CredentialsProvider {
+  public static final String NAME
+      = "org.apache.hadoop.fs.aliyun.oss.TemporaryAliyunCredentialsProvider";
+  private final String accessKeyId;
+  private final String accessKeySecret;
+  private final String securityToken;
+
+  public TemporaryAliyunCredentialsProvider(URI uri, Configuration conf) {
+    this.accessKeyId = conf.get(ACCESS_KEY, null);
+    this.accessKeySecret = conf.get(SECRET_KEY, null);
+    this.securityToken = conf.get(SECURITY_TOKEN, null);
+  }
+
+  @Override
+  public void setCredentials(Credentials creds) {
+
+  }
+
+  @Override
+  public Credentials getCredentials() {
+    if (!StringUtils.isEmpty(accessKeyId)
+        && !StringUtils.isEmpty(accessKeySecret)
+        && !StringUtils.isEmpty(securityToken)) {
+      return new DefaultCredentials(accessKeyId, accessKeySecret,
+          securityToken);
+    }
+    throw new InvalidCredentialsException(
+        "AccessKeyId, AccessKeySecret or SecurityToken is unset");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6bb741b9/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSTemporaryCredentials.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSTemporaryCredentials.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSTemporaryCredentials.java
new file mode 100644
index 0000000..ca2c0bc
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSTemporaryCredentials.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.aliyun.oss;
+
+import com.aliyun.oss.common.auth.Credentials;
+import com.aliyun.oss.common.auth.InvalidCredentialsException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.aliyun.oss.contract.OSSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
+import org.junit.Test;
+
+import java.net.URI;
+
+import static org.apache.hadoop.fs.aliyun.oss.Constants.ACCESS_KEY;
+import static org.apache.hadoop.fs.aliyun.oss.Constants.SECRET_KEY;
+import static org.apache.hadoop.fs.aliyun.oss.Constants.SECURITY_TOKEN;
+
+/**
+ * Tests use of temporary credentials (for example, Aliyun STS & Aliyun OSS).
+ * This test extends a class that "does things to the root directory", and
+ * should only be used against transient filesystems where you don't care about
+ * the data.
+ */
+public class TestOSSTemporaryCredentials extends AbstractFSContractTestBase {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new OSSContract(conf);
+  }
+
+  @Test
+  public void testTemporaryCredentialValidation() throws Throwable {
+    Configuration conf = new Configuration();
+    conf.set(ACCESS_KEY, "accessKeyId");
+    conf.set(SECRET_KEY, "accessKeySecret");
+    conf.set(SECURITY_TOKEN, "");
+    URI uri = getFileSystem().getUri();
+    TemporaryAliyunCredentialsProvider provider
+        = new TemporaryAliyunCredentialsProvider(uri, conf);
+    try {
+      Credentials credentials = provider.getCredentials();
+      fail("Expected a CredentialInitializationException, got " + credentials);
+    } catch (InvalidCredentialsException expected) {
+      // expected
+    }
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[14/23] hadoop git commit: Merge branch 'trunk' into HADOOP-12756

Posted by dr...@apache.org.
Merge branch 'trunk' into HADOOP-12756


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/005f4528
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/005f4528
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/005f4528

Branch: refs/heads/trunk
Commit: 005f4528c7b0aa4a48cf62b11e837be886f267d3
Parents: 7d4431c 3552c2b
Author: Kai Zheng <ka...@intel.com>
Authored: Tue Sep 20 08:09:29 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Tue Sep 20 08:09:29 2016 +0800

----------------------------------------------------------------------
 .../hadoop/crypto/key/KeyProviderFactory.java   |   3 +-
 .../fs/CommonConfigurationKeysPublic.java       |   8 +
 .../src/main/resources/core-default.xml         |   8 +
 .../src/site/markdown/DeprecatedProperties.md   |   1 +
 .../hadoop-kms/src/site/markdown/index.md.vm    |  10 +-
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  13 +-
 .../apache/hadoop/hdfs/HdfsConfiguration.java   |   3 +
 .../apache/hadoop/hdfs/KeyProviderCache.java    |   6 +-
 .../hdfs/client/HdfsClientConfigKeys.java       |   1 -
 .../org/apache/hadoop/test/TestHdfsHelper.java  |   4 +-
 .../hdfs/nfs/nfs3/TestRpcProgramNfs3.java       |   4 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   2 -
 .../src/main/resources/hdfs-default.xml         |   8 -
 .../src/site/markdown/TransparentEncryption.md  |   2 +-
 .../apache/hadoop/cli/TestCryptoAdminCLI.java   |   4 +-
 .../apache/hadoop/hdfs/TestAclsEndToEnd.java    |   3 +-
 .../org/apache/hadoop/hdfs/TestDFSUtil.java     |  12 +-
 .../apache/hadoop/hdfs/TestEncryptionZones.java |  17 +-
 .../hadoop/hdfs/TestEncryptionZonesWithHA.java  |   3 +-
 .../hadoop/hdfs/TestKeyProviderCache.java       |  10 +-
 .../hadoop/hdfs/TestReservedRawPaths.java       |   3 +-
 .../hdfs/TestSecureEncryptionZoneWithKMS.java   |   6 +-
 .../namenode/TestNestedEncryptionZones.java     |   4 +-
 .../namenode/metrics/TestNameNodeMetrics.java   |   3 +-
 .../src/site/markdown/tools/hadoop-aws/index.md |  44 +----
 .../containermanager/ContainerManagerImpl.java  |  68 ++++++-
 .../containermanager/container/Container.java   |   4 +
 .../container/ContainerEventType.java           |   1 +
 .../container/ContainerImpl.java                | 188 ++++++++++++++-----
 .../container/ContainerReInitEvent.java         |  20 +-
 .../TestContainerManagerWithLCE.java            |  42 ++++-
 .../containermanager/TestContainerManager.java  | 152 +++++++++++++--
 .../nodemanager/webapp/MockContainer.java       |  10 +
 33 files changed, 490 insertions(+), 177 deletions(-)
----------------------------------------------------------------------



---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[19/23] hadoop git commit: Merge branch 'trunk' into HADOOP-12756

Posted by dr...@apache.org.
Merge branch 'trunk' into HADOOP-12756


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/846c5ceb
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/846c5ceb
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/846c5ceb

Branch: refs/heads/trunk
Commit: 846c5ceb3a929ad7b2dcea9bef07299af17bdc84
Parents: a49b3be 964e546
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Sep 23 08:42:28 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Fri Sep 23 08:42:28 2016 +0800

----------------------------------------------------------------------
 .../apache/hadoop/io/ElasticByteBufferPool.java |    2 +-
 .../apache/hadoop/io/erasurecode/ECChunk.java   |   22 +
 .../io/erasurecode/rawcoder/CoderUtil.java      |    3 +
 .../AbstractDelegationTokenSecretManager.java   |    2 +-
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |   20 +-
 .../hadoop/hdfs/DFSStripedInputStream.java      |  654 +------
 .../hadoop/hdfs/PositionStripeReader.java       |  104 +
 .../hadoop/hdfs/StatefulStripeReader.java       |   95 +
 .../org/apache/hadoop/hdfs/StripeReader.java    |  463 +++++
 .../hadoop/hdfs/util/StripedBlockUtil.java      |  158 +-
 .../apache/hadoop/hdfs/TestEncryptionZones.java |   23 +-
 .../blockmanagement/TestBlockTokenWithDFS.java  |    8 +-
 .../TestBlockTokenWithDFSStriped.java           |   23 +-
 .../hdfs/tools/TestDFSZKFailoverController.java |   18 +-
 .../hadoop/hdfs/util/TestStripedBlockUtil.java  |    1 -
 .../dev-support/findbugs-exclude.xml            |   10 +
 .../apache/hadoop/yarn/webapp/Dispatcher.java   |    9 +
 .../org/apache/hadoop/yarn/webapp/WebApp.java   |    4 +-
 .../hadoop/yarn/webapp/view/RobotsTextPage.java |   39 +
 .../apache/hadoop/yarn/webapp/TestWebApp.java   |   26 +
 .../reservation/NoOverCommitPolicy.java         |   12 -
 .../exceptions/MismatchedUserException.java     |   46 -
 .../scheduler/capacity/AbstractCSQueue.java     |  378 ++--
 .../scheduler/capacity/LeafQueue.java           | 1819 ++++++++++--------
 .../scheduler/capacity/ParentQueue.java         |  825 ++++----
 .../scheduler/capacity/PlanQueue.java           |  122 +-
 .../scheduler/capacity/ReservationQueue.java    |   67 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java |    2 +-
 .../ReservationACLsTestBase.java                |    2 +
 .../reservation/TestNoOverCommitPolicy.java     |   21 -
 .../capacity/TestContainerResizing.java         |    4 +-
 .../server/TestContainerManagerSecurity.java    |    5 +
 32 files changed, 2781 insertions(+), 2206 deletions(-)
----------------------------------------------------------------------



---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[04/23] hadoop git commit: HADOOP-12756. Incorporate Aliyun OSS file system implementation. Contributed by Mingfei Shi and Lin Zhou

Posted by dr...@apache.org.
HADOOP-12756. Incorporate Aliyun OSS file system implementation. Contributed by Mingfei Shi and Lin Zhou


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a5d53422
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a5d53422
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a5d53422

Branch: refs/heads/trunk
Commit: a5d5342228050a778b20e95adf7885bdba39985d
Parents: 5f23abf
Author: Kai Zheng <ka...@intel.com>
Authored: Thu Aug 4 21:21:10 2016 +0800
Committer: Mingfei <mi...@intel.com>
Committed: Wed Sep 7 11:15:47 2016 +0800

----------------------------------------------------------------------
 .gitignore                                      |   2 +
 hadoop-project/pom.xml                          |  22 +
 .../dev-support/findbugs-exclude.xml            |  18 +
 hadoop-tools/hadoop-aliyun/pom.xml              | 133 +++
 .../fs/aliyun/oss/AliyunOSSFileSystem.java      | 847 +++++++++++++++++++
 .../fs/aliyun/oss/AliyunOSSInputStream.java     | 268 ++++++
 .../fs/aliyun/oss/AliyunOSSOutputStream.java    | 219 +++++
 .../hadoop/fs/aliyun/oss/AliyunOSSUtils.java    | 151 ++++
 .../apache/hadoop/fs/aliyun/oss/Constants.java  | 110 +++
 .../hadoop/fs/aliyun/oss/package-info.java      |  22 +
 .../hadoop/fs/aliyun/oss/OSSTestUtils.java      |  80 ++
 .../aliyun/oss/TestOSSFileSystemContract.java   | 253 ++++++
 .../fs/aliyun/oss/TestOSSInputStream.java       | 141 +++
 .../fs/aliyun/oss/TestOSSOutputStream.java      |  71 ++
 .../fs/aliyun/oss/contract/OSSContract.java     |  54 ++
 .../oss/contract/TestOSSContractCreate.java     |  41 +
 .../oss/contract/TestOSSContractDelete.java     |  34 +
 .../oss/contract/TestOSSContractMkdir.java      |  34 +
 .../oss/contract/TestOSSContractOpen.java       |  34 +
 .../oss/contract/TestOSSContractRename.java     |  35 +
 .../oss/contract/TestOSSContractSeek.java       |  34 +
 .../src/test/resources/contract/oss.xml         | 105 +++
 .../src/test/resources/core-site.xml            |  46 +
 .../src/test/resources/log4j.properties         |  23 +
 hadoop-tools/hadoop-tools-dist/pom.xml          |   6 +
 hadoop-tools/pom.xml                            |   1 +
 26 files changed, 2784 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index a5d69d0..194862b 100644
--- a/.gitignore
+++ b/.gitignore
@@ -31,3 +31,5 @@ hadoop-tools/hadoop-aws/src/test/resources/auth-keys.xml
 hadoop-tools/hadoop-aws/src/test/resources/contract-test-options.xml
 hadoop-tools/hadoop-azure/src/test/resources/azure-auth-keys.xml
 patchprocess/
+hadoop-tools/hadoop-aliyun/src/test/resources/auth-keys.xml
+hadoop-tools/hadoop-aliyun/src/test/resources/contract-test-options.xml

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 65e9672..29c2760 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -438,6 +438,12 @@
 
       <dependency>
         <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-aliyun</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-kms</artifactId>
         <version>${project.version}</version>
         <classifier>classes</classifier>
@@ -1004,6 +1010,22 @@
         <version>4.2.0</version>
      </dependency>
 
+      <dependency>
+        <groupId>com.aliyun.oss</groupId>
+        <artifactId>aliyun-sdk-oss</artifactId>
+        <version>2.2.1</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpclient</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commons-beanutils</groupId>
+            <artifactId>commons-beanutils</artifactId>
+          </exclusion>
+        </exclusions>
+     </dependency>
+
      <dependency>
        <groupId>xerces</groupId>
        <artifactId>xercesImpl</artifactId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aliyun/dev-support/findbugs-exclude.xml
new file mode 100644
index 0000000..40d78d0
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/dev-support/findbugs-exclude.xml
@@ -0,0 +1,18 @@
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+<FindBugsFilter>
+</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/pom.xml b/hadoop-tools/hadoop-aliyun/pom.xml
new file mode 100644
index 0000000..c87d13f
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/pom.xml
@@ -0,0 +1,133 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>3.0.0-alpha2-SNAPSHOT</version>
+    <relativePath>../../hadoop-project</relativePath>
+  </parent>
+  <artifactId>hadoop-aliyun</artifactId>
+  <name>Apache Hadoop Aliyun OSS support</name>
+  <packaging>jar</packaging>
+
+  <properties>
+    <file.encoding>UTF-8</file.encoding>
+    <downloadSources>true</downloadSources>
+  </properties>
+
+  <profiles>
+    <profile>
+      <id>tests-off</id>
+      <activation>
+        <file>
+          <missing>src/test/resources/auth-keys.xml</missing>
+        </file>
+      </activation>
+      <properties>
+        <maven.test.skip>true</maven.test.skip>
+      </properties>
+    </profile>
+    <profile>
+      <id>tests-on</id>
+      <activation>
+        <file>
+          <exists>src/test/resources/auth-keys.xml</exists>
+        </file>
+      </activation>
+      <properties>
+        <maven.test.skip>false</maven.test.skip>
+      </properties>
+    </profile>
+  </profiles>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <findbugsXmlOutput>true</findbugsXmlOutput>
+          <xmlOutput>true</xmlOutput>
+          <excludeFilterFile>${basedir}/dev-support/findbugs-exclude.xml
+          </excludeFilterFile>
+          <effort>Max</effort>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-project-info-reports-plugin</artifactId>
+        <configuration>
+          <dependencyDetailsEnabled>false</dependencyDetailsEnabled>
+          <dependencyLocationsEnabled>false</dependencyLocationsEnabled>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <forkedProcessTimeoutInSeconds>3600</forkedProcessTimeoutInSeconds>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>deplist</id>
+            <phase>compile</phase>
+            <goals>
+              <goal>list</goal>
+            </goals>
+            <configuration>
+              <!-- build a shellprofile -->
+              <outputFile>${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-optional.txt</outputFile>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.aliyun.oss</groupId>
+      <artifactId>aliyun-sdk-oss</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
new file mode 100644
index 0000000..30ddf8c
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
@@ -0,0 +1,847 @@
+/**
+ * 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.aliyun.oss;
+
+import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
+
+import com.aliyun.oss.common.auth.CredentialsProvider;
+import com.aliyun.oss.common.auth.DefaultCredentialProvider;
+import com.aliyun.oss.common.auth.DefaultCredentials;
+import java.io.ByteArrayInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.aliyun.oss.AliyunOSSUtils.UserInfo;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.ProviderUtils;
+import org.apache.hadoop.util.Progressable;
+
+import com.aliyun.oss.ClientConfiguration;
+import com.aliyun.oss.OSSClient;
+import com.aliyun.oss.OSSException;
+import com.aliyun.oss.common.comm.Protocol;
+import com.aliyun.oss.model.AbortMultipartUploadRequest;
+import com.aliyun.oss.model.CannedAccessControlList;
+import com.aliyun.oss.model.CompleteMultipartUploadRequest;
+import com.aliyun.oss.model.CompleteMultipartUploadResult;
+import com.aliyun.oss.model.CopyObjectResult;
+import com.aliyun.oss.model.DeleteObjectsRequest;
+import com.aliyun.oss.model.InitiateMultipartUploadRequest;
+import com.aliyun.oss.model.InitiateMultipartUploadResult;
+import com.aliyun.oss.model.ListObjectsRequest;
+import com.aliyun.oss.model.OSSObjectSummary;
+import com.aliyun.oss.model.ObjectListing;
+import com.aliyun.oss.model.ObjectMetadata;
+import com.aliyun.oss.model.PartETag;
+import com.aliyun.oss.model.UploadPartCopyRequest;
+import com.aliyun.oss.model.UploadPartCopyResult;
+
+/**
+ * Implementation of {@link FileSystem} for <a href="https://oss.aliyun.com">
+ * Aliyun OSS</a>, used to access OSS blob system in a filesystem style.
+ */
+public class AliyunOSSFileSystem extends FileSystem {
+
+  private URI uri;
+  private Path workingDir;
+  private OSSClient ossClient;
+  private String bucketName;
+  private long uploadPartSize;
+  private long multipartThreshold;
+  private int maxKeys;
+  private String serverSideEncryptionAlgorithm;
+
+  @Override
+  public FSDataOutputStream append(Path path, int bufferSize,
+      Progressable progress) throws IOException {
+    throw new IOException("Append is not supported!");
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      if (ossClient != null) {
+        ossClient.shutdown();
+      }
+    } finally {
+      super.close();
+    }
+  }
+
+  @Override
+  public FSDataOutputStream create(Path path, FsPermission permission,
+      boolean overwrite, int bufferSize, short replication, long blockSize,
+      Progressable progress) throws IOException {
+    String key = pathToKey(path);
+
+    if (!overwrite && exists(path)) {
+      throw new FileAlreadyExistsException(path + " already exists");
+    }
+
+    return new FSDataOutputStream(new AliyunOSSOutputStream(getConf(),
+        ossClient, bucketName, key, progress, statistics,
+        serverSideEncryptionAlgorithm), (Statistics)(null));
+  }
+
+  @Override
+  public boolean delete(Path path, boolean recursive) throws IOException {
+    FileStatus status;
+    try {
+      status = getFileStatus(path);
+    } catch (FileNotFoundException e) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Couldn't delete " + path + ": Does not exist!");
+      }
+      return false;
+    }
+
+    String key = pathToKey(status.getPath());
+    if (status.isDirectory()) {
+      if (!key.endsWith("/")) {
+        key += "/";
+      }
+      if (!recursive) {
+        FileStatus[] statuses = listStatus(status.getPath());
+        // Check whether it is an empty directory or not
+        if (statuses.length > 0) {
+          throw new IOException("Cannot remove directory" + path +
+              ": It is not empty!");
+        } else {
+          // Delete empty directory without '-r'
+          ossClient.deleteObject(bucketName, key);
+          statistics.incrementWriteOps(1);
+        }
+      } else {
+        ListObjectsRequest listRequest = new ListObjectsRequest(bucketName);
+        listRequest.setPrefix(key);
+        listRequest.setMaxKeys(maxKeys);
+
+        while (true) {
+          ObjectListing objects = ossClient.listObjects(listRequest);
+          statistics.incrementReadOps(1);
+          List<String> keysToDelete = new ArrayList<String>();
+          for (OSSObjectSummary objectSummary : objects.getObjectSummaries()) {
+            keysToDelete.add(objectSummary.getKey());
+          }
+          DeleteObjectsRequest deleteRequest =
+              new DeleteObjectsRequest(bucketName);
+          deleteRequest.setKeys(keysToDelete);
+          ossClient.deleteObjects(deleteRequest);
+          statistics.incrementWriteOps(1);
+          if (objects.isTruncated()) {
+            listRequest.setMarker(objects.getNextMarker());
+          } else {
+            break;
+          }
+        }
+      }
+    } else {
+      ossClient.deleteObject(bucketName, key);
+      statistics.incrementWriteOps(1);
+    }
+    //TODO: optimize logic here
+    try {
+      Path pPath = status.getPath().getParent();
+      FileStatus pStatus = getFileStatus(pPath);
+      if (pStatus.isDirectory()) {
+        return true;
+      } else {
+        throw new IOException("Path " + pPath +
+            " is assumed to be a directory!");
+      }
+    } catch (FileNotFoundException fnfe) {
+      // Make sure the parent directory exists
+      return mkdir(bucketName, pathToKey(status.getPath().getParent()));
+    }
+  }
+
+  @Override
+  public FileStatus getFileStatus(Path path) throws IOException {
+    Path qualifiedPath = path.makeQualified(uri, workingDir);
+    String key = pathToKey(qualifiedPath);
+
+    // Root always exists
+    if (key.length() == 0) {
+      return new FileStatus(0, true, 1, 0, 0, qualifiedPath);
+    }
+
+    ObjectMetadata meta = getObjectMetadata(key);
+    // If key not found and key does not end with "/"
+    if (meta == null && !key.endsWith("/")) {
+      // Case: dir + "/"
+      key += "/";
+      meta = getObjectMetadata(key);
+    }
+    if (meta == null) {
+      // Case: dir + "/" + file
+      ListObjectsRequest listRequest = new ListObjectsRequest(bucketName);
+      listRequest.setPrefix(key);
+      listRequest.setDelimiter("/");
+      listRequest.setMaxKeys(1);
+
+      ObjectListing listing = ossClient.listObjects(listRequest);
+      statistics.incrementReadOps(1);
+      if (!listing.getObjectSummaries().isEmpty() ||
+          !listing.getCommonPrefixes().isEmpty()) {
+        return new FileStatus(0, true, 1, 0, 0, qualifiedPath);
+      } else {
+        throw new FileNotFoundException(path + ": No such file or directory!");
+      }
+    } else if (objectRepresentsDirectory(key, meta.getContentLength())) {
+      return new FileStatus(0, true, 1, 0, 0, qualifiedPath);
+    } else {
+      return new FileStatus(meta.getContentLength(), false, 1,
+          getDefaultBlockSize(path), meta.getLastModified().getTime(),
+          qualifiedPath);
+    }
+  }
+
+  /**
+   * Return object metadata given object key.
+   *
+   * @param key object key
+   * @return return null if key does not exist
+   */
+  private ObjectMetadata getObjectMetadata(String key) {
+    try {
+      return ossClient.getObjectMetadata(bucketName, key);
+    } catch (OSSException osse) {
+      return null;
+    } finally {
+      statistics.incrementReadOps(1);
+    }
+  }
+
+  @Override
+  public String getScheme() {
+    return "oss";
+  }
+
+  @Override
+  public URI getUri() {
+    return uri;
+  }
+
+  @Override
+  public Path getWorkingDirectory() {
+    return workingDir;
+  }
+
+  @Deprecated
+  public long getDefaultBlockSize() {
+    return getConf().getLong(FS_OSS_BLOCK_SIZE_KEY, FS_OSS_BLOCK_SIZE_DEFAULT);
+  }
+
+  @Override
+  public String getCanonicalServiceName() {
+    // Does not support Token
+    return null;
+  }
+
+  /**
+   * Initialize new FileSystem.
+   *
+   * @param name the uri of the file system, including host, port, etc.
+   *
+   * @param conf configuration of the file system
+   * @throws IOException IO problems
+   */
+  public void initialize(URI name, Configuration conf) throws IOException {
+    super.initialize(name, conf);
+
+    uri = java.net.URI.create(name.getScheme() + "://" + name.getAuthority());
+    workingDir =
+        new Path("/user",
+            System.getProperty("user.name")).makeQualified(uri, null);
+
+    bucketName = name.getHost();
+
+    ClientConfiguration clientConf = new ClientConfiguration();
+    clientConf.setMaxConnections(conf.getInt(MAXIMUM_CONNECTIONS_KEY,
+        MAXIMUM_CONNECTIONS_DEFAULT));
+    boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS_KEY,
+        SECURE_CONNECTIONS_DEFAULT);
+    clientConf.setProtocol(secureConnections ? Protocol.HTTPS : Protocol.HTTP);
+    clientConf.setMaxErrorRetry(conf.getInt(MAX_ERROR_RETRIES_KEY,
+        MAX_ERROR_RETRIES_DEFAULT));
+    clientConf.setConnectionTimeout(conf.getInt(ESTABLISH_TIMEOUT_KEY,
+        ESTABLISH_TIMEOUT_DEFAULT));
+    clientConf.setSocketTimeout(conf.getInt(SOCKET_TIMEOUT_KEY,
+        SOCKET_TIMEOUT_DEFAULT));
+
+    String proxyHost = conf.getTrimmed(PROXY_HOST_KEY, "");
+    int proxyPort = conf.getInt(PROXY_PORT_KEY, -1);
+    if (!proxyHost.isEmpty()) {
+      clientConf.setProxyHost(proxyHost);
+      if (proxyPort >= 0) {
+        clientConf.setProxyPort(proxyPort);
+      } else {
+        if (secureConnections) {
+          LOG.warn("Proxy host set without port. Using HTTPS default 443");
+          clientConf.setProxyPort(443);
+        } else {
+          LOG.warn("Proxy host set without port. Using HTTP default 80");
+          clientConf.setProxyPort(80);
+        }
+      }
+      String proxyUsername = conf.getTrimmed(PROXY_USERNAME_KEY);
+      String proxyPassword = conf.getTrimmed(PROXY_PASSWORD_KEY);
+      if ((proxyUsername == null) != (proxyPassword == null)) {
+        String msg = "Proxy error: " + PROXY_USERNAME_KEY + " or " +
+            PROXY_PASSWORD_KEY + " set without the other.";
+        LOG.error(msg);
+        throw new IllegalArgumentException(msg);
+      }
+      clientConf.setProxyUsername(proxyUsername);
+      clientConf.setProxyPassword(proxyPassword);
+      clientConf.setProxyDomain(conf.getTrimmed(PROXY_DOMAIN_KEY));
+      clientConf.setProxyWorkstation(conf.getTrimmed(PROXY_WORKSTATION_KEY));
+    } else if (proxyPort >= 0) {
+      String msg = "Proxy error: " + PROXY_PORT_KEY + " set without " +
+          PROXY_HOST_KEY;
+      LOG.error(msg);
+      throw new IllegalArgumentException(msg);
+    }
+
+    String endPoint = conf.getTrimmed(ENDPOINT_KEY, "");
+    ossClient =
+        new OSSClient(endPoint, getCredentialsProvider(name, conf), clientConf);
+
+    maxKeys = conf.getInt(MAX_PAGING_KEYS_KEY, MAX_PAGING_KEYS_DEFAULT);
+    uploadPartSize = conf.getLong(MULTIPART_UPLOAD_SIZE_KEY,
+        MULTIPART_UPLOAD_SIZE_DEFAULT);
+    multipartThreshold = conf.getLong(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY,
+        MIN_MULTIPART_UPLOAD_THRESHOLD_DEFAULT);
+
+    if (uploadPartSize < 5 * 1024 * 1024) {
+      LOG.warn(MULTIPART_UPLOAD_SIZE_KEY + " must be at least 5 MB");
+      uploadPartSize = 5 * 1024 * 1024;
+    }
+
+    if (multipartThreshold < 5 * 1024 * 1024) {
+      LOG.warn(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY + " must be at least 5 MB");
+      multipartThreshold = 5 * 1024 * 1024;
+    }
+
+    if (multipartThreshold > 1024 * 1024 * 1024) {
+      LOG.warn(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY + " must be less than 1 GB");
+      multipartThreshold = 1024 * 1024 * 1024;
+    }
+
+    String cannedACLName = conf.get(CANNED_ACL_KEY, CANNED_ACL_DEFAULT);
+    if (!cannedACLName.isEmpty()) {
+      CannedAccessControlList cannedACL =
+          CannedAccessControlList.valueOf(cannedACLName);
+      ossClient.setBucketAcl(bucketName, cannedACL);
+    }
+
+    serverSideEncryptionAlgorithm =
+        conf.get(SERVER_SIDE_ENCRYPTION_ALGORITHM_KEY, "");
+
+    setConf(conf);
+  }
+
+  /**
+   * Create the default credential provider, or load in one explicitly
+   * identified in the configuration.
+   * @param name the uri of the file system
+   * @param conf configuration
+   * @return a credential provider
+   * @throws IOException on any problem. Class construction issues may be
+   * nested inside the IOE.
+   */
+  private CredentialsProvider getCredentialsProvider(URI name,
+      Configuration conf) throws IOException {
+    CredentialsProvider credentials;
+
+    String className = conf.getTrimmed(ALIYUN_OSS_CREDENTIALS_PROVIDER_KEY);
+    if (StringUtils.isEmpty(className)) {
+      Configuration newConf =
+          ProviderUtils.excludeIncompatibleCredentialProviders(conf,
+              AliyunOSSFileSystem.class);
+      String accessKey =
+          AliyunOSSUtils.getPassword(newConf, ACCESS_KEY,
+              UserInfo.EMPTY.getUser());
+      String secretKey =
+          AliyunOSSUtils.getPassword(newConf, SECRET_KEY,
+              UserInfo.EMPTY.getPassword());
+      credentials =
+          new DefaultCredentialProvider(
+              new DefaultCredentials(accessKey, secretKey));
+
+    } else {
+      try {
+        LOG.debug("Credential provider class is:" + className);
+        Class<?> credClass = Class.forName(className);
+        try {
+          credentials =
+              (CredentialsProvider)credClass.getDeclaredConstructor(
+                  URI.class, Configuration.class).newInstance(this.uri, conf);
+        } catch (NoSuchMethodException | SecurityException e) {
+          credentials =
+              (CredentialsProvider)credClass.getDeclaredConstructor()
+              .newInstance();
+        }
+      } catch (ClassNotFoundException e) {
+        throw new IOException(className + " not found.", e);
+      } catch (NoSuchMethodException | SecurityException e) {
+        throw new IOException(String.format("%s constructor exception.  A " +
+            "class specified in %s must provide an accessible constructor " +
+            "accepting URI and Configuration, or an accessible default " +
+            "constructor.", className, ALIYUN_OSS_CREDENTIALS_PROVIDER_KEY), e);
+      } catch (ReflectiveOperationException | IllegalArgumentException e) {
+        throw new IOException(className + " instantiation exception.", e);
+      }
+    }
+
+    return credentials;
+  }
+
+  /**
+   * Check if OSS object represents a directory.
+   *
+   * @param name object key
+   * @param size object content length
+   * @return true if object represents a directory
+   */
+  private boolean objectRepresentsDirectory(final String name,
+      final long size) {
+    return !name.isEmpty() && name.endsWith("/") && size == 0L;
+  }
+
+  /**
+   * Turns a path (relative or otherwise) into an OSS key.
+   *
+   * @param path the path of the file
+   * @return the key of the object that represent the file
+   */
+  private String pathToKey(Path path) {
+    if (!path.isAbsolute()) {
+      path = new Path(workingDir, path);
+    }
+
+    if (path.toUri().getScheme() != null && path.toUri().getPath().isEmpty()) {
+      return "";
+    }
+
+    return path.toUri().getPath().substring(1);
+  }
+
+  private Path keyToPath(String key) {
+    return new Path("/" + key);
+  }
+
+  @Override
+  public FileStatus[] listStatus(Path path) throws IOException {
+    String key = pathToKey(path);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("List status for path: " + path);
+    }
+
+    final List<FileStatus> result = new ArrayList<FileStatus>();
+    final FileStatus fileStatus = getFileStatus(path);
+
+    if (fileStatus.isDirectory()) {
+      if (!key.endsWith("/")) {
+        key = key + "/";
+      }
+
+      ListObjectsRequest listObjectsRequest =
+          new ListObjectsRequest(bucketName);
+      listObjectsRequest.setPrefix(key);
+      listObjectsRequest.setDelimiter("/");
+      listObjectsRequest.setMaxKeys(maxKeys);
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("listStatus: doing listObjects for directory " + key);
+      }
+
+      while (true) {
+        ObjectListing objects = ossClient.listObjects(listObjectsRequest);
+        statistics.incrementReadOps(1);
+        for (OSSObjectSummary objectSummary : objects.getObjectSummaries()) {
+          Path keyPath = keyToPath(objectSummary.getKey())
+              .makeQualified(uri, workingDir);
+          if (keyPath.equals(path)) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Ignoring: " + keyPath);
+            }
+            continue;
+          } else {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Adding: fi: " + keyPath);
+            }
+            result.add(new FileStatus(objectSummary.getSize(), false, 1,
+                getDefaultBlockSize(keyPath),
+                objectSummary.getLastModified().getTime(), keyPath));
+          }
+        }
+
+        for (String prefix : objects.getCommonPrefixes()) {
+          Path keyPath = keyToPath(prefix).makeQualified(uri, workingDir);
+          if (keyPath.equals(path)) {
+            continue;
+          } else {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Adding: rd: " + keyPath);
+            }
+            result.add(new FileStatus(0, true, 1, 0, 0, keyPath));
+          }
+        }
+
+        if (objects.isTruncated()) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("listStatus: list truncated - getting next batch");
+          }
+          listObjectsRequest.setMarker(objects.getNextMarker());
+          statistics.incrementReadOps(1);
+        } else {
+          break;
+        }
+      }
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Adding: rd (not a dir): " + path);
+      }
+      result.add(fileStatus);
+    }
+
+    return result.toArray(new FileStatus[result.size()]);
+  }
+
+  /**
+   * Used to create an empty file that represents an empty directory.
+   *
+   * @param bucketName the bucket this directory belongs to
+   * @param objectName directory path
+   * @return true if directory successfully created
+   * @throws IOException
+   */
+  private boolean mkdir(final String bucket, final String objectName)
+      throws IOException {
+    String dirName = objectName;
+    ObjectMetadata dirMeta = new ObjectMetadata();
+    byte[] buffer = new byte[0];
+    ByteArrayInputStream in = new ByteArrayInputStream(buffer);
+    dirMeta.setContentLength(0);
+    if (!objectName.endsWith("/")) {
+      dirName += "/";
+    }
+    try {
+      ossClient.putObject(bucket, dirName, in, dirMeta);
+      return true;
+    } finally {
+      in.close();
+    }
+  }
+
+  @Override
+  public boolean mkdirs(Path path, FsPermission permission)
+      throws IOException {
+    try {
+      FileStatus fileStatus = getFileStatus(path);
+
+      if (fileStatus.isDirectory()) {
+        return true;
+      } else {
+        throw new FileAlreadyExistsException("Path is a file: " + path);
+      }
+    } catch (FileNotFoundException e) {
+      validatePath(path);
+      String key = pathToKey(path);
+      return mkdir(bucketName, key);
+    }
+  }
+
+  /**
+   * Check whether the path is a valid path.
+   *
+   * @param path the path to be checked
+   * @throws IOException
+   */
+  private void validatePath(Path path) throws IOException {
+    Path fPart = path.getParent();
+    do {
+      try {
+        FileStatus fileStatus = getFileStatus(fPart);
+        if (fileStatus.isDirectory()) {
+          // If path exists and a directory, exit
+          break;
+        } else {
+          throw new FileAlreadyExistsException(String.format(
+              "Can't make directory for path '%s', it is a file.", fPart));
+        }
+      } catch (FileNotFoundException fnfe) {
+      }
+      fPart = fPart.getParent();
+    } while (fPart != null);
+  }
+
+  @Override
+  public FSDataInputStream open(Path path, int bufferSize) throws IOException {
+    final FileStatus fileStatus = getFileStatus(path);
+    if (fileStatus.isDirectory()) {
+      throw new FileNotFoundException("Can't open " + path +
+          " because it is a directory");
+    }
+
+    return new FSDataInputStream(new AliyunOSSInputStream(getConf(), ossClient,
+        bucketName, pathToKey(path), fileStatus.getLen(), statistics));
+  }
+
+  @Override
+  public boolean rename(Path srcPath, Path dstPath) throws IOException {
+    if (srcPath.isRoot()) {
+      // Cannot rename root of file system
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Cannot rename the root of a filesystem");
+      }
+      return false;
+    }
+    Path parent = dstPath.getParent();
+    while (parent != null && !srcPath.equals(parent)) {
+      parent = parent.getParent();
+    }
+    if (parent != null) {
+      return false;
+    }
+    FileStatus srcStatus = getFileStatus(srcPath);
+    FileStatus dstStatus;
+    try {
+      dstStatus = getFileStatus(dstPath);
+    } catch (FileNotFoundException fnde) {
+      dstStatus = null;
+    }
+    if (dstStatus == null) {
+      // If dst doesn't exist, check whether dst dir exists or not
+      dstStatus = getFileStatus(dstPath.getParent());
+      if (!dstStatus.isDirectory()) {
+        throw new IOException(String.format(
+            "Failed to rename %s to %s, %s is a file", srcPath, dstPath,
+            dstPath.getParent()));
+      }
+    } else {
+      if (srcStatus.getPath().equals(dstStatus.getPath())) {
+        return !srcStatus.isDirectory();
+      } else if (dstStatus.isDirectory()) {
+        // If dst is a directory
+        dstPath = new Path(dstPath, srcPath.getName());
+        FileStatus[] statuses;
+        try {
+          statuses = listStatus(dstPath);
+        } catch (FileNotFoundException fnde) {
+          statuses = null;
+        }
+        if (statuses != null && statuses.length > 0) {
+          // If dst exists and not a directory / not empty
+          throw new FileAlreadyExistsException(String.format(
+              "Failed to rename %s to %s, file already exists or not empty!",
+              srcPath, dstPath));
+        }
+      } else {
+        // If dst is not a directory
+        throw new FileAlreadyExistsException(String.format(
+            "Failed to rename %s to %s, file already exists!", srcPath,
+            dstPath));
+      }
+    }
+    if (srcStatus.isDirectory()) {
+      copyDirectory(srcPath, dstPath);
+    } else {
+      copyFile(srcPath, dstPath);
+    }
+    if (srcPath.equals(dstPath)) {
+      return true;
+    } else {
+      return delete(srcPath, true);
+    }
+  }
+
+  /**
+   * Copy file from source path to destination path.
+   * (the caller should make sure srcPath is a file and dstPath is valid.)
+   *
+   * @param srcPath source path
+   * @param dstPath destination path
+   * @return true if successfully copied
+   */
+  private boolean copyFile(Path srcPath, Path dstPath) {
+    String srcKey = pathToKey(srcPath);
+    String dstKey = pathToKey(dstPath);
+    return copyFile(srcKey, dstKey);
+  }
+
+  /**
+   * Copy an object from source key to destination key.
+   *
+   * @param srcKey source key
+   * @param dstKey destination key
+   * @return true if successfully copied
+   */
+  private boolean copyFile(String srcKey, String dstKey) {
+    ObjectMetadata objectMeta =
+        ossClient.getObjectMetadata(bucketName, srcKey);
+    long dataLen = objectMeta.getContentLength();
+    if (dataLen <= multipartThreshold) {
+      return singleCopy(srcKey, dstKey);
+    } else {
+      return multipartCopy(srcKey, dataLen, dstKey);
+    }
+  }
+
+  /**
+   * Use single copy to copy an oss object.
+   *
+   * @param srcKey source key
+   * @param dstKey destination key
+   * @return true if successfully copied
+   * (the caller should make sure srcPath is a file and dstPath is valid)
+   */
+  private boolean singleCopy(String srcKey, String dstKey) {
+    CopyObjectResult copyResult =
+        ossClient.copyObject(bucketName, srcKey, bucketName, dstKey);
+    LOG.debug(copyResult.getETag());
+    return true;
+  }
+
+  /**
+   * Use multipart copy to copy an oss object.
+   * (the caller should make sure srcPath is a file and dstPath is valid)
+   *
+   * @param srcKey source key
+   * @param dataLen data size of the object to copy
+   * @param dstKey destination key
+   * @return true if successfully copied, or false if upload is aborted
+   */
+  private boolean multipartCopy(String srcKey, long dataLen, String dstKey) {
+    int partNum = (int)(dataLen / uploadPartSize);
+    if (dataLen % uploadPartSize != 0) {
+      partNum++;
+    }
+    InitiateMultipartUploadRequest initiateMultipartUploadRequest =
+        new InitiateMultipartUploadRequest(bucketName, dstKey);
+    ObjectMetadata meta = new ObjectMetadata();
+    if (!serverSideEncryptionAlgorithm.isEmpty()) {
+      meta.setServerSideEncryption(serverSideEncryptionAlgorithm);
+    }
+    initiateMultipartUploadRequest.setObjectMetadata(meta);
+    InitiateMultipartUploadResult initiateMultipartUploadResult =
+        ossClient.initiateMultipartUpload(initiateMultipartUploadRequest);
+    String uploadId = initiateMultipartUploadResult.getUploadId();
+    List<PartETag> partETags = new ArrayList<PartETag>();
+    try {
+      for (int i = 0; i < partNum; i++) {
+        long skipBytes = uploadPartSize * i;
+        long size = (uploadPartSize < dataLen - skipBytes) ?
+            uploadPartSize : dataLen - skipBytes;
+        UploadPartCopyRequest partCopyRequest = new UploadPartCopyRequest();
+        partCopyRequest.setSourceBucketName(bucketName);
+        partCopyRequest.setSourceKey(srcKey);
+        partCopyRequest.setBucketName(bucketName);
+        partCopyRequest.setKey(dstKey);
+        partCopyRequest.setUploadId(uploadId);
+        partCopyRequest.setPartSize(size);
+        partCopyRequest.setBeginIndex(skipBytes);
+        partCopyRequest.setPartNumber(i + 1);
+        UploadPartCopyResult partCopyResult =
+            ossClient.uploadPartCopy(partCopyRequest);
+        statistics.incrementWriteOps(1);
+        partETags.add(partCopyResult.getPartETag());
+      }
+      CompleteMultipartUploadRequest completeMultipartUploadRequest =
+          new CompleteMultipartUploadRequest(bucketName, dstKey,
+          uploadId, partETags);
+      CompleteMultipartUploadResult completeMultipartUploadResult =
+          ossClient.completeMultipartUpload(completeMultipartUploadRequest);
+      LOG.debug(completeMultipartUploadResult.getETag());
+      return true;
+    } catch (Exception e) {
+      AbortMultipartUploadRequest abortMultipartUploadRequest =
+          new AbortMultipartUploadRequest(bucketName, dstKey, uploadId);
+      ossClient.abortMultipartUpload(abortMultipartUploadRequest);
+      return false;
+    }
+  }
+
+  /**
+   * Copy a directory from source path to destination path.
+   * (the caller should make sure srcPath is a directory, and dstPath is valid)
+   *
+   * @param srcPath source path
+   * @param dstPath destination path
+   * @return true if successfully copied
+   */
+  private boolean copyDirectory(Path srcPath, Path dstPath) {
+    String srcKey = pathToKey(srcPath);
+    String dstKey = pathToKey(dstPath);
+
+    if (!srcKey.endsWith("/")) {
+      srcKey = srcKey + "/";
+    }
+    if (!dstKey.endsWith("/")) {
+      dstKey = dstKey + "/";
+    }
+
+    if (dstKey.startsWith(srcKey)) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Cannot rename a directory to a subdirectory of self");
+      }
+      return false;
+    }
+
+    ListObjectsRequest listObjectsRequest = new ListObjectsRequest(bucketName);
+    listObjectsRequest.setPrefix(srcKey);
+    listObjectsRequest.setMaxKeys(maxKeys);
+
+    ObjectListing objects = ossClient.listObjects(listObjectsRequest);
+    statistics.incrementReadOps(1);
+    // Copy files from src folder to dst
+    while (true) {
+      for (OSSObjectSummary objectSummary : objects.getObjectSummaries()) {
+        String newKey =
+            dstKey.concat(objectSummary.getKey().substring(srcKey.length()));
+        copyFile(objectSummary.getKey(), newKey);
+      }
+      if (objects.isTruncated()) {
+        listObjectsRequest.setMarker(objects.getNextMarker());
+        statistics.incrementReadOps(1);
+      } else {
+        break;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public void setWorkingDirectory(Path dir) {
+    this.workingDir = dir;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
new file mode 100644
index 0000000..bcd00dc
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
@@ -0,0 +1,268 @@
+/**
+ * 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.aliyun.oss;
+
+import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileSystem.Statistics;
+
+import com.aliyun.oss.OSSClient;
+import com.aliyun.oss.model.GetObjectRequest;
+
+/**
+ * The input stream for OSS blob system.
+ * The class uses multi-part downloading to read data from the object content
+ * stream.
+ */
+public class AliyunOSSInputStream extends FSInputStream {
+  public static final Log LOG = LogFactory.getLog(AliyunOSSInputStream.class);
+  private static final int MAX_RETRIES = 10;
+  private final long downloadPartSize;
+
+  private String bucketName;
+  private String key;
+  private OSSClient ossClient;
+  private Statistics statistics;
+  private boolean closed;
+  private InputStream wrappedStream = null;
+  private long dataLen;
+  private long position;
+  private long partRemaining;
+
+  public AliyunOSSInputStream(Configuration conf, OSSClient client,
+      String bucketName, String key, Long dataLen, Statistics statistics)
+      throws IOException {
+    this.bucketName = bucketName;
+    this.key = key;
+    ossClient = client;
+    this.statistics = statistics;
+    this.dataLen = dataLen;
+    downloadPartSize = conf.getLong(MULTIPART_DOWNLOAD_SIZE_KEY,
+        MULTIPART_DOWNLOAD_SIZE_DEFAULT);
+    reopen(0);
+    closed = false;
+  }
+
+  /**
+   * Reopen the wrapped stream at give position, by seeking for
+   * data of a part length from object content stream.
+   *
+   * @param pos position from start of a file
+   * @throws IOException if failed to reopen
+   */
+  private synchronized void reopen(long pos) throws IOException {
+
+    long partLen;
+
+    if (pos < 0) {
+      throw new EOFException("Cannot seek at negtive position:" + pos);
+    } else if (pos > dataLen) {
+      throw new EOFException("Cannot seek after EOF, fileLen:" + dataLen +
+          " position:" + pos);
+    } else if (pos + downloadPartSize > dataLen) {
+      partLen = dataLen - pos;
+    } else {
+      partLen = downloadPartSize;
+    }
+
+    if (wrappedStream != null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Aborting old stream to open at pos " + pos);
+      }
+      wrappedStream.close();
+    }
+
+    GetObjectRequest request = new GetObjectRequest(bucketName, key);
+    request.setRange(pos, pos + partLen - 1);
+    wrappedStream = ossClient.getObject(request).getObjectContent();
+    if (wrappedStream == null) {
+      throw new IOException("Null IO stream");
+    }
+    position = pos;
+    partRemaining = partLen;
+  }
+
+  @Override
+  public synchronized int read() throws IOException {
+    checkNotClosed();
+
+    if (partRemaining <= 0 && position < dataLen) {
+      reopen(position);
+    }
+
+    int tries = MAX_RETRIES;
+    boolean retry;
+    int byteRead = -1;
+    do {
+      retry = false;
+      try {
+        byteRead = wrappedStream.read();
+      } catch (Exception e) {
+        handleReadException(e, --tries);
+        retry = true;
+      }
+    } while (retry);
+    if (byteRead >= 0) {
+      position++;
+      partRemaining--;
+    }
+
+    if (statistics != null && byteRead >= 0) {
+      statistics.incrementBytesRead(1);
+    }
+    return byteRead;
+  }
+
+
+  /**
+   * Check whether the input stream is closed.
+   *
+   * @throws IOException if stream is closed
+   */
+  private void checkNotClosed() throws IOException {
+    if (closed) {
+      throw new IOException("Stream is closed!");
+    }
+  }
+
+  @Override
+  public synchronized int read(byte[] buf, int off, int len)
+      throws IOException {
+    checkNotClosed();
+
+    if (buf == null) {
+      throw new NullPointerException();
+    } else if (off < 0 || len < 0 || len > buf.length - off) {
+      throw new IndexOutOfBoundsException();
+    } else if (len == 0) {
+      return 0;
+    }
+
+    int bytesRead = 0;
+    // Not EOF, and read not done
+    while (position < dataLen && bytesRead < len) {
+      if (partRemaining == 0) {
+        reopen(position);
+      }
+
+      int tries = MAX_RETRIES;
+      boolean retry;
+      int bytes = -1;
+      do {
+        retry = false;
+        try {
+          bytes = wrappedStream.read(buf, off + bytesRead, len - bytesRead);
+        } catch (Exception e) {
+          handleReadException(e, --tries);
+          retry = true;
+        }
+      } while (retry);
+
+      if (bytes > 0) {
+        bytesRead += bytes;
+        position += bytes;
+        partRemaining -= bytes;
+      } else if (partRemaining != 0) {
+        throw new IOException("Failed to read from stream. Remaining:" +
+            partRemaining);
+      }
+    }
+
+    if (statistics != null && bytesRead > 0) {
+      statistics.incrementBytesRead(bytesRead);
+    }
+
+    // Read nothing, but attempt to read something
+    if (bytesRead == 0 && len > 0) {
+      return -1;
+    } else {
+      return bytesRead;
+    }
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    if (closed) {
+      return;
+    }
+    closed = true;
+    if (wrappedStream != null) {
+      wrappedStream.close();
+    }
+  }
+
+  @Override
+  public synchronized int available() throws IOException {
+    checkNotClosed();
+
+    long remaining = dataLen - position;
+    if (remaining > Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    }
+    return (int)remaining;
+  }
+
+  @Override
+  public void seek(long pos) throws IOException {
+    checkNotClosed();
+    if (position == pos) {
+      return;
+    } else if (pos > position && pos < position + partRemaining) {
+      wrappedStream.skip(pos - position);
+      position = pos;
+    } else {
+      reopen(pos);
+    }
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    checkNotClosed();
+    return position;
+  }
+
+  @Override
+  public boolean seekToNewSource(long targetPos) throws IOException {
+    checkNotClosed();
+    return false;
+  }
+
+  private void handleReadException(Exception e, int tries) throws IOException{
+    if (tries == 0) {
+      throw new IOException(e);
+    }
+
+    LOG.warn("Some exceptions occurred in oss connection, try to reopen oss" +
+        " connection at position '" + position + "', " + e.getMessage());
+    try {
+      Thread.sleep(100);
+    } catch (InterruptedException e2) {
+      LOG.warn(e2.getMessage());
+    }
+    reopen(position);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
new file mode 100644
index 0000000..589e014
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
@@ -0,0 +1,219 @@
+/**
+ * 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.aliyun.oss;
+
+import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
+
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem.Statistics;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.util.Progressable;
+
+import com.aliyun.oss.OSSClient;
+import com.aliyun.oss.model.AbortMultipartUploadRequest;
+import com.aliyun.oss.model.CompleteMultipartUploadRequest;
+import com.aliyun.oss.model.CompleteMultipartUploadResult;
+import com.aliyun.oss.model.InitiateMultipartUploadRequest;
+import com.aliyun.oss.model.InitiateMultipartUploadResult;
+import com.aliyun.oss.model.ObjectMetadata;
+import com.aliyun.oss.model.PartETag;
+import com.aliyun.oss.model.PutObjectResult;
+import com.aliyun.oss.model.UploadPartRequest;
+import com.aliyun.oss.model.UploadPartResult;
+
+/**
+ * The output stream for OSS blob system.
+ * Data will be buffered on local disk, then uploaded to OSS in
+ * {@link #close()} method.
+ */
+public class AliyunOSSOutputStream extends OutputStream {
+  public static final Log LOG = LogFactory.getLog(AliyunOSSOutputStream.class);
+  private String bucketName;
+  private String key;
+  private Statistics statistics;
+  private Progressable progress;
+  private String serverSideEncryptionAlgorithm;
+  private long partSize;
+  private long partSizeThreshold;
+  private LocalDirAllocator dirAlloc;
+  private boolean closed;
+  private File tmpFile;
+  private BufferedOutputStream backupStream;
+  private OSSClient ossClient;
+
+  public AliyunOSSOutputStream(Configuration conf, OSSClient client,
+      String bucketName, String key, Progressable progress,
+      Statistics statistics, String serverSideEncryptionAlgorithm)
+      throws IOException {
+    this.bucketName = bucketName;
+    this.key = key;
+    // The caller cann't get any progress information
+    this.progress = progress;
+    ossClient = client;
+    this.statistics = statistics;
+    this.serverSideEncryptionAlgorithm = serverSideEncryptionAlgorithm;
+
+    partSize = conf.getLong(MULTIPART_UPLOAD_SIZE_KEY,
+        MULTIPART_UPLOAD_SIZE_DEFAULT);
+    partSizeThreshold = conf.getLong(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY,
+        MIN_MULTIPART_UPLOAD_THRESHOLD_DEFAULT);
+
+    if (conf.get(BUFFER_DIR_KEY) == null) {
+      conf.set(BUFFER_DIR_KEY, conf.get("hadoop.tmp.dir") + "/oss");
+    }
+    dirAlloc = new LocalDirAllocator(BUFFER_DIR_KEY);
+
+    tmpFile = dirAlloc.createTmpFileForWrite("output-",
+        LocalDirAllocator.SIZE_UNKNOWN, conf);
+    backupStream = new BufferedOutputStream(new FileOutputStream(tmpFile));
+    closed = false;
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    if (closed) {
+      return;
+    }
+    closed = true;
+    if (backupStream != null) {
+      backupStream.close();
+    }
+    long dataLen = tmpFile.length();
+    try {
+      if (dataLen <= partSizeThreshold) {
+        uploadObject();
+      } else {
+        multipartUploadObject();
+      }
+    } finally {
+      tmpFile.delete();
+    }
+  }
+
+  /**
+   * Upload temporary file as an OSS object, using single upload.
+   *
+   * @throws IOException
+   */
+  private void uploadObject() throws IOException {
+    File object = tmpFile.getAbsoluteFile();
+    FileInputStream fis = new FileInputStream(object);
+    ObjectMetadata meta = new ObjectMetadata();
+    meta.setContentLength(object.length());
+    if (!serverSideEncryptionAlgorithm.isEmpty()) {
+      meta.setServerSideEncryption(serverSideEncryptionAlgorithm);
+    }
+    try {
+      PutObjectResult result = ossClient.putObject(bucketName, key, fis, meta);
+      LOG.debug(result.getETag());
+      statistics.incrementWriteOps(1);
+    } finally {
+      fis.close();
+    }
+  }
+
+  /**
+   * Upload temporary file as an OSS object, using multipart upload.
+   *
+   * @throws IOException
+   */
+  private void multipartUploadObject() throws IOException {
+    File object = tmpFile.getAbsoluteFile();
+    long dataLen = object.length();
+    InitiateMultipartUploadRequest initiateMultipartUploadRequest =
+        new InitiateMultipartUploadRequest(bucketName, key);
+    ObjectMetadata meta = new ObjectMetadata();
+    //    meta.setContentLength(dataLen);
+    if (!serverSideEncryptionAlgorithm.isEmpty()) {
+      meta.setServerSideEncryption(serverSideEncryptionAlgorithm);
+    }
+    initiateMultipartUploadRequest.setObjectMetadata(meta);
+    InitiateMultipartUploadResult initiateMultipartUploadResult =
+        ossClient.initiateMultipartUpload(initiateMultipartUploadRequest);
+    int partNum = (int)(dataLen / partSize);
+    if (dataLen % partSize != 0) {
+      partNum += 1;
+    }
+    if (partNum > MULTIPART_UPLOAD_PART_NUM_LIMIT) {
+      throw new IOException("Number of parts " + partNum + " should not be " +
+          "bigger than limit " + MULTIPART_UPLOAD_PART_NUM_LIMIT);
+    }
+    List<PartETag> partETags = new ArrayList<PartETag>();
+    String uploadId = initiateMultipartUploadResult.getUploadId();
+
+    try {
+      for (int i = 0; i < partNum; i++) {
+        // TODO: Optimize this, avoid opening the object multiple times
+        FileInputStream fis = new FileInputStream(object);
+        try {
+          long skipBytes = partSize * i;
+          fis.skip(skipBytes);
+          long size = (partSize < dataLen - skipBytes) ?
+              partSize : dataLen - skipBytes;
+          UploadPartRequest uploadPartRequest = new UploadPartRequest();
+          uploadPartRequest.setBucketName(bucketName);
+          uploadPartRequest.setKey(key);
+          uploadPartRequest.setUploadId(uploadId);
+          uploadPartRequest.setInputStream(fis);
+          uploadPartRequest.setPartSize(size);
+          uploadPartRequest.setPartNumber(i + 1);
+          UploadPartResult uploadPartResult =
+              ossClient.uploadPart(uploadPartRequest);
+          statistics.incrementWriteOps(1);
+          partETags.add(uploadPartResult.getPartETag());
+        } finally {
+          fis.close();
+        }
+      }
+      CompleteMultipartUploadRequest completeMultipartUploadRequest =
+          new CompleteMultipartUploadRequest(bucketName, key,
+          uploadId, partETags);
+      CompleteMultipartUploadResult completeMultipartUploadResult =
+          ossClient.completeMultipartUpload(completeMultipartUploadRequest);
+      LOG.debug(completeMultipartUploadResult.getETag());
+    } catch (Exception e) {
+      AbortMultipartUploadRequest abortMultipartUploadRequest =
+          new AbortMultipartUploadRequest(bucketName, key, uploadId);
+      ossClient.abortMultipartUpload(abortMultipartUploadRequest);
+    }
+  }
+
+  @Override
+  public synchronized void flush() throws IOException {
+    backupStream.flush();
+  }
+
+  @Override
+  public synchronized void write(int b) throws IOException {
+    backupStream.write(b);
+    statistics.incrementBytesWritten(1);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
new file mode 100644
index 0000000..3f66a4f
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
@@ -0,0 +1,151 @@
+/**
+ * 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.aliyun.oss;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URI;
+import java.net.URLDecoder;
+import java.util.Objects;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Utility methods for Aliyun OSS code.
+ */
+final public class AliyunOSSUtils {
+  private AliyunOSSUtils() {
+  }
+
+  /**
+   * User information includes user name and password.
+   */
+  static public class UserInfo {
+    private final String user;
+    private final String password;
+
+    public static final UserInfo EMPTY = new UserInfo("", "");
+
+    public UserInfo(String user, String password) {
+      this.user = user;
+      this.password = password;
+    }
+
+    /**
+     * Predicate to verify user information is set.
+     * @return true if the username is defined (not null, not empty).
+     */
+    public boolean hasLogin() {
+      return StringUtils.isNotEmpty(user);
+    }
+
+    /**
+     * Equality test matches user and password.
+     * @param o other object
+     * @return true if the objects are considered equivalent.
+     */
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      UserInfo that = (UserInfo) o;
+      return Objects.equals(user, that.user) &&
+          Objects.equals(password, that.password);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(user, password);
+    }
+
+    public String getUser() {
+      return user;
+    }
+
+    public String getPassword() {
+      return password;
+    }
+  }
+
+  /**
+   * Used to get password from configuration, if default value is not available.
+   * @param conf configuration that contains password information
+   * @param key the key of the password
+   * @param val the default value of the key
+   * @return the value for the key
+   * @throws IOException if failed to get password from configuration
+   */
+  static public String getPassword(Configuration conf, String key, String val)
+      throws IOException {
+    if (StringUtils.isEmpty(val)) {
+      try {
+        final char[] pass = conf.getPassword(key);
+        if (pass != null) {
+          return (new String(pass)).trim();
+        } else {
+          return "";
+        }
+      } catch (IOException ioe) {
+        throw new IOException("Cannot find password option " + key, ioe);
+      }
+    } else {
+      return val;
+    }
+  }
+
+  /**
+   * Extract the user information details from a URI.
+   * @param name URI of the filesystem
+   * @return a login tuple, possibly empty.
+   */
+  public static UserInfo extractLoginDetails(URI name) {
+    try {
+      String authority = name.getAuthority();
+      if (authority == null) {
+        return UserInfo.EMPTY;
+      }
+      int loginIndex = authority.indexOf('@');
+      if (loginIndex < 0) {
+        // No user information
+        return UserInfo.EMPTY;
+      }
+      String login = authority.substring(0, loginIndex);
+      int loginSplit = login.indexOf(':');
+      if (loginSplit > 0) {
+        String user = login.substring(0, loginSplit);
+        String password = URLDecoder.decode(login.substring(loginSplit + 1),
+            "UTF-8");
+        return new UserInfo(user, password);
+      } else if (loginSplit == 0) {
+        // There is no user, just a password.
+        return UserInfo.EMPTY;
+      } else {
+        return new UserInfo(login, "");
+      }
+    } catch (UnsupportedEncodingException e) {
+      // This should never happen; translate it if it does.
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
new file mode 100644
index 0000000..4ee4cd4
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
@@ -0,0 +1,110 @@
+/**
+ * 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.aliyun.oss;
+
+/**
+ * ALL configuration constants for OSS filesystem.
+ */
+public final class Constants {
+
+  private Constants() {
+  }
+
+  // Class of credential provider
+  public static final String ALIYUN_OSS_CREDENTIALS_PROVIDER_KEY =
+      "fs.oss.credentials.provider";
+
+  // OSS access verification
+  public static final String ACCESS_KEY = "fs.oss.access.key";
+  public static final String SECRET_KEY = "fs.oss.secret.key";
+
+  // Number of simultaneous connections to oss
+  public static final String MAXIMUM_CONNECTIONS_KEY =
+      "fs.oss.connection.maximum";
+  public static final int MAXIMUM_CONNECTIONS_DEFAULT = 32;
+
+  // Connect to oss over ssl
+  public static final String SECURE_CONNECTIONS_KEY =
+      "fs.oss.connection.secure.enabled";
+  public static final boolean SECURE_CONNECTIONS_DEFAULT = true;
+
+  // Use a custom endpoint
+  public static final String ENDPOINT_KEY = "fs.oss.endpoint";
+
+  // Connect to oss through a proxy server
+  public static final String PROXY_HOST_KEY = "fs.oss.proxy.host";
+  public static final String PROXY_PORT_KEY = "fs.oss.proxy.port";
+  public static final String PROXY_USERNAME_KEY = "fs.oss.proxy.username";
+  public static final String PROXY_PASSWORD_KEY = "fs.oss.proxy.password";
+  public static final String PROXY_DOMAIN_KEY = "fs.oss.proxy.domain";
+  public static final String PROXY_WORKSTATION_KEY =
+      "fs.oss.proxy.workstation";
+
+  // Number of times we should retry errors
+  public static final String MAX_ERROR_RETRIES_KEY = "fs.oss.attempts.maximum";
+  public static final int MAX_ERROR_RETRIES_DEFAULT = 20;
+
+  // Time until we give up trying to establish a connection to oss
+  public static final String ESTABLISH_TIMEOUT_KEY =
+      "fs.oss.connection.establish.timeout";
+  public static final int ESTABLISH_TIMEOUT_DEFAULT = 50000;
+
+  // Time until we give up on a connection to oss
+  public static final String SOCKET_TIMEOUT_KEY = "fs.oss.connection.timeout";
+  public static final int SOCKET_TIMEOUT_DEFAULT = 200000;
+
+  // Number of records to get while paging through a directory listing
+  public static final String MAX_PAGING_KEYS_KEY = "fs.oss.paging.maximum";
+  public static final int MAX_PAGING_KEYS_DEFAULT = 500;
+
+  // Size of each of or multipart pieces in bytes
+  public static final String MULTIPART_UPLOAD_SIZE_KEY =
+      "fs.oss.multipart.upload.size";
+
+  public static final long MULTIPART_UPLOAD_SIZE_DEFAULT = 10 * 1024 * 1024;
+  public static final int MULTIPART_UPLOAD_PART_NUM_LIMIT = 1000;
+
+  // Minimum size in bytes before we start a multipart uploads or copy
+  public static final String MIN_MULTIPART_UPLOAD_THRESHOLD_KEY =
+      "fs.oss.multipart.upload.threshold";
+  public static final long MIN_MULTIPART_UPLOAD_THRESHOLD_DEFAULT =
+      20 * 1024 * 1024;
+
+  public static final String MULTIPART_DOWNLOAD_SIZE_KEY =
+      "fs.oss.multipart.download.size";
+
+  public static final long MULTIPART_DOWNLOAD_SIZE_DEFAULT = 100 * 1024;
+
+  // Comma separated list of directories
+  public static final String BUFFER_DIR_KEY = "fs.oss.buffer.dir";
+
+  // private | public-read | public-read-write | authenticated-read |
+  // log-delivery-write | bucket-owner-read | bucket-owner-full-control
+  public static final String CANNED_ACL_KEY = "fs.oss.acl.default";
+  public static final String CANNED_ACL_DEFAULT = "";
+
+  // OSS server-side encryption
+  public static final String SERVER_SIDE_ENCRYPTION_ALGORITHM_KEY =
+      "fs.oss.server-side-encryption-algorithm";
+
+  public static final String FS_OSS_BLOCK_SIZE_KEY = "fs.oss.block.size";
+  public static final int FS_OSS_BLOCK_SIZE_DEFAULT = 64 * 1024 * 1024;
+  public static final String FS_OSS = "oss";
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/package-info.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/package-info.java
new file mode 100644
index 0000000..234567b
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/package-info.java
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Aliyun OSS Filesystem.
+ */
+package org.apache.hadoop.fs.aliyun.oss;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/OSSTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/OSSTestUtils.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/OSSTestUtils.java
new file mode 100644
index 0000000..37ed831
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/OSSTestUtils.java
@@ -0,0 +1,80 @@
+/**
+ * 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.aliyun.oss;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.junit.internal.AssumptionViolatedException;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Date;
+import java.util.Random;
+
+/**
+ * Utility class for OSS Tests.
+ */
+public final class OSSTestUtils {
+
+  private OSSTestUtils() {
+  }
+
+  /**
+   * Create the test filesystem.
+   *
+   * If the test.fs.oss.name property is not set,
+   * tests will fail.
+   *
+   * @param conf configuration
+   * @return the FS
+   * @throws IOException
+   */
+  public static AliyunOSSFileSystem createTestFileSystem(Configuration conf)
+      throws IOException {
+    String fsname = conf.getTrimmed(
+        TestOSSFileSystemContract.TEST_FS_OSS_NAME, "");
+
+    boolean liveTest = !StringUtils.isEmpty(fsname);
+    URI testURI = null;
+    if (liveTest) {
+      testURI = URI.create(fsname);
+      liveTest = testURI.getScheme().equals(Constants.FS_OSS);
+    }
+
+    if (!liveTest) {
+      throw new AssumptionViolatedException("No test filesystem in "
+          + TestOSSFileSystemContract.TEST_FS_OSS_NAME);
+    }
+    AliyunOSSFileSystem ossfs = new AliyunOSSFileSystem();
+    ossfs.initialize(testURI, conf);
+    return ossfs;
+  }
+
+  /**
+   * Generate unique test path for multiple user tests.
+   *
+   * @return root test path
+   */
+  public static String generateUniqueTestPath() {
+    Long time = new Date().getTime();
+    Random rand = new Random();
+    return "/test_" + Long.toString(time) + "_"
+        + Long.toString(Math.abs(rand.nextLong()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemContract.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemContract.java
new file mode 100644
index 0000000..de4e5a9
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemContract.java
@@ -0,0 +1,253 @@
+/**
+ * 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.aliyun.oss;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileSystemContractBaseTest;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+/**
+ * Tests a live OSS system.
+ *
+ * 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 TestOSSFileSystemContract extends FileSystemContractBaseTest {
+
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(TestOSSFileSystemContract.class);
+
+  public static final String TEST_FS_OSS_NAME = "test.fs.oss.name";
+  private static String testRootPath = OSSTestUtils.generateUniqueTestPath();
+
+  @Override
+  public void setUp() throws Exception {
+    Configuration conf = new Configuration();
+    fs = OSSTestUtils.createTestFileSystem(conf);
+    super.setUp();
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    if (fs != null) {
+      fs.delete(super.path(testRootPath), true);
+    }
+    super.tearDown();
+  }
+
+  @Override
+  protected Path path(String path) {
+    if (path.startsWith("/")) {
+      return super.path(testRootPath + path);
+    } else {
+      return super.path(testRootPath + "/" + path);
+    }
+  }
+
+  @Override
+  public void testMkdirsWithUmask() throws Exception {
+    // not supported
+  }
+
+  /**
+   * Assert that root directory renames are not allowed.
+   *
+   * @throws Exception on failures
+   */
+  @Override
+  public void testRootDirAlwaysExists() throws Exception {
+    //this will throw an exception if the path is not found
+    fs.getFileStatus(super.path("/"));
+    //this catches overrides of the base exists() method that don't
+    //use getFileStatus() as an existence probe
+    assertTrue("FileSystem.exists() fails for root",
+        fs.exists(super.path("/")));
+  }
+
+  /**
+   * Assert that root directory renames are not allowed.
+   *
+   * @throws Exception on failures
+   */
+  @Override
+  public void testRenameRootDirForbidden() throws Exception {
+    if (!renameSupported()) {
+      return;
+    }
+    rename(super.path("/"),
+           super.path("/test/newRootDir"),
+           false, true, false);
+  }
+
+  public void testDeleteSubdir() throws IOException {
+    Path parentDir = this.path("/test/hadoop");
+    Path file = this.path("/test/hadoop/file");
+    Path subdir = this.path("/test/hadoop/subdir");
+    this.createFile(file);
+
+    assertTrue("Created subdir", this.fs.mkdirs(subdir));
+    assertTrue("File exists", this.fs.exists(file));
+    assertTrue("Parent dir exists", this.fs.exists(parentDir));
+    assertTrue("Subdir exists", this.fs.exists(subdir));
+
+    assertTrue("Deleted subdir", this.fs.delete(subdir, true));
+    assertTrue("Parent should exist", this.fs.exists(parentDir));
+
+    assertTrue("Deleted file", this.fs.delete(file, false));
+    assertTrue("Parent should exist", this.fs.exists(parentDir));
+  }
+
+
+  @Override
+  protected boolean renameSupported() {
+    return true;
+  }
+
+  @Override
+  public void testRenameNonExistentPath() throws Exception {
+    if (this.renameSupported()) {
+      Path src = this.path("/test/hadoop/path");
+      Path dst = this.path("/test/new/newpath");
+      try {
+        super.rename(src, dst, false, false, false);
+        fail("Should throw FileNotFoundException!");
+      } catch (FileNotFoundException e) {
+        // expected
+      }
+    }
+  }
+
+  @Override
+  public void testRenameFileMoveToNonExistentDirectory() throws Exception {
+    if (this.renameSupported()) {
+      Path src = this.path("/test/hadoop/file");
+      this.createFile(src);
+      Path dst = this.path("/test/new/newfile");
+      try {
+        super.rename(src, dst, false, true, false);
+        fail("Should throw FileNotFoundException!");
+      } catch (FileNotFoundException e) {
+        // expected
+      }
+    }
+  }
+
+  @Override
+  public void testRenameDirectoryMoveToNonExistentDirectory() throws Exception {
+    if (this.renameSupported()) {
+      Path src = this.path("/test/hadoop/dir");
+      this.fs.mkdirs(src);
+      Path dst = this.path("/test/new/newdir");
+      try {
+        super.rename(src, dst, false, true, false);
+        fail("Should throw FileNotFoundException!");
+      } catch (FileNotFoundException e) {
+        // expected
+      }
+    }
+  }
+
+  @Override
+  public void testRenameFileMoveToExistingDirectory() throws Exception {
+    super.testRenameFileMoveToExistingDirectory();
+  }
+
+  @Override
+  public void testRenameFileAsExistingFile() throws Exception {
+    if (this.renameSupported()) {
+      Path src = this.path("/test/hadoop/file");
+      this.createFile(src);
+      Path dst = this.path("/test/new/newfile");
+      this.createFile(dst);
+      try {
+        super.rename(src, dst, false, true, true);
+        fail("Should throw FileAlreadyExistsException");
+      } catch (FileAlreadyExistsException e) {
+        // expected
+      }
+    }
+  }
+
+  @Override
+  public void testRenameDirectoryAsExistingFile() throws Exception {
+    if (this.renameSupported()) {
+      Path src = this.path("/test/hadoop/dir");
+      this.fs.mkdirs(src);
+      Path dst = this.path("/test/new/newfile");
+      this.createFile(dst);
+      try {
+        super.rename(src, dst, false, true, true);
+        fail("Should throw FileAlreadyExistsException");
+      } catch (FileAlreadyExistsException e) {
+        // expected
+      }
+    }
+  }
+
+  public void testGetFileStatusFileAndDirectory() throws Exception {
+    Path filePath = this.path("/test/oss/file1");
+    this.createFile(filePath);
+    assertTrue("Should be file", this.fs.getFileStatus(filePath).isFile());
+    assertFalse("Should not be directory",
+        this.fs.getFileStatus(filePath).isDirectory());
+
+    Path dirPath = this.path("/test/oss/dir");
+    this.fs.mkdirs(dirPath);
+    assertTrue("Should be directory",
+        this.fs.getFileStatus(dirPath).isDirectory());
+    assertFalse("Should not be file", this.fs.getFileStatus(dirPath).isFile());
+  }
+
+  public void testMkdirsForExistingFile() throws Exception {
+    Path testFile = this.path("/test/hadoop/file");
+    assertFalse(this.fs.exists(testFile));
+    this.createFile(testFile);
+    assertTrue(this.fs.exists(testFile));
+    try {
+      this.fs.mkdirs(testFile);
+      fail("Should throw FileAlreadyExistsException!");
+    } catch (FileAlreadyExistsException e) {
+      // expected
+    }
+  }
+
+  public void testWorkingDirectory() throws Exception {
+    Path workDir = super.path(this.getDefaultWorkingDirectory());
+    assertEquals(workDir, this.fs.getWorkingDirectory());
+    this.fs.setWorkingDirectory(super.path("."));
+    assertEquals(workDir, this.fs.getWorkingDirectory());
+    this.fs.setWorkingDirectory(super.path(".."));
+    assertEquals(workDir.getParent(), this.fs.getWorkingDirectory());
+    Path relativeDir = super.path("hadoop");
+    this.fs.setWorkingDirectory(relativeDir);
+    assertEquals(relativeDir, this.fs.getWorkingDirectory());
+    Path absoluteDir = super.path("/test/hadoop");
+    this.fs.setWorkingDirectory(absoluteDir);
+    assertEquals(absoluteDir, this.fs.getWorkingDirectory());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSInputStream.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSInputStream.java
new file mode 100644
index 0000000..411cd57
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSInputStream.java
@@ -0,0 +1,141 @@
+/**
+ * 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.aliyun.oss;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.*;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Random;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests basic functionality for AliyunOSSInputStream, including seeking and
+ * reading files.
+ */
+public class TestOSSInputStream {
+
+  private FileSystem fs;
+
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(TestOSSInputStream.class);
+
+  private static String testRootPath = OSSTestUtils.generateUniqueTestPath();
+
+  @Rule
+  public Timeout testTimeout = new Timeout(30 * 60 * 1000);
+
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = new Configuration();
+    fs = OSSTestUtils.createTestFileSystem(conf);
+  }
+
+  @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);
+    }
+  }
+
+  @Test
+  public void testSeekFile() 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 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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5d53422/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSOutputStream.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSOutputStream.java
new file mode 100644
index 0000000..3951529
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSOutputStream.java
@@ -0,0 +1,71 @@
+/**
+ * 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.aliyun.oss;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.io.IOException;
+
+/**
+ * Tests regular and multi-part upload functionality for AliyunOSSOutputStream.
+ */
+public class TestOSSOutputStream {
+  private FileSystem fs;
+  private static String testRootPath = OSSTestUtils.generateUniqueTestPath();
+
+  @Rule
+  public Timeout testTimeout = new Timeout(30 * 60 * 1000);
+
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setLong(Constants.MIN_MULTIPART_UPLOAD_THRESHOLD_KEY, 5 * 1024 * 1024);
+    conf.setInt(Constants.MULTIPART_UPLOAD_SIZE_KEY, 5 * 1024 * 1024);
+    fs = OSSTestUtils.createTestFileSystem(conf);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (fs != null) {
+      fs.delete(new Path(testRootPath), true);
+    }
+  }
+
+  protected Path getTestPath() {
+    return new Path(testRootPath + "/testoss");
+  }
+
+  @Test
+  public void testRegularUpload() throws IOException {
+    ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 1024 * 1024);
+  }
+
+  @Test
+  public void testMultiPartUpload() throws IOException {
+    ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 6 * 1024 * 1024);
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[11/23] hadoop git commit: HADOOP-13610. Clean up AliyunOss integration tests. Contributed by Genmao Yu

Posted by dr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractDispCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractDispCp.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractDispCp.java
deleted file mode 100644
index eb0c5e0..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractDispCp.java
+++ /dev/null
@@ -1,44 +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.aliyun.oss.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
-
-import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
-
-/**
- * Contract test suite covering Aliyun OSS integration with DistCp.
- */
-public class TestOSSContractDispCp extends AbstractContractDistCpTest {
-
-  private static final long MULTIPART_SETTING = 8 * 1024 * 1024; // 8 MB
-
-  @Override
-  protected Configuration createConfiguration() {
-    Configuration newConf = super.createConfiguration();
-    newConf.setLong(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY, MULTIPART_SETTING);
-    newConf.setLong(MULTIPART_UPLOAD_SIZE_KEY, MULTIPART_SETTING);
-    return newConf;
-  }
-
-  @Override
-  protected OSSContract createContract(Configuration conf) {
-    return new OSSContract(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractGetFileStatus.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractGetFileStatus.java
deleted file mode 100644
index cc21a2e..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractGetFileStatus.java
+++ /dev/null
@@ -1,35 +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.aliyun.oss.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-/**
- * Test getFileStatus and related listing operations.
- */
-public class TestOSSContractGetFileStatus
-    extends AbstractContractGetFileStatusTest {
-
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new OSSContract(conf);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractMkdir.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractMkdir.java
deleted file mode 100644
index 1dcb7f0..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractMkdir.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.aliyun.oss.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-/**
- * OSS contract directory tests.
- */
-public class TestOSSContractMkdir extends AbstractContractMkdirTest {
-
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new OSSContract(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractOpen.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractOpen.java
deleted file mode 100644
index ee0c055..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractOpen.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.aliyun.oss.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-/**
- * OSS contract opening file tests.
- */
-public class TestOSSContractOpen extends AbstractContractOpenTest {
-
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new OSSContract(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractRename.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractRename.java
deleted file mode 100644
index 634fcf1..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractRename.java
+++ /dev/null
@@ -1,35 +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.aliyun.oss.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-/**
- * OSS contract renaming tests.
- */
-public class TestOSSContractRename extends AbstractContractRenameTest {
-
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new OSSContract(conf);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractRootDir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractRootDir.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractRootDir.java
deleted file mode 100644
index cbc262c..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractRootDir.java
+++ /dev/null
@@ -1,69 +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.aliyun.oss.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-
-/**
- * Root dir operations against an Aliyun OSS bucket.
- */
-public class TestOSSContractRootDir extends
-    AbstractContractRootDirectoryTest {
-
-  private static final Logger LOG =
-      LoggerFactory.getLogger(TestOSSContractRootDir.class);
-
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new OSSContract(conf);
-  }
-
-  @Override
-  public void testListEmptyRootDirectory() throws IOException {
-    for (int attempt = 1, maxAttempts = 10; attempt <= maxAttempts; ++attempt) {
-      try {
-        super.testListEmptyRootDirectory();
-        break;
-      } catch (AssertionError | FileNotFoundException e) {
-        if (attempt < maxAttempts) {
-          LOG.info("Attempt {} of {} for empty root directory test failed.  "
-              + "Attempting retry.", attempt, maxAttempts);
-          try {
-            Thread.sleep(1000);
-          } catch (InterruptedException e2) {
-            Thread.currentThread().interrupt();
-            fail("Test interrupted.");
-            break;
-          }
-        } else {
-          LOG.error(
-              "Empty root directory test failed {} attempts.  Failing test.",
-              maxAttempts);
-          throw e;
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractSeek.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractSeek.java
deleted file mode 100644
index 40ea772..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractSeek.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.aliyun.oss.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-/**
- * OSS contract seeking tests.
- */
-public class TestOSSContractSeek extends AbstractContractSeekTest {
-
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new OSSContract(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/resources/contract/aliyun-oss.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/resources/contract/aliyun-oss.xml b/hadoop-tools/hadoop-aliyun/src/test/resources/contract/aliyun-oss.xml
new file mode 100644
index 0000000..2bc34b7
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/resources/contract/aliyun-oss.xml
@@ -0,0 +1,105 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  ~ 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.
+  -->
+<configuration>
+    <property>
+        <name>fs.contract.test.random-seek-count</name>
+        <value>10</value>
+    </property>
+
+    <property>
+        <name>fs.contract.is-blobstore</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.is-case-sensitive</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.rename-returns-false-if-source-missing</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.rename-remove-dest-if-empty-dir</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-append</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-atomic-directory-delete</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-atomic-rename</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-block-locality</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-concat</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-seek</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-seek-on-closed-file</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.rejects-seek-past-eof</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-strict-exceptions</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-unix-permissions</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.rename-overwrites-dest</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.oss.multipart.download.size</name>
+        <value>102400</value>
+    </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/resources/contract/oss.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/resources/contract/oss.xml b/hadoop-tools/hadoop-aliyun/src/test/resources/contract/oss.xml
deleted file mode 100644
index 2bc34b7..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/resources/contract/oss.xml
+++ /dev/null
@@ -1,105 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-  ~ 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.
-  -->
-<configuration>
-    <property>
-        <name>fs.contract.test.random-seek-count</name>
-        <value>10</value>
-    </property>
-
-    <property>
-        <name>fs.contract.is-blobstore</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>fs.contract.is-case-sensitive</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>fs.contract.rename-returns-false-if-source-missing</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>fs.contract.rename-remove-dest-if-empty-dir</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>fs.contract.supports-append</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>fs.contract.supports-atomic-directory-delete</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>fs.contract.supports-atomic-rename</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>fs.contract.supports-block-locality</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>fs.contract.supports-concat</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>fs.contract.supports-seek</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>fs.contract.supports-seek-on-closed-file</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>fs.contract.rejects-seek-past-eof</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>fs.contract.supports-strict-exceptions</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>fs.contract.supports-unix-permissions</name>
-        <value>false</value>
-    </property>
-
-    <property>
-        <name>fs.contract.rename-overwrites-dest</name>
-        <value>true</value>
-    </property>
-
-    <property>
-        <name>fs.oss.multipart.download.size</name>
-        <value>102400</value>
-    </property>
-</configuration>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[12/23] hadoop git commit: HADOOP-13610. Clean up AliyunOss integration tests. Contributed by Genmao Yu

Posted by dr...@apache.org.
HADOOP-13610. Clean up AliyunOss integration tests. Contributed by Genmao Yu


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a1940464
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a1940464
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a1940464

Branch: refs/heads/trunk
Commit: a1940464a498d1e662e5c3843f2d31ce63ec726b
Parents: dbb28eb
Author: Kai Zheng <ka...@intel.com>
Authored: Sun Sep 18 19:10:48 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Sun Sep 18 19:10:48 2016 +0800

----------------------------------------------------------------------
 .../fs/aliyun/oss/AliyunOSSTestUtils.java       |  77 ++++++
 .../hadoop/fs/aliyun/oss/OSSTestUtils.java      |  80 ------
 .../oss/TestAliyunOSSFileSystemContract.java    | 239 ++++++++++++++++++
 .../oss/TestAliyunOSSFileSystemStore.java       | 121 +++++++++
 .../fs/aliyun/oss/TestAliyunOSSInputStream.java | 142 +++++++++++
 .../aliyun/oss/TestAliyunOSSOutputStream.java   |  91 +++++++
 .../oss/TestAliyunOSSTemporaryCredentials.java  |  65 +++++
 .../aliyun/oss/TestOSSFileSystemContract.java   | 243 -------------------
 .../fs/aliyun/oss/TestOSSFileSystemStore.java   | 121 ---------
 .../fs/aliyun/oss/TestOSSInputStream.java       | 141 -----------
 .../fs/aliyun/oss/TestOSSOutputStream.java      |  90 -------
 .../aliyun/oss/TestOSSTemporaryCredentials.java |  64 -----
 .../aliyun/oss/contract/AliyunOSSContract.java  |  49 ++++
 .../fs/aliyun/oss/contract/OSSContract.java     |  53 ----
 .../contract/TestAliyunOSSContractCreate.java   |  35 +++
 .../contract/TestAliyunOSSContractDelete.java   |  34 +++
 .../contract/TestAliyunOSSContractDispCp.java   |  44 ++++
 .../TestAliyunOSSContractGetFileStatus.java     |  35 +++
 .../contract/TestAliyunOSSContractMkdir.java    |  34 +++
 .../oss/contract/TestAliyunOSSContractOpen.java |  34 +++
 .../contract/TestAliyunOSSContractRename.java   |  35 +++
 .../contract/TestAliyunOSSContractRootDir.java  |  69 ++++++
 .../oss/contract/TestAliyunOSSContractSeek.java |  34 +++
 .../oss/contract/TestOSSContractCreate.java     |  35 ---
 .../oss/contract/TestOSSContractDelete.java     |  34 ---
 .../oss/contract/TestOSSContractDispCp.java     |  44 ----
 .../contract/TestOSSContractGetFileStatus.java  |  35 ---
 .../oss/contract/TestOSSContractMkdir.java      |  34 ---
 .../oss/contract/TestOSSContractOpen.java       |  34 ---
 .../oss/contract/TestOSSContractRename.java     |  35 ---
 .../oss/contract/TestOSSContractRootDir.java    |  69 ------
 .../oss/contract/TestOSSContractSeek.java       |  34 ---
 .../src/test/resources/contract/aliyun-oss.xml  | 105 ++++++++
 .../src/test/resources/contract/oss.xml         | 105 --------
 34 files changed, 1243 insertions(+), 1251 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSTestUtils.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSTestUtils.java
new file mode 100644
index 0000000..84dba6a
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSTestUtils.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.aliyun.oss;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.junit.internal.AssumptionViolatedException;
+
+import java.io.IOException;
+import java.net.URI;
+
+/**
+ * Utility class for Aliyun OSS Tests.
+ */
+public final class AliyunOSSTestUtils {
+
+  private AliyunOSSTestUtils() {
+  }
+
+  /**
+   * Create the test filesystem.
+   *
+   * If the test.fs.oss.name property is not set,
+   * tests will fail.
+   *
+   * @param conf configuration
+   * @return the FS
+   * @throws IOException
+   */
+  public static AliyunOSSFileSystem createTestFileSystem(Configuration conf)
+      throws IOException {
+    String fsname = conf.getTrimmed(
+        TestAliyunOSSFileSystemContract.TEST_FS_OSS_NAME, "");
+
+    boolean liveTest = !StringUtils.isEmpty(fsname);
+    URI testURI = null;
+    if (liveTest) {
+      testURI = URI.create(fsname);
+      liveTest = testURI.getScheme().equals(Constants.FS_OSS);
+    }
+
+    if (!liveTest) {
+      throw new AssumptionViolatedException("No test filesystem in "
+          + TestAliyunOSSFileSystemContract.TEST_FS_OSS_NAME);
+    }
+    AliyunOSSFileSystem ossfs = new AliyunOSSFileSystem();
+    ossfs.initialize(testURI, conf);
+    return ossfs;
+  }
+
+  /**
+   * Generate unique test path for multiple user tests.
+   *
+   * @return root test path
+   */
+  public static String generateUniqueTestPath() {
+    String testUniqueForkId = System.getProperty("test.unique.fork.id");
+    return testUniqueForkId == null ? "/test" :
+        "/" + testUniqueForkId + "/test";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/OSSTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/OSSTestUtils.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/OSSTestUtils.java
deleted file mode 100644
index 37ed831..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/OSSTestUtils.java
+++ /dev/null
@@ -1,80 +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.aliyun.oss;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.junit.internal.AssumptionViolatedException;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.Date;
-import java.util.Random;
-
-/**
- * Utility class for OSS Tests.
- */
-public final class OSSTestUtils {
-
-  private OSSTestUtils() {
-  }
-
-  /**
-   * Create the test filesystem.
-   *
-   * If the test.fs.oss.name property is not set,
-   * tests will fail.
-   *
-   * @param conf configuration
-   * @return the FS
-   * @throws IOException
-   */
-  public static AliyunOSSFileSystem createTestFileSystem(Configuration conf)
-      throws IOException {
-    String fsname = conf.getTrimmed(
-        TestOSSFileSystemContract.TEST_FS_OSS_NAME, "");
-
-    boolean liveTest = !StringUtils.isEmpty(fsname);
-    URI testURI = null;
-    if (liveTest) {
-      testURI = URI.create(fsname);
-      liveTest = testURI.getScheme().equals(Constants.FS_OSS);
-    }
-
-    if (!liveTest) {
-      throw new AssumptionViolatedException("No test filesystem in "
-          + TestOSSFileSystemContract.TEST_FS_OSS_NAME);
-    }
-    AliyunOSSFileSystem ossfs = new AliyunOSSFileSystem();
-    ossfs.initialize(testURI, conf);
-    return ossfs;
-  }
-
-  /**
-   * Generate unique test path for multiple user tests.
-   *
-   * @return root test path
-   */
-  public static String generateUniqueTestPath() {
-    Long time = new Date().getTime();
-    Random rand = new Random();
-    return "/test_" + Long.toString(time) + "_"
-        + Long.toString(Math.abs(rand.nextLong()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemContract.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemContract.java
new file mode 100644
index 0000000..ad8ef6e
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemContract.java
@@ -0,0 +1,239 @@
+/**
+ * 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.aliyun.oss;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileSystemContractBaseTest;
+import org.apache.hadoop.fs.Path;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+/**
+ * Tests a live Aliyun OSS system.
+ *
+ * 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 TestAliyunOSSFileSystemContract
+    extends FileSystemContractBaseTest {
+  public static final String TEST_FS_OSS_NAME = "test.fs.oss.name";
+  private static String testRootPath =
+      AliyunOSSTestUtils.generateUniqueTestPath();
+
+  @Override
+  public void setUp() throws Exception {
+    Configuration conf = new Configuration();
+    fs = AliyunOSSTestUtils.createTestFileSystem(conf);
+    super.setUp();
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    if (fs != null) {
+      fs.delete(super.path(testRootPath), true);
+    }
+    super.tearDown();
+  }
+
+  @Override
+  protected Path path(String path) {
+    if (path.startsWith("/")) {
+      return super.path(testRootPath + path);
+    } else {
+      return super.path(testRootPath + "/" + path);
+    }
+  }
+
+  @Override
+  public void testMkdirsWithUmask() throws Exception {
+    // not supported
+  }
+
+  @Override
+  public void testRootDirAlwaysExists() throws Exception {
+    //this will throw an exception if the path is not found
+    fs.getFileStatus(super.path("/"));
+    //this catches overrides of the base exists() method that don't
+    //use getFileStatus() as an existence probe
+    assertTrue("FileSystem.exists() fails for root",
+        fs.exists(super.path("/")));
+  }
+
+  @Override
+  public void testRenameRootDirForbidden() throws Exception {
+    if (!renameSupported()) {
+      return;
+    }
+    rename(super.path("/"),
+           super.path("/test/newRootDir"),
+           false, true, false);
+  }
+
+  public void testDeleteSubdir() throws IOException {
+    Path parentDir = this.path("/test/hadoop");
+    Path file = this.path("/test/hadoop/file");
+    Path subdir = this.path("/test/hadoop/subdir");
+    this.createFile(file);
+
+    assertTrue("Created subdir", this.fs.mkdirs(subdir));
+    assertTrue("File exists", this.fs.exists(file));
+    assertTrue("Parent dir exists", this.fs.exists(parentDir));
+    assertTrue("Subdir exists", this.fs.exists(subdir));
+
+    assertTrue("Deleted subdir", this.fs.delete(subdir, true));
+    assertTrue("Parent should exist", this.fs.exists(parentDir));
+
+    assertTrue("Deleted file", this.fs.delete(file, false));
+    assertTrue("Parent should exist", this.fs.exists(parentDir));
+  }
+
+
+  @Override
+  protected boolean renameSupported() {
+    return true;
+  }
+
+  @Override
+  public void testRenameNonExistentPath() throws Exception {
+    if (this.renameSupported()) {
+      Path src = this.path("/test/hadoop/path");
+      Path dst = this.path("/test/new/newpath");
+      try {
+        super.rename(src, dst, false, false, false);
+        fail("Should throw FileNotFoundException!");
+      } catch (FileNotFoundException e) {
+        // expected
+      }
+    }
+  }
+
+  @Override
+  public void testRenameFileMoveToNonExistentDirectory() throws Exception {
+    if (this.renameSupported()) {
+      Path src = this.path("/test/hadoop/file");
+      this.createFile(src);
+      Path dst = this.path("/test/new/newfile");
+      try {
+        super.rename(src, dst, false, true, false);
+        fail("Should throw FileNotFoundException!");
+      } catch (FileNotFoundException e) {
+        // expected
+      }
+    }
+  }
+
+  @Override
+  public void testRenameDirectoryMoveToNonExistentDirectory() throws Exception {
+    if (this.renameSupported()) {
+      Path src = this.path("/test/hadoop/dir");
+      this.fs.mkdirs(src);
+      Path dst = this.path("/test/new/newdir");
+      try {
+        super.rename(src, dst, false, true, false);
+        fail("Should throw FileNotFoundException!");
+      } catch (FileNotFoundException e) {
+        // expected
+      }
+    }
+  }
+
+  @Override
+  public void testRenameFileMoveToExistingDirectory() throws Exception {
+    super.testRenameFileMoveToExistingDirectory();
+  }
+
+  @Override
+  public void testRenameFileAsExistingFile() throws Exception {
+    if (this.renameSupported()) {
+      Path src = this.path("/test/hadoop/file");
+      this.createFile(src);
+      Path dst = this.path("/test/new/newfile");
+      this.createFile(dst);
+      try {
+        super.rename(src, dst, false, true, true);
+        fail("Should throw FileAlreadyExistsException");
+      } catch (FileAlreadyExistsException e) {
+        // expected
+      }
+    }
+  }
+
+  @Override
+  public void testRenameDirectoryAsExistingFile() throws Exception {
+    if (this.renameSupported()) {
+      Path src = this.path("/test/hadoop/dir");
+      this.fs.mkdirs(src);
+      Path dst = this.path("/test/new/newfile");
+      this.createFile(dst);
+      try {
+        super.rename(src, dst, false, true, true);
+        fail("Should throw FileAlreadyExistsException");
+      } catch (FileAlreadyExistsException e) {
+        // expected
+      }
+    }
+  }
+
+  public void testGetFileStatusFileAndDirectory() throws Exception {
+    Path filePath = this.path("/test/oss/file1");
+    this.createFile(filePath);
+    assertTrue("Should be file", this.fs.getFileStatus(filePath).isFile());
+    assertFalse("Should not be directory",
+        this.fs.getFileStatus(filePath).isDirectory());
+
+    Path dirPath = this.path("/test/oss/dir");
+    this.fs.mkdirs(dirPath);
+    assertTrue("Should be directory",
+        this.fs.getFileStatus(dirPath).isDirectory());
+    assertFalse("Should not be file", this.fs.getFileStatus(dirPath).isFile());
+  }
+
+  public void testMkdirsForExistingFile() throws Exception {
+    Path testFile = this.path("/test/hadoop/file");
+    assertFalse(this.fs.exists(testFile));
+    this.createFile(testFile);
+    assertTrue(this.fs.exists(testFile));
+    try {
+      this.fs.mkdirs(testFile);
+      fail("Should throw FileAlreadyExistsException!");
+    } catch (FileAlreadyExistsException e) {
+      // expected
+    }
+  }
+
+  public void testWorkingDirectory() throws Exception {
+    Path workDir = super.path(this.getDefaultWorkingDirectory());
+    assertEquals(workDir, this.fs.getWorkingDirectory());
+    this.fs.setWorkingDirectory(super.path("."));
+    assertEquals(workDir, this.fs.getWorkingDirectory());
+    this.fs.setWorkingDirectory(super.path(".."));
+    assertEquals(workDir.getParent(), this.fs.getWorkingDirectory());
+    Path relativeDir = super.path("hadoop");
+    this.fs.setWorkingDirectory(relativeDir);
+    assertEquals(relativeDir, this.fs.getWorkingDirectory());
+    Path absoluteDir = super.path("/test/hadoop");
+    this.fs.setWorkingDirectory(absoluteDir);
+    assertEquals(absoluteDir, this.fs.getWorkingDirectory());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemStore.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemStore.java
new file mode 100644
index 0000000..c849804
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemStore.java
@@ -0,0 +1,121 @@
+/**
+ * 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.aliyun.oss;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.*;
+import java.net.URI;
+import java.security.DigestInputStream;
+import java.security.DigestOutputStream;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeNotNull;
+
+/**
+ * Test the bridging logic between Hadoop's abstract filesystem and
+ * Aliyun OSS.
+ */
+public class TestAliyunOSSFileSystemStore {
+  private Configuration conf;
+  private AliyunOSSFileSystemStore store;
+  private AliyunOSSFileSystem fs;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();
+    fs = new AliyunOSSFileSystem();
+    fs.initialize(URI.create(conf.get("test.fs.oss.name")), conf);
+    store = fs.getStore();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    try {
+      store.purge("test");
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+  }
+
+  @BeforeClass
+  public static void checkSettings() throws Exception {
+    Configuration conf = new Configuration();
+    assumeNotNull(conf.get(Constants.ACCESS_KEY));
+    assumeNotNull(conf.get(Constants.SECRET_KEY));
+    assumeNotNull(conf.get("test.fs.oss.name"));
+  }
+
+  protected void writeRenameReadCompare(Path path, long len)
+      throws IOException, NoSuchAlgorithmException {
+    // If len > fs.oss.multipart.upload.threshold,
+    // we'll use a multipart upload copy
+    MessageDigest digest = MessageDigest.getInstance("MD5");
+    OutputStream out = new BufferedOutputStream(
+        new DigestOutputStream(fs.create(path, false), digest));
+    for (long i = 0; i < len; i++) {
+      out.write('Q');
+    }
+    out.flush();
+    out.close();
+
+    assertTrue("Exists", fs.exists(path));
+
+    Path copyPath = path.suffix(".copy");
+    fs.rename(path, copyPath);
+
+    assertTrue("Copy exists", fs.exists(copyPath));
+
+    // Download file from Aliyun OSS and compare the digest against the original
+    MessageDigest digest2 = MessageDigest.getInstance("MD5");
+    InputStream in = new BufferedInputStream(
+        new DigestInputStream(fs.open(copyPath), digest2));
+    long copyLen = 0;
+    while (in.read() != -1) {
+      copyLen++;
+    }
+    in.close();
+
+    assertEquals("Copy length matches original", len, copyLen);
+    assertArrayEquals("Digests match", digest.digest(), digest2.digest());
+  }
+
+  @Test
+  public void testSmallUpload() throws IOException, NoSuchAlgorithmException {
+    // Regular upload, regular copy
+    writeRenameReadCompare(new Path("/test/small"), 16384);
+  }
+
+  @Test
+  public void testLargeUpload()
+      throws IOException, NoSuchAlgorithmException {
+    // Multipart upload, multipart copy
+    writeRenameReadCompare(new Path("/test/xlarge"), 52428800L); // 50MB byte
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java
new file mode 100644
index 0000000..892eda0
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java
@@ -0,0 +1,142 @@
+/**
+ * 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.aliyun.oss;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.*;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Random;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests basic functionality for AliyunOSSInputStream, including seeking and
+ * reading files.
+ */
+public class TestAliyunOSSInputStream {
+
+  private FileSystem fs;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestAliyunOSSInputStream.class);
+
+  private static String testRootPath =
+      AliyunOSSTestUtils.generateUniqueTestPath();
+
+  @Rule
+  public Timeout testTimeout = new Timeout(30 * 60 * 1000);
+
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = new Configuration();
+    fs = AliyunOSSTestUtils.createTestFileSystem(conf);
+  }
+
+  @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);
+    }
+  }
+
+  @Test
+  public void testSeekFile() 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 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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSOutputStream.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSOutputStream.java
new file mode 100644
index 0000000..6b87d9c
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSOutputStream.java
@@ -0,0 +1,91 @@
+/**
+ * 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.aliyun.oss;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.io.IOException;
+
+/**
+ * Tests regular and multi-part upload functionality for AliyunOSSOutputStream.
+ */
+public class TestAliyunOSSOutputStream {
+  private FileSystem fs;
+  private static String testRootPath =
+      AliyunOSSTestUtils.generateUniqueTestPath();
+
+  @Rule
+  public Timeout testTimeout = new Timeout(30 * 60 * 1000);
+
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setLong(Constants.MIN_MULTIPART_UPLOAD_THRESHOLD_KEY, 5 * 1024 * 1024);
+    conf.setInt(Constants.MULTIPART_UPLOAD_SIZE_KEY, 5 * 1024 * 1024);
+    fs = AliyunOSSTestUtils.createTestFileSystem(conf);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (fs != null) {
+      fs.delete(new Path(testRootPath), true);
+    }
+  }
+
+  protected Path getTestPath() {
+    return new Path(testRootPath + "/test-aliyun-oss");
+  }
+
+  @Test
+  public void testRegularUpload() throws IOException {
+    ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 1024 * 1024);
+  }
+
+  @Test
+  public void testMultiPartUpload() throws IOException {
+    ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 6 * 1024 * 1024);
+  }
+
+  @Test
+  public void testMultiPartUploadLimit() throws IOException {
+    long partSize1 = AliyunOSSUtils.calculatePartSize(10 * 1024, 100 * 1024);
+    assert(10 * 1024 / partSize1 < Constants.MULTIPART_UPLOAD_PART_NUM_LIMIT);
+
+    long partSize2 = AliyunOSSUtils.calculatePartSize(200 * 1024, 100 * 1024);
+    assert(200 * 1024 / partSize2 < Constants.MULTIPART_UPLOAD_PART_NUM_LIMIT);
+
+    long partSize3 = AliyunOSSUtils.calculatePartSize(10000 * 100 * 1024,
+        100 * 1024);
+    assert(10000 * 100 * 1024 / partSize3
+        < Constants.MULTIPART_UPLOAD_PART_NUM_LIMIT);
+
+    long partSize4 = AliyunOSSUtils.calculatePartSize(10001 * 100 * 1024,
+        100 * 1024);
+    assert(10001 * 100 * 1024 / partSize4
+        < Constants.MULTIPART_UPLOAD_PART_NUM_LIMIT);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSTemporaryCredentials.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSTemporaryCredentials.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSTemporaryCredentials.java
new file mode 100644
index 0000000..e106aae
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSTemporaryCredentials.java
@@ -0,0 +1,65 @@
+/**
+ * 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.aliyun.oss;
+
+import com.aliyun.oss.common.auth.Credentials;
+import com.aliyun.oss.common.auth.InvalidCredentialsException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.aliyun.oss.contract.AliyunOSSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
+import org.junit.Test;
+
+import java.net.URI;
+
+import static org.apache.hadoop.fs.aliyun.oss.Constants.ACCESS_KEY;
+import static org.apache.hadoop.fs.aliyun.oss.Constants.SECRET_KEY;
+import static org.apache.hadoop.fs.aliyun.oss.Constants.SECURITY_TOKEN;
+
+/**
+ * Tests use of temporary credentials (for example, Aliyun STS & Aliyun OSS).
+ * This test extends a class that "does things to the root directory", and
+ * should only be used against transient filesystems where you don't care about
+ * the data.
+ */
+public class TestAliyunOSSTemporaryCredentials
+    extends AbstractFSContractTestBase {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new AliyunOSSContract(conf);
+  }
+
+  @Test
+  public void testTemporaryCredentialValidation() throws Throwable {
+    Configuration conf = new Configuration();
+    conf.set(ACCESS_KEY, "accessKeyId");
+    conf.set(SECRET_KEY, "accessKeySecret");
+    conf.set(SECURITY_TOKEN, "");
+    URI uri = getFileSystem().getUri();
+    TemporaryAliyunCredentialsProvider provider
+        = new TemporaryAliyunCredentialsProvider(uri, conf);
+    try {
+      Credentials credentials = provider.getCredentials();
+      fail("Expected a CredentialInitializationException, got " + credentials);
+    } catch (InvalidCredentialsException expected) {
+      // expected
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemContract.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemContract.java
deleted file mode 100644
index f234d50..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemContract.java
+++ /dev/null
@@ -1,243 +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.aliyun.oss;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FileSystemContractBaseTest;
-import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-
-/**
- * Tests a live OSS system.
- *
- * 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 TestOSSFileSystemContract extends FileSystemContractBaseTest {
-
-  protected static final Logger LOG =
-      LoggerFactory.getLogger(TestOSSFileSystemContract.class);
-
-  public static final String TEST_FS_OSS_NAME = "test.fs.oss.name";
-  private static String testRootPath = OSSTestUtils.generateUniqueTestPath();
-
-  @Override
-  public void setUp() throws Exception {
-    Configuration conf = new Configuration();
-    fs = OSSTestUtils.createTestFileSystem(conf);
-    super.setUp();
-  }
-
-  @Override
-  public void tearDown() throws Exception {
-    if (fs != null) {
-      fs.delete(super.path(testRootPath), true);
-    }
-    super.tearDown();
-  }
-
-  @Override
-  protected Path path(String path) {
-    if (path.startsWith("/")) {
-      return super.path(testRootPath + path);
-    } else {
-      return super.path(testRootPath + "/" + path);
-    }
-  }
-
-  @Override
-  public void testMkdirsWithUmask() throws Exception {
-    // not supported
-  }
-
-  @Override
-  public void testRootDirAlwaysExists() throws Exception {
-    //this will throw an exception if the path is not found
-    fs.getFileStatus(super.path("/"));
-    //this catches overrides of the base exists() method that don't
-    //use getFileStatus() as an existence probe
-    assertTrue("FileSystem.exists() fails for root",
-        fs.exists(super.path("/")));
-  }
-
-  @Override
-  public void testRenameRootDirForbidden() throws Exception {
-    if (!renameSupported()) {
-      return;
-    }
-    rename(super.path("/"),
-           super.path("/test/newRootDir"),
-           false, true, false);
-  }
-
-  public void testDeleteSubdir() throws IOException {
-    Path parentDir = this.path("/test/hadoop");
-    Path file = this.path("/test/hadoop/file");
-    Path subdir = this.path("/test/hadoop/subdir");
-    this.createFile(file);
-
-    assertTrue("Created subdir", this.fs.mkdirs(subdir));
-    assertTrue("File exists", this.fs.exists(file));
-    assertTrue("Parent dir exists", this.fs.exists(parentDir));
-    assertTrue("Subdir exists", this.fs.exists(subdir));
-
-    assertTrue("Deleted subdir", this.fs.delete(subdir, true));
-    assertTrue("Parent should exist", this.fs.exists(parentDir));
-
-    assertTrue("Deleted file", this.fs.delete(file, false));
-    assertTrue("Parent should exist", this.fs.exists(parentDir));
-  }
-
-
-  @Override
-  protected boolean renameSupported() {
-    return true;
-  }
-
-  @Override
-  public void testRenameNonExistentPath() throws Exception {
-    if (this.renameSupported()) {
-      Path src = this.path("/test/hadoop/path");
-      Path dst = this.path("/test/new/newpath");
-      try {
-        super.rename(src, dst, false, false, false);
-        fail("Should throw FileNotFoundException!");
-      } catch (FileNotFoundException e) {
-        // expected
-      }
-    }
-  }
-
-  @Override
-  public void testRenameFileMoveToNonExistentDirectory() throws Exception {
-    if (this.renameSupported()) {
-      Path src = this.path("/test/hadoop/file");
-      this.createFile(src);
-      Path dst = this.path("/test/new/newfile");
-      try {
-        super.rename(src, dst, false, true, false);
-        fail("Should throw FileNotFoundException!");
-      } catch (FileNotFoundException e) {
-        // expected
-      }
-    }
-  }
-
-  @Override
-  public void testRenameDirectoryMoveToNonExistentDirectory() throws Exception {
-    if (this.renameSupported()) {
-      Path src = this.path("/test/hadoop/dir");
-      this.fs.mkdirs(src);
-      Path dst = this.path("/test/new/newdir");
-      try {
-        super.rename(src, dst, false, true, false);
-        fail("Should throw FileNotFoundException!");
-      } catch (FileNotFoundException e) {
-        // expected
-      }
-    }
-  }
-
-  @Override
-  public void testRenameFileMoveToExistingDirectory() throws Exception {
-    super.testRenameFileMoveToExistingDirectory();
-  }
-
-  @Override
-  public void testRenameFileAsExistingFile() throws Exception {
-    if (this.renameSupported()) {
-      Path src = this.path("/test/hadoop/file");
-      this.createFile(src);
-      Path dst = this.path("/test/new/newfile");
-      this.createFile(dst);
-      try {
-        super.rename(src, dst, false, true, true);
-        fail("Should throw FileAlreadyExistsException");
-      } catch (FileAlreadyExistsException e) {
-        // expected
-      }
-    }
-  }
-
-  @Override
-  public void testRenameDirectoryAsExistingFile() throws Exception {
-    if (this.renameSupported()) {
-      Path src = this.path("/test/hadoop/dir");
-      this.fs.mkdirs(src);
-      Path dst = this.path("/test/new/newfile");
-      this.createFile(dst);
-      try {
-        super.rename(src, dst, false, true, true);
-        fail("Should throw FileAlreadyExistsException");
-      } catch (FileAlreadyExistsException e) {
-        // expected
-      }
-    }
-  }
-
-  public void testGetFileStatusFileAndDirectory() throws Exception {
-    Path filePath = this.path("/test/oss/file1");
-    this.createFile(filePath);
-    assertTrue("Should be file", this.fs.getFileStatus(filePath).isFile());
-    assertFalse("Should not be directory",
-        this.fs.getFileStatus(filePath).isDirectory());
-
-    Path dirPath = this.path("/test/oss/dir");
-    this.fs.mkdirs(dirPath);
-    assertTrue("Should be directory",
-        this.fs.getFileStatus(dirPath).isDirectory());
-    assertFalse("Should not be file", this.fs.getFileStatus(dirPath).isFile());
-  }
-
-  public void testMkdirsForExistingFile() throws Exception {
-    Path testFile = this.path("/test/hadoop/file");
-    assertFalse(this.fs.exists(testFile));
-    this.createFile(testFile);
-    assertTrue(this.fs.exists(testFile));
-    try {
-      this.fs.mkdirs(testFile);
-      fail("Should throw FileAlreadyExistsException!");
-    } catch (FileAlreadyExistsException e) {
-      // expected
-    }
-  }
-
-  public void testWorkingDirectory() throws Exception {
-    Path workDir = super.path(this.getDefaultWorkingDirectory());
-    assertEquals(workDir, this.fs.getWorkingDirectory());
-    this.fs.setWorkingDirectory(super.path("."));
-    assertEquals(workDir, this.fs.getWorkingDirectory());
-    this.fs.setWorkingDirectory(super.path(".."));
-    assertEquals(workDir.getParent(), this.fs.getWorkingDirectory());
-    Path relativeDir = super.path("hadoop");
-    this.fs.setWorkingDirectory(relativeDir);
-    assertEquals(relativeDir, this.fs.getWorkingDirectory());
-    Path absoluteDir = super.path("/test/hadoop");
-    this.fs.setWorkingDirectory(absoluteDir);
-    assertEquals(absoluteDir, this.fs.getWorkingDirectory());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemStore.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemStore.java
deleted file mode 100644
index 6331ed8..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSFileSystemStore.java
+++ /dev/null
@@ -1,121 +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.aliyun.oss;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.*;
-import java.net.URI;
-import java.security.DigestInputStream;
-import java.security.DigestOutputStream;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assume.assumeNotNull;
-
-/**
- * Test the bridging logic between Hadoop's abstract filesystem and
- * Aliyun OSS.
- */
-public class TestOSSFileSystemStore {
-  private Configuration conf;
-  private AliyunOSSFileSystemStore store;
-  private AliyunOSSFileSystem fs;
-
-  @Before
-  public void setUp() throws Exception {
-    conf = new Configuration();
-    fs = new AliyunOSSFileSystem();
-    fs.initialize(URI.create(conf.get("test.fs.oss.name")), conf);
-    store = fs.getStore();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    try {
-      store.purge("test");
-    } catch (Exception e) {
-      e.printStackTrace();
-      throw e;
-    }
-  }
-
-  @BeforeClass
-  public static void checkSettings() throws Exception {
-    Configuration conf = new Configuration();
-    assumeNotNull(conf.get("fs.oss.accessKeyId"));
-    assumeNotNull(conf.get("fs.oss.accessKeySecret"));
-    assumeNotNull(conf.get("test.fs.oss.name"));
-  }
-
-  protected void writeRenameReadCompare(Path path, long len)
-      throws IOException, NoSuchAlgorithmException {
-    // If len > fs.oss.multipart.upload.threshold,
-    // we'll use a multipart upload copy
-    MessageDigest digest = MessageDigest.getInstance("MD5");
-    OutputStream out = new BufferedOutputStream(
-        new DigestOutputStream(fs.create(path, false), digest));
-    for (long i = 0; i < len; i++) {
-      out.write('Q');
-    }
-    out.flush();
-    out.close();
-
-    assertTrue("Exists", fs.exists(path));
-
-    Path copyPath = path.suffix(".copy");
-    fs.rename(path, copyPath);
-
-    assertTrue("Copy exists", fs.exists(copyPath));
-
-    // Download file from Aliyun OSS and compare the digest against the original
-    MessageDigest digest2 = MessageDigest.getInstance("MD5");
-    InputStream in = new BufferedInputStream(
-        new DigestInputStream(fs.open(copyPath), digest2));
-    long copyLen = 0;
-    while (in.read() != -1) {
-      copyLen++;
-    }
-    in.close();
-
-    assertEquals("Copy length matches original", len, copyLen);
-    assertArrayEquals("Digests match", digest.digest(), digest2.digest());
-  }
-
-  @Test
-  public void testSmallUpload() throws IOException, NoSuchAlgorithmException {
-    // Regular upload, regular copy
-    writeRenameReadCompare(new Path("/test/small"), 16384);
-  }
-
-  @Test
-  public void testLargeUpload()
-      throws IOException, NoSuchAlgorithmException {
-    // Multipart upload, multipart copy
-    writeRenameReadCompare(new Path("/test/xlarge"), 52428800L); // 50MB byte
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSInputStream.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSInputStream.java
deleted file mode 100644
index 411cd57..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSInputStream.java
+++ /dev/null
@@ -1,141 +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.aliyun.oss;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.apache.hadoop.io.IOUtils;
-import org.junit.*;
-import org.junit.rules.Timeout;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Random;
-
-import static org.junit.Assert.assertTrue;
-
-/**
- * Tests basic functionality for AliyunOSSInputStream, including seeking and
- * reading files.
- */
-public class TestOSSInputStream {
-
-  private FileSystem fs;
-
-  protected static final Logger LOG =
-      LoggerFactory.getLogger(TestOSSInputStream.class);
-
-  private static String testRootPath = OSSTestUtils.generateUniqueTestPath();
-
-  @Rule
-  public Timeout testTimeout = new Timeout(30 * 60 * 1000);
-
-  @Before
-  public void setUp() throws Exception {
-    Configuration conf = new Configuration();
-    fs = OSSTestUtils.createTestFileSystem(conf);
-  }
-
-  @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);
-    }
-  }
-
-  @Test
-  public void testSeekFile() 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 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);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSOutputStream.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSOutputStream.java
deleted file mode 100644
index b33ab99..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSOutputStream.java
+++ /dev/null
@@ -1,90 +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.aliyun.oss;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-
-import java.io.IOException;
-
-/**
- * Tests regular and multi-part upload functionality for AliyunOSSOutputStream.
- */
-public class TestOSSOutputStream {
-  private FileSystem fs;
-  private static String testRootPath = OSSTestUtils.generateUniqueTestPath();
-
-  @Rule
-  public Timeout testTimeout = new Timeout(30 * 60 * 1000);
-
-  @Before
-  public void setUp() throws Exception {
-    Configuration conf = new Configuration();
-    conf.setLong(Constants.MIN_MULTIPART_UPLOAD_THRESHOLD_KEY, 5 * 1024 * 1024);
-    conf.setInt(Constants.MULTIPART_UPLOAD_SIZE_KEY, 5 * 1024 * 1024);
-    fs = OSSTestUtils.createTestFileSystem(conf);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (fs != null) {
-      fs.delete(new Path(testRootPath), true);
-    }
-  }
-
-  protected Path getTestPath() {
-    return new Path(testRootPath + "/testoss");
-  }
-
-  @Test
-  public void testRegularUpload() throws IOException {
-    ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 1024 * 1024);
-  }
-
-  @Test
-  public void testMultiPartUpload() throws IOException {
-    ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 6 * 1024 * 1024);
-  }
-
-  @Test
-  public void testMultiPartUploadLimit() throws IOException {
-    long partSize1 = AliyunOSSUtils.calculatePartSize(10 * 1024, 100 * 1024);
-    assert(10 * 1024 / partSize1 < Constants.MULTIPART_UPLOAD_PART_NUM_LIMIT);
-
-    long partSize2 = AliyunOSSUtils.calculatePartSize(200 * 1024, 100 * 1024);
-    assert(200 * 1024 / partSize2 < Constants.MULTIPART_UPLOAD_PART_NUM_LIMIT);
-
-    long partSize3 = AliyunOSSUtils.calculatePartSize(10000 * 100 * 1024,
-        100 * 1024);
-    assert(10000 * 100 * 1024 / partSize3
-        < Constants.MULTIPART_UPLOAD_PART_NUM_LIMIT);
-
-    long partSize4 = AliyunOSSUtils.calculatePartSize(10001 * 100 * 1024,
-        100 * 1024);
-    assert(10001 * 100 * 1024 / partSize4
-        < Constants.MULTIPART_UPLOAD_PART_NUM_LIMIT);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSTemporaryCredentials.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSTemporaryCredentials.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSTemporaryCredentials.java
deleted file mode 100644
index ca2c0bc..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestOSSTemporaryCredentials.java
+++ /dev/null
@@ -1,64 +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.aliyun.oss;
-
-import com.aliyun.oss.common.auth.Credentials;
-import com.aliyun.oss.common.auth.InvalidCredentialsException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.aliyun.oss.contract.OSSContract;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
-import org.junit.Test;
-
-import java.net.URI;
-
-import static org.apache.hadoop.fs.aliyun.oss.Constants.ACCESS_KEY;
-import static org.apache.hadoop.fs.aliyun.oss.Constants.SECRET_KEY;
-import static org.apache.hadoop.fs.aliyun.oss.Constants.SECURITY_TOKEN;
-
-/**
- * Tests use of temporary credentials (for example, Aliyun STS & Aliyun OSS).
- * This test extends a class that "does things to the root directory", and
- * should only be used against transient filesystems where you don't care about
- * the data.
- */
-public class TestOSSTemporaryCredentials extends AbstractFSContractTestBase {
-
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new OSSContract(conf);
-  }
-
-  @Test
-  public void testTemporaryCredentialValidation() throws Throwable {
-    Configuration conf = new Configuration();
-    conf.set(ACCESS_KEY, "accessKeyId");
-    conf.set(SECRET_KEY, "accessKeySecret");
-    conf.set(SECURITY_TOKEN, "");
-    URI uri = getFileSystem().getUri();
-    TemporaryAliyunCredentialsProvider provider
-        = new TemporaryAliyunCredentialsProvider(uri, conf);
-    try {
-      Credentials credentials = provider.getCredentials();
-      fail("Expected a CredentialInitializationException, got " + credentials);
-    } catch (InvalidCredentialsException expected) {
-      // expected
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/AliyunOSSContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/AliyunOSSContract.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/AliyunOSSContract.java
new file mode 100644
index 0000000..624c606
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/AliyunOSSContract.java
@@ -0,0 +1,49 @@
+/**
+ * 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.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
+
+/**
+ * The contract of Aliyun OSS: only enabled if the test bucket is provided.
+ */
+public class AliyunOSSContract extends AbstractBondedFSContract {
+
+  public static final String CONTRACT_XML = "contract/aliyun-oss.xml";
+
+  public AliyunOSSContract(Configuration conf) {
+    super(conf);
+    //insert the base features
+    addConfResource(CONTRACT_XML);
+  }
+
+  @Override
+  public String getScheme() {
+    return "oss";
+  }
+
+  @Override
+  public Path getTestPath() {
+    String testUniqueForkId = System.getProperty("test.unique.fork.id");
+    return testUniqueForkId == null ? super.getTestPath() :
+        new Path("/" + testUniqueForkId, "test");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/OSSContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/OSSContract.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/OSSContract.java
deleted file mode 100644
index f90a8bb..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/OSSContract.java
+++ /dev/null
@@ -1,53 +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.aliyun.oss.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.aliyun.oss.OSSTestUtils;
-import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
-
-/**
- * The contract of OSS: only enabled if the test bucket is provided.
- */
-public class OSSContract extends AbstractBondedFSContract {
-
-  public static final String CONTRACT_XML = "contract/oss.xml";
-  public static final String CONTRACT_TEST_OSS_FS_NAME =
-      "fs.contract.test.fs.oss";
-
-  private static String testPath = OSSTestUtils.generateUniqueTestPath();
-
-  public OSSContract(Configuration conf) {
-    super(conf);
-    //insert the base features
-    addConfResource(CONTRACT_XML);
-  }
-
-  @Override
-  public String getScheme() {
-    return "oss";
-  }
-
-  @Override
-  public Path getTestPath() {
-    Path path = new Path(testPath);
-    return path;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractCreate.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractCreate.java
new file mode 100644
index 0000000..88dd8cd
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractCreate.java
@@ -0,0 +1,35 @@
+/**
+ * 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.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Aliyun OSS contract creating tests.
+ */
+public class TestAliyunOSSContractCreate extends AbstractContractCreateTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new AliyunOSSContract(conf);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDelete.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDelete.java
new file mode 100644
index 0000000..1658d80
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDelete.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Aliyun OSS contract deleting tests.
+ */
+public class TestAliyunOSSContractDelete extends AbstractContractDeleteTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new AliyunOSSContract(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDispCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDispCp.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDispCp.java
new file mode 100644
index 0000000..4b482fc
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDispCp.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
+
+import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
+
+/**
+ * Contract test suite covering Aliyun OSS integration with DistCp.
+ */
+public class TestAliyunOSSContractDispCp extends AbstractContractDistCpTest {
+
+  private static final long MULTIPART_SETTING = 8 * 1024 * 1024; // 8 MB
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration newConf = super.createConfiguration();
+    newConf.setLong(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY, MULTIPART_SETTING);
+    newConf.setLong(MULTIPART_UPLOAD_SIZE_KEY, MULTIPART_SETTING);
+    return newConf;
+  }
+
+  @Override
+  protected AliyunOSSContract createContract(Configuration conf) {
+    return new AliyunOSSContract(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractGetFileStatus.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractGetFileStatus.java
new file mode 100644
index 0000000..c69124d
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractGetFileStatus.java
@@ -0,0 +1,35 @@
+/**
+ * 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.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Test getFileStatus and related listing operations.
+ */
+public class TestAliyunOSSContractGetFileStatus
+    extends AbstractContractGetFileStatusTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new AliyunOSSContract(conf);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractMkdir.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractMkdir.java
new file mode 100644
index 0000000..6cb7549
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractMkdir.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Aliyun OSS contract directory tests.
+ */
+public class TestAliyunOSSContractMkdir extends AbstractContractMkdirTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new AliyunOSSContract(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractOpen.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractOpen.java
new file mode 100644
index 0000000..099aba6
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractOpen.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Aliyun OSS contract opening file tests.
+ */
+public class TestAliyunOSSContractOpen extends AbstractContractOpenTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new AliyunOSSContract(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractRename.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractRename.java
new file mode 100644
index 0000000..e15b3ba
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractRename.java
@@ -0,0 +1,35 @@
+/**
+ * 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.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Aliyun OSS contract renaming tests.
+ */
+public class TestAliyunOSSContractRename extends AbstractContractRenameTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new AliyunOSSContract(conf);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractRootDir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractRootDir.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractRootDir.java
new file mode 100644
index 0000000..9faae37
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractRootDir.java
@@ -0,0 +1,69 @@
+/*
+ * 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.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+/**
+ * Root dir operations against an Aliyun OSS bucket.
+ */
+public class TestAliyunOSSContractRootDir extends
+    AbstractContractRootDirectoryTest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestAliyunOSSContractRootDir.class);
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new AliyunOSSContract(conf);
+  }
+
+  @Override
+  public void testListEmptyRootDirectory() throws IOException {
+    for (int attempt = 1, maxAttempts = 10; attempt <= maxAttempts; ++attempt) {
+      try {
+        super.testListEmptyRootDirectory();
+        break;
+      } catch (AssertionError | FileNotFoundException e) {
+        if (attempt < maxAttempts) {
+          LOG.info("Attempt {} of {} for empty root directory test failed.  "
+              + "Attempting retry.", attempt, maxAttempts);
+          try {
+            Thread.sleep(1000);
+          } catch (InterruptedException e2) {
+            Thread.currentThread().interrupt();
+            fail("Test interrupted.");
+            break;
+          }
+        } else {
+          LOG.error(
+              "Empty root directory test failed {} attempts.  Failing test.",
+              maxAttempts);
+          throw e;
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractSeek.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractSeek.java
new file mode 100644
index 0000000..b247ab1
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractSeek.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.aliyun.oss.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Aliyun OSS contract seeking tests.
+ */
+public class TestAliyunOSSContractSeek extends AbstractContractSeekTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new AliyunOSSContract(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractCreate.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractCreate.java
deleted file mode 100644
index ce927a9..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractCreate.java
+++ /dev/null
@@ -1,35 +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.aliyun.oss.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-/**
- * OSS contract creating tests.
- */
-public class TestOSSContractCreate extends AbstractContractCreateTest {
-
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new OSSContract(conf);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1940464/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractDelete.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractDelete.java
deleted file mode 100644
index 6a1eb40..0000000
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestOSSContractDelete.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.aliyun.oss.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-/**
- * OSS contract deleting tests.
- */
-public class TestOSSContractDelete extends AbstractContractDeleteTest {
-
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new OSSContract(conf);
-  }
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[20/23] hadoop git commit: HADOOP-13634. Some configuration in doc has been outdated. Contributed by Genmao Yu

Posted by dr...@apache.org.
HADOOP-13634. Some configuration in doc has been outdated. Contributed by Genmao Yu


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/26d5df39
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/26d5df39
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/26d5df39

Branch: refs/heads/trunk
Commit: 26d5df390cf976dcc1d17fc68d0fed789dc34e84
Parents: 846c5ce
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Sep 23 08:44:28 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Fri Sep 23 08:44:28 2016 +0800

----------------------------------------------------------------------
 .../src/site/markdown/tools/hadoop-aliyun/index.md            | 7 +------
 1 file changed, 1 insertion(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/26d5df39/hadoop-tools/hadoop-aliyun/src/site/markdown/tools/hadoop-aliyun/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/site/markdown/tools/hadoop-aliyun/index.md b/hadoop-tools/hadoop-aliyun/src/site/markdown/tools/hadoop-aliyun/index.md
index 4095e06..88c83b5 100644
--- a/hadoop-tools/hadoop-aliyun/src/site/markdown/tools/hadoop-aliyun/index.md
+++ b/hadoop-tools/hadoop-aliyun/src/site/markdown/tools/hadoop-aliyun/index.md
@@ -167,7 +167,7 @@ please raise your issues with them.
 
     <property>
       <name>fs.oss.paging.maximum</name>
-      <value>500</value>
+      <value>1000</value>
       <description>How many keys to request from Aliyun OSS when doing directory listings at a time.
       </description>
     </property>
@@ -196,11 +196,6 @@ please raise your issues with them.
     </property>
 
     <property>
-      <name>fs.oss.buffer.dir</name>
-      <description>Comma separated list of directories to buffer OSS data before uploading to Aliyun OSS</description>
-    </property>
-
-    <property>
       <name>fs.oss.acl.default</name>
       <value></vaule>
       <description>Set a canned ACL for bucket. Value may be private, public-read, public-read-write.


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org