فهرست منبع

HDFS-13170. Port webhdfs unmaskedpermission parameter to HTTPFS. Contributed by Stephen O'Donnell.

(cherry picked from commit 92c15e19272a8bb59f8704c44d4cd713d3a58bcb)
Xiao Chen 7 سال پیش
والد
کامیت
2d93a7381e

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java

@@ -104,6 +104,7 @@ public class HttpFSFileSystem extends FileSystem
   public static final String REPLICATION_PARAM = "replication";
   public static final String BLOCKSIZE_PARAM = "blocksize";
   public static final String PERMISSION_PARAM = "permission";
+  public static final String UNMASKED_PERMISSION_PARAM = "unmaskedpermission";
   public static final String ACLSPEC_PARAM = "aclspec";
   public static final String DESTINATION_PARAM = "destination";
   public static final String RECURSIVE_PARAM = "recursive";

+ 18 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.lib.service.FileSystemAccess;
 import org.apache.hadoop.util.StringUtils;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
+import org.apache.hadoop.fs.permission.FsCreateModes;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -473,6 +474,7 @@ public class FSOperations {
     private InputStream is;
     private Path path;
     private short permission;
+    private short unmaskedPermission;
     private boolean override;
     private short replication;
     private long blockSize;
@@ -486,12 +488,14 @@ public class FSOperations {
      * @param override if the file should be overriden if it already exist.
      * @param repl the replication factor for the file.
      * @param blockSize the block size for the file.
+     * @param unmaskedPerm unmasked permissions for the file
      */
     public FSCreate(InputStream is, String path, short perm, boolean override,
-                    short repl, long blockSize) {
+                    short repl, long blockSize, short unmaskedPerm) {
       this.is = is;
       this.path = new Path(path);
       this.permission = perm;
+      this.unmaskedPermission = unmaskedPerm;
       this.override = override;
       this.replication = repl;
       this.blockSize = blockSize;
@@ -515,6 +519,10 @@ public class FSOperations {
         blockSize = fs.getDefaultBlockSize(path);
       }
       FsPermission fsPermission = new FsPermission(permission);
+      if (unmaskedPermission != -1) {
+        fsPermission = FsCreateModes.create(fsPermission,
+            new FsPermission(unmaskedPermission));
+      }
       int bufferSize = fs.getConf().getInt(HTTPFS_BUFFER_SIZE_KEY,
           HTTP_BUFFER_SIZE_DEFAULT);
       OutputStream os = fs.create(path, fsPermission, override, bufferSize, replication, blockSize, null);
@@ -748,16 +756,20 @@ public class FSOperations {
 
     private Path path;
     private short permission;
+    private short unmaskedPermission;
 
     /**
      * Creates a mkdirs executor.
      *
      * @param path directory path to create.
      * @param permission permission to use.
+     * @param unmaskedPermission unmasked permissions for the directory
      */
-    public FSMkdirs(String path, short permission) {
+    public FSMkdirs(String path, short permission,
+        short unmaskedPermission) {
       this.path = new Path(path);
       this.permission = permission;
+      this.unmaskedPermission = unmaskedPermission;
     }
 
     /**
@@ -773,6 +785,10 @@ public class FSOperations {
     @Override
     public JSONObject execute(FileSystem fs) throws IOException {
       FsPermission fsPermission = new FsPermission(permission);
+      if (unmaskedPermission != -1) {
+        fsPermission = FsCreateModes.create(fsPermission,
+            new FsPermission(unmaskedPermission));
+      }
       boolean mkdirs = fs.mkdirs(path, fsPermission);
       return toJSON(HttpFSFileSystem.MKDIRS_JSON, mkdirs);
     }

+ 27 - 3
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java

@@ -66,9 +66,11 @@ public class HttpFSParametersProvider extends ParametersProvider {
     PARAMS_DEF.put(Operation.CONCAT, new Class[]{SourcesParam.class});
     PARAMS_DEF.put(Operation.TRUNCATE, new Class[]{NewLengthParam.class});
     PARAMS_DEF.put(Operation.CREATE,
-      new Class[]{PermissionParam.class, OverwriteParam.class,
-                  ReplicationParam.class, BlockSizeParam.class, DataParam.class});
-    PARAMS_DEF.put(Operation.MKDIRS, new Class[]{PermissionParam.class});
+        new Class[]{PermissionParam.class, OverwriteParam.class,
+            ReplicationParam.class, BlockSizeParam.class, DataParam.class,
+            UnmaskedPermissionParam.class});
+    PARAMS_DEF.put(Operation.MKDIRS, new Class[]{PermissionParam.class,
+        UnmaskedPermissionParam.class});
     PARAMS_DEF.put(Operation.RENAME, new Class[]{DestinationParam.class});
     PARAMS_DEF.put(Operation.SETOWNER,
         new Class[]{OwnerParam.class, GroupParam.class});
@@ -384,6 +386,28 @@ public class HttpFSParametersProvider extends ParametersProvider {
 
   }
 
+  /**
+   * Class for unmaskedpermission parameter.
+   */
+  @InterfaceAudience.Private
+  public static class UnmaskedPermissionParam extends ShortParam {
+
+    /**
+     * Parameter name.
+     */
+    public static final String NAME =
+        HttpFSFileSystem.UNMASKED_PERMISSION_PARAM;
+
+
+    /**
+     * Constructor.
+     */
+    public UnmaskedPermissionParam() {
+      super(NAME, (short) -1, 8);
+    }
+
+  }
+
   /**
    * Class for AclPermission parameter.
    */

+ 13 - 5
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.PolicyNameParam
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.RecursiveParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.ReplicationParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.SourcesParam;
+import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.UnmaskedPermissionParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.SnapshotNameParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrEncodingParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrNameParam;
@@ -578,6 +579,8 @@ public class HttpFSServer {
         } else {
           Short permission = params.get(PermissionParam.NAME,
                                          PermissionParam.class);
+          Short unmaskedPermission = params.get(UnmaskedPermissionParam.NAME,
+              UnmaskedPermissionParam.class);
           Boolean override = params.get(OverwriteParam.NAME,
                                         OverwriteParam.class);
           Short replication = params.get(ReplicationParam.NAME,
@@ -586,11 +589,13 @@ public class HttpFSServer {
                                       BlockSizeParam.class);
           FSOperations.FSCreate command =
             new FSOperations.FSCreate(is, path, permission, override,
-                                      replication, blockSize);
+                replication, blockSize, unmaskedPermission);
           fsExecute(user, command);
           AUDIT_LOG.info(
-            "[{}] permission [{}] override [{}] replication [{}] blockSize [{}]",
-            new Object[]{path, permission, override, replication, blockSize});
+              "[{}] permission [{}] override [{}] "+
+              "replication [{}] blockSize [{}] unmaskedpermission [{}]",
+              new Object[]{path, permission,  override, replication, blockSize,
+                  unmaskedPermission});
           response = Response.status(Response.Status.CREATED).build();
         }
         break;
@@ -646,10 +651,13 @@ public class HttpFSServer {
       case MKDIRS: {
         Short permission = params.get(PermissionParam.NAME,
                                        PermissionParam.class);
+        Short unmaskedPermission = params.get(UnmaskedPermissionParam.NAME,
+            UnmaskedPermissionParam.class);
         FSOperations.FSMkdirs command =
-          new FSOperations.FSMkdirs(path, permission);
+            new FSOperations.FSMkdirs(path, permission, unmaskedPermission);
         JSONObject json = fsExecute(user, command);
-        AUDIT_LOG.info("[{}] permission [{}]", path, permission);
+        AUDIT_LOG.info("[{}] permission [{}] unmaskedpermission [{}]",
+            path, permission, unmaskedPermission);
         response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
         break;
       }

+ 174 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java

@@ -41,6 +41,7 @@ import java.net.URL;
 import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
@@ -48,6 +49,11 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.XAttrCodec;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclEntryScope;
+import org.apache.hadoop.fs.permission.AclEntryType;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.web.WebHdfsConstants;
 import org.apache.hadoop.lib.server.Service;
 import org.apache.hadoop.lib.server.ServiceException;
@@ -406,6 +412,19 @@ public class TestHttpFSServer extends HFSTestCase {
    * @throws Exception
    */
   private void createWithHttp(String filename, String perms) throws Exception {
+    createWithHttp(filename, perms, null);
+  }
+
+  /**
+   * Talks to the http interface to create a file.
+   *
+   * @param filename The file to create
+   * @param perms The permission field, if any (may be null)
+   * @param unmaskedPerms The unmaskedPermission field, if any (may be null)
+   * @throws Exception
+   */
+  private void createWithHttp(String filename, String perms,
+      String unmaskedPerms) throws Exception {
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     // Remove leading / from filename
     if (filename.charAt(0) == '/') {
@@ -421,6 +440,9 @@ public class TestHttpFSServer extends HFSTestCase {
               "/webhdfs/v1/{0}?user.name={1}&permission={2}&op=CREATE",
               filename, user, perms);
     }
+    if (unmaskedPerms != null) {
+      pathOps = pathOps+"&unmaskedpermission="+unmaskedPerms;
+    }
     URL url = new URL(TestJettyHelper.getJettyURL(), pathOps);
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
     conn.addRequestProperty("Content-Type", "application/octet-stream");
@@ -429,6 +451,41 @@ public class TestHttpFSServer extends HFSTestCase {
     Assert.assertEquals(HttpURLConnection.HTTP_CREATED, conn.getResponseCode());
   }
 
+  /**
+   * Talks to the http interface to create a directory.
+   *
+   * @param dirname The directory to create
+   * @param perms The permission field, if any (may be null)
+   * @param unmaskedPerms The unmaskedPermission field, if any (may be null)
+   * @throws Exception
+   */
+  private void createDirWithHttp(String dirname, String perms,
+      String unmaskedPerms) throws Exception {
+    String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
+    // Remove leading / from filename
+    if (dirname.charAt(0) == '/') {
+      dirname = dirname.substring(1);
+    }
+    String pathOps;
+    if (perms == null) {
+      pathOps = MessageFormat.format(
+              "/webhdfs/v1/{0}?user.name={1}&op=MKDIRS",
+              dirname, user);
+    } else {
+      pathOps = MessageFormat.format(
+              "/webhdfs/v1/{0}?user.name={1}&permission={2}&op=MKDIRS",
+              dirname, user, perms);
+    }
+    if (unmaskedPerms != null) {
+      pathOps = pathOps+"&unmaskedpermission="+unmaskedPerms;
+    }
+    URL url = new URL(TestJettyHelper.getJettyURL(), pathOps);
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestMethod("PUT");
+    conn.connect();
+    Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+  }
+
   /**
    * Talks to the http interface to get the json output of a *STATUS command
    * on the given file.
@@ -577,6 +634,27 @@ public class TestHttpFSServer extends HFSTestCase {
     }
   }
 
+  /**
+   *
+   * @param stat AclStatus object from a call to getAclStatus
+   * @param name The name of the ACL being searched for
+   * @return The AclEntry if found, or null otherwise
+   * @throws IOException
+   */
+  private AclEntry findAclWithName(AclStatus stat, String name)
+      throws IOException{
+    AclEntry relevantAcl = null;
+    Iterator<AclEntry> it = stat.getEntries().iterator();
+    while (it.hasNext()) {
+      AclEntry e = it.next();
+      if (e.getName().equals(name)) {
+        relevantAcl = e;
+        break;
+      }
+    }
+    return relevantAcl;
+  }
+
   /**
    * Validate that files are created with 755 permissions when no
    * 'permissions' attribute is specified, and when 'permissions'
@@ -837,6 +915,102 @@ public class TestHttpFSServer extends HFSTestCase {
     Assert.assertEquals(-1, is.read());
   }
 
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testCreateFileWithUnmaskedPermissions() throws Exception {
+    createHttpFSServer(false, false);
+
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    // Create a folder with a default acl default:user2:rw-
+    fs.mkdirs(new Path("/tmp"));
+    AclEntry acl = new org.apache.hadoop.fs.permission.AclEntry.Builder()
+        .setType(AclEntryType.USER)
+        .setScope(AclEntryScope.DEFAULT)
+        .setName("user2")
+        .setPermission(FsAction.READ_WRITE)
+        .build();
+    fs.setAcl(new Path("/tmp"), new ArrayList<AclEntry>(Arrays.asList(acl)));
+
+    String notUnmaskedFile = "/tmp/notUnmasked";
+    String unmaskedFile = "/tmp/unmasked";
+
+    // Create a file inside the folder. It should inherit the default acl
+    // but the mask should affect the ACL permissions. The mask is controlled
+    // by the group permissions, which are 0, and hence the mask will make
+    // the effective permission of the inherited ACL be NONE.
+    createWithHttp(notUnmaskedFile, "700");
+
+    // Pull the relevant ACL from the FS object and check the mask has affected
+    // its permissions.
+    AclStatus aclStatus = fs.getAclStatus(new Path(notUnmaskedFile));
+    AclEntry theAcl = findAclWithName(aclStatus, "user2");
+
+    Assert.assertNotNull(theAcl);
+    Assert.assertEquals(FsAction.NONE,
+        aclStatus.getEffectivePermission(theAcl));
+
+    // Create another file, this time pass a mask of 777. Now the inherited
+    // permissions should be as expected
+    createWithHttp(unmaskedFile, "700", "777");
+
+    aclStatus = fs.getAclStatus(new Path(unmaskedFile));
+    theAcl = findAclWithName(aclStatus, "user2");
+
+    Assert.assertNotNull(theAcl);
+    Assert.assertEquals(FsAction.READ_WRITE,
+        aclStatus.getEffectivePermission(theAcl));
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testMkdirWithUnmaskedPermissions() throws Exception {
+    createHttpFSServer(false, false);
+
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    // Create a folder with a default acl default:user2:rw-
+    fs.mkdirs(new Path("/tmp"));
+    AclEntry acl = new org.apache.hadoop.fs.permission.AclEntry.Builder()
+        .setType(AclEntryType.USER)
+        .setScope(AclEntryScope.DEFAULT)
+        .setName("user2")
+        .setPermission(FsAction.READ_WRITE)
+        .build();
+    fs.setAcl(new Path("/tmp"), new ArrayList<AclEntry>(Arrays.asList(acl)));
+
+    String notUnmaskedDir = "/tmp/notUnmaskedDir";
+    String unmaskedDir = "/tmp/unmaskedDir";
+
+    // Create a file inside the folder. It should inherit the default acl
+    // but the mask should affect the ACL permissions. The mask is controlled
+    // by the group permissions, which are 0, and hence the mask will make
+    // the effective permission of the inherited ACL be NONE.
+    createDirWithHttp(notUnmaskedDir, "700", null);
+
+    // Pull the relevant ACL from the FS object and check the mask has affected
+    // its permissions.
+    AclStatus aclStatus = fs.getAclStatus(new Path(notUnmaskedDir));
+    AclEntry theAcl = findAclWithName(aclStatus, "user2");
+
+    Assert.assertNotNull(theAcl);
+    Assert.assertEquals(FsAction.NONE,
+        aclStatus.getEffectivePermission(theAcl));
+
+    // Create another file, this time pass a mask of 777. Now the inherited
+    // permissions should be as expected
+    createDirWithHttp(unmaskedDir, "700", "777");
+
+    aclStatus = fs.getAclStatus(new Path(unmaskedDir));
+    theAcl = findAclWithName(aclStatus, "user2");
+
+    Assert.assertNotNull(theAcl);
+    Assert.assertEquals(FsAction.READ_WRITE,
+        aclStatus.getEffectivePermission(theAcl));
+  }
+
   @Test
   @TestDir
   @TestJetty