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 xg...@apache.org on 2017/06/05 20:31:01 UTC

[12/50] [abbrv] hadoop git commit: HADOOP-14442. Owner support for ranger-wasb integration. Contributed by Varada Hemeswari

HADOOP-14442. Owner support for ranger-wasb integration. Contributed by Varada Hemeswari


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

Branch: refs/heads/YARN-5734
Commit: e47993cc5547130a6b5235d4315f805fbd939694
Parents: 67efc84
Author: Mingliang Liu <li...@apache.org>
Authored: Fri May 26 17:52:56 2017 -0700
Committer: Xuan <xg...@apache.org>
Committed: Mon Jun 5 13:29:27 2017 -0700

----------------------------------------------------------------------
 .../hadoop/fs/azure/NativeAzureFileSystem.java  |  48 +++++++-
 .../fs/azure/RemoteWasbAuthorizerImpl.java      |  12 +-
 .../fs/azure/WasbAuthorizerInterface.java       |   3 +-
 .../hadoop/fs/azure/MockWasbAuthorizerImpl.java |  52 ++++++--
 .../TestNativeAzureFileSystemAuthorization.java |  90 ++++----------
 ...veAzureFileSystemAuthorizationWithOwner.java | 122 +++++++++++++++++++
 6 files changed, 247 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e47993cc/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
index 0ba47ef..b61baab 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
@@ -200,7 +200,7 @@ public class NativeAzureFileSystem extends FileSystem {
       JsonNode oldFolderName = json.get("OldFolderName");
       JsonNode newFolderName = json.get("NewFolderName");
       if (oldFolderName == null || newFolderName == null) {
-    	  this.committed = false;
+        this.committed = false;
       } else {
         this.srcKey = oldFolderName.textValue();
         this.dstKey = newFolderName.textValue();
@@ -349,7 +349,7 @@ public class NativeAzureFileSystem extends FileSystem {
 
       return contents;
     }
-    
+
     /**
      * This is an exact copy of org.codehaus.jettison.json.JSONObject.quote 
      * method.
@@ -639,7 +639,7 @@ public class NativeAzureFileSystem extends FileSystem {
     return "wasb";
   }
 
-  
+
   /**
    * <p>
    * A {@link FileSystem} for reading and writing files stored on <a
@@ -1441,12 +1441,14 @@ public class NativeAzureFileSystem extends FileSystem {
       requestingAccessForPath = requestingAccessForPath.makeQualified(getUri(), getWorkingDirectory());
       originalPath = originalPath.makeQualified(getUri(), getWorkingDirectory());
 
-      if (!this.authorizer.authorize(requestingAccessForPath.toString(), accessType.toString())) {
+      String owner = getOwnerForPath(requestingAccessForPath);
+
+      if (!this.authorizer.authorize(requestingAccessForPath.toString(), accessType.toString(), owner)) {
         throw new WasbAuthorizationException(operation
             + " operation for Path : " + originalPath.toString() + " not allowed");
       }
 
-    }
+   }
   }
 
   /**
@@ -3173,4 +3175,40 @@ public class NativeAzureFileSystem extends FileSystem {
     // Return to the caller with the randomized key.
     return randomizedKey;
   }
+
+  /*
+   * Helper method to retrieve owner information for a given path.
+   * The method returns empty string in case the file is not found or the metadata does not contain owner information
+  */
+  @VisibleForTesting
+  public String getOwnerForPath(Path absolutePath) throws IOException {
+    String owner = "";
+    FileMetadata meta = null;
+    String key = pathToKey(absolutePath);
+    try {
+
+      meta = store.retrieveMetadata(key);
+
+      if (meta != null) {
+        owner = meta.getPermissionStatus().getUserName();
+        LOG.debug("Retrieved '{}' as owner for path - {}", owner, absolutePath);
+      } else {
+        // meta will be null if file/folder doen not exist
+        LOG.debug("Cannot find file/folder - '{}'. Returning owner as empty string", absolutePath);
+      }
+    } catch(IOException ex) {
+
+          Throwable innerException = NativeAzureFileSystemHelper.checkForAzureStorageException(ex);
+          boolean isfileNotFoundException = innerException instanceof StorageException
+            && NativeAzureFileSystemHelper.isFileNotFoundException((StorageException) innerException);
+
+          // should not throw when the exception is related to blob/container/file/folder not found
+          if (!isfileNotFoundException) {
+            String errorMsg = "Could not retrieve owner information for path - " + absolutePath;
+            LOG.error(errorMsg);
+            throw new IOException(errorMsg, ex);
+          }
+      }
+    return owner;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e47993cc/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/RemoteWasbAuthorizerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/RemoteWasbAuthorizerImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/RemoteWasbAuthorizerImpl.java
index ea08b2b..3c912d7 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/RemoteWasbAuthorizerImpl.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/RemoteWasbAuthorizerImpl.java
@@ -87,6 +87,12 @@ public class RemoteWasbAuthorizerImpl implements WasbAuthorizerInterface {
   private static final String DELEGATION_TOKEN_QUERY_PARAM_NAME =
       "delegation";
 
+  /**
+   *  Query parameter name for sending owner of the specific resource {@value}
+   */
+  private static final String WASB_RESOURCE_OWNER_QUERY_PARAM_NAME =
+      "wasb_resource_owner";
+
   private WasbRemoteCallHelper remoteCallHelper = null;
   private String delegationToken;
   private boolean isSecurityEnabled;
@@ -119,7 +125,7 @@ public class RemoteWasbAuthorizerImpl implements WasbAuthorizerInterface {
   }
 
   @Override
-  public boolean authorize(String wasbAbsolutePath, String accessType)
+  public boolean authorize(String wasbAbsolutePath, String accessType, String resourceOwner)
       throws WasbAuthorizationException, IOException {
 
       try {
@@ -140,6 +146,10 @@ public class RemoteWasbAuthorizerImpl implements WasbAuthorizerInterface {
           uriBuilder.addParameter(DELEGATION_TOKEN_QUERY_PARAM_NAME,
               delegationToken);
         }
+        if (resourceOwner != null && StringUtils.isNotEmpty(resourceOwner)) {
+          uriBuilder.addParameter(WASB_RESOURCE_OWNER_QUERY_PARAM_NAME,
+              resourceOwner);
+        }
 
         String responseBody = null;
         UserGroupInformation ugi = UserGroupInformation.getCurrentUser();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e47993cc/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/WasbAuthorizerInterface.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/WasbAuthorizerInterface.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/WasbAuthorizerInterface.java
index 57d7516..af0e954 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/WasbAuthorizerInterface.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/WasbAuthorizerInterface.java
@@ -43,10 +43,11 @@ public interface WasbAuthorizerInterface {
 
    * @param wasbAbolutePath : Absolute WASB Path used for access.
    * @param accessType : Type of access
+   * @param owner : owner of the file/folder specified in the wasb path
    * @return : true - If access allowed false - If access is not allowed.
    * @throws WasbAuthorizationException - On authorization exceptions
    * @throws IOException - When not able to reach the authorizer
    */
-  boolean authorize(String wasbAbolutePath, String accessType)
+  boolean authorize(String wasbAbolutePath, String accessType, String owner)
       throws WasbAuthorizationException, IOException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e47993cc/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockWasbAuthorizerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockWasbAuthorizerImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockWasbAuthorizerImpl.java
index 0b3422c..90a6b51 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockWasbAuthorizerImpl.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockWasbAuthorizerImpl.java
@@ -22,7 +22,9 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.regex.Pattern;
 
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.fs.Path;
 
 /**
@@ -32,8 +34,9 @@ import org.apache.hadoop.fs.Path;
 public class MockWasbAuthorizerImpl implements WasbAuthorizerInterface {
 
   private Map<AuthorizationComponent, Boolean> authRules;
+  private boolean performOwnerMatch;
 
-  // The full qualified URL to the root directory
+ // The full qualified URL to the root directory
   private String qualifiedPrefixUrl;
 
   public MockWasbAuthorizerImpl(NativeAzureFileSystem fs) {
@@ -43,15 +46,22 @@ public class MockWasbAuthorizerImpl implements WasbAuthorizerInterface {
 
   @Override
   public void init(Configuration conf) {
+    init(conf, false);
+  }
+
+  /*
+  authorization matches owner with currentUserShortName while evaluating auth rules
+  if currentUserShortName is set to a string that is not empty
+  */
+  public void init(Configuration conf, boolean matchOwner) {
     authRules = new HashMap<AuthorizationComponent, Boolean>();
+    this.performOwnerMatch = matchOwner;
   }
 
   public void addAuthRule(String wasbAbsolutePath,
       String accessType, boolean access) {
-
-    wasbAbsolutePath = qualifiedPrefixUrl + wasbAbsolutePath;
-
-    AuthorizationComponent component = wasbAbsolutePath.endsWith("*")
+        wasbAbsolutePath = qualifiedPrefixUrl + wasbAbsolutePath;
+        AuthorizationComponent component = wasbAbsolutePath.endsWith("*")
         ? new AuthorizationComponent("^" + wasbAbsolutePath.replace("*", ".*"), accessType)
         : new AuthorizationComponent(wasbAbsolutePath, accessType);
 
@@ -59,18 +69,40 @@ public class MockWasbAuthorizerImpl implements WasbAuthorizerInterface {
   }
 
   @Override
-  public boolean authorize(String wasbAbsolutePath, String accessType)
+  public boolean authorize(String wasbAbsolutePath, String accessType, String owner)
       throws WasbAuthorizationException {
 
     if (wasbAbsolutePath.endsWith(NativeAzureFileSystem.FolderRenamePending.SUFFIX)) {
       return true;
     }
 
+    String currentUserShortName = "";
+    if (this.performOwnerMatch) {
+      try {
+        UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+        currentUserShortName = ugi.getShortUserName();
+      } catch (Exception e) {
+        //no op
+      }
+    }
+
+    // In case of root("/"), owner match does not happen because owner is returned as empty string.
+    // we try to force owner match just for purpose of tests to make sure all operations work seemlessly with owner.
+    if (this.performOwnerMatch
+      && StringUtils.equalsIgnoreCase(wasbAbsolutePath, qualifiedPrefixUrl + "/")) {
+      owner = currentUserShortName;
+    }
+
+    boolean shouldEvaluateOwnerAccess = owner != null && !owner.isEmpty()
+      && this.performOwnerMatch;
+
+    boolean isOwnerMatch =  StringUtils.equalsIgnoreCase(currentUserShortName, owner);
+
     AuthorizationComponent component =
         new AuthorizationComponent(wasbAbsolutePath, accessType);
 
     if (authRules.containsKey(component)) {
-      return authRules.get(component);
+      return shouldEvaluateOwnerAccess ? isOwnerMatch && authRules.get(component) : authRules.get(component);
     } else {
       // Regex-pattern match if we don't have a straight match
       for (Map.Entry<AuthorizationComponent, Boolean> entry : authRules.entrySet()) {
@@ -79,12 +111,16 @@ public class MockWasbAuthorizerImpl implements WasbAuthorizerInterface {
         String keyAccess = key.getAccessType();
 
         if (keyPath.endsWith("*") && Pattern.matches(keyPath, wasbAbsolutePath) && keyAccess.equals(accessType)) {
-          return entry.getValue();
+          return shouldEvaluateOwnerAccess ? isOwnerMatch && entry.getValue() : entry.getValue();
         }
       }
       return false;
     }
   }
+
+  public void deleteAllAuthRules() {
+    authRules.clear();
+  }
 }
 
 class AuthorizationComponent {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e47993cc/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorization.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorization.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorization.java
index f7a2eb7..a0276cb5 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorization.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorization.java
@@ -29,6 +29,7 @@ import org.junit.Rule;
 import org.junit.Test;
 
 import org.junit.rules.ExpectedException;
+import com.google.common.annotations.VisibleForTesting;
 
 import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.KEY_USE_SECURE_MODE;
 
@@ -38,6 +39,9 @@ import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.KEY_USE_SECU
 public class TestNativeAzureFileSystemAuthorization
   extends AbstractWasbTestBase {
 
+  @VisibleForTesting
+  protected MockWasbAuthorizerImpl authorizer;
+
   @Override
   protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
     Configuration conf = new Configuration();
@@ -54,9 +58,8 @@ public class TestNativeAzureFileSystemAuthorization
     Assume.assumeTrue("Test valid when both SecureMode and Authorization are enabled .. skipping",
         useSecureMode && useAuthorization);
 
-    Assume.assumeTrue(
-        useSecureMode && useAuthorization
-    );
+    authorizer = new MockWasbAuthorizerImpl(fs);
+    authorizer.init(null);
   }
 
 
@@ -66,12 +69,12 @@ public class TestNativeAzureFileSystemAuthorization
   /**
    * Setup up permissions to allow a recursive delete for cleanup purposes.
    */
-  private void allowRecursiveDelete(NativeAzureFileSystem fs, MockWasbAuthorizerImpl authorizer, String path) {
+  protected void allowRecursiveDelete(NativeAzureFileSystem fs, String path) {
 
     int index = path.lastIndexOf('/');
     String parent = (index == 0) ? "/" : path.substring(0, index);
 
-    authorizer.init(null);
+    authorizer.deleteAllAuthRules();
     authorizer.addAuthRule(parent, WasbAuthorizationOperations.WRITE.toString(), true);
     authorizer.addAuthRule((path.endsWith("*") ? path : path+"*"), WasbAuthorizationOperations.WRITE.toString(), true);
     fs.updateWasbAuthorizer(authorizer);
@@ -80,7 +83,7 @@ public class TestNativeAzureFileSystemAuthorization
   /**
    * Setup the expected exception class, and exception message that the test is supposed to fail with
    */
-  private void setExpectedFailureMessage(String operation, Path path) {
+  protected void setExpectedFailureMessage(String operation, Path path) {
     expectedEx.expect(WasbAuthorizationException.class);
     expectedEx.expectMessage(String.format("%s operation for Path : %s not allowed",
         operation, path.makeQualified(fs.getUri(), fs.getWorkingDirectory())));
@@ -98,8 +101,6 @@ public class TestNativeAzureFileSystemAuthorization
     Path parentDir = new Path("/");
     Path testPath = new Path(parentDir, "test.dat");
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
     authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.READ.toString(), true);
     fs.updateWasbAuthorizer(authorizer);
@@ -126,8 +127,6 @@ public class TestNativeAzureFileSystemAuthorization
     Path parentDir = new Path("/testCreateAccessCheckPositive/1/2/3");
     Path testPath = new Path(parentDir, "test.dat");
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
     authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.READ.toString(), true);
     fs.updateWasbAuthorizer(authorizer);
@@ -137,7 +136,7 @@ public class TestNativeAzureFileSystemAuthorization
       ContractTestUtils.assertPathExists(fs, "testPath was not created", testPath);
     }
     finally {
-      allowRecursiveDelete(fs, authorizer, "/testCreateAccessCheckPositive");
+      allowRecursiveDelete(fs, "/testCreateAccessCheckPositive");
       fs.delete(new Path("/testCreateAccessCheckPositive"), true);
     }
   }
@@ -156,8 +155,6 @@ public class TestNativeAzureFileSystemAuthorization
 
     setExpectedFailureMessage("create", testPath);
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
     authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.READ.toString(), true);
     fs.updateWasbAuthorizer(authorizer);
@@ -186,8 +183,6 @@ public class TestNativeAzureFileSystemAuthorization
     Path parentDir = new Path("/");
     Path testPath = new Path(parentDir, "test.dat");
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
     authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.READ.toString(), true);
     authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.WRITE.toString(), true);
@@ -219,8 +214,6 @@ public class TestNativeAzureFileSystemAuthorization
 
     setExpectedFailureMessage("create", testPath);
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), false);
     fs.updateWasbAuthorizer(authorizer);
 
@@ -229,7 +222,7 @@ public class TestNativeAzureFileSystemAuthorization
     }
     finally {
       /* Provide permissions to cleanup in case the file got created */
-      allowRecursiveDelete(fs, authorizer, parentDir.toString());
+      allowRecursiveDelete(fs, parentDir.toString());
       fs.delete(parentDir, true);
     }
   }
@@ -245,8 +238,6 @@ public class TestNativeAzureFileSystemAuthorization
     Path intermediateFolders = new Path(parentDir, "1/2/3/");
     Path testPath = new Path(intermediateFolders, "test.dat");
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
     authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.READ.toString(), true);
     fs.updateWasbAuthorizer(authorizer);
@@ -256,7 +247,7 @@ public class TestNativeAzureFileSystemAuthorization
       fs.listStatus(testPath);
     }
     finally {
-      allowRecursiveDelete(fs, authorizer, parentDir.toString());
+      allowRecursiveDelete(fs, parentDir.toString());
       fs.delete(parentDir, true);
     }
   }
@@ -274,8 +265,6 @@ public class TestNativeAzureFileSystemAuthorization
 
     setExpectedFailureMessage("liststatus", testPath);
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
     authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.READ.toString(), false);
     fs.updateWasbAuthorizer(authorizer);
@@ -285,7 +274,7 @@ public class TestNativeAzureFileSystemAuthorization
       fs.listStatus(testPath);
     }
     finally {
-      allowRecursiveDelete(fs, authorizer, parentDir.toString());
+      allowRecursiveDelete(fs, parentDir.toString());
       fs.delete(parentDir, true);
     }
   }
@@ -301,8 +290,6 @@ public class TestNativeAzureFileSystemAuthorization
     Path srcPath = new Path(parentDir, "test1.dat");
     Path dstPath = new Path(parentDir, "test2.dat");
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true); /* to create parentDir */
     authorizer.addAuthRule(parentDir.toString(), WasbAuthorizationOperations.WRITE.toString(), true); /* for rename */
     authorizer.addAuthRule(srcPath.toString(), WasbAuthorizationOperations.READ.toString(), true); /* for exists */
@@ -317,7 +304,7 @@ public class TestNativeAzureFileSystemAuthorization
       ContractTestUtils.assertPathDoesNotExist(fs, "sourcePath exists after rename!", srcPath);
     }
     finally {
-      allowRecursiveDelete(fs, authorizer, parentDir.toString());
+      allowRecursiveDelete(fs, parentDir.toString());
       fs.delete(parentDir, true);
     }
   }
@@ -335,8 +322,6 @@ public class TestNativeAzureFileSystemAuthorization
 
     setExpectedFailureMessage("rename", srcPath);
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true); /* to create parent dir */
     authorizer.addAuthRule(parentDir.toString(), WasbAuthorizationOperations.WRITE.toString(), false);
     authorizer.addAuthRule(srcPath.toString(), WasbAuthorizationOperations.READ.toString(), true);
@@ -351,7 +336,7 @@ public class TestNativeAzureFileSystemAuthorization
     } finally {
       ContractTestUtils.assertPathExists(fs, "sourcePath does not exist after rename failure!", srcPath);
 
-      allowRecursiveDelete(fs, authorizer, parentDir.toString());
+      allowRecursiveDelete(fs, parentDir.toString());
       fs.delete(parentDir, true);
     }
   }
@@ -370,8 +355,6 @@ public class TestNativeAzureFileSystemAuthorization
 
     setExpectedFailureMessage("rename", dstPath);
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true); /* to create parent dir */
     authorizer.addAuthRule(parentSrcDir.toString(), WasbAuthorizationOperations.WRITE.toString(), true);
     authorizer.addAuthRule(parentDstDir.toString(), WasbAuthorizationOperations.WRITE.toString(), false);
@@ -386,7 +369,7 @@ public class TestNativeAzureFileSystemAuthorization
       ContractTestUtils.assertPathDoesNotExist(fs, "destPath does not exist", dstPath);
     } finally {
       ContractTestUtils.assertPathExists(fs, "sourcePath does not exist after rename !", srcPath);
-      allowRecursiveDelete(fs, authorizer, parentSrcDir.toString());
+      allowRecursiveDelete(fs, parentSrcDir.toString());
       fs.delete(parentSrcDir, true);
     }
   }
@@ -403,8 +386,6 @@ public class TestNativeAzureFileSystemAuthorization
     Path parentDstDir = new Path("/testRenameAccessCheckPositiveDst");
     Path dstPath = new Path(parentDstDir, "test2.dat");
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true); /* to create parent dirs */
     authorizer.addAuthRule(parentSrcDir.toString(), WasbAuthorizationOperations.WRITE.toString(), true);
     authorizer.addAuthRule(parentDstDir.toString(), WasbAuthorizationOperations.WRITE.toString(), true);
@@ -420,10 +401,10 @@ public class TestNativeAzureFileSystemAuthorization
       ContractTestUtils.assertPathDoesNotExist(fs, "sourcePath does not exist", srcPath);
       ContractTestUtils.assertPathExists(fs, "destPath does not exist", dstPath);
     } finally {
-      allowRecursiveDelete(fs, authorizer, parentSrcDir.toString());
+      allowRecursiveDelete(fs, parentSrcDir.toString());
       fs.delete(parentSrcDir, true);
 
-      allowRecursiveDelete(fs, authorizer, parentDstDir.toString());
+      allowRecursiveDelete(fs, parentDstDir.toString());
       fs.delete(parentDstDir, true);
     }
   }
@@ -438,8 +419,6 @@ public class TestNativeAzureFileSystemAuthorization
     Path parentDir = new Path("/testReadAccessCheckPositive");
     Path testPath = new Path(parentDir, "test.dat");
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
     authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.READ.toString(), true);
     fs.updateWasbAuthorizer(authorizer);
@@ -463,7 +442,7 @@ public class TestNativeAzureFileSystemAuthorization
       if(inputStream != null) {
         inputStream.close();
       }
-      allowRecursiveDelete(fs, authorizer, parentDir.toString());
+      allowRecursiveDelete(fs, parentDir.toString());
       fs.delete(parentDir, true);
     }
   }
@@ -481,8 +460,6 @@ public class TestNativeAzureFileSystemAuthorization
 
     setExpectedFailureMessage("read", testPath);
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
     authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.READ.toString(), false);
     fs.updateWasbAuthorizer(authorizer);
@@ -505,7 +482,7 @@ public class TestNativeAzureFileSystemAuthorization
       if (inputStream != null) {
         inputStream.close();
       }
-      allowRecursiveDelete(fs, authorizer, parentDir.toString());
+      allowRecursiveDelete(fs, parentDir.toString());
       fs.delete(parentDir, true);
     }
   }
@@ -520,8 +497,6 @@ public class TestNativeAzureFileSystemAuthorization
     Path parentDir = new Path("/");
     Path testPath = new Path(parentDir, "test.dat");
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
     authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.READ.toString(), true);
     fs.updateWasbAuthorizer(authorizer);
@@ -547,8 +522,6 @@ public class TestNativeAzureFileSystemAuthorization
 
     setExpectedFailureMessage("delete", testPath);
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
     authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.READ.toString(), true);
     fs.updateWasbAuthorizer(authorizer);
@@ -558,7 +531,7 @@ public class TestNativeAzureFileSystemAuthorization
 
 
       /* Remove permissions for delete to force failure */
-      authorizer.init(null);
+      authorizer.deleteAllAuthRules();
       authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), false);
       fs.updateWasbAuthorizer(authorizer);
 
@@ -566,7 +539,7 @@ public class TestNativeAzureFileSystemAuthorization
     }
     finally {
       /* Restore permissions to force a successful delete */
-      authorizer.init(null);
+      authorizer.deleteAllAuthRules();
       authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
       authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.READ.toString(), true);
       fs.updateWasbAuthorizer(authorizer);
@@ -587,8 +560,6 @@ public class TestNativeAzureFileSystemAuthorization
     Path parentDir = new Path("/testDeleteIntermediateFolder");
     Path testPath = new Path(parentDir, "1/2/test.dat");
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true); // for create and delete
     authorizer.addAuthRule("/testDeleteIntermediateFolder*",
         WasbAuthorizationOperations.WRITE.toString(), true); // for recursive delete
@@ -602,7 +573,7 @@ public class TestNativeAzureFileSystemAuthorization
       ContractTestUtils.assertPathDoesNotExist(fs, "testPath exists after deletion!", parentDir);
     }
     finally {
-      allowRecursiveDelete(fs, authorizer, parentDir.toString());
+      allowRecursiveDelete(fs, parentDir.toString());
       fs.delete(parentDir, true);
     }
   }
@@ -616,8 +587,6 @@ public class TestNativeAzureFileSystemAuthorization
 
     Path testPath = new Path("/");
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.READ.toString(), true);
     fs.updateWasbAuthorizer(authorizer);
 
@@ -635,8 +604,6 @@ public class TestNativeAzureFileSystemAuthorization
 
     setExpectedFailureMessage("getFileStatus", testPath);
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.READ.toString(), false);
     fs.updateWasbAuthorizer(authorizer);
 
@@ -652,8 +619,6 @@ public class TestNativeAzureFileSystemAuthorization
 
     Path testPath = new Path("/testMkdirsAccessCheckPositive/1/2/3");
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
     authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.READ.toString(), true);
     fs.updateWasbAuthorizer(authorizer);
@@ -663,7 +628,7 @@ public class TestNativeAzureFileSystemAuthorization
       ContractTestUtils.assertIsDirectory(fs, testPath);
     }
     finally {
-      allowRecursiveDelete(fs, authorizer, "/testMkdirsAccessCheckPositive");
+      allowRecursiveDelete(fs, "/testMkdirsAccessCheckPositive");
       fs.delete(new Path("/testMkdirsAccessCheckPositive"), true);
     }
   }
@@ -679,8 +644,6 @@ public class TestNativeAzureFileSystemAuthorization
 
     setExpectedFailureMessage("mkdirs", testPath);
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), false);
     authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.READ.toString(), true);
     fs.updateWasbAuthorizer(authorizer);
@@ -690,12 +653,11 @@ public class TestNativeAzureFileSystemAuthorization
       ContractTestUtils.assertPathDoesNotExist(fs, "testPath was not created", testPath);
     }
     finally {
-      allowRecursiveDelete(fs, authorizer, "/testMkdirsAccessCheckNegative");
+      allowRecursiveDelete(fs, "/testMkdirsAccessCheckNegative");
       fs.delete(new Path("/testMkdirsAccessCheckNegative"), true);
     }
   }
 
-
   /**
    * Positive test triple slash format (wasb:///) access check
    * @throws Throwable
@@ -705,8 +667,6 @@ public class TestNativeAzureFileSystemAuthorization
 
     Path testPath = new Path("/");
 
-    MockWasbAuthorizerImpl authorizer = new MockWasbAuthorizerImpl(fs);
-    authorizer.init(null);
     authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.READ.toString(), true);
     fs.updateWasbAuthorizer(authorizer);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e47993cc/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorizationWithOwner.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorizationWithOwner.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorizationWithOwner.java
new file mode 100644
index 0000000..3329e67
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorizationWithOwner.java
@@ -0,0 +1,122 @@
+/**
+ * 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.azure;
+
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import java.security.PrivilegedExceptionAction;
+
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+import org.junit.Before;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test class that runs wasb authorization tests with owner check enabled.
+ */
+public class TestNativeAzureFileSystemAuthorizationWithOwner
+  extends TestNativeAzureFileSystemAuthorization {
+
+  @Before
+  public void beforeMethod() {
+    super.beforeMethod();
+    authorizer.init(null, true);
+  }
+
+  /**
+   * Test case when owner matches current user
+   */
+  @Test
+  public void testOwnerPermissionPositive() throws Throwable {
+
+    Path parentDir = new Path("/testOwnerPermissionPositive");
+    Path testPath = new Path(parentDir, "test.data");
+
+    authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
+    authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.READ.toString(), true);
+    authorizer.addAuthRule(parentDir.toString(), WasbAuthorizationOperations.WRITE.toString(), true);
+    // additional rule used for assertPathExists
+    authorizer.addAuthRule(parentDir.toString(), WasbAuthorizationOperations.READ.toString(), true);
+    fs.updateWasbAuthorizer(authorizer);
+
+    try {
+      // creates parentDir with owner as current user
+      fs.mkdirs(parentDir);
+      ContractTestUtils.assertPathExists(fs, "parentDir does not exist", parentDir);
+
+      fs.create(testPath);
+      fs.getFileStatus(testPath);
+      ContractTestUtils.assertPathExists(fs, "testPath does not exist", testPath);
+
+    } finally {
+      allowRecursiveDelete(fs, parentDir.toString());
+      fs.delete(parentDir, true);
+    }
+  }
+
+  /**
+   * Negative test case for owner does not match current user
+   */
+  @Test
+  public void testOwnerPermissionNegative() throws Throwable {
+    expectedEx.expect(WasbAuthorizationException.class);
+
+    Path parentDir = new Path("/testOwnerPermissionNegative");
+    Path childDir = new Path(parentDir, "childDir");
+
+    setExpectedFailureMessage("mkdirs", childDir);
+
+    authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
+    authorizer.addAuthRule(parentDir.toString(), WasbAuthorizationOperations.WRITE.toString(), true);
+
+    fs.updateWasbAuthorizer(authorizer);
+
+    try{
+      fs.mkdirs(parentDir);
+      UserGroupInformation ugiSuperUser = UserGroupInformation.createUserForTesting(
+          "testuser", new String[] {});
+
+      ugiSuperUser.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+          fs.mkdirs(childDir);
+          return null;
+        }
+      });
+
+    } finally {
+       allowRecursiveDelete(fs, parentDir.toString());
+       fs.delete(parentDir, true);
+    }
+  }
+
+  /**
+   * Test to verify that retrieving owner information does not
+   * throw when file/folder does not exist
+   */
+  @Test
+  public void testRetrievingOwnerDoesNotFailWhenFileDoesNotExist() throws Throwable {
+
+    Path testdirectory = new Path("/testDirectory123454565");
+
+    String owner = fs.getOwnerForPath(testdirectory);
+    assertEquals("", owner);
+  }
+}
+


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