浏览代码

HADOOP-13591. Unit test failure in TestOSSContractGetFileStatus and TestOSSContractRootDir. Contributed by Genmao Yu

Kai Zheng 8 年之前
父节点
当前提交
08b37603d9

+ 38 - 50
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;

+ 22 - 14
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);

+ 18 - 6
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;
+    }
+  }
 }

+ 1 - 1
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);

+ 10 - 0
hadoop-tools/hadoop-aliyun/src/test/resources/contract/aliyun-oss.xml

@@ -98,6 +98,16 @@
         <value>true</value>
     </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>